mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
dbms: Server: Feature development. [#METR-19266]
This commit is contained in:
parent
f52403f623
commit
4ff528140a
@ -171,6 +171,11 @@ public:
|
|||||||
return name;
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool hasSpecialSupportForNullValues() const override
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
{
|
{
|
||||||
|
@ -2,8 +2,9 @@
|
|||||||
#include <unistd.h>
|
#include <unistd.h>
|
||||||
|
|
||||||
#include <DB/Functions/FunctionsMiscellaneous.h>
|
#include <DB/Functions/FunctionsMiscellaneous.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/PrettyBlockOutputStream.h>
|
#include <DB/DataStreams/PrettyBlockOutputStream.h>
|
||||||
|
#include <DB/Columns/ColumnNullable.h>
|
||||||
|
#include <DB/DataTypes/NullSymbol.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -46,20 +47,39 @@ void PrettyBlockOutputStream::calculateWidths(Block & block, Widths_t & max_widt
|
|||||||
|
|
||||||
column.column = block.getByPosition(i + columns).column;
|
column.column = block.getByPosition(i + columns).column;
|
||||||
|
|
||||||
if (const ColumnUInt64 * col = typeid_cast<const ColumnUInt64 *>(&*column.column))
|
IColumn * observed_col;
|
||||||
|
if (column.column.get()->isNullable())
|
||||||
|
{
|
||||||
|
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(*(column.column.get()));
|
||||||
|
observed_col = nullable_col.getNestedColumn().get();
|
||||||
|
}
|
||||||
|
else
|
||||||
|
observed_col = column.column.get();
|
||||||
|
|
||||||
|
if (const ColumnUInt64 * col = typeid_cast<const ColumnUInt64 *>(observed_col))
|
||||||
{
|
{
|
||||||
const ColumnUInt64::Container_t & res = col->getData();
|
const ColumnUInt64::Container_t & res = col->getData();
|
||||||
for (size_t j = 0; j < rows; ++j)
|
for (size_t j = 0; j < rows; ++j)
|
||||||
if (res[j] > max_widths[i])
|
{
|
||||||
max_widths[i] = res[j];
|
Field f = (*(block.getByPosition(columns + i).column))[j];
|
||||||
|
|
||||||
|
size_t cur_width;
|
||||||
|
if (f.isNull())
|
||||||
|
cur_width = NullSymbol::Escaped::length;
|
||||||
|
else
|
||||||
|
cur_width = res[j];
|
||||||
|
|
||||||
|
if (cur_width > max_widths[i])
|
||||||
|
max_widths[i] = cur_width;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
else if (const ColumnConstUInt64 * col = typeid_cast<const ColumnConstUInt64 *>(&*column.column))
|
else if (const ColumnConstUInt64 * col = typeid_cast<const ColumnConstUInt64 *>(observed_col))
|
||||||
{
|
{
|
||||||
UInt64 res = col->getData();
|
UInt64 res = col->getData();
|
||||||
max_widths[i] = res;
|
max_widths[i] = res;
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + column.column->getName()
|
throw Exception("Illegal column " + observed_col->getName()
|
||||||
+ " of result of function " + visible_width_func.getName(),
|
+ " of result of function " + visible_width_func.getName(),
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
#include <DB/Functions/FunctionsArithmetic.h>
|
#include <DB/Functions/FunctionsArithmetic.h>
|
||||||
#include <DB/Functions/FunctionsMiscellaneous.h>
|
#include <DB/Functions/FunctionsMiscellaneous.h>
|
||||||
#include <DB/DataTypes/DataTypeEnum.h>
|
#include <DB/DataTypes/DataTypeEnum.h>
|
||||||
|
#include <DB/Columns/ColumnNull.h>
|
||||||
#include <ext/enumerate.hpp>
|
#include <ext/enumerate.hpp>
|
||||||
|
|
||||||
|
|
||||||
@ -368,6 +369,8 @@ void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & argu
|
|||||||
*/
|
*/
|
||||||
block.getByPosition(result).column = std::make_shared<ColumnConstUInt64>(rows, 10);
|
block.getByPosition(result).column = std::make_shared<ColumnConstUInt64>(rows, 10);
|
||||||
}
|
}
|
||||||
|
else if (typeid_cast<const ColumnNull *>(&*column) != nullptr)
|
||||||
|
block.getByPosition(result).column = std::make_shared<ColumnConstUInt64>(rows, strlen_constexpr("NULL"));
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of argument of function " + getName(),
|
+ " of argument of function " + getName(),
|
||||||
|
Loading…
Reference in New Issue
Block a user