mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Cut dependendy (code must depend on Functions only via FunctionFactory) [#METR-2944].
This commit is contained in:
parent
0eb5d53133
commit
b4e18e2222
@ -7,6 +7,7 @@ namespace DB
|
|||||||
{
|
{
|
||||||
|
|
||||||
class WriteBuffer;
|
class WriteBuffer;
|
||||||
|
class Context;
|
||||||
|
|
||||||
|
|
||||||
/** Выводит результат в виде красивых таблиц.
|
/** Выводит результат в виде красивых таблиц.
|
||||||
@ -15,7 +16,7 @@ class PrettyBlockOutputStream : public IBlockOutputStream
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
/// no_escapes - не использовать ANSI escape sequences - для отображения в браузере, а не в консоли.
|
/// no_escapes - не использовать ANSI escape sequences - для отображения в браузере, а не в консоли.
|
||||||
PrettyBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_);
|
PrettyBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_, const Context & context_);
|
||||||
|
|
||||||
void write(const Block & block) override;
|
void write(const Block & block) override;
|
||||||
void writeSuffix() override;
|
void writeSuffix() override;
|
||||||
@ -36,13 +37,15 @@ protected:
|
|||||||
|
|
||||||
WriteBuffer & ostr;
|
WriteBuffer & ostr;
|
||||||
size_t max_rows;
|
size_t max_rows;
|
||||||
size_t total_rows;
|
size_t total_rows = 0;
|
||||||
size_t terminal_width;
|
size_t terminal_width = 0;
|
||||||
|
|
||||||
bool no_escapes;
|
bool no_escapes;
|
||||||
|
|
||||||
Block totals;
|
Block totals;
|
||||||
Block extremes;
|
Block extremes;
|
||||||
|
|
||||||
|
const Context & context;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -11,8 +11,8 @@ namespace DB
|
|||||||
class PrettyCompactBlockOutputStream : public PrettyBlockOutputStream
|
class PrettyCompactBlockOutputStream : public PrettyBlockOutputStream
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
PrettyCompactBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_)
|
PrettyCompactBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_, const Context & context_)
|
||||||
: PrettyBlockOutputStream(ostr_, no_escapes_, max_rows_) {}
|
: PrettyBlockOutputStream(ostr_, no_escapes_, max_rows_, context_) {}
|
||||||
|
|
||||||
void write(const Block & block) override;
|
void write(const Block & block) override;
|
||||||
|
|
||||||
|
@ -12,8 +12,8 @@ namespace DB
|
|||||||
class PrettyCompactMonoBlockOutputStream : public PrettyCompactBlockOutputStream
|
class PrettyCompactMonoBlockOutputStream : public PrettyCompactBlockOutputStream
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
PrettyCompactMonoBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_)
|
PrettyCompactMonoBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_, const Context & context_)
|
||||||
: PrettyCompactBlockOutputStream(ostr_, no_escapes_, max_rows_) {}
|
: PrettyCompactBlockOutputStream(ostr_, no_escapes_, max_rows_, context_) {}
|
||||||
|
|
||||||
void write(const Block & block) override;
|
void write(const Block & block) override;
|
||||||
void writeSuffix() override;
|
void writeSuffix() override;
|
||||||
|
@ -11,8 +11,8 @@ namespace DB
|
|||||||
class PrettySpaceBlockOutputStream : public PrettyBlockOutputStream
|
class PrettySpaceBlockOutputStream : public PrettyBlockOutputStream
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
PrettySpaceBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_)
|
PrettySpaceBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_, const Context & context_)
|
||||||
: PrettyBlockOutputStream(ostr_, no_escapes_, max_rows_) {}
|
: PrettyBlockOutputStream(ostr_, no_escapes_, max_rows_, context_) {}
|
||||||
|
|
||||||
void write(const Block & block) override;
|
void write(const Block & block) override;
|
||||||
void writeSuffix() override;
|
void writeSuffix() override;
|
||||||
|
@ -9,14 +9,16 @@ namespace DB
|
|||||||
{
|
{
|
||||||
|
|
||||||
class WriteBuffer;
|
class WriteBuffer;
|
||||||
|
class Context;
|
||||||
|
|
||||||
|
|
||||||
/** Поток для вывода данных в формате "каждое значение на своей строке".
|
/** Stream to output data in format "each value in separate row".
|
||||||
|
* Usable to show few rows with many columns.
|
||||||
*/
|
*/
|
||||||
class VerticalRowOutputStream : public IRowOutputStream
|
class VerticalRowOutputStream : public IRowOutputStream
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
VerticalRowOutputStream(WriteBuffer & ostr_, const Block & sample_);
|
VerticalRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const Context & context);
|
||||||
|
|
||||||
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
|
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
|
||||||
void writeRowStartDelimiter() override;
|
void writeRowStartDelimiter() override;
|
||||||
@ -29,22 +31,21 @@ protected:
|
|||||||
|
|
||||||
WriteBuffer & ostr;
|
WriteBuffer & ostr;
|
||||||
const Block sample;
|
const Block sample;
|
||||||
Names names;
|
size_t field_number = 0;
|
||||||
size_t field_number;
|
size_t row_number = 0;
|
||||||
size_t row_number;
|
|
||||||
|
|
||||||
using Pads_t = std::vector<String>;
|
using NamesAndPaddings = std::vector<String>;
|
||||||
Pads_t pads;
|
NamesAndPaddings names_and_paddings;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
/** То же самое, но строки выводятся без экранирования.
|
/** Same but values are printed without escaping.
|
||||||
*/
|
*/
|
||||||
class VerticalRawRowOutputStream : public VerticalRowOutputStream
|
class VerticalRawRowOutputStream final : public VerticalRowOutputStream
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
VerticalRawRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
|
VerticalRawRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const Context & context)
|
||||||
: VerticalRowOutputStream(ostr_, sample_) {}
|
: VerticalRowOutputStream(ostr_, sample_, context) {}
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
void writeValue(const IColumn & column, const IDataType & type, size_t row_num) const override;
|
void writeValue(const IColumn & column, const IDataType & type, size_t row_num) const override;
|
||||||
|
@ -75,24 +75,6 @@ namespace ErrorCodes
|
|||||||
*/
|
*/
|
||||||
|
|
||||||
|
|
||||||
static inline UInt64 stringWidth(const UInt8 * pos, const UInt8 * end)
|
|
||||||
{
|
|
||||||
UInt64 res = 0;
|
|
||||||
for (; pos < end; ++pos)
|
|
||||||
{
|
|
||||||
if (*pos == '\b' || *pos == '\f' || *pos == '\n' || *pos == '\r' || *pos == '\t' || *pos == '\0' || *pos == '\'' || *pos == '\\')
|
|
||||||
++res;
|
|
||||||
if (*pos <= 0x7F || *pos >= 0xC0)
|
|
||||||
++res;
|
|
||||||
}
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
static inline void stringWidthConstant(const String & data, UInt64 & res)
|
|
||||||
{
|
|
||||||
res = stringWidth(reinterpret_cast<const UInt8 *>(data.data()), reinterpret_cast<const UInt8 *>(data.data()) + data.size());
|
|
||||||
}
|
|
||||||
|
|
||||||
class FunctionCurrentDatabase : public IFunction
|
class FunctionCurrentDatabase : public IFunction
|
||||||
{
|
{
|
||||||
const String db_name;
|
const String db_name;
|
||||||
|
@ -120,23 +120,23 @@ static BlockOutputStreamPtr getOutputImpl(const String & name, WriteBuffer & buf
|
|||||||
else if (name == "CSVWithNames")
|
else if (name == "CSVWithNames")
|
||||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<CSVRowOutputStream>(buf, sample, true));
|
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<CSVRowOutputStream>(buf, sample, true));
|
||||||
else if (name == "Pretty")
|
else if (name == "Pretty")
|
||||||
return std::make_shared<PrettyBlockOutputStream>(buf, false, settings.output_format_pretty_max_rows);
|
return std::make_shared<PrettyBlockOutputStream>(buf, false, settings.output_format_pretty_max_rows, context);
|
||||||
else if (name == "PrettyCompact")
|
else if (name == "PrettyCompact")
|
||||||
return std::make_shared<PrettyCompactBlockOutputStream>(buf, false, settings.output_format_pretty_max_rows);
|
return std::make_shared<PrettyCompactBlockOutputStream>(buf, false, settings.output_format_pretty_max_rows, context);
|
||||||
else if (name == "PrettyCompactMonoBlock")
|
else if (name == "PrettyCompactMonoBlock")
|
||||||
return std::make_shared<PrettyCompactMonoBlockOutputStream>(buf, false, settings.output_format_pretty_max_rows);
|
return std::make_shared<PrettyCompactMonoBlockOutputStream>(buf, false, settings.output_format_pretty_max_rows, context);
|
||||||
else if (name == "PrettySpace")
|
else if (name == "PrettySpace")
|
||||||
return std::make_shared<PrettySpaceBlockOutputStream>(buf, false, settings.output_format_pretty_max_rows);
|
return std::make_shared<PrettySpaceBlockOutputStream>(buf, false, settings.output_format_pretty_max_rows, context);
|
||||||
else if (name == "PrettyNoEscapes")
|
else if (name == "PrettyNoEscapes")
|
||||||
return std::make_shared<PrettyBlockOutputStream>(buf, true, settings.output_format_pretty_max_rows);
|
return std::make_shared<PrettyBlockOutputStream>(buf, true, settings.output_format_pretty_max_rows, context);
|
||||||
else if (name == "PrettyCompactNoEscapes")
|
else if (name == "PrettyCompactNoEscapes")
|
||||||
return std::make_shared<PrettyCompactBlockOutputStream>(buf, true, settings.output_format_pretty_max_rows);
|
return std::make_shared<PrettyCompactBlockOutputStream>(buf, true, settings.output_format_pretty_max_rows, context);
|
||||||
else if (name == "PrettySpaceNoEscapes")
|
else if (name == "PrettySpaceNoEscapes")
|
||||||
return std::make_shared<PrettySpaceBlockOutputStream>(buf, true, settings.output_format_pretty_max_rows);
|
return std::make_shared<PrettySpaceBlockOutputStream>(buf, true, settings.output_format_pretty_max_rows, context);
|
||||||
else if (name == "Vertical")
|
else if (name == "Vertical")
|
||||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<VerticalRowOutputStream>(buf, sample));
|
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<VerticalRowOutputStream>(buf, sample, context));
|
||||||
else if (name == "VerticalRaw")
|
else if (name == "VerticalRaw")
|
||||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<VerticalRawRowOutputStream>(buf, sample));
|
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<VerticalRawRowOutputStream>(buf, sample, context));
|
||||||
else if (name == "Values")
|
else if (name == "Values")
|
||||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<ValuesRowOutputStream>(buf));
|
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<ValuesRowOutputStream>(buf));
|
||||||
else if (name == "JSON")
|
else if (name == "JSON")
|
||||||
|
@ -1,15 +1,18 @@
|
|||||||
#include <sys/ioctl.h>
|
#include <sys/ioctl.h>
|
||||||
#include <unistd.h>
|
#include <unistd.h>
|
||||||
|
|
||||||
#include <DB/Functions/FunctionsMiscellaneous.h>
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataStreams/PrettyBlockOutputStream.h>
|
#include <DB/DataStreams/PrettyBlockOutputStream.h>
|
||||||
|
#include <DB/Columns/ColumnConst.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
PrettyBlockOutputStream::PrettyBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_)
|
PrettyBlockOutputStream::PrettyBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_, const Context & context_)
|
||||||
: ostr(ostr_), max_rows(max_rows_), total_rows(0), terminal_width(0), no_escapes(no_escapes_)
|
: ostr(ostr_), max_rows(max_rows_), no_escapes(no_escapes_), context(context_)
|
||||||
{
|
{
|
||||||
struct winsize w;
|
struct winsize w;
|
||||||
if (0 == ioctl(STDOUT_FILENO, TIOCGWINSZ, &w))
|
if (0 == ioctl(STDOUT_FILENO, TIOCGWINSZ, &w))
|
||||||
@ -31,20 +34,19 @@ void PrettyBlockOutputStream::calculateWidths(Block & block, Widths_t & max_widt
|
|||||||
max_widths.resize(columns);
|
max_widths.resize(columns);
|
||||||
name_widths.resize(columns);
|
name_widths.resize(columns);
|
||||||
|
|
||||||
FunctionVisibleWidth visible_width_func;
|
FunctionPtr visible_width_func = FunctionFactory::instance().get("visibleWidth", context);
|
||||||
DataTypePtr visible_width_type = std::make_shared<DataTypeUInt64>();
|
|
||||||
|
|
||||||
/// Вычислим ширину всех значений
|
/// Calculate widths of all values.
|
||||||
for (size_t i = 0; i < columns; ++i)
|
for (size_t i = 0; i < columns; ++i)
|
||||||
{
|
{
|
||||||
ColumnWithTypeAndName column;
|
ColumnWithTypeAndName column;
|
||||||
column.type = visible_width_type;
|
column.type = std::make_shared<DataTypeUInt64>();
|
||||||
column.name = "visibleWidth(" + block.getByPosition(i).name + ")";
|
column.name = "visibleWidth(" + block.unsafeGetByPosition(i).name + ")";
|
||||||
|
|
||||||
size_t result_number = block.columns();
|
size_t result_number = block.columns();
|
||||||
block.insert(column);
|
block.insert(column);
|
||||||
|
|
||||||
visible_width_func.execute(block, {i}, result_number);
|
visible_width_func->execute(block, {i}, result_number);
|
||||||
column.column = block.getByPosition(result_number).column;
|
column.column = block.getByPosition(result_number).column;
|
||||||
|
|
||||||
if (const ColumnUInt64 * col = typeid_cast<const ColumnUInt64 *>(&*column.column))
|
if (const ColumnUInt64 * col = typeid_cast<const ColumnUInt64 *>(&*column.column))
|
||||||
@ -63,13 +65,26 @@ void PrettyBlockOutputStream::calculateWidths(Block & block, Widths_t & max_widt
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + column.column->getName()
|
throw Exception("Illegal column " + column.column->getName()
|
||||||
+ " of result of function " + visible_width_func.getName(),
|
+ " of result of function " + visible_width_func->getName(),
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
/// И не только значений, но и их имён
|
/// And also calculate widths for names of columns.
|
||||||
stringWidthConstant(block.getByPosition(i).name, name_widths[i]);
|
{
|
||||||
if (name_widths[i] > max_widths[i])
|
const String & name = block.unsafeGetByPosition(i).name;
|
||||||
max_widths[i] = name_widths[i];
|
|
||||||
|
Block block_with_name
|
||||||
|
{
|
||||||
|
{ std::make_shared<ColumnConstString>(1, name), std::make_shared<DataTypeString>(), "name" },
|
||||||
|
{ nullptr, std::make_shared<DataTypeUInt64>(), "width" }
|
||||||
|
};
|
||||||
|
|
||||||
|
visible_width_func->execute(block_with_name, {0}, 1);
|
||||||
|
|
||||||
|
name_widths[i] = typeid_cast<const ColumnConstUInt64 &>(*block_with_name.unsafeGetByPosition(1).column).getData();
|
||||||
|
|
||||||
|
if (name_widths[i] > max_widths[i])
|
||||||
|
max_widths[i] = name_widths[i];
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,33 +1,53 @@
|
|||||||
#include <DB/Functions/FunctionsMiscellaneous.h>
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
|
#include <DB/Columns/ColumnConst.h>
|
||||||
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/IO/WriteHelpers.h>
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/VerticalRowOutputStream.h>
|
#include <DB/DataStreams/VerticalRowOutputStream.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
VerticalRowOutputStream::VerticalRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
|
VerticalRowOutputStream::VerticalRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const Context & context)
|
||||||
: ostr(ostr_), sample(sample_), field_number(0), row_number(0)
|
: ostr(ostr_), sample(sample_)
|
||||||
{
|
{
|
||||||
size_t columns = sample.columns();
|
size_t columns = sample.columns();
|
||||||
names.resize(columns);
|
|
||||||
|
|
||||||
using Widths_t = std::vector<size_t>;
|
using Widths_t = std::vector<size_t>;
|
||||||
Widths_t name_widths(columns);
|
Widths_t name_widths(columns);
|
||||||
size_t max_name_width = 0;
|
size_t max_name_width = 0;
|
||||||
|
|
||||||
|
FunctionPtr visible_width_func = FunctionFactory::instance().get("visibleWidth", context);
|
||||||
|
|
||||||
for (size_t i = 0; i < columns; ++i)
|
for (size_t i = 0; i < columns; ++i)
|
||||||
{
|
{
|
||||||
names[i] = sample.getByPosition(i).name;
|
{
|
||||||
stringWidthConstant(names[i], name_widths[i]);
|
Block block_with_name
|
||||||
|
{
|
||||||
|
{ std::make_shared<ColumnConstString>(1, sample.unsafeGetByPosition(i).name), std::make_shared<DataTypeString>(), "name" },
|
||||||
|
{ nullptr, std::make_shared<DataTypeUInt64>(), "width" }
|
||||||
|
};
|
||||||
|
|
||||||
|
visible_width_func->execute(block_with_name, {0}, 1);
|
||||||
|
|
||||||
|
name_widths[i] = (*block_with_name.unsafeGetByPosition(1).column)[0].get<UInt64>();
|
||||||
|
}
|
||||||
|
|
||||||
if (name_widths[i] > max_name_width)
|
if (name_widths[i] > max_name_width)
|
||||||
max_name_width = name_widths[i];
|
max_name_width = name_widths[i];
|
||||||
}
|
}
|
||||||
|
|
||||||
pads.resize(columns);
|
names_and_paddings.resize(columns);
|
||||||
for (size_t i = 0; i < columns; ++i)
|
for (size_t i = 0; i < columns; ++i)
|
||||||
pads[i] = String(max_name_width - name_widths[i], ' ');
|
{
|
||||||
|
WriteBufferFromString out(names_and_paddings[i]);
|
||||||
|
writeEscapedString(sample.unsafeGetByPosition(i).name, out);
|
||||||
|
writeCString(": ", out);
|
||||||
|
}
|
||||||
|
|
||||||
|
for (size_t i = 0; i < columns; ++i)
|
||||||
|
names_and_paddings[i].resize(max_name_width + strlen(": "), ' ');
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -39,13 +59,10 @@ void VerticalRowOutputStream::flush()
|
|||||||
|
|
||||||
void VerticalRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num)
|
void VerticalRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num)
|
||||||
{
|
{
|
||||||
writeEscapedString(names[field_number], ostr);
|
writeString(names_and_paddings[field_number], ostr);
|
||||||
writeCString(": ", ostr);
|
|
||||||
writeString(pads[field_number], ostr);
|
|
||||||
|
|
||||||
writeValue(column, type, row_num);
|
writeValue(column, type, row_num);
|
||||||
|
|
||||||
writeChar('\n', ostr);
|
writeChar('\n', ostr);
|
||||||
|
|
||||||
++field_number;
|
++field_number;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user