mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
dbms: cleanup [#METR-19266]
This commit is contained in:
parent
74ccf71a2a
commit
c4f348619d
@ -165,6 +165,11 @@ public:
|
||||
static constexpr auto name = "visibleWidth";
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionVisibleWidth>(); }
|
||||
|
||||
bool hasSpecialSupportForNulls() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
/// Получить имя функции.
|
||||
String getName() const override
|
||||
{
|
||||
@ -184,6 +189,10 @@ public:
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
|
||||
|
||||
private:
|
||||
/// Internal version of visibleWidth.
|
||||
void perform(Block & block, const ColumnNumbers & arguments, size_t result);
|
||||
};
|
||||
|
||||
|
||||
|
@ -3,8 +3,6 @@
|
||||
|
||||
#include <DB/Functions/FunctionsMiscellaneous.h>
|
||||
#include <DB/DataStreams/PrettyBlockOutputStream.h>
|
||||
#include <DB/Columns/ColumnNullable.h>
|
||||
#include <DB/DataTypes/NullSymbol.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -47,51 +45,20 @@ void PrettyBlockOutputStream::calculateWidths(Block & block, Widths_t & max_widt
|
||||
|
||||
column.column = block.getByPosition(i + columns).column;
|
||||
|
||||
auto has_null_value = [](const ColumnPtr & col, size_t row)
|
||||
{
|
||||
if (col->isNullable())
|
||||
{
|
||||
const ColumnNullable & nullable_col = static_cast<const ColumnNullable &>(*col);
|
||||
if (nullable_col.isNullAt(row))
|
||||
return true;
|
||||
}
|
||||
else if (col->isNull())
|
||||
return true;
|
||||
|
||||
return false;
|
||||
};
|
||||
|
||||
IColumn * observed_col;
|
||||
if (column.column->isNullable())
|
||||
{
|
||||
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(*column.column);
|
||||
observed_col = nullable_col.getNestedColumn().get();
|
||||
}
|
||||
else
|
||||
observed_col = column.column.get();
|
||||
|
||||
if (const ColumnUInt64 * col = typeid_cast<const ColumnUInt64 *>(observed_col))
|
||||
if (const ColumnUInt64 * col = typeid_cast<const ColumnUInt64 *>(&*column.column))
|
||||
{
|
||||
const ColumnUInt64::Container_t & res = col->getData();
|
||||
for (size_t j = 0; j < rows; ++j)
|
||||
{
|
||||
size_t cur_width;
|
||||
if (has_null_value(block.getByPosition(i).column, j))
|
||||
cur_width = NullSymbol::Escaped::length;
|
||||
else
|
||||
cur_width = res[j];
|
||||
|
||||
if (cur_width > max_widths[i])
|
||||
max_widths[i] = cur_width;
|
||||
}
|
||||
if (res[j] > max_widths[i])
|
||||
max_widths[i] = res[j];
|
||||
}
|
||||
else if (const ColumnConstUInt64 * col = typeid_cast<const ColumnConstUInt64 *>(observed_col))
|
||||
else if (const ColumnConstUInt64 * col = typeid_cast<const ColumnConstUInt64 *>(&*column.column))
|
||||
{
|
||||
UInt64 res = col->getData();
|
||||
max_widths[i] = res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + observed_col->getName()
|
||||
throw Exception("Illegal column " + column.column->getName()
|
||||
+ " of result of function " + visible_width_func.getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
|
@ -2,8 +2,6 @@
|
||||
#include <unistd.h>
|
||||
|
||||
#include <DB/DataStreams/PrettyCompactBlockOutputStream.h>
|
||||
#include <DB/DataTypes/NullSymbol.h>
|
||||
#include <DB/Columns/ColumnNullable.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -81,20 +79,6 @@ void PrettyCompactBlockOutputStream::writeRow(
|
||||
const Widths_t & max_widths,
|
||||
const Widths_t & name_widths)
|
||||
{
|
||||
auto has_null_value = [](const ColumnPtr & col, size_t row)
|
||||
{
|
||||
if (col->isNullable())
|
||||
{
|
||||
const ColumnNullable & nullable_col = static_cast<const ColumnNullable &>(*col);
|
||||
if (nullable_col.isNullAt(row))
|
||||
return true;
|
||||
}
|
||||
else if (col->isNull())
|
||||
return true;
|
||||
|
||||
return false;
|
||||
};
|
||||
|
||||
size_t columns = max_widths.size();
|
||||
|
||||
writeCString("│ ", ostr);
|
||||
@ -106,46 +90,19 @@ void PrettyCompactBlockOutputStream::writeRow(
|
||||
|
||||
const ColumnWithTypeAndName & col = block.getByPosition(j);
|
||||
|
||||
size_t width;
|
||||
|
||||
if (has_null_value(col.column, row_id))
|
||||
width = NullSymbol::Escaped::length;
|
||||
else
|
||||
{
|
||||
ColumnPtr res_col = block.getByPosition(columns + j).column;
|
||||
|
||||
IColumn * observed_col;
|
||||
if (res_col->isNullable())
|
||||
{
|
||||
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(*res_col);
|
||||
observed_col = nullable_col.getNestedColumn().get();
|
||||
}
|
||||
else
|
||||
observed_col = res_col.get();
|
||||
|
||||
if (const ColumnUInt64 * concrete_col = typeid_cast<const ColumnUInt64 *>(observed_col))
|
||||
{
|
||||
const ColumnUInt64::Container_t & res = concrete_col->getData();
|
||||
width = res[row_id];
|
||||
}
|
||||
else if (const ColumnConstUInt64 * concrete_col = typeid_cast<const ColumnConstUInt64 *>(observed_col))
|
||||
{
|
||||
UInt64 res = concrete_col->getData();
|
||||
width = res;
|
||||
}
|
||||
else
|
||||
throw Exception{"Illegal column " + observed_col->getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
if (col.type->isNumeric())
|
||||
{
|
||||
size_t width = get<UInt64>((*block.getByPosition(columns + j).column)[row_id]);
|
||||
for (size_t k = 0; k < max_widths[j] - width; ++k)
|
||||
writeChar(' ', ostr);
|
||||
|
||||
col.type->serializeTextEscaped(*col.column.get(), row_id, ostr);
|
||||
}
|
||||
else
|
||||
{
|
||||
col.type->serializeTextEscaped(*col.column.get(), row_id, ostr);
|
||||
|
||||
size_t width = get<UInt64>((*block.getByPosition(columns + j).column)[row_id]);
|
||||
for (size_t k = 0; k < max_widths[j] - width; ++k)
|
||||
writeChar(' ', ostr);
|
||||
}
|
||||
|
@ -4,6 +4,9 @@
|
||||
#include <DB/Functions/FunctionsArithmetic.h>
|
||||
#include <DB/Functions/FunctionsMiscellaneous.h>
|
||||
#include <DB/DataTypes/DataTypeEnum.h>
|
||||
#include <DB/DataTypes/NullSymbol.h>
|
||||
#include <DB/DataTypes/DataTypeNullable.h>
|
||||
#include <DB/Columns/ColumnNullable.h>
|
||||
#include <ext/enumerate.hpp>
|
||||
|
||||
|
||||
@ -173,6 +176,73 @@ namespace VisibleWidth
|
||||
|
||||
|
||||
void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
auto & element = block.getByPosition(arguments[0]);
|
||||
|
||||
auto & res_element = block.getByPosition(result);
|
||||
auto & res_col = res_element.column;
|
||||
|
||||
size_t row_count = block.rowsInFirstColumn();
|
||||
|
||||
if (element.column->isNull())
|
||||
{
|
||||
/// The input column has the Null type.
|
||||
res_col = std::make_shared<ColumnConstUInt64>(row_count, NullSymbol::Escaped::length);
|
||||
}
|
||||
else if (element.column->isNullable())
|
||||
{
|
||||
/// Perform visibleWidth on a block that holds the nested column
|
||||
/// of the input column.
|
||||
auto & nullable_col = static_cast<ColumnNullable &>(*element.column);
|
||||
auto & nested_col = nullable_col.getNestedColumn();
|
||||
|
||||
auto & nullable_type = static_cast<DataTypeNullable &>(*element.type);
|
||||
auto & nested_type = nullable_type.getNestedType();
|
||||
|
||||
Block block_with_nested_col =
|
||||
{
|
||||
{
|
||||
nested_col,
|
||||
nested_type,
|
||||
element.name
|
||||
},
|
||||
|
||||
{
|
||||
nullptr,
|
||||
res_element.type,
|
||||
""
|
||||
}
|
||||
};
|
||||
|
||||
perform(block_with_nested_col, {0, 1}, 1);
|
||||
|
||||
/// Create the result. If any row of the input column holds a NULL value,
|
||||
/// we assign the corresponding row of the result the length of the NULL
|
||||
/// symbol.
|
||||
res_col = std::make_shared<ColumnUInt64>(row_count);
|
||||
auto & res_data = static_cast<ColumnUInt64 &>(*res_col).getData();
|
||||
|
||||
const auto & src = static_cast<const ColumnUInt64 &>(
|
||||
*block_with_nested_col.unsafeGetByPosition(1).column
|
||||
).getData();
|
||||
|
||||
for (size_t row = 0; row < row_count; ++row)
|
||||
{
|
||||
if (nullable_col.isNullAt(row))
|
||||
res_data[row] = NullSymbol::Escaped::length;
|
||||
else
|
||||
res_data[row] = src[row];
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// The input column has an ordinary type.
|
||||
perform(block, arguments, result);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void FunctionVisibleWidth::perform(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
const DataTypePtr type = block.getByPosition(arguments[0]).type;
|
||||
|
Loading…
Reference in New Issue
Block a user