Skip to content

Commit

Permalink
Merge pull request #52520 from zvonand/revert-52450-remove-to-decimal…
Browse files Browse the repository at this point in the history
…-string

Fix `toDecimalString` function
  • Loading branch information
robot-ch-test-poll2 authored Jul 26, 2023
2 parents 3387b02 + 4f7bdf3 commit 7bcef0a
Show file tree
Hide file tree
Showing 7 changed files with 448 additions and 13 deletions.
38 changes: 38 additions & 0 deletions docs/en/sql-reference/functions/type-conversion-functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -945,6 +945,44 @@ 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)
Expand Down
38 changes: 38 additions & 0 deletions docs/ru/sql-reference/functions/type-conversion-functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -762,6 +762,44 @@ 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}
Expand Down
22 changes: 22 additions & 0 deletions src/Functions/FunctionToDecimalString.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
#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);
}

}
262 changes: 262 additions & 0 deletions src/Functions/FunctionToDecimalString.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,262 @@
#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_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 ColumnsWithTypeAndName & arguments) const override
{
FunctionArgumentDescriptors mandatory_args = {
{"Value", &isNumber<IDataType>, nullptr, "Number"},
{"precision", &isNativeInteger<IDataType>, &isColumnConst, "const Integer"}
};

validateFunctionArgumentTypes(*this, arguments, mandatory_args, {});

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();
}

/// 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 <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 * precision_col = checkAndGetColumn<ColumnVector<UInt8>>(arguments[1].column.get());
const auto * precision_col_const = checkAndGetColumnConst<ColumnVector<UInt8>>(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 if (precision_col)
vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale);
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of second argument of function formatDecimal", arguments[1].column->getName());
}
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 if (precision_col)
vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets);
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of second argument of function formatDecimal", arguments[1].column->getName());

}
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName());
}

return result_col;
}
};

}
Loading

0 comments on commit 7bcef0a

Please sign in to comment.