dbms: added function bar [#METR-12249].

This commit is contained in:
Alexey Milovidov 2014-08-09 05:51:03 +04:00
parent 506427e0a0
commit 8038d2ab66
2 changed files with 211 additions and 10 deletions

View File

@ -50,6 +50,8 @@ namespace DB
* не предназначена для пользователя, а используется только как prerequisites для функций высшего порядка.
*
* sleep(n) - спит n секунд каждый блок.
*
* bar(x, min, max, width) - рисует полосу из количества символов, пропорционального (x - min) и равного width при x == max.
*/
@ -274,7 +276,7 @@ public:
{
const IColumn & argument = *block.getByPosition(arguments[0]).column;
if (!argument.isConst())
throw Exception("Argument for function 'materialize' must be constant.", ErrorCodes::ILLEGAL_COLUMN);
throw Exception("Argument for function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = dynamic_cast<const IColumnConst &>(argument).convertToFullColumn();
}
@ -354,7 +356,7 @@ public:
DataTypePtr getReturnType(const DataTypes & arguments) const
{
if (arguments.size() < 2)
throw Exception("Function tuple requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception("Function " + getName() + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new DataTypeTuple(arguments);
}
@ -386,18 +388,18 @@ public:
ExpressionActions::Actions & out_prerequisites)
{
if (arguments.size() != 2)
throw Exception("Function tupleElement requires exactly two arguments: tuple and element index.",
throw Exception("Function " + getName() + " requires exactly two arguments: tuple and element index.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const ColumnConstUInt8 * index_col = typeid_cast<const ColumnConstUInt8 *>(&*arguments[1].column);
if (!index_col)
throw Exception("Second argument to tupleElement must be a constant UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception("Second argument to " + getName() + " must be a constant UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
size_t index = index_col->getData();
const DataTypeTuple * tuple = typeid_cast<const DataTypeTuple *>(&*arguments[0].type);
if (!tuple)
throw Exception("First argument for function tupleElement must be tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception("First argument for function " + getName() + " must be tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (index == 0)
throw Exception("Indices in tuples are 1-based.", ErrorCodes::ILLEGAL_INDEX);
@ -417,10 +419,10 @@ public:
const ColumnConstUInt8 * index_col = typeid_cast<const ColumnConstUInt8 *>(&*block.getByPosition(arguments[1]).column);
if (!tuple_col)
throw Exception("First argument for function tupleElement must be tuple.", ErrorCodes::ILLEGAL_COLUMN);
throw Exception("First argument for function " + getName() + " must be tuple.", ErrorCodes::ILLEGAL_COLUMN);
if (!index_col)
throw Exception("Second argument for function tupleElement must be UInt8 constant literal.", ErrorCodes::ILLEGAL_COLUMN);
throw Exception("Second argument for function " + getName() + " must be UInt8 constant literal.", ErrorCodes::ILLEGAL_COLUMN);
size_t index = index_col->getData();
if (index == 0)
@ -472,11 +474,11 @@ public:
DataTypePtr getReturnType(const DataTypes & arguments) const
{
if (arguments.size() != 1)
throw Exception("Function arrayJoin requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception("Function " + getName() + " requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const DataTypeArray * arr = typeid_cast<const DataTypeArray *>(&*arguments[0]);
if (!arr)
throw Exception("Argument for function arrayJoin must be Array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception("Argument for function " + getName() + " must be Array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return arr->getNestedType()->clone();
}
@ -484,7 +486,7 @@ public:
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
throw Exception("Function arrayJoin must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL);
throw Exception("Function " + getName() + " must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL);
}
};
@ -539,4 +541,202 @@ class FunctionReplicate : public IFunction
}
};
class FunctionBar : public IFunction
{
public:
/// Получить имя функции.
String getName() const
{
return "bar";
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const
{
if (arguments.size() != 3 && arguments.size() != 4)
throw Exception("Function " + getName() + " requires from 3 or 4 parameters: value, min_value, max_value, [max_width_of_bar = 80]. Passed "
+ toString(arguments.size()) + ".",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!arguments[0]->isNumeric() || !arguments[1]->isNumeric() || !arguments[2]->isNumeric()
|| (arguments.size() == 4 && !arguments[3]->isNumeric()))
throw Exception("All arguments for function " + getName() + " must be numeric.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return new DataTypeString;
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
Int64 min = extractConstant<Int64>(block, arguments, 1, "Second"); /// Уровень значения, при котором полоска имеет нулевую длину.
Int64 max = extractConstant<Int64>(block, arguments, 2, "Third"); /// Уровень значения, при котором полоска имеет максимальную длину.
/// Максимальная ширина полоски в символах, по-умолчанию.
Float64 max_width = arguments.size() == 4
? extractConstant<Float64>(block, arguments, 3, "Fourth")
: 80;
if (max_width < 1)
throw Exception("Max_width argument must be >= 1.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (max_width > 1000)
throw Exception("Too large max_width.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
const auto & src = *block.getByPosition(arguments[0]).column;
if (src.isConst())
{
auto res_column = new ColumnConstString(block.rowsInFirstColumn(), "");
block.getByPosition(result).column = res_column;
if ( executeConstNumber<UInt8> (src, *res_column, min, max, max_width)
|| executeConstNumber<UInt16> (src, *res_column, min, max, max_width)
|| executeConstNumber<UInt32> (src, *res_column, min, max, max_width)
|| executeConstNumber<UInt64> (src, *res_column, min, max, max_width)
|| executeConstNumber<Int8> (src, *res_column, min, max, max_width)
|| executeConstNumber<Int16> (src, *res_column, min, max, max_width)
|| executeConstNumber<Int32> (src, *res_column, min, max, max_width)
|| executeConstNumber<Int64> (src, *res_column, min, max, max_width)
|| executeConstNumber<Float32> (src, *res_column, min, max, max_width)
|| executeConstNumber<Float64> (src, *res_column, min, max, max_width))
{
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
else
{
auto res_column = new ColumnString;
block.getByPosition(result).column = res_column;
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))
{
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
}
private:
template <typename T>
T extractConstant(Block & block, const ColumnNumbers & arguments, size_t argument_pos, const char * which_argument) const
{
const auto & column = *block.getByPosition(arguments[argument_pos]).column;
if (!column.isConst())
throw Exception(which_argument + String(" argument for function ") + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
return apply_visitor(FieldVisitorConvertToNumber<T>(), column[0]);
}
static constexpr size_t BAR_CHAR_SIZE = strlen("");
template <typename T>
static Float64 barWidth(T x, Int64 min, Int64 max, Float64 max_width)
{
if (x <= min)
return 0;
if (x >= max)
return max_width;
return (x - min) * max_width / (max - min);
}
static size_t barWidthInBytes(Float64 width)
{
return ceil(width - 1.0 / 8) * BAR_CHAR_SIZE;
}
/// В dst должно быть место для barWidthInBytes(width) символов и завершающего нуля.
static void renderBar(Float64 width, char * dst)
{
size_t floor_width = floor(width);
for (size_t i = 0; i < floor_width; ++i)
{
memcpy(dst, "", BAR_CHAR_SIZE);
dst += BAR_CHAR_SIZE;
}
size_t remainder = floor((width - floor_width) * 8);
if (remainder)
{
memcpy(dst, &"▏▎▍▌▋▋▊▉"[(remainder - 1) * BAR_CHAR_SIZE], BAR_CHAR_SIZE);
dst += BAR_CHAR_SIZE;
}
*dst = 0;
}
template <typename T>
static void fill(const PODArray<T> & src, ColumnString::Chars_t & dst_chars, ColumnString::Offsets_t & dst_offsets,
Int64 min, Int64 max, Float64 max_width)
{
size_t size = src.size();
size_t current_offset = 0;
dst_offsets.resize(size);
dst_chars.reserve(size * (barWidthInBytes(max_width) + 1)); /// строки 0-terminated.
for (size_t i = 0; i < size; ++i)
{
Float64 width = barWidth(src[i], min, max, max_width);
size_t next_size = current_offset + barWidthInBytes(width) + 1;
dst_chars.resize(next_size);
renderBar(width, reinterpret_cast<char *>(&dst_chars[current_offset]));
current_offset = next_size;
dst_offsets[i] = current_offset;
}
}
template <typename T>
static void fill(T src, String & dst_chars,
Int64 min, Int64 max, Float64 max_width)
{
Float64 width = barWidth(src, min, max, max_width);
dst_chars.resize(barWidthInBytes(width));
renderBar(width, &dst_chars[0]);
}
template <typename T>
static bool executeNumber(const IColumn & src, ColumnString & dst, Int64 min, Int64 max, Float64 max_width)
{
if (const ColumnVector<T> * col = typeid_cast<const ColumnVector<T> *>(&src))
{
fill(col->getData(), dst.getChars(), dst.getOffsets(), min, max, max_width);
return true;
}
else
return false;
}
template <typename T>
static bool executeConstNumber(const IColumn & src, ColumnConstString & dst, Int64 min, Int64 max, Float64 max_width)
{
if (const ColumnConst<T> * col = typeid_cast<const ColumnConst<T> *>(&src))
{
fill(col->getData(), dst.getData(), min, max, max_width);
return true;
}
else
return false;
}
};
}

View File

@ -181,6 +181,7 @@ FunctionPtr FunctionFactory::get(
else if (name == "hostName") return new FunctionHostName;
else if (name == "visibleWidth") return new FunctionVisibleWidth;
else if (name == "bar") return new FunctionBar;
else if (name == "toTypeName") return new FunctionToTypeName;
else if (name == "blockSize") return new FunctionBlockSize;
else if (name == "sleep") return new FunctionSleep;