mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Remove toDecimalString
This commit is contained in:
parent
7af9af1a3f
commit
5f4756fb33
@ -945,44 +945,6 @@ Result:
|
||||
└────────────┴───────┘
|
||||
```
|
||||
|
||||
## toDecimalString
|
||||
|
||||
Converts a numeric value to String with the number of fractional digits in the output specified by the user.
|
||||
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
toDecimalString(number, scale)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
|
||||
- `number` — Value to be represented as String, [Int, UInt](/docs/en/sql-reference/data-types/int-uint.md), [Float](/docs/en/sql-reference/data-types/float.md), [Decimal](/docs/en/sql-reference/data-types/decimal.md),
|
||||
- `scale` — Number of fractional digits, [UInt8](/docs/en/sql-reference/data-types/int-uint.md).
|
||||
* Maximum scale for [Decimal](/docs/en/sql-reference/data-types/decimal.md) and [Int, UInt](/docs/en/sql-reference/data-types/int-uint.md) types is 77 (it is the maximum possible number of significant digits for Decimal),
|
||||
* Maximum scale for [Float](/docs/en/sql-reference/data-types/float.md) is 60.
|
||||
|
||||
**Returned value**
|
||||
|
||||
- Input value represented as [String](/docs/en/sql-reference/data-types/string.md) with given number of fractional digits (scale).
|
||||
The number is rounded up or down according to common arithmetic in case requested scale is smaller than original number's scale.
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
SELECT toDecimalString(CAST('64.32', 'Float64'), 5);
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```response
|
||||
┌toDecimalString(CAST('64.32', 'Float64'), 5)─┐
|
||||
│ 64.32000 │
|
||||
└─────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## reinterpretAsUInt(8\|16\|32\|64)
|
||||
|
||||
## reinterpretAsInt(8\|16\|32\|64)
|
||||
|
@ -762,44 +762,6 @@ SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut;
|
||||
└────────────┴───────┘
|
||||
```
|
||||
|
||||
## toDecimalString
|
||||
|
||||
Принимает любой численный тип первым аргументом, возвращает строковое десятичное представление числа с точностью, заданной вторым аргументом.
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
``` sql
|
||||
toDecimalString(number, scale)
|
||||
```
|
||||
|
||||
**Параметры**
|
||||
|
||||
- `number` — Значение любого числового типа: [Int, UInt](/docs/ru/sql-reference/data-types/int-uint.md), [Float](/docs/ru/sql-reference/data-types/float.md), [Decimal](/docs/ru/sql-reference/data-types/decimal.md),
|
||||
- `scale` — Требуемое количество десятичных знаков после запятой, [UInt8](/docs/ru/sql-reference/data-types/int-uint.md).
|
||||
* Значение `scale` для типов [Decimal](/docs/ru/sql-reference/data-types/decimal.md) и [Int, UInt](/docs/ru/sql-reference/data-types/int-uint.md) должно не превышать 77 (так как это наибольшее количество значимых символов для этих типов),
|
||||
* Значение `scale` для типа [Float](/docs/ru/sql-reference/data-types/float.md) не должно превышать 60.
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
- Строка ([String](/docs/en/sql-reference/data-types/string.md)), представляющая собой десятичное представление входного числа с заданной длиной дробной части.
|
||||
При необходимости число округляется по стандартным правилам арифметики.
|
||||
|
||||
**Пример использования**
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT toDecimalString(CAST('64.32', 'Float64'), 5);
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```response
|
||||
┌─toDecimalString(CAST('64.32', 'Float64'), 5)┐
|
||||
│ 64.32000 │
|
||||
└─────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## reinterpretAsUInt(8\|16\|32\|64) {#reinterpretasuint8163264}
|
||||
|
||||
## reinterpretAsInt(8\|16\|32\|64) {#reinterpretasint8163264}
|
||||
|
@ -1,22 +0,0 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionToDecimalString.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
REGISTER_FUNCTION(ToDecimalString)
|
||||
{
|
||||
factory.registerFunction<FunctionToDecimalString>(
|
||||
FunctionDocumentation{
|
||||
.description=R"(
|
||||
Returns string representation of a number. First argument is the number of any numeric type,
|
||||
second argument is the desired number of digits in fractional part. Returns String.
|
||||
|
||||
)",
|
||||
.examples{{"toDecimalString", "SELECT toDecimalString(2.1456,2)", ""}},
|
||||
.categories{"String"}
|
||||
}, FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
@ -1,312 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER;
|
||||
}
|
||||
|
||||
class FunctionToDecimalString : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "toDecimalString";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionToDecimalString>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isNumber(*arguments[0]))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal first argument for formatDecimal function: got {}, expected numeric type",
|
||||
arguments[0]->getName());
|
||||
|
||||
if (!isUInt8(*arguments[1]))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal second argument for formatDecimal function: got {}, expected UInt8",
|
||||
arguments[1]->getName());
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
private:
|
||||
/// For operations with Integer/Float
|
||||
template <typename FromVectorType>
|
||||
void vectorConstant(const FromVectorType & vec_from, UInt8 precision,
|
||||
ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const
|
||||
{
|
||||
size_t input_rows_count = vec_from.size();
|
||||
result_offsets.resize(input_rows_count);
|
||||
|
||||
/// Buffer is used here and in functions below because resulting size cannot be precisely anticipated,
|
||||
/// and buffer resizes on-the-go. Also, .count() provided by buffer is convenient in this case.
|
||||
WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to);
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
format(vec_from[i], buf_to, precision);
|
||||
result_offsets[i] = buf_to.count();
|
||||
}
|
||||
|
||||
buf_to.finalize();
|
||||
}
|
||||
|
||||
template <typename FirstArgVectorType>
|
||||
void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector<UInt8>::Container & vec_precision,
|
||||
ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const
|
||||
{
|
||||
size_t input_rows_count = vec_from.size();
|
||||
result_offsets.resize(input_rows_count);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to);
|
||||
|
||||
constexpr size_t max_digits = std::numeric_limits<UInt256>::digits10;
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
if (vec_precision[i] > max_digits)
|
||||
throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER,
|
||||
"Too many fractional digits requested, shall not be more than {}", max_digits);
|
||||
format(vec_from[i], buf_to, vec_precision[i]);
|
||||
result_offsets[i] = buf_to.count();
|
||||
}
|
||||
|
||||
buf_to.finalize();
|
||||
}
|
||||
|
||||
template <typename FirstArgType>
|
||||
void constantVector(const FirstArgType & value_from, const ColumnVector<UInt8>::Container & vec_precision,
|
||||
ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const
|
||||
{
|
||||
size_t input_rows_count = vec_precision.size();
|
||||
result_offsets.resize(input_rows_count);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to);
|
||||
|
||||
constexpr size_t max_digits = std::numeric_limits<UInt256>::digits10;
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
if (vec_precision[i] > max_digits)
|
||||
throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER,
|
||||
"Too many fractional digits requested, shall not be more than {}", max_digits);
|
||||
format(value_from, buf_to, vec_precision[i]);
|
||||
result_offsets[i] = buf_to.count();
|
||||
}
|
||||
|
||||
buf_to.finalize();
|
||||
}
|
||||
|
||||
/// For operations with Decimal
|
||||
template <typename FirstArgVectorType>
|
||||
void vectorConstant(const FirstArgVectorType & vec_from, UInt8 precision,
|
||||
ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const
|
||||
{
|
||||
/// There are no more than 77 meaning digits (as it is the max length of UInt256). So we can limit it with 77.
|
||||
constexpr size_t max_digits = std::numeric_limits<UInt256>::digits10;
|
||||
if (precision > max_digits)
|
||||
throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER,
|
||||
"Too many fractional digits requested for Decimal, must not be more than {}", max_digits);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to);
|
||||
size_t input_rows_count = vec_from.size();
|
||||
result_offsets.resize(input_rows_count);
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
writeText(vec_from[i], from_scale, buf_to, true, true, precision);
|
||||
writeChar(0, buf_to);
|
||||
result_offsets[i] = buf_to.count();
|
||||
}
|
||||
buf_to.finalize();
|
||||
}
|
||||
|
||||
template <typename FirstArgVectorType>
|
||||
void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector<UInt8>::Container & vec_precision,
|
||||
ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const
|
||||
{
|
||||
size_t input_rows_count = vec_from.size();
|
||||
result_offsets.resize(input_rows_count);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to);
|
||||
|
||||
constexpr size_t max_digits = std::numeric_limits<UInt256>::digits10;
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
if (vec_precision[i] > max_digits)
|
||||
throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER,
|
||||
"Too many fractional digits requested for Decimal, must not be more than {}", max_digits);
|
||||
writeText(vec_from[i], from_scale, buf_to, true, true, vec_precision[i]);
|
||||
writeChar(0, buf_to);
|
||||
result_offsets[i] = buf_to.count();
|
||||
}
|
||||
buf_to.finalize();
|
||||
}
|
||||
|
||||
template <typename FirstArgType>
|
||||
void constantVector(const FirstArgType & value_from, const ColumnVector<UInt8>::Container & vec_precision,
|
||||
ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const
|
||||
{
|
||||
size_t input_rows_count = vec_precision.size();
|
||||
result_offsets.resize(input_rows_count);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars> buf_to(vec_to);
|
||||
|
||||
constexpr size_t max_digits = std::numeric_limits<UInt256>::digits10;
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
if (vec_precision[i] > max_digits)
|
||||
throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER,
|
||||
"Too many fractional digits requested for Decimal, must not be more than {}", max_digits);
|
||||
writeText(value_from, from_scale, buf_to, true, true, vec_precision[i]);
|
||||
writeChar(0, buf_to);
|
||||
result_offsets[i] = buf_to.count();
|
||||
}
|
||||
buf_to.finalize();
|
||||
}
|
||||
|
||||
template <is_floating_point T>
|
||||
static void format(T value, DB::WriteBuffer & out, UInt8 precision)
|
||||
{
|
||||
/// Maximum of 60 is hard-coded in 'double-conversion/double-conversion.h' for floating point values,
|
||||
/// Catch this here to give user a more reasonable error.
|
||||
if (precision > 60)
|
||||
throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER,
|
||||
"Too high precision requested for Float, must not be more than 60, got {}", Int8(precision));
|
||||
|
||||
DB::DoubleConverter<false>::BufferType buffer;
|
||||
double_conversion::StringBuilder builder{buffer, sizeof(buffer)};
|
||||
|
||||
const auto result = DB::DoubleConverter<false>::instance().ToFixed(value, precision, &builder);
|
||||
|
||||
if (!result)
|
||||
throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, "Error processing number: {}", value);
|
||||
|
||||
out.write(buffer, builder.position());
|
||||
writeChar(0, out);
|
||||
}
|
||||
|
||||
template <is_integer T>
|
||||
static void format(T value, DB::WriteBuffer & out, UInt8 precision)
|
||||
{
|
||||
/// Fractional part for Integer is just trailing zeros. Let's limit it with 77 (like with Decimals).
|
||||
constexpr size_t max_digits = std::numeric_limits<UInt256>::digits10;
|
||||
if (precision > max_digits)
|
||||
throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER,
|
||||
"Too many fractional digits requested, shall not be more than {}", max_digits);
|
||||
writeText(value, out);
|
||||
if (precision > 0) [[likely]]
|
||||
{
|
||||
writeChar('.', out);
|
||||
for (int i = 0; i < precision; ++i)
|
||||
writeChar('0', out);
|
||||
writeChar(0, out);
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
switch (arguments[0].type->getTypeId())
|
||||
{
|
||||
case TypeIndex::UInt8: return executeType<UInt8>(arguments);
|
||||
case TypeIndex::UInt16: return executeType<UInt16>(arguments);
|
||||
case TypeIndex::UInt32: return executeType<UInt32>(arguments);
|
||||
case TypeIndex::UInt64: return executeType<UInt64>(arguments);
|
||||
case TypeIndex::UInt128: return executeType<UInt128>(arguments);
|
||||
case TypeIndex::UInt256: return executeType<UInt256>(arguments);
|
||||
case TypeIndex::Int8: return executeType<Int8>(arguments);
|
||||
case TypeIndex::Int16: return executeType<Int16>(arguments);
|
||||
case TypeIndex::Int32: return executeType<Int32>(arguments);
|
||||
case TypeIndex::Int64: return executeType<Int64>(arguments);
|
||||
case TypeIndex::Int128: return executeType<Int128>(arguments);
|
||||
case TypeIndex::Int256: return executeType<Int256>(arguments);
|
||||
case TypeIndex::Float32: return executeType<Float32>(arguments);
|
||||
case TypeIndex::Float64: return executeType<Float64>(arguments);
|
||||
case TypeIndex::Decimal32: return executeType<Decimal32>(arguments);
|
||||
case TypeIndex::Decimal64: return executeType<Decimal64>(arguments);
|
||||
case TypeIndex::Decimal128: return executeType<Decimal128>(arguments);
|
||||
case TypeIndex::Decimal256: return executeType<Decimal256>(arguments);
|
||||
default:
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",
|
||||
arguments[0].column->getName(), getName());
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
ColumnPtr executeType(const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
const auto * from_col_const = typeid_cast<const ColumnConst *>(arguments[0].column.get());
|
||||
const auto * precision_col = checkAndGetColumn<ColumnVector<UInt8>>(arguments[1].column.get());
|
||||
const auto * precision_col_const = typeid_cast<const ColumnConst *>(arguments[1].column.get());
|
||||
|
||||
auto result_col = ColumnString::create();
|
||||
auto * result_col_string = assert_cast<ColumnString *>(result_col.get());
|
||||
ColumnString::Chars & result_chars = result_col_string->getChars();
|
||||
ColumnString::Offsets & result_offsets = result_col_string->getOffsets();
|
||||
|
||||
if constexpr (is_decimal<T>)
|
||||
{
|
||||
const auto * from_col = checkAndGetColumn<ColumnDecimal<T>>(arguments[0].column.get());
|
||||
UInt8 from_scale = from_col->getScale();
|
||||
|
||||
if (from_col)
|
||||
{
|
||||
if (precision_col_const)
|
||||
vectorConstant(from_col->getData(), precision_col_const->template getValue<UInt8>(), result_chars, result_offsets, from_scale);
|
||||
else
|
||||
vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale);
|
||||
}
|
||||
else if (from_col_const)
|
||||
constantVector(from_col_const->template getValue<T>(), precision_col->getData(), result_chars, result_offsets, from_scale);
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName());
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto * from_col = checkAndGetColumn<ColumnVector<T>>(arguments[0].column.get());
|
||||
if (from_col)
|
||||
{
|
||||
if (precision_col_const)
|
||||
vectorConstant(from_col->getData(), precision_col_const->template getValue<UInt8>(), result_chars, result_offsets);
|
||||
else
|
||||
vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets);
|
||||
}
|
||||
else if (from_col_const)
|
||||
constantVector(from_col_const->template getValue<T>(), precision_col->getData(), result_chars, result_offsets);
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName());
|
||||
}
|
||||
|
||||
return result_col;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -905,26 +905,26 @@ inline void writeText(const IPv4 & x, WriteBuffer & buf) { writeIPv4Text(x, buf)
|
||||
inline void writeText(const IPv6 & x, WriteBuffer & buf) { writeIPv6Text(x, buf); }
|
||||
|
||||
template <typename T>
|
||||
void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros,
|
||||
bool fixed_fractional_length, UInt32 fractional_length)
|
||||
void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros)
|
||||
{
|
||||
/// If it's big integer, but the number of digits is small,
|
||||
/// use the implementation for smaller integers for more efficient arithmetic.
|
||||
|
||||
if constexpr (std::is_same_v<T, Int256>)
|
||||
{
|
||||
if (x <= std::numeric_limits<UInt32>::max())
|
||||
{
|
||||
writeDecimalFractional(static_cast<UInt32>(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length);
|
||||
writeDecimalFractional(static_cast<UInt32>(x), scale, ostr, trailing_zeros);
|
||||
return;
|
||||
}
|
||||
else if (x <= std::numeric_limits<UInt64>::max())
|
||||
{
|
||||
writeDecimalFractional(static_cast<UInt64>(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length);
|
||||
writeDecimalFractional(static_cast<UInt64>(x), scale, ostr, trailing_zeros);
|
||||
return;
|
||||
}
|
||||
else if (x <= std::numeric_limits<UInt128>::max())
|
||||
{
|
||||
writeDecimalFractional(static_cast<UInt128>(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length);
|
||||
writeDecimalFractional(static_cast<UInt128>(x), scale, ostr, trailing_zeros);
|
||||
return;
|
||||
}
|
||||
}
|
||||
@ -932,53 +932,35 @@ void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool
|
||||
{
|
||||
if (x <= std::numeric_limits<UInt32>::max())
|
||||
{
|
||||
writeDecimalFractional(static_cast<UInt32>(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length);
|
||||
writeDecimalFractional(static_cast<UInt32>(x), scale, ostr, trailing_zeros);
|
||||
return;
|
||||
}
|
||||
else if (x <= std::numeric_limits<UInt64>::max())
|
||||
{
|
||||
writeDecimalFractional(static_cast<UInt64>(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length);
|
||||
writeDecimalFractional(static_cast<UInt64>(x), scale, ostr, trailing_zeros);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
constexpr size_t max_digits = std::numeric_limits<UInt256>::digits10;
|
||||
assert(scale <= max_digits);
|
||||
assert(fractional_length <= max_digits);
|
||||
|
||||
char buf[max_digits];
|
||||
memset(buf, '0', std::max(scale, fractional_length));
|
||||
memset(buf, '0', scale);
|
||||
|
||||
T value = x;
|
||||
Int32 last_nonzero_pos = 0;
|
||||
|
||||
if (fixed_fractional_length && fractional_length < scale)
|
||||
{
|
||||
T new_value = value / DecimalUtils::scaleMultiplier<Int256>(scale - fractional_length - 1);
|
||||
auto round_carry = new_value % 10;
|
||||
value = new_value / 10;
|
||||
if (round_carry >= 5)
|
||||
value += 1;
|
||||
}
|
||||
|
||||
for (Int32 pos = fixed_fractional_length ? std::min(scale - 1, fractional_length - 1) : scale - 1; pos >= 0; --pos)
|
||||
for (Int32 pos = scale - 1; pos >= 0; --pos)
|
||||
{
|
||||
auto remainder = value % 10;
|
||||
value /= 10;
|
||||
|
||||
if (remainder != 0 && last_nonzero_pos == 0)
|
||||
last_nonzero_pos = pos;
|
||||
|
||||
buf[pos] += static_cast<char>(remainder);
|
||||
}
|
||||
|
||||
writeChar('.', ostr);
|
||||
ostr.write(buf, fixed_fractional_length ? fractional_length : (trailing_zeros ? scale : last_nonzero_pos + 1));
|
||||
ostr.write(buf, trailing_zeros ? scale : last_nonzero_pos + 1);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void writeText(Decimal<T> x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros,
|
||||
bool fixed_fractional_length = false, UInt32 fractional_length = 0)
|
||||
void writeText(Decimal<T> x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros)
|
||||
{
|
||||
T part = DecimalUtils::getWholePart(x, scale);
|
||||
|
||||
@ -989,7 +971,7 @@ void writeText(Decimal<T> x, UInt32 scale, WriteBuffer & ostr, bool trailing_zer
|
||||
|
||||
writeIntText(part, ostr);
|
||||
|
||||
if (scale || (fixed_fractional_length && fractional_length > 0))
|
||||
if (scale)
|
||||
{
|
||||
part = DecimalUtils::getFractionalPart(x, scale);
|
||||
if (part || trailing_zeros)
|
||||
@ -997,7 +979,7 @@ void writeText(Decimal<T> x, UInt32 scale, WriteBuffer & ostr, bool trailing_zer
|
||||
if (part < 0)
|
||||
part *= T(-1);
|
||||
|
||||
writeDecimalFractional(part, scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length);
|
||||
writeDecimalFractional(part, scale, ostr, trailing_zeros);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1,21 +0,0 @@
|
||||
2.00000000000000000000000000000000000000000000000000000000000000000000000000000
|
||||
2.12
|
||||
-2.00000000000000000000000000000000000000000000000000000000000000000000000000000
|
||||
-2.12
|
||||
2.987600000000000033395508580724708735942840576171875000000000
|
||||
2.15
|
||||
-2.987600000000000033395508580724708735942840576171875000000000
|
||||
-2.15
|
||||
64.1230010986
|
||||
64.2340000000
|
||||
-64.1230010986
|
||||
-64.2340000000
|
||||
-32.345
|
||||
32.34500000000000000000000000000000000000000000000000000000000000000000000000000
|
||||
32.46
|
||||
-64.5671232345
|
||||
128.78932312332132985464
|
||||
-128.78932312332132985464
|
||||
128.78932312332132985464000000000000000000000000000000000000000000000000000000000
|
||||
128.7893231233
|
||||
-128.78932312332132985464123123789323123321329854600000000000000000000000000000000
|
@ -1,35 +0,0 @@
|
||||
-- Regular types
|
||||
SELECT toDecimalString(2, 77); -- more digits required than exist
|
||||
SELECT toDecimalString(2.123456, 2); -- rounding
|
||||
SELECT toDecimalString(-2, 77); -- more digits required than exist
|
||||
SELECT toDecimalString(-2.123456, 2); -- rounding
|
||||
|
||||
SELECT toDecimalString(2.9876, 60); -- more digits required than exist (took 60 as it is float by default)
|
||||
SELECT toDecimalString(2.1456, 2); -- rounding
|
||||
SELECT toDecimalString(-2.9876, 60); -- more digits required than exist
|
||||
SELECT toDecimalString(-2.1456, 2); -- rounding
|
||||
|
||||
-- Float32 and Float64 tests. No sense to test big float precision -- the result will be a mess anyway.
|
||||
SELECT toDecimalString(64.123::Float32, 10);
|
||||
SELECT toDecimalString(64.234::Float64, 10);
|
||||
SELECT toDecimalString(-64.123::Float32, 10);
|
||||
SELECT toDecimalString(-64.234::Float64, 10);
|
||||
|
||||
-- Decimals
|
||||
SELECT toDecimalString(-32.345::Decimal32(3), 3);
|
||||
SELECT toDecimalString(32.345::Decimal32(3), 77); -- more digits required than exist
|
||||
SELECT toDecimalString(32.456::Decimal32(3), 2); -- rounding
|
||||
SELECT toDecimalString('-64.5671232345'::Decimal64(10), 10);
|
||||
SELECT toDecimalString('128.78932312332132985464'::Decimal128(20), 20);
|
||||
SELECT toDecimalString('-128.78932312332132985464123123'::Decimal128(26), 20); -- rounding
|
||||
SELECT toDecimalString('128.78932312332132985464'::Decimal128(20), 77); -- more digits required than exist
|
||||
SELECT toDecimalString('128.789323123321329854641231237893231233213298546'::Decimal256(45), 10); -- rounding
|
||||
SELECT toDecimalString('-128.789323123321329854641231237893231233213298546'::Decimal256(45), 77); -- more digits required than exist
|
||||
|
||||
-- Max number of decimal fractional digits is defined as 77 for Int/UInt/Decimal and 60 for Float.
|
||||
-- These values shall work OK.
|
||||
SELECT toDecimalString('32.32'::Float32, 61); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER}
|
||||
SELECT toDecimalString('64.64'::Float64, 61); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER}
|
||||
SELECT toDecimalString('88'::UInt8, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER}
|
||||
SELECT toDecimalString('646464'::Int256, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER}
|
||||
SELECT toDecimalString('-128.789323123321329854641231237893231233213298546'::Decimal256(45), 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER}
|
Loading…
Reference in New Issue
Block a user