2018-07-20 19:05:07 +00:00
|
|
|
#include <DataTypes/DataTypesDecimal.h>
|
2019-09-26 15:12:40 +00:00
|
|
|
|
|
|
|
#include <Common/assert_cast.h>
|
|
|
|
#include <Common/typeid_cast.h>
|
|
|
|
#include <Core/DecimalFunctions.h>
|
2018-07-20 19:05:07 +00:00
|
|
|
#include <DataTypes/DataTypeFactory.h>
|
2019-02-19 20:01:31 +00:00
|
|
|
#include <Formats/ProtobufReader.h>
|
2019-01-23 19:41:18 +00:00
|
|
|
#include <Formats/ProtobufWriter.h>
|
2018-07-20 19:05:07 +00:00
|
|
|
#include <IO/ReadHelpers.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2019-05-16 16:41:10 +00:00
|
|
|
#include <IO/readDecimalText.h>
|
2018-08-22 13:22:56 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2019-09-26 15:12:40 +00:00
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Parsers/IAST.h>
|
2018-07-20 19:05:07 +00:00
|
|
|
|
2019-09-26 15:12:40 +00:00
|
|
|
#include <type_traits>
|
2019-08-21 02:28:04 +00:00
|
|
|
|
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;
|
2020-05-17 05:12:33 +00:00
|
|
|
extern const int DECIMAL_OVERFLOW;
|
2018-07-20 19:05:07 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template <typename T>
|
2018-12-13 13:41:47 +00:00
|
|
|
std::string DataTypeDecimal<T>::doGetName() const
|
2018-07-20 19:05:07 +00:00
|
|
|
{
|
|
|
|
std::stringstream ss;
|
2020-03-10 18:16:14 +00:00
|
|
|
ss << "Decimal(" << this->precision << ", " << this->scale << ")";
|
2018-07-20 19:05:07 +00:00
|
|
|
return ss.str();
|
|
|
|
}
|
|
|
|
|
2019-10-02 10:54:59 +00:00
|
|
|
|
2018-07-20 19:05:07 +00:00
|
|
|
template <typename T>
|
2018-07-23 20:19:26 +00:00
|
|
|
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))
|
2019-10-02 10:54:59 +00:00
|
|
|
return this->scale == ptype->getScale();
|
2018-07-25 19:38:21 +00:00
|
|
|
return false;
|
2018-07-20 19:05:07 +00:00
|
|
|
}
|
|
|
|
|
2019-01-25 13:06:21 +00:00
|
|
|
template <typename T>
|
2019-01-25 14:16:23 +00:00
|
|
|
DataTypePtr DataTypeDecimal<T>::promoteNumericType() const
|
2019-01-25 13:06:21 +00:00
|
|
|
{
|
2019-01-25 14:16:23 +00:00
|
|
|
using PromotedType = DataTypeDecimal<Decimal128>;
|
2019-10-02 05:53:38 +00:00
|
|
|
return std::make_shared<PromotedType>(PromotedType::maxPrecision(), this->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
|
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
T value = assert_cast<const ColumnType &>(column).getData()[row_num];
|
2019-10-02 10:54:59 +00:00
|
|
|
writeText(value, this->scale, ostr);
|
2018-08-21 18:25:38 +00:00
|
|
|
}
|
2018-07-20 19:05:07 +00:00
|
|
|
|
2019-05-15 18:50:35 +00:00
|
|
|
template <typename T>
|
2020-05-17 03:16:58 +00:00
|
|
|
bool DataTypeDecimal<T>::tryReadText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale)
|
2019-05-15 18:50:35 +00:00
|
|
|
{
|
|
|
|
UInt32 unread_scale = scale;
|
2020-05-17 03:18:52 +00:00
|
|
|
if (!tryReadDecimalText(istr, x, precision, unread_scale))
|
2020-05-17 03:16:58 +00:00
|
|
|
return false;
|
2019-12-11 14:38:32 +00:00
|
|
|
|
2020-05-17 03:16:58 +00:00
|
|
|
if (common::mulOverflow(x.value, T::getScaleMultiplier(unread_scale), x.value))
|
|
|
|
return false;
|
|
|
|
|
|
|
|
return true;
|
2019-05-15 18:50:35 +00:00
|
|
|
}
|
|
|
|
|
2018-07-20 19:05:07 +00:00
|
|
|
template <typename T>
|
2020-05-17 03:16:58 +00:00
|
|
|
void DataTypeDecimal<T>::readText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale, bool csv)
|
2018-07-20 19:05:07 +00:00
|
|
|
{
|
2018-07-23 20:19:26 +00:00
|
|
|
UInt32 unread_scale = scale;
|
2019-05-15 13:51:17 +00:00
|
|
|
if (csv)
|
|
|
|
readCSVDecimalText(istr, x, precision, unread_scale);
|
|
|
|
else
|
|
|
|
readDecimalText(istr, x, precision, unread_scale);
|
2020-05-17 03:16:58 +00:00
|
|
|
|
|
|
|
if (common::mulOverflow(x.value, T::getScaleMultiplier(unread_scale), x.value))
|
|
|
|
throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW);
|
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);
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnType &>(column).getData().push_back(x);
|
2018-07-20 19:05:07 +00:00
|
|
|
}
|
|
|
|
|
2019-05-15 13:51:17 +00:00
|
|
|
template <typename T>
|
|
|
|
void DataTypeDecimal<T>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
|
|
|
{
|
|
|
|
T x;
|
|
|
|
readText(x, istr, true);
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnType &>(column).getData().push_back(x);
|
2019-05-15 13:51:17 +00:00
|
|
|
}
|
2018-07-20 19:05:07 +00:00
|
|
|
|
2018-07-30 18:10:38 +00:00
|
|
|
template <typename T>
|
2020-05-17 03:16:58 +00:00
|
|
|
T DataTypeDecimal<T>::parseFromString(const String & str) const
|
2018-07-30 18:10:38 +00:00
|
|
|
{
|
|
|
|
ReadBufferFromMemory buf(str.data(), str.size());
|
|
|
|
T x;
|
2019-10-02 10:54:59 +00:00
|
|
|
UInt32 unread_scale = this->scale;
|
|
|
|
readDecimalText(buf, x, this->precision, unread_scale, true);
|
2020-05-17 03:16:58 +00:00
|
|
|
|
|
|
|
if (common::mulOverflow(x.value, T::getScaleMultiplier(unread_scale), x.value))
|
|
|
|
throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW);
|
2018-07-20 19:05:07 +00:00
|
|
|
|
2018-07-30 18:10:38 +00:00
|
|
|
return x;
|
|
|
|
}
|
|
|
|
|
2019-01-23 19:41:18 +00:00
|
|
|
template <typename T>
|
2019-02-26 14:06:05 +00:00
|
|
|
void DataTypeDecimal<T>::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
|
2019-01-23 19:41:18 +00:00
|
|
|
{
|
2019-02-26 14:06:05 +00:00
|
|
|
if (value_index)
|
|
|
|
return;
|
2019-10-02 10:54:59 +00:00
|
|
|
value_index = static_cast<bool>(protobuf.writeDecimal(assert_cast<const ColumnType &>(column).getData()[row_num], this->scale));
|
2019-01-23 19:41:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-19 20:01:31 +00:00
|
|
|
template <typename T>
|
|
|
|
void DataTypeDecimal<T>::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const
|
|
|
|
{
|
|
|
|
row_added = false;
|
|
|
|
T decimal;
|
2019-10-02 10:54:59 +00:00
|
|
|
if (!protobuf.readDecimal(decimal, this->precision, this->scale))
|
2019-02-19 20:01:31 +00:00
|
|
|
return;
|
|
|
|
|
2019-08-21 02:28:04 +00:00
|
|
|
auto & container = assert_cast<ColumnType &>(column).getData();
|
2019-02-19 20:01:31 +00:00
|
|
|
if (allow_add_row)
|
|
|
|
{
|
|
|
|
container.emplace_back(decimal);
|
|
|
|
row_added = true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
container.back() = decimal;
|
|
|
|
}
|
|
|
|
|
2018-07-20 19:05:07 +00:00
|
|
|
|
2020-03-10 18:16:14 +00:00
|
|
|
static DataTypePtr create(const ASTPtr & arguments)
|
2018-07-20 19:05:07 +00:00
|
|
|
{
|
|
|
|
if (!arguments || arguments->children.size() != 2)
|
2019-10-30 10:10:51 +00:00
|
|
|
throw Exception("Decimal data type family must have exactly two arguments: precision and scale",
|
2018-07-20 19:05:07 +00:00
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
2019-03-11 13:22:51 +00:00
|
|
|
const auto * precision = arguments->children[0]->as<ASTLiteral>();
|
|
|
|
const auto * scale = arguments->children[1]->as<ASTLiteral>();
|
2018-07-20 19:05:07 +00:00
|
|
|
|
|
|
|
if (!precision || precision->value.getType() != Field::Types::UInt64 ||
|
|
|
|
!scale || !(scale->value.getType() == Field::Types::Int64 || scale->value.getType() == Field::Types::UInt64))
|
2019-10-30 10:10:51 +00:00
|
|
|
throw Exception("Decimal data type family must have two numbers as its arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2018-07-20 19:05:07 +00:00
|
|
|
|
|
|
|
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
|
|
|
|
2020-03-10 18:16:14 +00:00
|
|
|
return createDecimal<DataTypeDecimal>(precision_value, scale_value);
|
2018-07-20 19:05:07 +00:00
|
|
|
}
|
|
|
|
|
2018-08-22 16:16:39 +00:00
|
|
|
template <typename T>
|
2020-03-10 18:16:14 +00:00
|
|
|
static DataTypePtr createExact(const ASTPtr & arguments)
|
2018-08-22 16:16:39 +00:00
|
|
|
{
|
|
|
|
if (!arguments || arguments->children.size() != 1)
|
2019-10-30 10:10:51 +00:00
|
|
|
throw Exception("Decimal data type family must have exactly two arguments: precision and scale",
|
2018-08-22 16:16:39 +00:00
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
2019-03-11 13:22:51 +00:00
|
|
|
const auto * scale_arg = arguments->children[0]->as<ASTLiteral>();
|
2018-08-22 16:16:39 +00:00
|
|
|
|
|
|
|
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);
|
|
|
|
|
2019-12-06 21:05:31 +00:00
|
|
|
UInt64 precision = DecimalUtils::maxPrecision<T>();
|
2018-08-22 16:16:39 +00:00
|
|
|
UInt64 scale = scale_arg->value.get<UInt64>();
|
|
|
|
|
2020-03-10 18:16:14 +00:00
|
|
|
return createDecimal<DataTypeDecimal>(precision, scale);
|
2018-08-22 16:16:39 +00:00
|
|
|
}
|
2018-07-20 19:05:07 +00:00
|
|
|
|
|
|
|
void registerDataTypeDecimal(DataTypeFactory & factory)
|
|
|
|
{
|
2019-07-05 22:18:37 +00:00
|
|
|
factory.registerDataType("Decimal32", createExact<Decimal32>, DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerDataType("Decimal64", createExact<Decimal64>, DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerDataType("Decimal128", createExact<Decimal128>, DataTypeFactory::CaseInsensitive);
|
2018-08-22 16:16:39 +00:00
|
|
|
|
2018-07-20 19:05:07 +00:00
|
|
|
factory.registerDataType("Decimal", create, DataTypeFactory::CaseInsensitive);
|
2018-07-30 18:10:38 +00:00
|
|
|
factory.registerAlias("DEC", "Decimal", DataTypeFactory::CaseInsensitive);
|
2020-05-12 13:29:20 +00:00
|
|
|
factory.registerAlias("NUMERIC", "Decimal", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("FIXED", "Decimal", DataTypeFactory::CaseInsensitive);
|
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
|
|
|
|
|
|
|
}
|