\Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2018-09-26 22:48:59 +03:00
commit 836c20b2d3
4 changed files with 129 additions and 8 deletions

View File

@ -31,6 +31,28 @@ template <> constexpr size_t maxDecimalPrecision<Decimal128>() { return 38; }
DataTypePtr createDecimal(UInt64 precision, UInt64 scale);
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;
}
/// Implements Decimal(P, S), where P is precision, S is scale.
/// Maximum precisions for underlying types are:
@ -201,7 +223,7 @@ inline const DataTypeDecimal<T> * checkDecimal(const IDataType & data_type)
return typeid_cast<const DataTypeDecimal<T> *>(&data_type);
}
inline UInt32 getDecimalScale(const IDataType & data_type)
inline UInt32 getDecimalScale(const IDataType & data_type, UInt32 default_value = std::numeric_limits<UInt32>::max())
{
if (auto * decimal_type = checkDecimal<Decimal32>(data_type))
return decimal_type->getScale();
@ -209,7 +231,7 @@ inline UInt32 getDecimalScale(const IDataType & data_type)
return decimal_type->getScale();
if (auto * decimal_type = checkDecimal<Decimal128>(data_type))
return decimal_type->getScale();
return std::numeric_limits<UInt32>::max();
return default_value;
}
///

View File

@ -231,22 +231,53 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
if (have_decimal32 || have_decimal64 || have_decimal128)
{
bool all_are_decimals = type_ids.size() == (have_decimal32 + have_decimal64 + have_decimal128);
if (!all_are_decimals)
throw Exception(getExceptionMessagePrefix(types) + " because some of them are Decimals and some are not",
UInt32 num_supported = have_decimal32 + have_decimal64 + have_decimal128;
std::vector<TypeIndex> int_ids = {TypeIndex::Int8, TypeIndex::UInt8, TypeIndex::Int16, TypeIndex::UInt16,
TypeIndex::Int32, TypeIndex::UInt32, TypeIndex::Int64, TypeIndex::UInt64};
std::vector<UInt32> num_ints(int_ids.size(), 0);
TypeIndex max_int = TypeIndex::Nothing;
for (size_t i = 0; i < int_ids.size(); ++i)
{
UInt32 num = type_ids.count(int_ids[i]);
num_ints[i] = num;
num_supported += num;
if (num)
max_int = int_ids[i];
}
if (num_supported != type_ids.size())
throw Exception(getExceptionMessagePrefix(types) + " because some of them have no lossless convertion to Decimal",
ErrorCodes::NO_COMMON_TYPE);
UInt32 max_scale = 0;
for (const auto & type : types)
{
UInt32 scale = getDecimalScale(*type);
UInt32 scale = getDecimalScale(*type, 0);
if (scale > max_scale)
max_scale = scale;
}
if (have_decimal128)
UInt32 min_precision = max_scale + leastDecimalPrecisionFor(max_int);
/// special cases Int32 -> Dec32, Int64 -> Dec64
if (max_scale == 0)
{
if (max_int == TypeIndex::Int32)
min_precision = DataTypeDecimal<Decimal32>::maxPrecision();
else if (max_int == TypeIndex::Int64)
min_precision = DataTypeDecimal<Decimal64>::maxPrecision();
}
if (min_precision > DataTypeDecimal<Decimal128>::maxPrecision())
throw Exception(getExceptionMessagePrefix(types) + " because the least supertype is Decimal("
+ toString(min_precision) + ',' + toString(max_scale) + ')',
ErrorCodes::NO_COMMON_TYPE);
if (have_decimal128 || min_precision > DataTypeDecimal<Decimal64>::maxPrecision())
return std::make_shared<DataTypeDecimal<Decimal128>>(DataTypeDecimal<Decimal128>::maxPrecision(), max_scale);
if (have_decimal64)
if (have_decimal64 || min_precision > DataTypeDecimal<Decimal32>::maxPrecision())
return std::make_shared<DataTypeDecimal<Decimal64>>(DataTypeDecimal<Decimal64>::maxPrecision(), max_scale);
return std::make_shared<DataTypeDecimal<Decimal32>>(DataTypeDecimal<Decimal32>::maxPrecision(), max_scale);
}

View File

@ -60,3 +60,33 @@ Tuple(Decimal(9, 1), Decimal(18, 1), Decimal(38, 1)) Decimal(9, 1) Decimal(18, 1
-123456789.123456784 2 2 2
0.123456789123456784 2 2 2
-0.123456789112345680 2 2 2
Decimal(9, 5)
Decimal(9, 5)
Decimal(9, 4)
Decimal(9, 4)
Decimal(18, 7)
Decimal(18, 7)
Decimal(18, 5)
Decimal(18, 5)
Decimal(18, 4)
Decimal(18, 4)
Decimal(38, 4)
Decimal(38, 4)
Decimal(18, 4)
Decimal(18, 4)
Decimal(18, 4)
Decimal(18, 4)
Decimal(18, 4)
Decimal(18, 4)
Decimal(38, 4)
Decimal(38, 4)
Decimal(38, 4)
Decimal(38, 4)
Decimal(38, 4)
Decimal(38, 4)
Decimal(38, 4)
Decimal(38, 4)
Decimal(38, 4)
Decimal(38, 4)
Decimal(9, 0)
Decimal(18, 0)

View File

@ -112,4 +112,42 @@ SELECT toDecimal64(-123456789.123456789, 9) AS x, countEqual([x+1, x, x], x), co
SELECT toDecimal128(0.123456789123456789, 18) AS x, countEqual([x+1, x, x], x), countEqual([x, x-1, x], x), countEqual([x, x], x-0);
SELECT toDecimal128(-0.1234567891123456789, 18) AS x, countEqual([x+1, x, x], x), countEqual([x, x-1, x], x), countEqual([x, x], x+0);
SELECT toTypeName(x) FROM (SELECT toDecimal32('1234.5', 5) AS x UNION ALL SELECT toInt8(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal32('1234.5', 5) AS x UNION ALL SELECT toUInt8(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal32('12345.0', 4) AS x UNION ALL SELECT toInt16(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal32('12345.0', 4) AS x UNION ALL SELECT toUInt16(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal32('12.345', 7) AS x UNION ALL SELECT toInt8(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal32('12.345', 7) AS x UNION ALL SELECT toUInt8(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal32('1234.5', 5) AS x UNION ALL SELECT toInt16(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal32('1234.5', 5) AS x UNION ALL SELECT toUInt16(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal32('12345.00', 4) AS x UNION ALL SELECT toInt32(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal32('12345.00', 4) AS x UNION ALL SELECT toUInt32(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal32('12345.00', 4) AS x UNION ALL SELECT toInt64(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal32('12345.00', 4) AS x UNION ALL SELECT toUInt64(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal64('12345.00', 4) AS x UNION ALL SELECT toInt8(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal64('12345.00', 4) AS x UNION ALL SELECT toUInt8(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal64('12345.00', 4) AS x UNION ALL SELECT toInt16(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal64('12345.00', 4) AS x UNION ALL SELECT toUInt16(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal64('12345.00', 4) AS x UNION ALL SELECT toInt32(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal64('12345.00', 4) AS x UNION ALL SELECT toUInt32(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal64('12345.00', 4) AS x UNION ALL SELECT toInt64(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal64('12345.00', 4) AS x UNION ALL SELECT toUInt64(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal128('12345.00', 4) AS x UNION ALL SELECT toInt8(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal128('12345.00', 4) AS x UNION ALL SELECT toUInt8(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal128('12345.00', 4) AS x UNION ALL SELECT toInt16(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal128('12345.00', 4) AS x UNION ALL SELECT toUInt16(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal128('12345.00', 4) AS x UNION ALL SELECT toInt32(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal128('12345.00', 4) AS x UNION ALL SELECT toUInt32(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal128('12345.00', 4) AS x UNION ALL SELECT toInt64(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal128('12345.00', 4) AS x UNION ALL SELECT toUInt64(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal32('12345', 0) AS x UNION ALL SELECT toInt32(0) AS x) WHERE x = 0;
SELECT toTypeName(x) FROM (SELECT toDecimal64('12345', 0) AS x UNION ALL SELECT toInt64(0) AS x) WHERE x = 0;
DROP TABLE IF EXISTS test.decimal;