Merge branch 'ClickHouse:master' into fix_in_operator_type_conversion

This commit is contained in:
pn 2024-06-18 14:45:55 +08:00 committed by GitHub
commit e3c7085ff7
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
44 changed files with 3035 additions and 101 deletions

View File

@ -89,10 +89,6 @@ function configure()
# since we run clickhouse from root
sudo chown root: /var/lib/clickhouse
# Set more frequent update period of asynchronous metrics to more frequently update information about real memory usage (less chance of OOM).
echo "<clickhouse><asynchronous_metrics_update_period_s>1</asynchronous_metrics_update_period_s></clickhouse>" \
> /etc/clickhouse-server/config.d/asynchronous_metrics_update_period_s.xml
local total_mem
total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB
total_mem=$(( total_mem*1024 )) # bytes

View File

@ -1490,6 +1490,8 @@ Differs from [PrettySpaceNoEscapes](#prettyspacenoescapes) in that up to 10,000
- [output_format_pretty_color](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_color) - use ANSI escape sequences to paint colors in Pretty formats. Default value - `true`.
- [output_format_pretty_grid_charset](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_grid_charset) - Charset for printing grid borders. Available charsets: ASCII, UTF-8. Default value - `UTF-8`.
- [output_format_pretty_row_numbers](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_row_numbers) - Add row numbers before each row for pretty output format. Default value - `true`.
- [output_format_pretty_display_footer_column_names](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names) - Display column names in the footer if table contains many rows. Default value - `true`.
- [output_format_pretty_display_footer_column_names_min_rows](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names_min_rows) - Sets the minimum number of rows for which a footer will be displayed if [output_format_pretty_display_footer_column_names](/docs/en/operations/settings/settings-formats.md/#output_format_pretty_display_footer_column_names) is enabled. Default value - 50.
## RowBinary {#rowbinary}

View File

@ -1706,6 +1706,43 @@ Result:
└────────────┘
```
## output_format_pretty_display_footer_column_names
Display column names in the footer if there are many table rows.
Possible values:
- 0 — No column names are displayed in the footer.
- 1 — Column names are displayed in the footer if row count is greater than or equal to the threshold value set by [output_format_pretty_display_footer_column_names_min_rows](#output_format_pretty_display_footer_column_names_min_rows) (50 by default).
Default value: `1`.
**Example**
Query:
```sql
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 1000);
```
Result:
```response
┌─number─┬─toTypeName(number)─┐
1. │ 0 │ UInt64 │
2. │ 1 │ UInt64 │
3. │ 2 │ UInt64 │
...
999. │ 998 │ UInt64 │
1000. │ 999 │ UInt64 │
└─number─┴─toTypeName(number)─┘
```
## output_format_pretty_display_footer_column_names_min_rows
Sets the minimum number of rows for which a footer with column names will be displayed if setting [output_format_pretty_display_footer_column_names](#output_format_pretty_display_footer_column_names) is enabled.
Default value: `50`.
## Template format settings {#template-format-settings}
### format_template_resultset {#format_template_resultset}

View File

@ -0,0 +1,265 @@
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <Core/ServerSettings.h>
#include <Core/ColumnWithTypeAndName.h>
#include <Common/ArenaAllocator.h>
#include <Common/assert_cast.h>
#include <Interpreters/castColumn.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
}
namespace
{
struct GroupConcatDataBase
{
UInt64 data_size = 0;
UInt64 allocated_size = 0;
char * data = nullptr;
void checkAndUpdateSize(UInt64 add, Arena * arena)
{
if (data_size + add >= allocated_size)
{
auto old_size = allocated_size;
allocated_size = std::max(2 * allocated_size, data_size + add);
data = arena->realloc(data, old_size, allocated_size);
}
}
void insertChar(const char * str, UInt64 str_size, Arena * arena)
{
checkAndUpdateSize(str_size, arena);
memcpy(data + data_size, str, str_size);
data_size += str_size;
}
};
struct GroupConcatData : public GroupConcatDataBase
{
using Offset = UInt64;
using Allocator = MixedAlignedArenaAllocator<alignof(Offset), 4096>;
using Offsets = PODArray<Offset, 32, Allocator>;
/// offset[i * 2] - beginning of the i-th row, offset[i * 2 + 1] - end of the i-th row
Offsets offsets;
UInt64 num_rows = 0;
UInt64 getSize(size_t i) const { return offsets[i * 2 + 1] - offsets[i * 2]; }
UInt64 getString(size_t i) const { return offsets[i * 2]; }
void insert(const IColumn * column, const SerializationPtr & serialization, size_t row_num, Arena * arena)
{
WriteBufferFromOwnString buff;
serialization->serializeText(*column, row_num, buff, {});
auto string = buff.stringView();
checkAndUpdateSize(string.size(), arena);
memcpy(data + data_size, string.data(), string.size());
offsets.push_back(data_size, arena);
data_size += string.size();
offsets.push_back(data_size, arena);
num_rows++;
}
};
template <bool has_limit>
class GroupConcatImpl final
: public IAggregateFunctionDataHelper<GroupConcatData, GroupConcatImpl<has_limit>>
{
static constexpr auto name = "groupConcat";
SerializationPtr serialization;
UInt64 limit;
const String delimiter;
public:
GroupConcatImpl(const DataTypePtr & data_type_, const Array & parameters_, UInt64 limit_, const String & delimiter_)
: IAggregateFunctionDataHelper<GroupConcatData, GroupConcatImpl<has_limit>>(
{data_type_}, parameters_, std::make_shared<DataTypeString>())
, serialization(this->argument_types[0]->getDefaultSerialization())
, limit(limit_)
, delimiter(delimiter_)
{
}
String getName() const override { return name; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
auto & cur_data = this->data(place);
if constexpr (has_limit)
if (cur_data.num_rows >= limit)
return;
if (cur_data.data_size != 0)
cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena);
cur_data.insert(columns[0], serialization, row_num, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & cur_data = this->data(place);
auto & rhs_data = this->data(rhs);
if (rhs_data.data_size == 0)
return;
if constexpr (has_limit)
{
UInt64 new_elems_count = std::min(rhs_data.num_rows, limit - cur_data.num_rows);
for (UInt64 i = 0; i < new_elems_count; ++i)
{
if (cur_data.data_size != 0)
cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena);
cur_data.offsets.push_back(cur_data.data_size, arena);
cur_data.insertChar(rhs_data.data + rhs_data.getString(i), rhs_data.getSize(i), arena);
cur_data.num_rows++;
cur_data.offsets.push_back(cur_data.data_size, arena);
}
}
else
{
if (cur_data.data_size != 0)
cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena);
cur_data.insertChar(rhs_data.data, rhs_data.data_size, arena);
}
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
auto & cur_data = this->data(place);
writeVarUInt(cur_data.data_size, buf);
writeVarUInt(cur_data.allocated_size, buf);
buf.write(cur_data.data, cur_data.data_size);
if constexpr (has_limit)
{
writeVarUInt(cur_data.num_rows, buf);
for (const auto & offset : cur_data.offsets)
writeVarUInt(offset, buf);
}
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
auto & cur_data = this->data(place);
readVarUInt(cur_data.data_size, buf);
readVarUInt(cur_data.allocated_size, buf);
buf.readStrict(cur_data.data, cur_data.data_size);
if constexpr (has_limit)
{
readVarUInt(cur_data.num_rows, buf);
cur_data.offsets.resize_exact(cur_data.num_rows * 2, arena);
for (auto & offset : cur_data.offsets)
readVarUInt(offset, buf);
}
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & cur_data = this->data(place);
if (cur_data.data_size == 0)
{
auto column_nullable = IColumn::mutate(makeNullable(to.getPtr()));
column_nullable->insertDefault();
return;
}
auto & column_string = assert_cast<ColumnString &>(to);
column_string.insertData(cur_data.data, cur_data.data_size);
}
bool allocatesMemoryInArena() const override { return true; }
};
AggregateFunctionPtr createAggregateFunctionGroupConcat(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);
bool has_limit = false;
UInt64 limit = 0;
String delimiter;
if (parameters.size() > 2)
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION,
"Incorrect number of parameters for aggregate function {}, should be 0, 1 or 2, got: {}", name, parameters.size());
if (!parameters.empty())
{
auto type = parameters[0].getType();
if (type != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First parameter for aggregate function {} should be string", name);
delimiter = parameters[0].get<String>();
}
if (parameters.size() == 2)
{
auto type = parameters[1].getType();
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number", name);
if ((type == Field::Types::Int64 && parameters[1].get<Int64>() <= 0) ||
(type == Field::Types::UInt64 && parameters[1].get<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number, got: {}", name, parameters[1].get<Int64>());
has_limit = true;
limit = parameters[1].get<UInt64>();
}
if (has_limit)
return std::make_shared<GroupConcatImpl</* has_limit= */ true>>(argument_types[0], parameters, limit, delimiter);
else
return std::make_shared<GroupConcatImpl</* has_limit= */ false>>(argument_types[0], parameters, limit, delimiter);
}
}
void registerAggregateFunctionGroupConcat(AggregateFunctionFactory & factory)
{
AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true };
factory.registerFunction("groupConcat", { createAggregateFunctionGroupConcat, properties });
factory.registerAlias("group_concat", "groupConcat", AggregateFunctionFactory::CaseInsensitive);
}
}

View File

@ -19,6 +19,7 @@ void registerAggregateFunctionGroupArraySorted(AggregateFunctionFactory & factor
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory &);
void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &);
void registerAggregateFunctionGroupArrayIntersect(AggregateFunctionFactory &);
void registerAggregateFunctionGroupConcat(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantile(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory &);
@ -120,6 +121,7 @@ void registerAggregateFunctions()
registerAggregateFunctionGroupUniqArray(factory);
registerAggregateFunctionGroupArrayInsertAt(factory);
registerAggregateFunctionGroupArrayIntersect(factory);
registerAggregateFunctionGroupConcat(factory);
registerAggregateFunctionsQuantile(factory);
registerAggregateFunctionsQuantileDeterministic(factory);
registerAggregateFunctionsQuantileExact(factory);

View File

@ -1145,6 +1145,8 @@ class IColumn;
M(UInt64, output_format_pretty_max_value_width_apply_for_single_value, false, "Only cut values (see the `output_format_pretty_max_value_width` setting) when it is not a single value in a block. Otherwise output it entirely, which is useful for the `SHOW CREATE TABLE` query.", 0) \
M(UInt64Auto, output_format_pretty_color, "auto", "Use ANSI escape sequences in Pretty formats. 0 - disabled, 1 - enabled, 'auto' - enabled if a terminal.", 0) \
M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \
M(UInt64, output_format_pretty_display_footer_column_names, true, "Display column names in the footer if there are 999 or more rows.", 0) \
M(UInt64, output_format_pretty_display_footer_column_names_min_rows, 50, "Sets the minimum threshold value of rows for which to enable displaying column names in the footer. 50 (default)", 0) \
M(UInt64, output_format_parquet_row_group_size, 1000000, "Target row group size in rows.", 0) \
M(UInt64, output_format_parquet_row_group_size_bytes, 512 * 1024 * 1024, "Target row group size in bytes, before compression.", 0) \
M(Bool, output_format_parquet_string_as_string, true, "Use Parquet String type instead of Binary for String columns.", 0) \

View File

@ -108,6 +108,8 @@ static const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges
{"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"},
{"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"},
{"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"},
{"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."},
{"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."},
{"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."},
{"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."},
{"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."},

View File

@ -175,8 +175,7 @@ Columns DirectDictionary<dictionary_key_type>::getColumns(
if (!mask_filled)
(*default_mask)[requested_key_index] = 1;
Field value{};
result_column->insert(value);
result_column->insertDefault();
}
else
{

View File

@ -181,6 +181,8 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
format_settings.pretty.highlight_digit_groups = settings.output_format_pretty_highlight_digit_groups;
format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers;
format_settings.pretty.output_format_pretty_single_large_number_tip_threshold = settings.output_format_pretty_single_large_number_tip_threshold;
format_settings.pretty.output_format_pretty_display_footer_column_names = settings.output_format_pretty_display_footer_column_names;
format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows = settings.output_format_pretty_display_footer_column_names_min_rows;
format_settings.protobuf.input_flatten_google_wrappers = settings.input_format_protobuf_flatten_google_wrappers;
format_settings.protobuf.output_nullables_with_google_wrappers = settings.output_format_protobuf_nullables_with_google_wrappers;
format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference;

View File

@ -289,6 +289,8 @@ struct FormatSettings
bool output_format_pretty_row_numbers = false;
UInt64 output_format_pretty_single_large_number_tip_threshold = 1'000'000;
UInt64 output_format_pretty_display_footer_column_names = 1;
UInt64 output_format_pretty_display_footer_column_names_min_rows = 50;
enum class Charset : uint8_t
{

View File

@ -116,6 +116,12 @@ struct GridSymbols
const char * dash = "";
const char * bold_bar = "";
const char * bar = "";
const char * bold_right_separator_footer = "";
const char * bold_left_separator_footer = "";
const char * bold_middle_separator_footer = "";
const char * bold_left_bottom_corner = "";
const char * bold_right_bottom_corner = "";
const char * bold_bottom_separator = "";
};
GridSymbols utf8_grid_symbols;
@ -182,20 +188,23 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
Widths name_widths;
calculateWidths(header, chunk, widths, max_widths, name_widths);
const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ?
utf8_grid_symbols :
ascii_grid_symbols;
const GridSymbols & grid_symbols
= format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : ascii_grid_symbols;
/// Create separators
WriteBufferFromOwnString top_separator;
WriteBufferFromOwnString middle_names_separator;
WriteBufferFromOwnString middle_values_separator;
WriteBufferFromOwnString bottom_separator;
WriteBufferFromOwnString footer_top_separator;
WriteBufferFromOwnString footer_bottom_separator;
top_separator << grid_symbols.bold_left_top_corner;
middle_names_separator << grid_symbols.bold_left_separator;
middle_values_separator << grid_symbols.left_separator;
bottom_separator << grid_symbols.left_bottom_corner;
footer_top_separator << grid_symbols.bold_left_separator_footer;
footer_bottom_separator << grid_symbols.bold_left_bottom_corner;
for (size_t i = 0; i < num_columns; ++i)
{
if (i != 0)
@ -204,6 +213,8 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
middle_names_separator << grid_symbols.bold_middle_separator;
middle_values_separator << grid_symbols.middle_separator;
bottom_separator << grid_symbols.bottom_separator;
footer_top_separator << grid_symbols.bold_middle_separator_footer;
footer_bottom_separator << grid_symbols.bold_bottom_separator;
}
for (size_t j = 0; j < max_widths[i] + 2; ++j)
@ -212,17 +223,23 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
middle_names_separator << grid_symbols.bold_dash;
middle_values_separator << grid_symbols.dash;
bottom_separator << grid_symbols.dash;
footer_top_separator << grid_symbols.bold_dash;
footer_bottom_separator << grid_symbols.bold_dash;
}
}
top_separator << grid_symbols.bold_right_top_corner << "\n";
middle_names_separator << grid_symbols.bold_right_separator << "\n";
middle_values_separator << grid_symbols.right_separator << "\n";
bottom_separator << grid_symbols.right_bottom_corner << "\n";
footer_top_separator << grid_symbols.bold_right_separator_footer << "\n";
footer_bottom_separator << grid_symbols.bold_right_bottom_corner << "\n";
std::string top_separator_s = top_separator.str();
std::string middle_names_separator_s = middle_names_separator.str();
std::string middle_values_separator_s = middle_values_separator.str();
std::string bottom_separator_s = bottom_separator.str();
std::string footer_top_separator_s = footer_top_separator.str();
std::string footer_bottom_separator_s = footer_bottom_separator.str();
if (format_settings.pretty.output_format_pretty_row_numbers)
{
@ -239,6 +256,8 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
}
/// Names
auto write_names = [&]() -> void
{
writeCString(grid_symbols.bold_bar, out);
writeCString(" ", out);
for (size_t i = 0; i < num_columns; ++i)
@ -276,6 +295,8 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
writeCString(" ", out);
writeCString(grid_symbols.bold_bar, out);
writeCString("\n", out);
};
write_names();
if (format_settings.pretty.output_format_pretty_row_numbers)
{
@ -317,9 +338,15 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
if (j != 0)
writeCString(grid_symbols.bar, out);
const auto & type = *header.getByPosition(j).type;
writeValueWithPadding(*columns[j], *serializations[j], i,
writeValueWithPadding(
*columns[j],
*serializations[j],
i,
widths[j].empty() ? max_widths[j] : widths[j][i],
max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type));
max_widths[j],
cut_to_width,
type.shouldAlignRightInPrettyFormats(),
isNumber(type));
}
writeCString(grid_symbols.bar, out);
@ -332,8 +359,33 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
/// Write left blank
writeString(String(row_number_width, ' '), out);
}
writeString(bottom_separator_s, out);
/// output column names in the footer
if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names)
{
writeString(footer_top_separator_s, out);
if (format_settings.pretty.output_format_pretty_row_numbers)
{
/// Write left blank
writeString(String(row_number_width, ' '), out);
}
/// output header names
write_names();
if (format_settings.pretty.output_format_pretty_row_numbers)
{
/// Write left blank
writeString(String(row_number_width, ' '), out);
}
writeString(footer_bottom_separator_s, out);
}
else
{
writeString(bottom_separator_s, out);
}
total_rows += num_rows;
}

View File

@ -57,7 +57,8 @@ PrettyCompactBlockOutputFormat::PrettyCompactBlockOutputFormat(WriteBuffer & out
void PrettyCompactBlockOutputFormat::writeHeader(
const Block & block,
const Widths & max_widths,
const Widths & name_widths)
const Widths & name_widths,
const bool write_footer)
{
if (format_settings.pretty.output_format_pretty_row_numbers)
{
@ -70,6 +71,9 @@ void PrettyCompactBlockOutputFormat::writeHeader(
ascii_grid_symbols;
/// Names
if (write_footer)
writeCString(grid_symbols.left_bottom_corner, out);
else
writeCString(grid_symbols.left_top_corner, out);
writeCString(grid_symbols.dash, out);
for (size_t i = 0; i < max_widths.size(); ++i)
@ -77,6 +81,9 @@ void PrettyCompactBlockOutputFormat::writeHeader(
if (i != 0)
{
writeCString(grid_symbols.dash, out);
if (write_footer)
writeCString(grid_symbols.bottom_separator, out);
else
writeCString(grid_symbols.top_separator, out);
writeCString(grid_symbols.dash, out);
}
@ -107,6 +114,9 @@ void PrettyCompactBlockOutputFormat::writeHeader(
}
}
writeCString(grid_symbols.dash, out);
if (write_footer)
writeCString(grid_symbols.right_bottom_corner, out);
else
writeCString(grid_symbols.right_top_corner, out);
writeCString("\n", out);
}
@ -195,13 +205,19 @@ void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind po
Widths name_widths;
calculateWidths(header, chunk, widths, max_widths, name_widths);
writeHeader(header, max_widths, name_widths);
writeHeader(header, max_widths, name_widths, false);
for (size_t i = 0; i < num_rows && total_rows + i < max_rows; ++i)
writeRow(i, header, chunk, widths, max_widths);
if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names)
{
writeHeader(header, max_widths, name_widths, true);
}
else
{
writeBottom(max_widths);
}
total_rows += num_rows;
}

View File

@ -17,7 +17,7 @@ public:
String getName() const override { return "PrettyCompactBlockOutputFormat"; }
private:
void writeHeader(const Block & block, const Widths & max_widths, const Widths & name_widths);
void writeHeader(const Block & block, const Widths & max_widths, const Widths & name_widths, bool write_footer);
void writeBottom(const Widths & max_widths);
void writeRow(
size_t row_num,

View File

@ -36,6 +36,8 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port
if (format_settings.pretty.output_format_pretty_row_numbers)
writeString(String(row_number_width, ' '), out);
/// Names
auto write_names = [&](const bool is_footer) -> void
{
for (size_t i = 0; i < num_columns; ++i)
{
if (i != 0)
@ -68,7 +70,12 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port
writeChar(' ', out);
}
}
if (!is_footer)
writeCString("\n\n", out);
else
writeCString("\n", out);
};
write_names(false);
for (size_t row = 0; row < num_rows && total_rows + row < max_rows; ++row)
{
@ -95,11 +102,19 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port
writeValueWithPadding(
*columns[column], *serializations[column], row, cur_width, max_widths[column], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type));
}
writeReadableNumberTip(chunk);
writeChar('\n', out);
}
/// Write blank line between last row and footer
if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names)
writeCString("\n", out);
/// Write left blank
if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_row_numbers && format_settings.pretty.output_format_pretty_display_footer_column_names)
writeString(String(row_number_width, ' '), out);
/// Write footer
if ((num_rows >= format_settings.pretty.output_format_pretty_display_footer_column_names_min_rows) && format_settings.pretty.output_format_pretty_display_footer_column_names)
write_names(true);
total_rows += num_rows;
}

View File

@ -9,3 +9,5 @@
01287_max_execution_speed
# Check after ConstantNode refactoring
02154_parser_backtracking
02944_variant_as_common_type
02942_variant_cast

View File

@ -512,6 +512,7 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried(
), error
@pytest.mark.skip(reason="test is flaky, waiting ClickHouse/issues/64451")
def test_query_is_canceled_with_inf_retries(cluster, broken_s3):
node = cluster.instances["node_with_inf_s3_retries"]

View File

@ -1,4 +1,5 @@
SET output_format_pretty_color=1;
SET output_format_pretty_display_footer_column_names=0;
SELECT 1 FORMAT PrettySpace;
SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;
SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SET output_format_pretty_color = 0;
SHOW SETTING output_format_pretty_color;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SET output_format_pretty_color = 1;
SELECT number AS hello, toString(number) AS world, (hello, world) AS tuple, nullIf(hello % 3, 0) AS sometimes_nulls FROM system.numbers LIMIT 10 SETTINGS max_block_size = 5 FORMAT Pretty;

View File

@ -1,4 +1,4 @@
SET output_format_pretty_color=1, output_format_pretty_highlight_digit_groups=0;
SET output_format_pretty_color=1, output_format_pretty_highlight_digit_groups=0, output_format_pretty_display_footer_column_names=0;
SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT Pretty;
SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettyCompact;
SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettySpace;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SELECT
s,
parseDateTimeBestEffortOrNull(s, 'UTC') AS a,

View File

@ -1,2 +1,3 @@
SET output_format_pretty_display_footer_column_names=0;
SET output_format_pretty_max_column_pad_width = 250;
SELECT range(number) FROM system.numbers LIMIT 100 FORMAT PrettyCompactNoEscapes;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SELECT
s,
parseDateTimeBestEffortOrNull(s, 'UTC') AS a,

View File

@ -1,4 +1,5 @@
SET output_format_pretty_color = 1, output_format_pretty_max_value_width_apply_for_single_value = 1, output_format_pretty_row_numbers = 0;
SET output_format_pretty_display_footer_column_names=0;
SELECT 'привет' AS x, 'мир' AS y FORMAT Pretty;
SET output_format_pretty_max_value_width = 5;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SELECT 'parseDateTimeBestEffortUS';
SELECT

View File

@ -1,5 +1,6 @@
SET output_format_pretty_color=1;
SET output_format_pretty_row_numbers=0;
SET output_format_pretty_display_footer_column_names=0;
SELECT * FROM numbers(10) FORMAT Pretty;
SELECT * FROM numbers(10) FORMAT PrettyCompact;
SELECT * FROM numbers(10) FORMAT PrettyCompactMonoBlock;

View File

@ -1,3 +1,4 @@
set output_format_pretty_display_footer_column_names=0;
set output_format_write_statistics=0;
select * from numbers(100) settings max_result_rows = 1; -- { serverError TOO_MANY_ROWS_OR_BYTES }

View File

@ -3,6 +3,6 @@
'PrettySpaceNoEscapesMonoBlock'] -%}
select '{{ format }}';
select number as x, number + 1 as y from numbers(4) settings max_block_size=2, output_format_pretty_color=1 format {{ format }};
select number as x, number + 1 as y from numbers(4) settings max_block_size=2, output_format_pretty_color=1, output_format_pretty_display_footer_column_names=0 format {{ format }};
{% endfor -%}

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SELECT 'parseDateTime64BestEffortUS';
SELECT

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SET output_format_pretty_color=1;
SET read_in_order_two_level_merge_threshold=1000000;

View File

@ -1,5 +1,4 @@
set allow_experimental_variant_type=1;
set allow_experimental_analyzer=0; -- It's currently doesn't work with analyzer because of the way it works with constants, but it will be refactored and fixed in future
select NULL::Variant(String, UInt64);
select 42::UInt64::Variant(String, UInt64);

View File

@ -1,5 +1,3 @@
set allow_experimental_analyzer=0; -- The result type for if function with constant is different with analyzer. It wil be fixed after refactoring around constants in analyzer.
set allow_experimental_variant_type=1;
set use_variant_as_common_type=1;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SELECT 1_000_000 as a FORMAT Pretty;
SELECT 1_000_000 as a FORMAT PrettyNoEscapes;
SELECT 1_000_000 as a FORMAT PrettyMonoBlock;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SET output_format_pretty_row_numbers = 0;
SELECT exp10(number) * (number % 2 ? 1 : -1) FROM numbers(30) FORMAT PrettySpace SETTINGS output_format_pretty_color = 1;

View File

@ -0,0 +1,14 @@
0 95 abc [1,2,3]
1 \N a [993,986,979,972]
2 123 makson95 []
95123
abcamakson95
[1,2,3][993,986,979,972][]
95,123
abc,a,makson95
[1,2,3],[993,986,979,972]
\N
951239512395123
abc,a,makson95,abc,a,makson95,abc,a,makson95
[1,2,3][993,986,979,972][][1,2,3][993,986,979,972][][1,2,3][993,986,979,972][]
488890

View File

@ -0,0 +1,40 @@
DROP TABLE IF EXISTS test_groupConcat;
CREATE TABLE test_groupConcat
(
id UInt64,
p_int Int32 NULL,
p_string String,
p_array Array(Int32)
) ENGINE = MergeTree ORDER BY id;
SET max_insert_threads = 1, max_threads = 1, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
INSERT INTO test_groupConcat VALUES (0, 95, 'abc', [1, 2, 3]), (1, NULL, 'a', [993, 986, 979, 972]), (2, 123, 'makson95', []);
SELECT * FROM test_groupConcat;
SELECT groupConcat(p_int) FROM test_groupConcat;
SELECT groupConcat(p_string) FROM test_groupConcat;
SELECT groupConcat(p_array) FROM test_groupConcat;
SELECT groupConcat(',')(p_int) FROM test_groupConcat;
SELECT groupConcat(',')(p_string) FROM test_groupConcat;
SELECT groupConcat(',', 2)(p_array) FROM test_groupConcat;
SELECT groupConcat(p_int) FROM test_groupConcat WHERE id = 1;
INSERT INTO test_groupConcat VALUES (0, 95, 'abc', [1, 2, 3]), (1, NULL, 'a', [993, 986, 979, 972]), (2, 123, 'makson95', []);
INSERT INTO test_groupConcat VALUES (0, 95, 'abc', [1, 2, 3]), (1, NULL, 'a', [993, 986, 979, 972]), (2, 123, 'makson95', []);
SELECT groupConcat(p_int) FROM test_groupConcat;
SELECT groupConcat(',')(p_string) FROM test_groupConcat;
SELECT groupConcat(p_array) FROM test_groupConcat;
SELECT groupConcat(123)(number) FROM numbers(10); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT groupConcat(',', '3')(number) FROM numbers(10); -- { serverError BAD_ARGUMENTS }
SELECT groupConcat(',', 0)(number) FROM numbers(10); -- { serverError BAD_ARGUMENTS }
SELECT groupConcat(',', -1)(number) FROM numbers(10); -- { serverError BAD_ARGUMENTS }
SELECT groupConcat(',', 3, 3)(number) FROM numbers(10); -- { serverError TOO_MANY_ARGUMENTS_FOR_FUNCTION }
SELECT length(groupConcat(number)) FROM numbers(100000);
DROP TABLE IF EXISTS test_groupConcat;

View File

@ -1,3 +1,4 @@
SET output_format_pretty_display_footer_column_names=0;
SELECT 123456789 AS x FORMAT PrettyCompact;
SELECT toNullable(123456789) AS x FORMAT PrettyCompact;
SELECT toLowCardinality(toNullable(123456789)) AS x FORMAT PrettyCompact;

View File

@ -5,4 +5,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CUR_DIR"/../shell_config.sh
# default output_format_pretty_max_rows is 10K
$CLICKHOUSE_LOCAL -q "select * from numbers(100e3) format PrettySpace settings max_threads=1" | wc -l
$CLICKHOUSE_LOCAL -q "select * from numbers(100e3) format PrettySpace settings max_threads=1, output_format_pretty_display_footer_column_names=0" | wc -l

View File

@ -0,0 +1,4 @@
{% for type in ['Int8', 'Int16', 'Int32', 'Int64', 'UInt8', 'UInt16', 'UInt32', 'UInt64'] -%}
{'xxx':56}
{56:'xxx'}
{% endfor -%}

View File

@ -0,0 +1,31 @@
DROP TABLE IF EXISTS boom_filter_map_1;
DROP TABLE IF EXISTS boom_filter_map_2;
{% for type in ['Int8', 'Int16', 'Int32', 'Int64', 'UInt8', 'UInt16', 'UInt32', 'UInt64'] -%}
CREATE TABLE boom_filter_map_1
(
`m` Map(String, {{ type }}),
INDEX index_models_value_bloom_filter mapValues(m) TYPE bloom_filter GRANULARITY 1
)
ENGINE = MergeTree
ORDER BY tuple();
CREATE TABLE boom_filter_map_2
(
`m` Map({{ type }}, String),
INDEX index_models_value_bloom_filter mapKeys(m) TYPE bloom_filter GRANULARITY 1
)
ENGINE = MergeTree
ORDER BY tuple();
INSERT INTO boom_filter_map_1 (m) values (map('xxx', 56));
INSERT INTO boom_filter_map_2 (m) values (map(56, 'xxx'));
SELECT m FROM boom_filter_map_1 WHERE (m['xxx']) = 56;
SELECT m FROM boom_filter_map_2 WHERE (m[56]) = 'xxx';
DROP TABLE IF EXISTS boom_filter_map_1;
DROP TABLE IF EXISTS boom_filter_map_2;
{% endfor -%}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,19 @@
-- https://github.com/ClickHouse/ClickHouse/issues/65035
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 49) FORMAT Pretty;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 10) FORMAT Pretty SETTINGS output_format_pretty_display_footer_column_names_min_rows=9;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT Pretty SETTINGS output_format_pretty_display_footer_column_names=0;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT Pretty;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapes;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyMonoBlock;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapesMonoBlock;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyNoEscapesMonoBlock;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompact SETTINGS output_format_pretty_display_footer_column_names=0;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompact;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactNoEscapes;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactMonoBlock;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettyCompactNoEscapesMonoBlock;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpace SETTINGS output_format_pretty_display_footer_column_names=0;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpace;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceNoEscapes;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceMonoBlock;
SELECT *, toTypeName(*) FROM (SELECT * FROM system.numbers LIMIT 100) FORMAT PrettySpaceNoEscapesMonoBlock;

View File

@ -0,0 +1,37 @@
-- https://github.com/ClickHouse/ClickHouse/issues/65201
SET short_circuit_function_evaluation='enable';
DROP DICTIONARY IF EXISTS direct_dictionary_simple_key_simple_attributes;
DROP TABLE IF EXISTS simple_key_simple_attributes_source_table;
CREATE TABLE simple_key_simple_attributes_source_table
(
id UInt64,
value_first String,
value_second String
)
ENGINE = TinyLog;
INSERT INTO simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
INSERT INTO simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
INSERT INTO simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
CREATE DICTIONARY direct_dictionary_simple_key_simple_attributes
(
`id` UInt64,
`value_first` String DEFAULT 'value_first_default',
`value_second` String DEFAULT 'value_second_default'
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(TABLE 'simple_key_simple_attributes_source_table'))
LAYOUT(DIRECT());
SELECT
toUInt128(1),
dictGetOrDefault('direct_dictionary_simple_key_simple_attributes', 'value_second', number, toString(toFixedString(toFixedString(toFixedString(materialize(toNullable('default')), 7), 7), toNullable(toNullable(toNullable(toUInt128(7))))))) AS value_second
FROM system.numbers LIMIT 255
FORMAT Null;
DROP DICTIONARY IF EXISTS direct_dictionary_simple_key_simple_attributes;
DROP TABLE IF EXISTS simple_key_simple_attributes_source_table;