Decimal round functions [CLICKHOUSE-3765]

This commit is contained in:
chertus 2018-09-10 16:52:18 +03:00
parent 6846d33b88
commit 9828813523
7 changed files with 287 additions and 105 deletions

View File

@ -260,13 +260,13 @@ inline UInt32 getDecimalScale(const IDataType & data_type)
///
template <typename DataType> constexpr bool IsDecimal = false;
template <> constexpr bool IsDecimal<DataTypeDecimal<Decimal32>> = true;
template <> constexpr bool IsDecimal<DataTypeDecimal<Decimal64>> = true;
template <> constexpr bool IsDecimal<DataTypeDecimal<Decimal128>> = true;
template <typename DataType> constexpr bool IsDataTypeDecimal = false;
template <> constexpr bool IsDataTypeDecimal<DataTypeDecimal<Decimal32>> = true;
template <> constexpr bool IsDataTypeDecimal<DataTypeDecimal<Decimal64>> = true;
template <> constexpr bool IsDataTypeDecimal<DataTypeDecimal<Decimal128>> = true;
template <typename FromDataType, typename ToDataType>
inline std::enable_if_t<IsDecimal<FromDataType> && IsDecimal<ToDataType>, typename ToDataType::FieldType>
inline std::enable_if_t<IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>, typename ToDataType::FieldType>
convertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to)
{
ToDataType type_to(ToDataType::maxPrecision(), scale_to);
@ -285,7 +285,7 @@ convertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_fro
}
template <typename FromDataType, typename ToDataType>
inline std::enable_if_t<IsDecimal<FromDataType> && !IsDecimal<ToDataType>, typename ToDataType::FieldType>
inline std::enable_if_t<IsDataTypeDecimal<FromDataType> && !IsDataTypeDecimal<ToDataType>, typename ToDataType::FieldType>
convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale [[maybe_unused]])
{
if (scale > FromDataType::maxPrecision())
@ -298,7 +298,7 @@ convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale
}
template <typename FromDataType, typename ToDataType>
inline std::enable_if_t<!IsDecimal<FromDataType> && IsDecimal<ToDataType>, typename ToDataType::FieldType>
inline std::enable_if_t<!IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>, typename ToDataType::FieldType>
convertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale [[maybe_unused]])
{
if (scale > ToDataType::maxPrecision())

View File

@ -30,4 +30,16 @@ using DataTypeInt64 = DataTypeNumber<Int64>;
using DataTypeFloat32 = DataTypeNumber<Float32>;
using DataTypeFloat64 = DataTypeNumber<Float64>;
template <typename DataType> constexpr bool IsDataTypeNumber = false;
template <> constexpr bool IsDataTypeNumber<DataTypeNumber<UInt8>> = true;
template <> constexpr bool IsDataTypeNumber<DataTypeNumber<UInt16>> = true;
template <> constexpr bool IsDataTypeNumber<DataTypeNumber<UInt32>> = true;
template <> constexpr bool IsDataTypeNumber<DataTypeNumber<UInt64>> = true;
template <> constexpr bool IsDataTypeNumber<DataTypeNumber<Int8>> = true;
template <> constexpr bool IsDataTypeNumber<DataTypeNumber<Int16>> = true;
template <> constexpr bool IsDataTypeNumber<DataTypeNumber<Int32>> = true;
template <> constexpr bool IsDataTypeNumber<DataTypeNumber<Int64>> = true;
template <> constexpr bool IsDataTypeNumber<DataTypeNumber<Float32>> = true;
template <> constexpr bool IsDataTypeNumber<DataTypeNumber<Float64>> = true;
}

View File

@ -1020,14 +1020,14 @@ public:
/// DateTime, but if both operands are Dates, their type must be the same (e.g. Date - DateTime is invalid).
using ResultDataType = Switch<
/// Decimal cases
Case<!allow_decimal && (IsDecimal<LeftDataType> || IsDecimal<RightDataType>), InvalidType>,
Case<IsDecimal<LeftDataType> && IsDecimal<RightDataType> && UseLeftDecimal<LeftDataType, RightDataType>, LeftDataType>,
Case<IsDecimal<LeftDataType> && IsDecimal<RightDataType>, RightDataType>,
Case<IsDecimal<LeftDataType> && !IsDecimal<RightDataType> && IsIntegral<RightDataType>, LeftDataType>,
Case<!IsDecimal<LeftDataType> && IsDecimal<RightDataType> && IsIntegral<LeftDataType>, RightDataType>,
Case<!allow_decimal && (IsDataTypeDecimal<LeftDataType> || IsDataTypeDecimal<RightDataType>), InvalidType>,
Case<IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType> && UseLeftDecimal<LeftDataType, RightDataType>, LeftDataType>,
Case<IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType>, RightDataType>,
Case<IsDataTypeDecimal<LeftDataType> && !IsDataTypeDecimal<RightDataType> && IsIntegral<RightDataType>, LeftDataType>,
Case<!IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType> && IsIntegral<LeftDataType>, RightDataType>,
/// Decimal <op> Real is not supported (traditional DBs convert Decimal <op> Real to Real)
Case<IsDecimal<LeftDataType> && !IsDecimal<RightDataType> && !IsIntegral<RightDataType>, InvalidType>,
Case<!IsDecimal<LeftDataType> && IsDecimal<RightDataType> && !IsIntegral<LeftDataType>, InvalidType>,
Case<IsDataTypeDecimal<LeftDataType> && !IsDataTypeDecimal<RightDataType> && !IsIntegral<RightDataType>, InvalidType>,
Case<!IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType> && !IsIntegral<LeftDataType>, InvalidType>,
/// number <op> number -> see corresponding impl
Case<!IsDateOrDateTime<LeftDataType> && !IsDateOrDateTime<RightDataType>,
DataTypeFromFieldType<typename Op::ResultType>>,
@ -1311,16 +1311,16 @@ public:
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
if constexpr (!std::is_same_v<ResultDataType, InvalidType>)
{
if constexpr (IsDecimal<LeftDataType> && IsDecimal<RightDataType>)
if constexpr (IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType>)
{
constexpr bool is_multiply = std::is_same_v<Op<UInt8, UInt8>, MultiplyImpl<UInt8, UInt8>>;
constexpr bool is_division = std::is_same_v<Op<UInt8, UInt8>, DivideFloatingImpl<UInt8, UInt8>>;
ResultDataType result_type = decimalResultType(left, right, is_multiply, is_division);
type_res = std::make_shared<ResultDataType>(result_type.getPrecision(), result_type.getScale());
}
else if constexpr (IsDecimal<LeftDataType>)
else if constexpr (IsDataTypeDecimal<LeftDataType>)
type_res = std::make_shared<LeftDataType>(left.getPrecision(), left.getScale());
else if constexpr (IsDecimal<RightDataType>)
else if constexpr (IsDataTypeDecimal<RightDataType>)
type_res = std::make_shared<RightDataType>(right.getPrecision(), right.getScale());
else
type_res = std::make_shared<ResultDataType>();
@ -1366,7 +1366,7 @@ public:
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
if constexpr (!std::is_same_v<ResultDataType, InvalidType>)
{
constexpr bool result_is_decimal = IsDecimal<LeftDataType> || IsDecimal<RightDataType>;
constexpr bool result_is_decimal = IsDataTypeDecimal<LeftDataType> || IsDataTypeDecimal<RightDataType>;
constexpr bool is_multiply = std::is_same_v<Op<UInt8, UInt8>, MultiplyImpl<UInt8, UInt8>>;
constexpr bool is_division = std::is_same_v<Op<UInt8, UInt8>, DivideFloatingImpl<UInt8, UInt8>>;
@ -1378,7 +1378,7 @@ public:
using ColVecResult = std::conditional_t<IsDecimalNumber<ResultType>, ColumnDecimal<ResultType>, ColumnVector<ResultType>>;
/// Decimal operations need scale. Operations are on result type.
using OpImpl = std::conditional_t<IsDecimal<ResultDataType>,
using OpImpl = std::conditional_t<IsDataTypeDecimal<ResultDataType>,
DecimalBinaryOperation<T0, T1, Op, ResultType>,
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>>;
@ -1394,7 +1394,7 @@ public:
ResultDataType type = decimalResultType(left, right, is_multiply, is_division);
typename ResultDataType::FieldType scale_a = type.scaleFactorFor(left, is_multiply);
typename ResultDataType::FieldType scale_b = type.scaleFactorFor(right, is_multiply || is_division);
if constexpr (IsDecimal<RightDataType> && is_division)
if constexpr (IsDataTypeDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
auto res = OpImpl::constant_constant(col_left->template getValue<T0>(), col_right->template getValue<T1>(),
@ -1435,7 +1435,7 @@ public:
typename ResultDataType::FieldType scale_a = type.scaleFactorFor(left, is_multiply);
typename ResultDataType::FieldType scale_b = type.scaleFactorFor(right, is_multiply || is_division);
if constexpr (IsDecimal<RightDataType> && is_division)
if constexpr (IsDataTypeDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
OpImpl::constant_vector(col_left_const->template getValue<T0>(), col_right->getData(), vec_res,
@ -1455,7 +1455,7 @@ public:
typename ResultDataType::FieldType scale_a = type.scaleFactorFor(left, is_multiply);
typename ResultDataType::FieldType scale_b = type.scaleFactorFor(right, is_multiply || is_division);
if constexpr (IsDecimal<RightDataType> && is_division)
if constexpr (IsDataTypeDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
{
@ -1501,7 +1501,7 @@ public:
using RightDataType = std::decay_t<decltype(right)>;
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;
return !std::is_same_v<ResultDataType, InvalidType> && !IsDecimal<ResultDataType> && OpSpec::compilable;
return !std::is_same_v<ResultDataType, InvalidType> && !IsDataTypeDecimal<ResultDataType> && OpSpec::compilable;
});
}
@ -1514,7 +1514,7 @@ public:
using RightDataType = std::decay_t<decltype(right)>;
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;
if constexpr (!std::is_same_v<ResultDataType, InvalidType> && !IsDecimal<ResultDataType> && OpSpec::compilable)
if constexpr (!std::is_same_v<ResultDataType, InvalidType> && !IsDataTypeDecimal<ResultDataType> && OpSpec::compilable)
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto type = std::make_shared<ResultDataType>();
@ -1584,7 +1584,7 @@ public:
using DataType = std::decay_t<decltype(type)>;
using T0 = typename DataType::FieldType;
if constexpr (IsDecimal<DataType>)
if constexpr (IsDataTypeDecimal<DataType>)
{
if constexpr (!allow_decimal)
return false;
@ -1607,7 +1607,7 @@ public:
using DataType = std::decay_t<decltype(type)>;
using T0 = typename DataType::FieldType;
if constexpr (IsDecimal<DataType>)
if constexpr (IsDataTypeDecimal<DataType>)
{
if constexpr (allow_decimal)
{
@ -1647,7 +1647,7 @@ public:
return castType(arguments[0].get(), [&](const auto & type)
{
using DataType = std::decay_t<decltype(type)>;
return !IsDecimal<DataType> && Op<typename DataType::FieldType>::compilable;
return !IsDataTypeDecimal<DataType> && Op<typename DataType::FieldType>::compilable;
});
}
@ -1659,7 +1659,7 @@ public:
using DataType = std::decay_t<decltype(type)>;
using T0 = typename DataType::FieldType;
using T1 = typename Op<T0>::ResultType;
if constexpr (!std::is_same_v<T1, InvalidType> && !IsDecimal<DataType> && Op<T0>::compilable)
if constexpr (!std::is_same_v<T1, InvalidType> && !IsDecimalDataType<DataType> && Op<T0>::compilable)
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * v = nativeCast(b, types[0], values[0](), std::make_shared<DataTypeNumber<T1>>());

View File

@ -110,7 +110,7 @@ struct ConvertImpl
if (const ColVecFrom * col_from = checkAndGetColumn<ColVecFrom>(named_from.column.get()))
{
typename ColVecTo::MutablePtr col_to = nullptr;
if constexpr (IsDecimal<ToDataType>)
if constexpr (IsDataTypeDecimal<ToDataType>)
{
UInt32 scale = additions;
col_to = ColVecTo::create(0, scale);
@ -125,11 +125,11 @@ struct ConvertImpl
for (size_t i = 0; i < size; ++i)
{
if constexpr (IsDecimal<FromDataType> && IsDecimal<ToDataType>)
if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>)
vec_to[i] = convertDecimals<FromDataType, ToDataType>(vec_from[i], vec_from.getScale(), vec_to.getScale());
else if constexpr (IsDecimal<FromDataType>)
else if constexpr (IsDataTypeDecimal<FromDataType>)
vec_to[i] = convertFromDecimal<FromDataType, ToDataType>(vec_from[i], vec_from.getScale());
else if constexpr (IsDecimal<ToDataType>)
else if constexpr (IsDataTypeDecimal<ToDataType>)
vec_to[i] = convertToDecimal<FromDataType, ToDataType>(vec_from[i], vec_to.getScale());
else
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
@ -490,7 +490,7 @@ struct ConvertThroughParsing
size_t size = input_rows_count;
typename ColVecTo::MutablePtr col_to = nullptr;
if constexpr (IsDecimal<ToDataType>)
if constexpr (IsDataTypeDecimal<ToDataType>)
{
UInt32 scale = additions;
ToDataType check_bounds_in_ctor(ToDataType::maxPrecision(), scale);
@ -533,7 +533,7 @@ struct ConvertThroughParsing
ReadBufferFromMemory read_buffer(&(*chars)[current_offset], string_size);
if constexpr (IsDecimal<ToDataType>)
if constexpr (IsDataTypeDecimal<ToDataType>)
{
ToDataType::readText(vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale());
}
@ -880,7 +880,7 @@ private:
using LeftDataType = typename Types::LeftType;
using RightDataType = typename Types::RightType;
if constexpr (IsDecimal<RightDataType>)
if constexpr (IsDataTypeDecimal<RightDataType>)
{
if (arguments.size() != 2)
throw Exception{"Function " + getName() + " expects 2 arguments for Decimal.",

View File

@ -425,7 +425,6 @@ struct IntegerRoundingImpl
{
private:
using Op = IntegerRoundingComputation<T, rounding_mode, scale_mode>;
using Data = T;
public:
template <size_t scale>
@ -476,70 +475,103 @@ public:
}
};
template <typename T, RoundingMode rounding_mode, ScaleMode scale_mode>
using FunctionRoundingImpl = std::conditional_t<std::is_floating_point_v<T>,
FloatRoundingImpl<T, rounding_mode, scale_mode>,
IntegerRoundingImpl<T, rounding_mode, scale_mode>>;
template <typename T, RoundingMode rounding_mode>
class DecimalRounding
{
using NativeType= typename T::NativeType;
using Op = IntegerRoundingComputation<NativeType, rounding_mode, ScaleMode::Negative>;
using Container = typename ColumnDecimal<T>::Container;
public:
static NO_INLINE void apply(const Container & in, Container & out, Int64 scale_arg)
{
scale_arg = in.getScale() - scale_arg;
if (scale_arg > 0)
{
size_t scale = pow(10, scale_arg);
const NativeType * __restrict p_in = reinterpret_cast<const NativeType *>(in.data());
const NativeType * end_in = reinterpret_cast<const NativeType *>(in.data()) + in.size();
NativeType * __restrict p_out = reinterpret_cast<NativeType *>(out.data());
while (p_in < end_in)
{
Op::compute(p_in, scale, p_out);
++p_in;
++p_out;
}
}
else
memcpy(out.data(), in.data(), in.size() * sizeof(T));
}
};
/** Select the appropriate processing algorithm depending on the scale.
*/
template <typename T, RoundingMode rounding_mode>
struct Dispatcher
class Dispatcher
{
static void apply(Block & block, const ColumnVector<T> * col, const ColumnNumbers & arguments, size_t result)
template <ScaleMode scale_mode>
using FunctionRoundingImpl = std::conditional_t<std::is_floating_point_v<T>,
FloatRoundingImpl<T, rounding_mode, scale_mode>,
IntegerRoundingImpl<T, rounding_mode, scale_mode>>;
static void apply(Block & block, const ColumnVector<T> * col, Int64 scale_arg, size_t result)
{
size_t scale = 1;
Int64 scale_arg = 0;
if (arguments.size() == 2)
{
const IColumn & scale_column = *block.getByPosition(arguments[1]).column;
if (!scale_column.isColumnConst())
throw Exception("Scale argument for rounding functions must be constant.", ErrorCodes::ILLEGAL_COLUMN);
Field scale_field = static_cast<const ColumnConst &>(scale_column).getField();
if (scale_field.getType() != Field::Types::UInt64
&& scale_field.getType() != Field::Types::Int64)
throw Exception("Scale argument for rounding functions must have integer type.", ErrorCodes::ILLEGAL_COLUMN);
scale_arg = scale_field.get<Int64>();
}
auto col_res = ColumnVector<T>::create();
typename ColumnVector<T>::Container & vec_res = col_res->getData();
vec_res.resize(col->getData().size());
if (vec_res.empty())
if (!vec_res.empty())
{
block.getByPosition(result).column = std::move(col_res);
return;
}
if (scale_arg == 0)
{
scale = 1;
FunctionRoundingImpl<T, rounding_mode, ScaleMode::Zero>::apply(col->getData(), scale, vec_res);
size_t scale = 1;
FunctionRoundingImpl<ScaleMode::Zero>::apply(col->getData(), scale, vec_res);
}
else if (scale_arg > 0)
{
scale = pow(10, scale_arg);
FunctionRoundingImpl<T, rounding_mode, ScaleMode::Positive>::apply(col->getData(), scale, vec_res);
size_t scale = pow(10, scale_arg);
FunctionRoundingImpl<ScaleMode::Positive>::apply(col->getData(), scale, vec_res);
}
else
{
scale = pow(10, -scale_arg);
FunctionRoundingImpl<T, rounding_mode, ScaleMode::Negative>::apply(col->getData(), scale, vec_res);
size_t scale = pow(10, -scale_arg);
FunctionRoundingImpl<ScaleMode::Negative>::apply(col->getData(), scale, vec_res);
}
}
block.getByPosition(result).column = std::move(col_res);
}
static void apply(Block & block, const ColumnDecimal<T> * col, Int64 scale_arg, size_t result)
{
const typename ColumnDecimal<T>::Container & vec_src = col->getData();
auto col_res = ColumnDecimal<T>::create(vec_src.size(), vec_src.getScale());
auto & vec_res = col_res->getData();
if (!vec_res.empty())
DecimalRounding<T, rounding_mode>::apply(col->getData(), vec_res, scale_arg);
block.getByPosition(result).column = std::move(col_res);
}
public:
static void apply(Block & block, const IColumn * column, Int64 scale_arg, size_t result)
{
if constexpr (IsNumber<T>)
apply(block, checkAndGetColumn<ColumnVector<T>>(column), scale_arg, result);
else if constexpr (IsDecimalNumber<T>)
apply(block, checkAndGetColumn<ColumnDecimal<T>>(column), scale_arg, result);
}
};
/** A template for functions that round the value of an input parameter of type
* (U)Int8/16/32/64 or Float32/64, and accept an additional optional
* parameter (default is 0).
* (U)Int8/16/32/64, Float32/64 or Decimal32/64/128, and accept an additional optional parameter (default is 0).
*/
template <typename Name, RoundingMode rounding_mode>
class FunctionRounding : public IFunction
@ -548,18 +580,6 @@ public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRounding>(); }
private:
template <typename T>
bool executeForType(Block & block, const ColumnNumbers & arguments, size_t result)
{
if (auto col = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
{
Dispatcher<T, rounding_mode>::apply(block, col, arguments, result);
return true;
}
return false;
}
public:
String getName() const override
{
@ -578,31 +598,56 @@ public:
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (const auto & type : arguments)
if (!isNumber(type))
if (!isNumber(type) && !isDecimal(type))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return arguments[0];
}
static Int64 getScaleArg(Block & block, const ColumnNumbers & arguments)
{
if (arguments.size() == 2)
{
const IColumn & scale_column = *block.getByPosition(arguments[1]).column;
if (!scale_column.isColumnConst())
throw Exception("Scale argument for rounding functions must be constant.", ErrorCodes::ILLEGAL_COLUMN);
Field scale_field = static_cast<const ColumnConst &>(scale_column).getField();
if (scale_field.getType() != Field::Types::UInt64
&& scale_field.getType() != Field::Types::Int64)
throw Exception("Scale argument for rounding functions must have integer type.", ErrorCodes::ILLEGAL_COLUMN);
return scale_field.get<Int64>();
}
return 0;
}
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (!( executeForType<UInt8>(block, arguments, result)
|| executeForType<UInt16>(block, arguments, result)
|| executeForType<UInt32>(block, arguments, result)
|| executeForType<UInt64>(block, arguments, result)
|| executeForType<Int8>(block, arguments, result)
|| executeForType<Int16>(block, arguments, result)
|| executeForType<Int32>(block, arguments, result)
|| executeForType<Int64>(block, arguments, result)
|| executeForType<Float32>(block, arguments, result)
|| executeForType<Float64>(block, arguments, result)))
const ColumnWithTypeAndName & column = block.getByPosition(arguments[0]);
Int64 scale_arg = getScaleArg(block, arguments);
auto call = [&](const auto & types) -> bool
{
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
using Types = std::decay_t<decltype(types)>;
using DataType = typename Types::LeftType;
if constexpr (IsDataTypeNumber<DataType> || IsDataTypeDecimal<DataType>)
{
using FieldType = typename DataType::FieldType;
Dispatcher<FieldType, rounding_mode>::apply(block, column.column.get(), scale_arg, result);
return true;
}
return false;
};
if (!callOnIndexAndDataType<void>(column.type->getTypeId(), call))
{
throw Exception("Illegal column " + column.name + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
}

View File

@ -0,0 +1,60 @@
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12345.0000 12345.6000 12345.6700 12345.6780 12345.6789 12345.6789
12345.6789 12345.0000 12345.6000 12345.6700 12345.6780 12345.6789 12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12345.0000 -12345.6000 -12345.6700 -12345.6780 -12345.6789 -12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12345.0000 -12345.6000 -12345.6700 -12345.6780 -12345.6789 -12345.6789
12345.6789 12350.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12350.0000 12400.0000 13000.0000 20000.0000 100000.0000
12345.6789 12340.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12340.0000 12300.0000 12000.0000 10000.0000 0.0000
-12345.6789 -12350.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12340.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12350.0000 -12400.0000 -13000.0000 -20000.0000 -100000.0000
-12345.6789 -12340.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12345.0000 12345.6000 12345.6700 12345.6780 12345.6789 12345.6789
12345.6789 12345.0000 12345.6000 12345.6700 12345.6780 12345.6789 12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12345.0000 -12345.6000 -12345.6700 -12345.6780 -12345.6789 -12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12345.0000 -12345.6000 -12345.6700 -12345.6780 -12345.6789 -12345.6789
12345.6789 12350.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12350.0000 12400.0000 13000.0000 20000.0000 100000.0000
12345.6789 12340.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12340.0000 12300.0000 12000.0000 10000.0000 0.0000
-12345.6789 -12350.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12340.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12350.0000 -12400.0000 -13000.0000 -20000.0000 -100000.0000
-12345.6789 -12340.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12345.0000 12345.6000 12345.6700 12345.6780 12345.6789 12345.6789
12345.6789 12345.0000 12345.6000 12345.6700 12345.6780 12345.6789 12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12345.0000 -12345.6000 -12345.6700 -12345.6780 -12345.6789 -12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12345.0000 -12345.6000 -12345.6700 -12345.6780 -12345.6789 -12345.6789
12345.6789 12350.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12350.0000 12400.0000 13000.0000 20000.0000 100000.0000
12345.6789 12340.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12340.0000 12300.0000 12000.0000 10000.0000 0.0000
-12345.6789 -12350.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12340.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12350.0000 -12400.0000 -13000.0000 -20000.0000 -100000.0000
-12345.6789 -12340.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
123456789.123456784 -123456789.123456784 123456789.000000000 -123456789.000000000 123456789.123460000 -123456789.123460000 123500000.000000000 -123500000.000000000
123456789.123456784 -123456789.123456784 123456790.000000000 -123456789.000000000 123456789.123460000 -123456789.123450000 123500000.000000000 -123400000.000000000
123456789.123456784 -123456789.123456784 123456789.000000000 -123456790.000000000 123456789.123450000 -123456789.123460000 123400000.000000000 -123500000.000000000
123456789.123456784 -123456789.123456784 123456789.000000000 -123456789.000000000 123456789.123450000 -123456789.123450000 123400000.000000000 -123400000.000000000
12345678901234567525491324606.797053952 -12345678901234567525491324606.797053952 12345678901234567525491324607.000000000 -12345678901234567525491324607.000000000 12345678901234567525491324606.797000000 -12345678901234567525491324606.797000000
12345678901234567525491324606.797053952 -12345678901234567525491324606.797053952 12345678901234567525491324607.000000000 -12345678901234567525491324606.000000000 12345678901234567525491324606.798000000 -12345678901234567525491324606.797000000
12345678901234567525491324606.797053952 -12345678901234567525491324606.797053952 12345678901234567525491324606.000000000 -12345678901234567525491324607.000000000 12345678901234567525491324606.797000000 -12345678901234567525491324606.798000000
12345678901234567525491324606.797053952 -12345678901234567525491324606.797053952 12345678901234567525491324606.000000000 -12345678901234567525491324606.000000000 12345678901234567525491324606.797000000 -12345678901234567525491324606.797000000
12345678901234567525491324606.797053952 -12345678901234567525491324606.797053952 12345678901234567525491324607.000000000 -12345678901234567525491324607.000000000 12345678901234567525491325000.000000000 -12345678901234567525491325000.000000000
12345678901234567525491324606.797053952 -12345678901234567525491324606.797053952 12345678901234567525491324607.000000000 -12345678901234567525491324606.000000000 12345678901234567525491325000.000000000 -12345678901234567525491324000.000000000
12345678901234567525491324606.797053952 -12345678901234567525491324606.797053952 12345678901234567525491324606.000000000 -12345678901234567525491324607.000000000 12345678901234567525491324000.000000000 -12345678901234567525491325000.000000000
12345678901234567525491324606.797053952 -12345678901234567525491324606.797053952 12345678901234567525491324606.000000000 -12345678901234567525491324606.000000000 12345678901234567525491324000.000000000 -12345678901234567525491324000.000000000

View File

@ -0,0 +1,65 @@
SELECT toDecimal32(12345.6789, 4) AS x, round(x), round(x, 1), round(x, 2), round(x, 3), round(x, 4), round(x, 5);
SELECT toDecimal32(12345.6789, 4) AS x, ceil(x), ceil(x, 1), ceil(x, 2), ceil(x, 3), ceil(x, 4), ceil(x, 5);
SELECT toDecimal32(12345.6789, 4) AS x, floor(x), floor(x, 1), floor(x, 2), floor(x, 3), floor(x, 4), floor(x, 5);
SELECT toDecimal32(12345.6789, 4) AS x, trunc(x), trunc(x, 1), trunc(x, 2), trunc(x, 3), trunc(x, 4), trunc(x, 5);
SELECT toDecimal32(-12345.6789, 4) AS x, round(x), round(x, 1), round(x, 2), round(x, 3), round(x, 4), round(x, 5);
SELECT toDecimal32(-12345.6789, 4) AS x, ceil(x), ceil(x, 1), ceil(x, 2), ceil(x, 3), ceil(x, 4), ceil(x, 5);
SELECT toDecimal32(-12345.6789, 4) AS x, floor(x), floor(x, 1), floor(x, 2), floor(x, 3), floor(x, 4), floor(x, 5);
SELECT toDecimal32(-12345.6789, 4) AS x, trunc(x), trunc(x, 1), trunc(x, 2), trunc(x, 3), trunc(x, 4), trunc(x, 5);
SELECT toDecimal32(12345.6789, 4) AS x, round(x, -1), round(x, -2), round(x, -3), round(x, -4), round(x, -5);
SELECT toDecimal32(12345.6789, 4) AS x, ceil(x, -1), ceil(x, -2), ceil(x, -3), ceil(x, -4), ceil(x, -5);
SELECT toDecimal32(12345.6789, 4) AS x, floor(x, -1), floor(x, -2), floor(x, -3), floor(x, -4), floor(x, -5);
SELECT toDecimal32(12345.6789, 4) AS x, trunc(x, -1), trunc(x, -2), trunc(x, -3), trunc(x, -4), trunc(x, -5);
SELECT toDecimal32(-12345.6789, 4) AS x, round(x, -1), round(x, -2), round(x, -3), round(x, -4), round(x, -5);
SELECT toDecimal32(-12345.6789, 4) AS x, ceil(x, -1), ceil(x, -2), ceil(x, -3), ceil(x, -4), ceil(x, -5);
SELECT toDecimal32(-12345.6789, 4) AS x, floor(x, -1), floor(x, -2), floor(x, -3), floor(x, -4), floor(x, -5);
SELECT toDecimal32(-12345.6789, 4) AS x, trunc(x, -1), trunc(x, -2), trunc(x, -3), trunc(x, -4), trunc(x, -5);
SELECT toDecimal64(12345.6789, 4) AS x, round(x), round(x, 1), round(x, 2), round(x, 3), round(x, 4), round(x, 5);
SELECT toDecimal64(12345.6789, 4) AS x, ceil(x), ceil(x, 1), ceil(x, 2), ceil(x, 3), ceil(x, 4), ceil(x, 5);
SELECT toDecimal64(12345.6789, 4) AS x, floor(x), floor(x, 1), floor(x, 2), floor(x, 3), floor(x, 4), floor(x, 5);
SELECT toDecimal64(12345.6789, 4) AS x, trunc(x), trunc(x, 1), trunc(x, 2), trunc(x, 3), trunc(x, 4), trunc(x, 5);
SELECT toDecimal64(-12345.6789, 4) AS x, round(x), round(x, 1), round(x, 2), round(x, 3), round(x, 4), round(x, 5);
SELECT toDecimal64(-12345.6789, 4) AS x, ceil(x), ceil(x, 1), ceil(x, 2), ceil(x, 3), ceil(x, 4), ceil(x, 5);
SELECT toDecimal64(-12345.6789, 4) AS x, floor(x), floor(x, 1), floor(x, 2), floor(x, 3), floor(x, 4), floor(x, 5);
SELECT toDecimal64(-12345.6789, 4) AS x, trunc(x), trunc(x, 1), trunc(x, 2), trunc(x, 3), trunc(x, 4), trunc(x, 5);
SELECT toDecimal64(12345.6789, 4) AS x, round(x, -1), round(x, -2), round(x, -3), round(x, -4), round(x, -5);
SELECT toDecimal64(12345.6789, 4) AS x, ceil(x, -1), ceil(x, -2), ceil(x, -3), ceil(x, -4), ceil(x, -5);
SELECT toDecimal64(12345.6789, 4) AS x, floor(x, -1), floor(x, -2), floor(x, -3), floor(x, -4), floor(x, -5);
SELECT toDecimal64(12345.6789, 4) AS x, trunc(x, -1), trunc(x, -2), trunc(x, -3), trunc(x, -4), trunc(x, -5);
SELECT toDecimal64(-12345.6789, 4) AS x, round(x, -1), round(x, -2), round(x, -3), round(x, -4), round(x, -5);
SELECT toDecimal64(-12345.6789, 4) AS x, ceil(x, -1), ceil(x, -2), ceil(x, -3), ceil(x, -4), ceil(x, -5);
SELECT toDecimal64(-12345.6789, 4) AS x, floor(x, -1), floor(x, -2), floor(x, -3), floor(x, -4), floor(x, -5);
SELECT toDecimal64(-12345.6789, 4) AS x, trunc(x, -1), trunc(x, -2), trunc(x, -3), trunc(x, -4), trunc(x, -5);
SELECT toDecimal128(12345.6789, 4) AS x, round(x), round(x, 1), round(x, 2), round(x, 3), round(x, 4), round(x, 5);
SELECT toDecimal128(12345.6789, 4) AS x, ceil(x), ceil(x, 1), ceil(x, 2), ceil(x, 3), ceil(x, 4), ceil(x, 5);
SELECT toDecimal128(12345.6789, 4) AS x, floor(x), floor(x, 1), floor(x, 2), floor(x, 3), floor(x, 4), floor(x, 5);
SELECT toDecimal128(12345.6789, 4) AS x, trunc(x), trunc(x, 1), trunc(x, 2), trunc(x, 3), trunc(x, 4), trunc(x, 5);
SELECT toDecimal128(-12345.6789, 4) AS x, round(x), round(x, 1), round(x, 2), round(x, 3), round(x, 4), round(x, 5);
SELECT toDecimal128(-12345.6789, 4) AS x, ceil(x), ceil(x, 1), ceil(x, 2), ceil(x, 3), ceil(x, 4), ceil(x, 5);
SELECT toDecimal128(-12345.6789, 4) AS x, floor(x), floor(x, 1), floor(x, 2), floor(x, 3), floor(x, 4), floor(x, 5);
SELECT toDecimal128(-12345.6789, 4) AS x, trunc(x), trunc(x, 1), trunc(x, 2), trunc(x, 3), trunc(x, 4), trunc(x, 5);
SELECT toDecimal128(12345.6789, 4) AS x, round(x, -1), round(x, -2), round(x, -3), round(x, -4), round(x, -5);
SELECT toDecimal128(12345.6789, 4) AS x, ceil(x, -1), ceil(x, -2), ceil(x, -3), ceil(x, -4), ceil(x, -5);
SELECT toDecimal128(12345.6789, 4) AS x, floor(x, -1), floor(x, -2), floor(x, -3), floor(x, -4), floor(x, -5);
SELECT toDecimal128(12345.6789, 4) AS x, trunc(x, -1), trunc(x, -2), trunc(x, -3), trunc(x, -4), trunc(x, -5);
SELECT toDecimal128(-12345.6789, 4) AS x, round(x, -1), round(x, -2), round(x, -3), round(x, -4), round(x, -5);
SELECT toDecimal128(-12345.6789, 4) AS x, ceil(x, -1), ceil(x, -2), ceil(x, -3), ceil(x, -4), ceil(x, -5);
SELECT toDecimal128(-12345.6789, 4) AS x, floor(x, -1), floor(x, -2), floor(x, -3), floor(x, -4), floor(x, -5);
SELECT toDecimal128(-12345.6789, 4) AS x, trunc(x, -1), trunc(x, -2), trunc(x, -3), trunc(x, -4), trunc(x, -5);
SELECT toDecimal64(123456789.123456789, 9) AS x, -x AS y, round(x), round(y), round(x, 5), round(y, 5), round(x, -5), round(y, -5);
SELECT toDecimal64(123456789.123456789, 9) AS x, -x AS y, ceil(x), ceil(y), ceil(x, 5), ceil(y, 5), ceil(x, -5), ceil(y, -5);
SELECT toDecimal64(123456789.123456789, 9) AS x, -x AS y, floor(x), floor(y), floor(x, 5), floor(y, 5), floor(x, -5), floor(y, -5);
SELECT toDecimal64(123456789.123456789, 9) AS x, -x AS y, trunc(x), trunc(y), trunc(x, 5), trunc(y, 5), trunc(x, -5), trunc(y, -5);
SELECT toDecimal128(12345678901234567890123456789.123456789, 9) AS x, -x AS y, round(x), round(y), round(x, 3), round(y, 3);
SELECT toDecimal128(12345678901234567890123456789.123456789, 9) AS x, -x AS y, ceil(x), ceil(y), ceil(x, 3), ceil(y, 3);
SELECT toDecimal128(12345678901234567890123456789.123456789, 9) AS x, -x AS y, floor(x), floor(y), floor(x, 3), floor(y, 3);
SELECT toDecimal128(12345678901234567890123456789.123456789, 9) AS x, -x AS y, trunc(x), trunc(y), trunc(x, 3), trunc(y, 3);
SELECT toDecimal128(12345678901234567890123456789.123456789, 9) AS x, -x AS y, round(x), round(y), round(x, -3), round(y, -3);
SELECT toDecimal128(12345678901234567890123456789.123456789, 9) AS x, -x AS y, ceil(x), ceil(y), ceil(x, -3), ceil(y, -3);
SELECT toDecimal128(12345678901234567890123456789.123456789, 9) AS x, -x AS y, floor(x), floor(y), floor(x, -3), floor(y, -3);
SELECT toDecimal128(12345678901234567890123456789.123456789, 9) AS x, -x AS y, trunc(x), trunc(y), trunc(x, -3), trunc(y, -3);