mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Support non-const scale arguments in rounding functions
This commit is contained in:
parent
6af3b9ac12
commit
830acd3a89
@ -9,8 +9,8 @@ sidebar_label: Rounding
|
||||
## floor(x\[, N\])
|
||||
|
||||
Returns the largest round number that is less than or equal to `x`. A round number is a multiple of 1/10N, or the nearest number of the appropriate data type if 1 / 10N isn’t exact.
|
||||
‘N’ is an integer constant, optional parameter. By default it is zero, which means to round to an integer.
|
||||
‘N’ may be negative.
|
||||
`N` is an integer, optional parameter. By default it is zero, which means to round to an integer.
|
||||
`N` may be negative.
|
||||
|
||||
Examples: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.`
|
||||
|
||||
@ -70,7 +70,7 @@ round(expression [, decimal_places])
|
||||
**Arguments**
|
||||
|
||||
- `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../data-types/index.md#data_types).
|
||||
- `decimal-places` — An integer value.
|
||||
- `decimal-places` — The number of decimal places to round to.
|
||||
- If `decimal-places > 0` then the function rounds the value to the right of the decimal point.
|
||||
- If `decimal-places < 0` then the function rounds the value to the left of the decimal point.
|
||||
- If `decimal-places = 0` then the function rounds the value to integer. In this case the argument can be omitted.
|
||||
@ -264,7 +264,7 @@ Result:
|
||||
|
||||
## roundDuration
|
||||
|
||||
Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set of commonly used durations: `1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000`.
|
||||
Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set of commonly used durations: `1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000`.
|
||||
|
||||
**Syntax**
|
||||
|
||||
|
@ -10,7 +10,7 @@ sidebar_label: "Функции округления"
|
||||
|
||||
Возвращает наибольшее круглое число, которое меньше или равно, чем x.
|
||||
Круглым называется число, кратное 1 / 10N или ближайшее к нему число соответствующего типа данных, если 1 / 10N не представимо точно.
|
||||
N - целочисленная константа, не обязательный параметр. По умолчанию - ноль, что означает - округлять до целого числа.
|
||||
N - целочисленный аргумент, не обязательный параметр. По умолчанию - ноль, что означает - округлять до целого числа.
|
||||
N может быть отрицательным.
|
||||
|
||||
Примеры: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.`
|
||||
|
@ -187,13 +187,13 @@ struct IntegerRoundingComputation
|
||||
};
|
||||
|
||||
|
||||
#ifdef __SSE4_1__
|
||||
|
||||
template <typename T>
|
||||
template <typename T, bool is_vectorized = true>
|
||||
class BaseFloatRoundingComputation;
|
||||
|
||||
#ifdef __SSE4_1__
|
||||
|
||||
template <>
|
||||
class BaseFloatRoundingComputation<Float32>
|
||||
class BaseFloatRoundingComputation<Float32, true>
|
||||
{
|
||||
public:
|
||||
using ScalarType = Float32;
|
||||
@ -214,7 +214,7 @@ public:
|
||||
};
|
||||
|
||||
template <>
|
||||
class BaseFloatRoundingComputation<Float64>
|
||||
class BaseFloatRoundingComputation<Float64, true>
|
||||
{
|
||||
public:
|
||||
using ScalarType = Float64;
|
||||
@ -234,9 +234,9 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
#else
|
||||
#endif
|
||||
|
||||
/// Implementation for ARM. Not vectorized.
|
||||
/// Implementation for ARM/columnar scale argument. Not vectorized.
|
||||
|
||||
inline float roundWithMode(float x, RoundingMode mode)
|
||||
{
|
||||
@ -265,7 +265,7 @@ inline double roundWithMode(double x, RoundingMode mode)
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
class BaseFloatRoundingComputation
|
||||
class BaseFloatRoundingComputation<T, false>
|
||||
{
|
||||
public:
|
||||
using ScalarType = T;
|
||||
@ -285,15 +285,13 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
#endif
|
||||
|
||||
|
||||
/** Implementation of low-level round-off functions for floating-point values.
|
||||
*/
|
||||
template <typename T, RoundingMode rounding_mode, ScaleMode scale_mode>
|
||||
class FloatRoundingComputation : public BaseFloatRoundingComputation<T>
|
||||
template <typename T, RoundingMode rounding_mode, ScaleMode scale_mode, bool is_vectorized>
|
||||
class FloatRoundingComputation : public BaseFloatRoundingComputation<T, is_vectorized>
|
||||
{
|
||||
using Base = BaseFloatRoundingComputation<T>;
|
||||
using Base = BaseFloatRoundingComputation<T, is_vectorized>;
|
||||
|
||||
public:
|
||||
static void compute(const T * __restrict in, const typename Base::VectorType & scale, T * __restrict out)
|
||||
@ -325,15 +323,22 @@ struct FloatRoundingImpl
|
||||
private:
|
||||
static_assert(!is_decimal<T>);
|
||||
|
||||
using Op = FloatRoundingComputation<T, rounding_mode, scale_mode>;
|
||||
using Data = std::array<T, Op::data_count>;
|
||||
template <bool is_vectorized =
|
||||
#ifdef __SSE4_1__
|
||||
true
|
||||
#else
|
||||
false
|
||||
#endif
|
||||
>
|
||||
using Op = FloatRoundingComputation<T, rounding_mode, scale_mode, is_vectorized>;
|
||||
using Data = std::array<T, Op<>::data_count>;
|
||||
using ColumnType = ColumnVector<T>;
|
||||
using Container = typename ColumnType::Container;
|
||||
|
||||
public:
|
||||
static NO_INLINE void apply(const Container & in, size_t scale, Container & out)
|
||||
{
|
||||
auto mm_scale = Op::prepare(scale);
|
||||
auto mm_scale = Op<>::prepare(scale);
|
||||
|
||||
const size_t data_count = std::tuple_size<Data>();
|
||||
|
||||
@ -345,7 +350,7 @@ public:
|
||||
|
||||
while (p_in < limit)
|
||||
{
|
||||
Op::compute(p_in, mm_scale, p_out);
|
||||
Op<>::compute(p_in, mm_scale, p_out);
|
||||
p_in += data_count;
|
||||
p_out += data_count;
|
||||
}
|
||||
@ -358,10 +363,17 @@ public:
|
||||
size_t tail_size_bytes = (end_in - p_in) * sizeof(*p_in);
|
||||
|
||||
memcpy(&tmp_src, p_in, tail_size_bytes);
|
||||
Op::compute(reinterpret_cast<T *>(&tmp_src), mm_scale, reinterpret_cast<T *>(&tmp_dst));
|
||||
Op<>::compute(reinterpret_cast<T *>(&tmp_src), mm_scale, reinterpret_cast<T *>(&tmp_dst));
|
||||
memcpy(p_out, &tmp_dst, tail_size_bytes);
|
||||
}
|
||||
}
|
||||
|
||||
static void applyOne(const T* __restrict in, size_t scale, T* __restrict out)
|
||||
{
|
||||
using ScalarOp = Op<false>;
|
||||
auto s = ScalarOp::prepare(scale);
|
||||
ScalarOp::compute(in, s, out);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename T, RoundingMode rounding_mode, ScaleMode scale_mode, TieBreakingMode tie_breaking_mode>
|
||||
@ -417,6 +429,11 @@ public:
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected 'scale' parameter passed to function");
|
||||
}
|
||||
}
|
||||
|
||||
static void applyOne(const T* __restrict in, size_t scale, T* __restrict out)
|
||||
{
|
||||
Op::compute(in, scale, out);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -452,11 +469,40 @@ public:
|
||||
memcpy(out.data(), in.data(), in.size() * sizeof(T));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
static void applyOne(const NativeType* __restrict in, UInt32 in_scale, NativeType* __restrict out, Scale scale_arg)
|
||||
{
|
||||
scale_arg = in_scale - scale_arg;
|
||||
if (scale_arg > 0)
|
||||
{
|
||||
auto scale = intExp10OfSize<NativeType>(scale_arg);
|
||||
Op::compute(in, scale, out);
|
||||
}
|
||||
else
|
||||
{
|
||||
memcpy(out, in, sizeof(T));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
/** Select the appropriate processing algorithm depending on the scale.
|
||||
*/
|
||||
inline void validateScale(Int64 scale64)
|
||||
{
|
||||
if (scale64 > std::numeric_limits<Scale>::max() || scale64 < std::numeric_limits<Scale>::min())
|
||||
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale argument for rounding function is too large");
|
||||
}
|
||||
|
||||
inline Scale getScaleArg(const ColumnConst* scale_col)
|
||||
{
|
||||
const auto& scale_field = scale_col->getField();
|
||||
|
||||
Int64 scale64 = scale_field.get<Int64>();
|
||||
validateScale(scale64);
|
||||
|
||||
return scale64;
|
||||
}
|
||||
|
||||
template <typename T, RoundingMode rounding_mode, TieBreakingMode tie_breaking_mode>
|
||||
struct Dispatcher
|
||||
{
|
||||
@ -465,9 +511,10 @@ struct Dispatcher
|
||||
FloatRoundingImpl<T, rounding_mode, scale_mode>,
|
||||
IntegerRoundingImpl<T, rounding_mode, scale_mode, tie_breaking_mode>>;
|
||||
|
||||
static ColumnPtr apply(const IColumn * col_general, Scale scale_arg)
|
||||
template <typename ScaleType>
|
||||
static ColumnPtr apply(const IColumn * data_col, const IColumn * scale_col = nullptr)
|
||||
{
|
||||
const auto & col = checkAndGetColumn<ColumnVector<T>>(*col_general);
|
||||
const auto & col = checkAndGetColumn<ColumnVector<T>>(*data_col);
|
||||
auto col_res = ColumnVector<T>::create();
|
||||
|
||||
typename ColumnVector<T>::Container & vec_res = col_res->getData();
|
||||
@ -475,20 +522,62 @@ struct Dispatcher
|
||||
|
||||
if (!vec_res.empty())
|
||||
{
|
||||
if (scale_arg == 0)
|
||||
using ColVecScale = ColumnVector<ScaleType>;
|
||||
if (scale_col == nullptr || isColumnConst(*scale_col))
|
||||
{
|
||||
size_t scale = 1;
|
||||
FunctionRoundingImpl<ScaleMode::Zero>::apply(col.getData(), scale, vec_res);
|
||||
auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst<ColVecScale>(scale_col));
|
||||
if (scale_arg == 0)
|
||||
{
|
||||
size_t scale = 1;
|
||||
FunctionRoundingImpl<ScaleMode::Zero>::apply(col.getData(), scale, vec_res);
|
||||
}
|
||||
else if (scale_arg > 0)
|
||||
{
|
||||
size_t scale = intExp10(scale_arg);
|
||||
FunctionRoundingImpl<ScaleMode::Positive>::apply(col.getData(), scale, vec_res);
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t scale = intExp10(-scale_arg);
|
||||
FunctionRoundingImpl<ScaleMode::Negative>::apply(col.getData(), scale, vec_res);
|
||||
}
|
||||
}
|
||||
else if (scale_arg > 0)
|
||||
else if (const auto scale_typed = checkAndGetColumn<ColVecScale>(scale_col))
|
||||
{
|
||||
size_t scale = intExp10(scale_arg);
|
||||
FunctionRoundingImpl<ScaleMode::Positive>::apply(col.getData(), scale, vec_res);
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t scale = intExp10(-scale_arg);
|
||||
FunctionRoundingImpl<ScaleMode::Negative>::apply(col.getData(), scale, vec_res);
|
||||
const auto & in = col.getData();
|
||||
const auto & scale_data = scale_typed->getData();
|
||||
const size_t count = in.size();
|
||||
|
||||
const T * end_in = in.data() + count;
|
||||
const T * __restrict p_in = in.data();
|
||||
const ScaleType * __restrict p_scale = scale_data.data();
|
||||
T * __restrict p_out = vec_res.data();
|
||||
|
||||
while (p_in < end_in)
|
||||
{
|
||||
Int64 scale64 = *p_scale;
|
||||
validateScale(scale64);
|
||||
Scale raw_scale = scale64;
|
||||
|
||||
if (raw_scale == 0)
|
||||
{
|
||||
size_t scale = 1;
|
||||
FunctionRoundingImpl<ScaleMode::Zero>::applyOne(p_in, scale, p_out);
|
||||
}
|
||||
else if (raw_scale > 0)
|
||||
{
|
||||
size_t scale = intExp10(raw_scale);
|
||||
FunctionRoundingImpl<ScaleMode::Positive>::applyOne(p_in, scale, p_out);
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t scale = intExp10(-raw_scale);
|
||||
FunctionRoundingImpl<ScaleMode::Negative>::applyOne(p_in, scale, p_out);
|
||||
}
|
||||
++p_in;
|
||||
++p_scale;
|
||||
++p_out;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -500,16 +589,46 @@ template <is_decimal T, RoundingMode rounding_mode, TieBreakingMode tie_breaking
|
||||
struct Dispatcher<T, rounding_mode, tie_breaking_mode>
|
||||
{
|
||||
public:
|
||||
static ColumnPtr apply(const IColumn * col_general, Scale scale_arg)
|
||||
template <typename ScaleType>
|
||||
static ColumnPtr apply(const IColumn * data_col, const IColumn* scale_col = nullptr)
|
||||
{
|
||||
const auto & col = checkAndGetColumn<ColumnDecimal<T>>(*col_general);
|
||||
const auto & col = checkAndGetColumn<ColumnDecimal<T>>(*data_col);
|
||||
const typename ColumnDecimal<T>::Container & vec_src = col.getData();
|
||||
|
||||
auto col_res = ColumnDecimal<T>::create(vec_src.size(), col.getScale());
|
||||
auto & vec_res = col_res->getData();
|
||||
|
||||
if (!vec_res.empty())
|
||||
DecimalRoundingImpl<T, rounding_mode, tie_breaking_mode>::apply(col.getData(), col.getScale(), vec_res, scale_arg);
|
||||
{
|
||||
using ColVecScale = ColumnVector<ScaleType>;
|
||||
if (scale_col == nullptr || isColumnConst(*scale_col))
|
||||
{
|
||||
auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst<ColVecScale>(scale_col));
|
||||
DecimalRoundingImpl<T, rounding_mode, tie_breaking_mode>::apply(col.getData(), col.getScale(), vec_res, scale_arg);
|
||||
}
|
||||
else if (const auto scale_typed = checkAndGetColumn<ColVecScale>(scale_col))
|
||||
{
|
||||
const auto & scale = scale_typed->getData();
|
||||
const size_t count = vec_src.size();
|
||||
|
||||
using NativeType = typename T::NativeType;
|
||||
const NativeType * __restrict p_in = reinterpret_cast<const NativeType *>(vec_src.data());
|
||||
const ScaleType * __restrict p_scale = scale.data();
|
||||
const NativeType * end_in = p_in + count;
|
||||
NativeType * __restrict p_out = reinterpret_cast<NativeType *>(vec_res.data());
|
||||
while (p_in < end_in)
|
||||
{
|
||||
Int64 scale64 = *p_scale;
|
||||
validateScale(scale64);
|
||||
Scale raw_scale = scale64;
|
||||
|
||||
DecimalRoundingImpl<T, rounding_mode, tie_breaking_mode>::applyOne(p_in, col.getScale(), p_out, raw_scale);
|
||||
++p_in;
|
||||
++p_scale;
|
||||
++p_out;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return col_res;
|
||||
}
|
||||
@ -550,50 +669,41 @@ public:
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
static Scale getScaleArg(const ColumnsWithTypeAndName & arguments)
|
||||
{
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const IColumn & scale_column = *arguments[1].column;
|
||||
if (!isColumnConst(scale_column))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must be constant");
|
||||
|
||||
Field scale_field = assert_cast<const ColumnConst &>(scale_column).getField();
|
||||
if (scale_field.getType() != Field::Types::UInt64
|
||||
&& scale_field.getType() != Field::Types::Int64)
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must have integer type");
|
||||
|
||||
Int64 scale64 = scale_field.get<Int64>();
|
||||
if (scale64 > std::numeric_limits<Scale>::max()
|
||||
|| scale64 < std::numeric_limits<Scale>::min())
|
||||
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale argument for rounding function is too large");
|
||||
|
||||
return scale64;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & column = arguments[0];
|
||||
Scale scale_arg = getScaleArg(arguments);
|
||||
const ColumnWithTypeAndName & data_column = arguments[0];
|
||||
|
||||
ColumnPtr res;
|
||||
auto call = [&](const auto & types) -> bool
|
||||
auto callData = [&](const auto & types) -> bool
|
||||
{
|
||||
using Types = std::decay_t<decltype(types)>;
|
||||
using DataType = typename Types::LeftType;
|
||||
using DataType = typename Types::RightType;
|
||||
|
||||
if constexpr (IsDataTypeNumber<DataType> || IsDataTypeDecimal<DataType>)
|
||||
if (arguments.size() > 1)
|
||||
{
|
||||
using FieldType = typename DataType::FieldType;
|
||||
res = Dispatcher<FieldType, rounding_mode, tie_breaking_mode>::apply(column.column.get(), scale_arg);
|
||||
const ColumnWithTypeAndName & scale_column = arguments[1];
|
||||
|
||||
auto callScale = [&](const auto & scaleTypes) -> bool
|
||||
{
|
||||
using ScaleTypes = std::decay_t<decltype(scaleTypes)>;
|
||||
using ScaleType = typename ScaleTypes::RightType;
|
||||
|
||||
if (isColumnConst(*data_column.column) && !isColumnConst(*scale_column.column))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale column must be const for const data column");
|
||||
|
||||
res = Dispatcher<DataType, rounding_mode, tie_breaking_mode>::template apply<ScaleType>(data_column.column.get(), scale_column.column.get());
|
||||
return true;
|
||||
};
|
||||
|
||||
TypeIndex right_index = scale_column.type->getTypeId();
|
||||
if (!callOnBasicType<void, true, false, false, false>(right_index, callScale))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must have integer type");
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
res = Dispatcher<DataType, rounding_mode, tie_breaking_mode>::template apply<int>(data_column.column.get());
|
||||
return true;
|
||||
};
|
||||
|
||||
#if !defined(__SSE4_1__)
|
||||
@ -605,10 +715,9 @@ public:
|
||||
throw Exception(ErrorCodes::CANNOT_SET_ROUNDING_MODE, "Cannot set floating point rounding mode");
|
||||
#endif
|
||||
|
||||
if (!callOnIndexAndDataType<void>(column.type->getTypeId(), call))
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", column.name, getName());
|
||||
}
|
||||
TypeIndex left_index = data_column.type->getTypeId();
|
||||
if (!callOnBasicType<void, true, true, true, false>(left_index, callData))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", data_column.name, getName());
|
||||
|
||||
return res;
|
||||
}
|
||||
|
2165
tests/queries/0_stateless/03165_round_scale_as_column.reference
Normal file
2165
tests/queries/0_stateless/03165_round_scale_as_column.reference
Normal file
File diff suppressed because it is too large
Load Diff
125
tests/queries/0_stateless/03165_round_scale_as_column.sql
Normal file
125
tests/queries/0_stateless/03165_round_scale_as_column.sql
Normal file
@ -0,0 +1,125 @@
|
||||
-- Regression test that functions round(), roundBankers(), floor(), ceil() and trunc() work with default 'scale' argument (= the 2nd argument)
|
||||
SELECT toUInt8(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20;
|
||||
SELECT toUInt16(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20;
|
||||
SELECT toUInt32(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20;
|
||||
SELECT toUInt64(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20;
|
||||
SELECT toInt8(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20;
|
||||
SELECT toInt16(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20;
|
||||
SELECT toInt32(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20;
|
||||
SELECT toInt64(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20;
|
||||
SELECT toFloat32(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20;
|
||||
SELECT toFloat64(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20;
|
||||
|
||||
SELECT toFloat32((number - 10) / 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20;
|
||||
SELECT toFloat64((number - 10) / 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20;
|
||||
|
||||
-- Test that functions round(), roundBankers(), floor(), ceil() and trunc() accept non-const 'scale' argument
|
||||
SELECT toFloat32((number - 10) / 10) AS x, round(x, materialize(1)), roundBankers(x, materialize(1)), floor(x, materialize(1)), ceil(x, materialize(1)), trunc(x, materialize(1)) FROM system.numbers LIMIT 20;
|
||||
SELECT toFloat64((number - 10) / 10) AS x, round(x, materialize(1)), roundBankers(x, materialize(1)), floor(x, materialize(1)), ceil(x, materialize(1)), trunc(x, materialize(1)) FROM system.numbers LIMIT 20;
|
||||
SELECT toUInt8(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20;
|
||||
SELECT toUInt16(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20;
|
||||
SELECT toUInt32(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20;
|
||||
SELECT toUInt64(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20;
|
||||
|
||||
SELECT toInt8(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20;
|
||||
SELECT toInt16(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20;
|
||||
SELECT toInt32(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20;
|
||||
SELECT toInt64(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20;
|
||||
SELECT toFloat32(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20;
|
||||
SELECT toFloat64(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20;
|
||||
|
||||
SELECT toUInt8(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20;
|
||||
SELECT toUInt16(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20;
|
||||
SELECT toUInt32(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20;
|
||||
SELECT toUInt64(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20;
|
||||
SELECT toInt8(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20;
|
||||
SELECT toInt16(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20;
|
||||
SELECT toInt32(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20;
|
||||
SELECT toInt64(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20;
|
||||
SELECT toFloat32(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20;
|
||||
SELECT toFloat64(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20;
|
||||
|
||||
SELECT toString('CHECKPOINT1');
|
||||
|
||||
DROP TABLE IF EXISTS tround;
|
||||
|
||||
CREATE TABLE tround (
|
||||
id Int32,
|
||||
scale Int16,
|
||||
u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64,
|
||||
i8 Int8, i16 Int16, i32 Int32, i64 Int64,
|
||||
f32 Float32, f64 Float64
|
||||
) ENGINE = Memory;
|
||||
|
||||
INSERT INTO tround SELECT number , 0, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20;
|
||||
INSERT INTO tround SELECT number+20 , 0, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20;
|
||||
INSERT INTO tround SELECT number+40 , -1, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20;
|
||||
INSERT INTO tround SELECT number+60 , -1, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20;
|
||||
INSERT INTO tround SELECT number+80 , -2, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20;
|
||||
INSERT INTO tround SELECT number+100, -2, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20;
|
||||
|
||||
INSERT INTO tround SELECT number+200, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10;
|
||||
INSERT INTO tround SELECT number+210, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10;
|
||||
|
||||
INSERT INTO tround VALUES (300, 4, 2, 2, 2, 2, 2, 2, 2, 2, 2.0, 2.0);
|
||||
INSERT INTO tround VALUES (301, 4, 20, 20, 20, 20, 20, 20, 20, 20, 20.0, 20.0);
|
||||
INSERT INTO tround VALUES (302, 4, 200, 200, 200, 200, 200, 200, 200, 200, 200.0, 200.0);
|
||||
INSERT INTO tround VALUES (303, 4, 5, 5, 5, 5, 5, 5, 5, 5, 5.0, 5.0);
|
||||
INSERT INTO tround VALUES (304, 4, 50, 50, 50, 50, 50, 50, 50, 50, 50.0, 50.0);
|
||||
INSERT INTO tround VALUES (305, 4, 500, 500, 500, 500, 500, 500, 500, 500, 500.0, 500.0);
|
||||
|
||||
SELECT toString('id u8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)');
|
||||
SELECT id, u8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id;
|
||||
SELECT toString('id u16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)');
|
||||
SELECT id, u16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id;
|
||||
SELECT toString('id u32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)');
|
||||
SELECT id, u32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id;
|
||||
SELECT toString('id u64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)');
|
||||
SELECT id, u64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id;
|
||||
SELECT toString('id i8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)');
|
||||
SELECT id, i8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id;
|
||||
SELECT toString('id i16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)');
|
||||
SELECT id, i16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id;
|
||||
SELECT toString('id i32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)');
|
||||
SELECT id, i32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id;
|
||||
SELECT toString('id i64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)');
|
||||
SELECT id, i64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id;
|
||||
SELECT toString('id f32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)');
|
||||
SELECT id, f32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id;
|
||||
SELECT toString('id f64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)');
|
||||
SELECT id, f64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id;
|
||||
|
||||
DROP TABLE IF EXISTS tround;
|
||||
--
|
||||
SELECT toString('CHECKPOINT2');
|
||||
|
||||
DROP TABLE IF EXISTS tround2;
|
||||
|
||||
CREATE TABLE tround2 (
|
||||
id Int32,
|
||||
scale Int16,
|
||||
d32 Decimal32(4), d64 Decimal64(4), d128 Decimal128(4), d256 Decimal256(4)
|
||||
) ENGINE = Memory;
|
||||
|
||||
INSERT INTO tround2 VALUES (1, 6, toDecimal32('42.42', 4), toDecimal64('42.42', 4), toDecimal128('42.42', 4), toDecimal256('42.42', 4));
|
||||
INSERT INTO tround2 SELECT 2 , 6, cos(d32), cos(d64), cos(d128), cos(d256) FROM tround2 WHERE id = 1;
|
||||
INSERT INTO tround2 SELECT 3 , 6, sqrt(d32), sqrt(d64), sqrt(d128), sqrt(d256) FROM tround2 WHERE id = 1;
|
||||
INSERT INTO tround2 SELECT 4 , 6, lgamma(d32), lgamma(d64), lgamma(d128), lgamma(d256) FROM tround2 WHERE id = 1;
|
||||
INSERT INTO tround2 SELECT 5 , 6, tgamma(d32)/1e50, tgamma(d64)/1e50, tgamma(d128)/1e50, tgamma(d256)/1e50 FROM tround2 WHERE id = 1;
|
||||
INSERT INTO tround2 SELECT 6 , 8, sin(d32), sin(d64), sin(d128), sin(d256) FROM tround2 WHERE id = 1;
|
||||
INSERT INTO tround2 SELECT 7 , 8, cos(d32), cos(d64), cos(d128), cos(d256) FROM tround2 WHERE id = 1;
|
||||
INSERT INTO tround2 SELECT 8 , 8, log(d32), log(d64), log(d128), log(d256) FROM tround2 WHERE id = 1;
|
||||
INSERT INTO tround2 SELECT 9 , 8, log2(d32), log2(d64), log2(d128), log2(d256) FROM tround2 WHERE id = 1;
|
||||
INSERT INTO tround2 SELECT 10, 8, log10(d32), log10(d64), log10(d128), log10(d256) FROM tround2 WHERE id = 1;
|
||||
|
||||
SELECT id, round(d32, scale), round(d64, scale), round(d128, scale), round(d256, scale) FROM tround2 ORDER BY id;
|
||||
|
||||
DROP TABLE IF EXISTS tround2;
|
||||
|
||||
SELECT round(1, 1);
|
||||
SELECT round(materialize(1), materialize(1));
|
||||
SELECT round(1, materialize(1)); --{serverError ILLEGAL_COLUMN}
|
||||
SELECT round(materialize(1), 1);
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user