mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge branch 'master' into revert-15259-revert-14962-log-avoid-deadlock
This commit is contained in:
commit
da943ef95f
@ -300,7 +300,7 @@ public:
|
|||||||
const char * ptr_end = reinterpret_cast<const char *>(&*from_end);
|
const char * ptr_end = reinterpret_cast<const char *>(&*from_end);
|
||||||
|
|
||||||
/// Also it's safe if the range is empty.
|
/// Also it's safe if the range is empty.
|
||||||
assert(!((ptr_begin >= c_start && ptr_begin <= c_end) || (ptr_end >= c_start && ptr_end <= c_end)) || (ptr_begin == ptr_end));
|
assert(!((ptr_begin >= c_start && ptr_begin < c_end) || (ptr_end > c_start && ptr_end <= c_end)) || (ptr_begin == ptr_end));
|
||||||
#endif
|
#endif
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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)
|
||||||
|
@ -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.
|
||||||
|
@ -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;
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -11,7 +11,7 @@
|
|||||||
179 -0.06
|
179 -0.06
|
||||||
|
|
||||||
██
|
██
|
||||||
████
|
████▏
|
||||||
██████▏
|
██████▏
|
||||||
████████▎
|
████████▎
|
||||||
██████████▎
|
██████████▎
|
||||||
@ -19,25 +19,25 @@
|
|||||||
██████████████▍
|
██████████████▍
|
||||||
████████████████▌
|
████████████████▌
|
||||||
██████████████████▌
|
██████████████████▌
|
||||||
████████████████████▌
|
████████████████████▋
|
||||||
██████████████████████▋
|
██████████████████████▋
|
||||||
████████████████████████▋
|
████████████████████████▋
|
||||||
██████████████████████████▌
|
██████████████████████████▌
|
||||||
████████████████████████████▍
|
████████████████████████████▍
|
||||||
██████████████████████████████▍
|
██████████████████████████████▍
|
||||||
████████████████████████████████▎
|
████████████████████████████████▍
|
||||||
██████████████████████████████████▎
|
██████████████████████████████████▎
|
||||||
████████████████████████████████████▏
|
████████████████████████████████████▏
|
||||||
██████████████████████████████████████
|
██████████████████████████████████████
|
||||||
███████████████████████████████████████▊
|
███████████████████████████████████████▊
|
||||||
█████████████████████████████████████████▋
|
█████████████████████████████████████████▋
|
||||||
███████████████████████████████████████████▌
|
███████████████████████████████████████████▌
|
||||||
█████████████████████████████████████████████▎
|
█████████████████████████████████████████████▍
|
||||||
███████████████████████████████████████████████
|
███████████████████████████████████████████████▏
|
||||||
████████████████████████████████████████████████▋
|
████████████████████████████████████████████████▊
|
||||||
██████████████████████████████████████████████████▌
|
██████████████████████████████████████████████████▌
|
||||||
████████████████████████████████████████████████████▏
|
████████████████████████████████████████████████████▏
|
||||||
█████████████████████████████████████████████████████▋
|
█████████████████████████████████████████████████████▊
|
||||||
███████████████████████████████████████████████████████▍
|
███████████████████████████████████████████████████████▍
|
||||||
█████████████████████████████████████████████████████████
|
█████████████████████████████████████████████████████████
|
||||||
██████████████████████████████████████████████████████████▌
|
██████████████████████████████████████████████████████████▌
|
||||||
@ -50,8 +50,8 @@
|
|||||||
████████████████████████████████████████████████████████████████████
|
████████████████████████████████████████████████████████████████████
|
||||||
█████████████████████████████████████████████████████████████████████▏
|
█████████████████████████████████████████████████████████████████████▏
|
||||||
██████████████████████████████████████████████████████████████████████▎
|
██████████████████████████████████████████████████████████████████████▎
|
||||||
███████████████████████████████████████████████████████████████████████▎
|
███████████████████████████████████████████████████████████████████████▍
|
||||||
████████████████████████████████████████████████████████████████████████▎
|
████████████████████████████████████████████████████████████████████████▍
|
||||||
█████████████████████████████████████████████████████████████████████████▎
|
█████████████████████████████████████████████████████████████████████████▎
|
||||||
██████████████████████████████████████████████████████████████████████████▏
|
██████████████████████████████████████████████████████████████████████████▏
|
||||||
███████████████████████████████████████████████████████████████████████████
|
███████████████████████████████████████████████████████████████████████████
|
||||||
@ -61,11 +61,11 @@
|
|||||||
█████████████████████████████████████████████████████████████████████████████▌
|
█████████████████████████████████████████████████████████████████████████████▌
|
||||||
█████████████████████████████████████████████████████████████████████████████▊
|
█████████████████████████████████████████████████████████████████████████████▊
|
||||||
██████████████████████████████████████████████████████████████████████████████▎
|
██████████████████████████████████████████████████████████████████████████████▎
|
||||||
██████████████████████████████████████████████████████████████████████████████▌
|
██████████████████████████████████████████████████████████████████████████████▋
|
||||||
██████████████████████████████████████████████████████████████████████████████▋
|
██████████████████████████████████████████████████████████████████████████████▋
|
||||||
██████████████████████████████████████████████████████████████████████████████▊
|
██████████████████████████████████████████████████████████████████████████████▊
|
||||||
██████████████████████████████████████████████████████████████████████████████▊
|
██████████████████████████████████████████████████████████████████████████████▊
|
||||||
██████████████████████████████████████████████████████████████████████████████▋
|
██████████████████████████████████████████████████████████████████████████████▊
|
||||||
██████████████████████████████████████████████████████████████████████████████▋
|
██████████████████████████████████████████████████████████████████████████████▋
|
||||||
██████████████████████████████████████████████████████████████████████████████▍
|
██████████████████████████████████████████████████████████████████████████████▍
|
||||||
██████████████████████████████████████████████████████████████████████████████
|
██████████████████████████████████████████████████████████████████████████████
|
||||||
@ -75,7 +75,7 @@
|
|||||||
███████████████████████████████████████████████████████████████████████████▌
|
███████████████████████████████████████████████████████████████████████████▌
|
||||||
██████████████████████████████████████████████████████████████████████████▌
|
██████████████████████████████████████████████████████████████████████████▌
|
||||||
█████████████████████████████████████████████████████████████████████████▌
|
█████████████████████████████████████████████████████████████████████████▌
|
||||||
████████████████████████████████████████████████████████████████████████▎
|
████████████████████████████████████████████████████████████████████████▍
|
||||||
███████████████████████████████████████████████████████████████████████
|
███████████████████████████████████████████████████████████████████████
|
||||||
█████████████████████████████████████████████████████████████████████▋
|
█████████████████████████████████████████████████████████████████████▋
|
||||||
████████████████████████████████████████████████████████████████████
|
████████████████████████████████████████████████████████████████████
|
||||||
@ -97,5 +97,5 @@
|
|||||||
██████████████████████▋
|
██████████████████████▋
|
||||||
██████████████████▌
|
██████████████████▌
|
||||||
██████████████▏
|
██████████████▏
|
||||||
█████████▌
|
█████████▋
|
||||||
████▊
|
████▊
|
||||||
|
@ -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;
|
||||||
|
Loading…
Reference in New Issue
Block a user