ClickHouse/dbms/src/DataTypes/DataTypesDecimal.cpp

263 lines
8.5 KiB
C++
Raw Normal View History

2018-07-20 19:05:07 +00:00
#include <type_traits>
#include <common/intExp.h>
2018-07-20 19:05:07 +00:00
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeFactory.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/readFloatText.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#include <Interpreters/Context.h>
2018-07-20 19:05:07 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
}
bool decimalCheckComparisonOverflow(const Context & context) { return context.getSettingsRef().decimal_check_comparison_overflow; }
bool decimalCheckArithmeticOverflow(const Context & context) { return context.getSettingsRef().decimal_check_arithmetic_overflow; }
2018-07-20 19:05:07 +00:00
//
template <typename T>
std::string DataTypeDecimal<T>::getName() const
{
std::stringstream ss;
ss << "Decimal(" << precision << ", " << scale << ")";
return ss.str();
}
template <typename T>
bool DataTypeDecimal<T>::equals(const IDataType & rhs) const
2018-07-20 19:05:07 +00:00
{
2018-07-25 19:38:21 +00:00
if (auto * ptype = typeid_cast<const DataTypeDecimal<T> *>(&rhs))
return scale == ptype->getScale();
return false;
2018-07-20 19:05:07 +00:00
}
template <typename T>
2018-08-21 18:25:38 +00:00
void DataTypeDecimal<T>::writeText(T value, WriteBuffer & ostr) const
2018-07-20 19:05:07 +00:00
{
2018-08-07 13:57:28 +00:00
if (value < T(0))
{
2018-08-07 13:57:28 +00:00
value *= T(-1);
writeChar('-', ostr); /// avoid crop leading minus when whole part is zero
}
2018-07-20 19:05:07 +00:00
2018-08-07 13:57:28 +00:00
writeIntText(static_cast<typename T::NativeType>(wholePart(value)), ostr);
if (scale)
{
writeChar('.', ostr);
String str_fractional(scale, '0');
2018-08-07 13:57:28 +00:00
for (Int32 pos = scale - 1; pos >= 0; --pos, value /= T(10))
str_fractional[pos] += value % T(10);
ostr.write(str_fractional.data(), scale);
}
2018-07-20 19:05:07 +00:00
}
2018-08-21 18:25:38 +00:00
template <typename T>
void DataTypeDecimal<T>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{
T value = static_cast<const ColumnType &>(column).getData()[row_num];
writeText(value, ostr);
}
2018-07-20 19:05:07 +00:00
template <typename T>
2018-08-21 18:25:38 +00:00
void DataTypeDecimal<T>::readText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale)
2018-07-20 19:05:07 +00:00
{
UInt32 unread_scale = scale;
readDecimalText(istr, x, precision, unread_scale);
x *= getScaleMultiplier(unread_scale);
2018-08-21 18:25:38 +00:00
}
template <typename T>
void DataTypeDecimal<T>::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
T x;
readText(x, istr);
2018-07-25 19:38:21 +00:00
static_cast<ColumnType &>(column).getData().push_back(x);
2018-07-20 19:05:07 +00:00
}
template <typename T>
T DataTypeDecimal<T>::parseFromString(const String & str) const
{
ReadBufferFromMemory buf(str.data(), str.size());
T x;
UInt32 unread_scale = scale;
readDecimalText(buf, x, precision, unread_scale, true);
x *= getScaleMultiplier(unread_scale);
return x;
}
2018-07-20 19:05:07 +00:00
template <typename T>
void DataTypeDecimal<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
2018-08-21 04:31:35 +00:00
FieldType x = get<DecimalField>(field);
2018-07-20 19:05:07 +00:00
writeBinary(x, ostr);
}
template <typename T>
void DataTypeDecimal<T>::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
2018-08-07 13:57:28 +00:00
const FieldType & x = static_cast<const ColumnType &>(column).getData()[row_num];
writeBinary(x, ostr);
2018-07-20 19:05:07 +00:00
}
template <typename T>
void DataTypeDecimal<T>::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const
{
2018-07-25 19:38:21 +00:00
const typename ColumnType::Container & x = typeid_cast<const ColumnType &>(column).getData();
2018-07-20 19:05:07 +00:00
size_t size = x.size();
if (limit == 0 || offset + limit > size)
limit = size - offset;
2018-08-07 13:57:28 +00:00
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(FieldType) * limit);
2018-07-20 19:05:07 +00:00
}
template <typename T>
void DataTypeDecimal<T>::deserializeBinary(Field & field, ReadBuffer & istr) const
{
2018-08-07 13:57:28 +00:00
typename FieldType::NativeType x;
2018-07-20 19:05:07 +00:00
readBinary(x, istr);
2018-08-21 04:31:35 +00:00
field = DecimalField(T(x), scale);
2018-07-20 19:05:07 +00:00
}
template <typename T>
void DataTypeDecimal<T>::deserializeBinary(IColumn & column, ReadBuffer & istr) const
{
2018-08-07 13:57:28 +00:00
typename FieldType::NativeType x;
2018-07-20 19:05:07 +00:00
readBinary(x, istr);
2018-08-07 13:57:28 +00:00
static_cast<ColumnType &>(column).getData().push_back(FieldType(x));
2018-07-20 19:05:07 +00:00
}
template <typename T>
void DataTypeDecimal<T>::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double ) const
{
2018-07-25 19:38:21 +00:00
typename ColumnType::Container & x = typeid_cast<ColumnType &>(column).getData();
2018-07-20 19:05:07 +00:00
size_t initial_size = x.size();
x.resize(initial_size + limit);
2018-08-07 13:57:28 +00:00
size_t size = istr.readBig(reinterpret_cast<char*>(&x[initial_size]), sizeof(FieldType) * limit);
x.resize(initial_size + size / sizeof(FieldType));
2018-07-20 19:05:07 +00:00
}
template <typename T>
Field DataTypeDecimal<T>::getDefault() const
{
2018-08-21 04:31:35 +00:00
return DecimalField(T(0), scale);
2018-07-20 19:05:07 +00:00
}
template <typename T>
MutableColumnPtr DataTypeDecimal<T>::createColumn() const
{
auto column = ColumnType::create();
column->getData().setScale(scale);
return column;
2018-07-20 19:05:07 +00:00
}
//
2018-08-21 18:25:38 +00:00
DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value)
{
2018-08-21 18:55:36 +00:00
if (precision_value < minDecimalPrecision() || precision_value > maxDecimalPrecision<Decimal128>())
2018-08-21 18:25:38 +00:00
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 presicion are not supported", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
2018-08-21 18:55:36 +00:00
if (precision_value <= maxDecimalPrecision<Decimal32>())
return std::make_shared<DataTypeDecimal<Decimal32>>(precision_value, scale_value);
else if (precision_value <= maxDecimalPrecision<Decimal64>())
return std::make_shared<DataTypeDecimal<Decimal64>>(precision_value, scale_value);
return std::make_shared<DataTypeDecimal<Decimal128>>(precision_value, scale_value);
2018-08-21 18:25:38 +00:00
}
2018-07-20 19:05:07 +00:00
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",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const ASTLiteral * precision = typeid_cast<const ASTLiteral *>(arguments->children[0].get());
const ASTLiteral * scale = typeid_cast<const ASTLiteral *>(arguments->children[1].get());
if (!precision || precision->value.getType() != Field::Types::UInt64 ||
!scale || !(scale->value.getType() == Field::Types::Int64 || scale->value.getType() == Field::Types::UInt64))
throw Exception("Decimal data type family must have a two numbers as its arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
UInt64 precision_value = precision->value.get<UInt64>();
2018-08-21 18:25:38 +00:00
UInt64 scale_value = scale->value.get<UInt64>();
2018-07-20 19:05:07 +00:00
2018-08-21 18:25:38 +00:00
return createDecimal(precision_value, scale_value);
2018-07-20 19:05:07 +00:00
}
template <typename T>
static DataTypePtr createExect(const ASTPtr & arguments)
{
if (!arguments || arguments->children.size() != 1)
throw Exception("Decimal data type family must have exactly two arguments: precision and scale",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const ASTLiteral * scale_arg = typeid_cast<const ASTLiteral *>(arguments->children[0].get());
if (!scale_arg || !(scale_arg->value.getType() == Field::Types::Int64 || scale_arg->value.getType() == Field::Types::UInt64))
throw Exception("Decimal data type family must have a two numbers as its arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
UInt64 precision = maxDecimalPrecision<T>();
UInt64 scale = scale_arg->value.get<UInt64>();
return createDecimal(precision, scale);
}
2018-07-20 19:05:07 +00:00
void registerDataTypeDecimal(DataTypeFactory & factory)
{
factory.registerDataType("Decimal32", createExect<Decimal32>, DataTypeFactory::CaseInsensitive);
factory.registerDataType("Decimal64", createExect<Decimal64>, DataTypeFactory::CaseInsensitive);
factory.registerDataType("Decimal128", createExect<Decimal128>, DataTypeFactory::CaseInsensitive);
2018-07-20 19:05:07 +00:00
factory.registerDataType("Decimal", create, DataTypeFactory::CaseInsensitive);
factory.registerAlias("DEC", "Decimal", DataTypeFactory::CaseInsensitive);
2018-07-20 19:05:07 +00:00
}
template <>
2018-08-21 04:31:35 +00:00
Decimal32 DataTypeDecimal<Decimal32>::getScaleMultiplier(UInt32 scale_)
{
return common::exp10_i32(scale_);
}
template <>
2018-08-21 04:31:35 +00:00
Decimal64 DataTypeDecimal<Decimal64>::getScaleMultiplier(UInt32 scale_)
{
return common::exp10_i64(scale_);
}
template <>
2018-08-21 04:31:35 +00:00
Decimal128 DataTypeDecimal<Decimal128>::getScaleMultiplier(UInt32 scale_)
{
return common::exp10_i128(scale_);
}
2018-07-20 19:05:07 +00:00
/// Explicit template instantiations.
2018-08-21 04:31:35 +00:00
template class DataTypeDecimal<Decimal32>;
template class DataTypeDecimal<Decimal64>;
template class DataTypeDecimal<Decimal128>;
2018-07-20 19:05:07 +00:00
}