Simplify code of function "bar"

This commit is contained in:
Alexey Milovidov 2021-01-02 23:40:15 +03:00
parent aff724ea7d
commit 820af3a48d
4 changed files with 39 additions and 92 deletions

View File

@ -5,33 +5,25 @@
#include <common/arithmeticOverflow.h> #include <common/arithmeticOverflow.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/UnicodeBar.h> #include <Common/UnicodeBar.h>
#include <Common/NaNUtils.h>
#include <iostream>
namespace DB
{
namespace ErrorCodes
{
extern const int PARAMETER_OUT_OF_BOUND;
}
}
namespace UnicodeBar namespace UnicodeBar
{ {
double getWidth(Int64 x, Int64 min, Int64 max, double max_width) double getWidth(double x, double min, double max, double max_width)
{ {
if (isNaN(x))
return 0;
if (x <= min) if (x <= min)
return 0; return 0;
if (x >= max) if (x >= max)
return max_width; return max_width;
/// The case when max - min overflows return (x - min) / (max - min) * max_width;
Int64 max_difference;
if (common::subOverflow(max, min, max_difference))
throw DB::Exception(DB::ErrorCodes::PARAMETER_OUT_OF_BOUND, "The arguments to render unicode bar will lead to arithmetic overflow");
return (x - min) * max_width / max_difference;
} }
size_t getWidthInBytes(double width) size_t getWidthInBytes(double width)

View File

@ -10,7 +10,7 @@
*/ */
namespace UnicodeBar namespace UnicodeBar
{ {
double getWidth(Int64 x, Int64 min, Int64 max, double max_width); double getWidth(double x, double min, double max, double max_width);
size_t getWidthInBytes(double width); size_t getWidthInBytes(double width);
/// In `dst` there must be a space for barWidthInBytes(width) characters and a trailing zero. /// In `dst` there must be a space for barWidthInBytes(width) characters and a trailing zero.

View File

@ -2,7 +2,6 @@
#include <Functions/FunctionFactory.h> #include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Common/UnicodeBar.h> #include <Common/UnicodeBar.h>
#include <Common/FieldVisitors.h> #include <Common/FieldVisitors.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
@ -57,23 +56,30 @@ public:
+ ".", + ".",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!isNativeNumber(arguments[0]) || !isNativeNumber(arguments[1]) || !isNativeNumber(arguments[2]) if (!isNumber(arguments[0]) || !isNumber(arguments[1]) || !isNumber(arguments[2])
|| (arguments.size() == 4 && !isNativeNumber(arguments[3]))) || (arguments.size() == 4 && !isNumber(arguments[3])))
throw Exception("All arguments for function " + getName() + " must be numeric.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); throw Exception("All arguments for function " + getName() + " must be numeric.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>(); return std::make_shared<DataTypeString>();
} }
bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {3}; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{ {
Int64 min = extractConstant<Int64>(arguments, 1, "Second"); /// The level at which the line has zero length. /// The maximum width of the bar in characters.
Int64 max = extractConstant<Int64>(arguments, 2, "Third"); /// The level at which the line has the maximum length. Float64 max_width = 80; /// Motivated by old-school terminal size.
/// The maximum width of the bar in characters, by default. if (arguments.size() == 4)
Float64 max_width = arguments.size() == 4 ? extractConstant<Float64>(arguments, 3, "Fourth") : 80; {
const auto & max_width_column = *arguments[3].column;
if (!isColumnConst(max_width_column))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Fourth argument for function {} must be constant", getName());
max_width = max_width_column.getFloat64(0);
}
if (isNaN(max_width)) if (isNaN(max_width))
throw Exception("Argument 'max_width' must not be NaN", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); throw Exception("Argument 'max_width' must not be NaN", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -86,83 +92,32 @@ public:
const auto & src = *arguments[0].column; const auto & src = *arguments[0].column;
auto res_column = ColumnString::create();
if (executeNumber<UInt8>(src, *res_column, min, max, max_width)
|| executeNumber<UInt16>(src, *res_column, min, max, max_width)
|| executeNumber<UInt32>(src, *res_column, min, max, max_width)
|| executeNumber<UInt64>(src, *res_column, min, max, max_width)
|| executeNumber<Int8>(src, *res_column, min, max, max_width)
|| executeNumber<Int16>(src, *res_column, min, max, max_width)
|| executeNumber<Int32>(src, *res_column, min, max, max_width)
|| executeNumber<Int64>(src, *res_column, min, max, max_width)
|| executeNumber<Float32>(src, *res_column, min, max, max_width)
|| executeNumber<Float64>(src, *res_column, min, max, max_width))
{
return res_column;
}
else
throw Exception(
"Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
private:
template <typename T>
T extractConstant(const ColumnsWithTypeAndName & arguments, size_t argument_pos, const char * which_argument) const
{
const auto & column = *arguments[argument_pos].column;
if (!isColumnConst(column))
throw Exception(
which_argument + String(" argument for function ") + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
return applyVisitor(FieldVisitorConvertToNumber<T>(), column[0]);
}
template <typename T>
static void fill(const PaddedPODArray<T> & src,
ColumnString::Chars & dst_chars,
ColumnString::Offsets & dst_offsets,
Int64 min,
Int64 max,
Float64 max_width)
{
size_t size = src.size();
size_t current_offset = 0; size_t current_offset = 0;
dst_offsets.resize(size); auto res_column = ColumnString::create();
dst_chars.reserve(size * (UnicodeBar::getWidthInBytes(max_width) + 1)); /// lines 0-terminated.
for (size_t i = 0; i < size; ++i) ColumnString::Chars & dst_chars = res_column->getChars();
ColumnString::Offsets & dst_offsets = res_column->getOffsets();
dst_offsets.resize(input_rows_count);
dst_chars.reserve(input_rows_count * (UnicodeBar::getWidthInBytes(max_width) + 1)); /// strings are 0-terminated.
for (size_t i = 0; i < input_rows_count; ++i)
{ {
Float64 width = UnicodeBar::getWidth(src[i], min, max, max_width); Float64 width = UnicodeBar::getWidth(
src.getFloat64(i),
arguments[1].column->getFloat64(i),
arguments[2].column->getFloat64(i),
max_width);
size_t next_size = current_offset + UnicodeBar::getWidthInBytes(width) + 1; size_t next_size = current_offset + UnicodeBar::getWidthInBytes(width) + 1;
dst_chars.resize(next_size); dst_chars.resize(next_size);
UnicodeBar::render(width, reinterpret_cast<char *>(&dst_chars[current_offset])); UnicodeBar::render(width, reinterpret_cast<char *>(&dst_chars[current_offset]));
current_offset = next_size; current_offset = next_size;
dst_offsets[i] = current_offset; dst_offsets[i] = current_offset;
} }
}
template <typename T> return res_column;
static void fill(T src, String & dst_chars, Int64 min, Int64 max, Float64 max_width)
{
Float64 width = UnicodeBar::getWidth(src, min, max, max_width);
dst_chars.resize(UnicodeBar::getWidthInBytes(width));
UnicodeBar::render(width, dst_chars.data());
}
template <typename T>
static bool executeNumber(const IColumn & src, ColumnString & dst, Int64 min, Int64 max, Float64 max_width)
{
if (const ColumnVector<T> * col = checkAndGetColumn<ColumnVector<T>>(&src))
{
fill(col->getData(), dst.getChars(), dst.getOffsets(), min, max, max_width);
return true;
}
else
return false;
} }
}; };

View File

@ -1 +1 @@
SELECT bar((greatCircleAngle(100, -1, number, number) - number) * 2, -9223372036854775808, 1023, 100) FROM numbers(1048575); -- { serverError 12 } SELECT bar((greatCircleAngle(100, -1, number, number) - number) * 2, -9223372036854775808, 1023, 100) FROM numbers(1048575) FORMAT Null;