mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Fix wrong cases of numbers pretty printing
Add a test Revert changes from another branch Add a test Better test Revert wrong changes
This commit is contained in:
parent
6fb40d6765
commit
a2e89c8be7
@ -1090,6 +1090,7 @@ class IColumn;
|
||||
M(UInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \
|
||||
M(UInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \
|
||||
M(UInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \
|
||||
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_parquet_row_group_size, 1000000, "Target row group size in rows.", 0) \
|
||||
|
@ -105,6 +105,7 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
|
||||
{"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"},
|
||||
{"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"},
|
||||
{"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."},
|
||||
{"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."},
|
||||
}},
|
||||
{"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"},
|
||||
{"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"},
|
||||
|
@ -167,6 +167,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
|
||||
format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width;
|
||||
format_settings.pretty.max_rows = settings.output_format_pretty_max_rows;
|
||||
format_settings.pretty.max_value_width = settings.output_format_pretty_max_value_width;
|
||||
format_settings.pretty.max_value_width_apply_for_single_value = settings.output_format_pretty_max_value_width_apply_for_single_value;
|
||||
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.protobuf.input_flatten_google_wrappers = settings.input_format_protobuf_flatten_google_wrappers;
|
||||
|
@ -275,6 +275,7 @@ struct FormatSettings
|
||||
UInt64 max_rows = 10000;
|
||||
UInt64 max_column_pad_width = 250;
|
||||
UInt64 max_value_width = 10000;
|
||||
UInt64 max_value_width_apply_for_single_value = false;
|
||||
SettingFieldUInt64Auto color{"auto"};
|
||||
|
||||
bool output_format_pretty_row_numbers = false;
|
||||
|
@ -164,6 +164,10 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
|
||||
const auto & columns = chunk.getColumns();
|
||||
const auto & header = getPort(port_kind).getHeader();
|
||||
|
||||
size_t cut_to_width = format_settings.pretty.max_value_width;
|
||||
if (!format_settings.pretty.max_value_width_apply_for_single_value && num_rows == 1 && num_columns == 1 && total_rows == 0)
|
||||
cut_to_width = 0;
|
||||
|
||||
WidthsPerColumn widths;
|
||||
Widths max_widths;
|
||||
Widths name_widths;
|
||||
@ -303,7 +307,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
|
||||
const auto & type = *header.getByPosition(j).type;
|
||||
writeValueWithPadding(*columns[j], *serializations[j], i,
|
||||
widths[j].empty() ? max_widths[j] : widths[j][i],
|
||||
max_widths[j], type.shouldAlignRightInPrettyFormats());
|
||||
max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats());
|
||||
}
|
||||
|
||||
writeCString(grid_symbols.bar, out);
|
||||
@ -324,7 +328,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
|
||||
|
||||
void PrettyBlockOutputFormat::writeValueWithPadding(
|
||||
const IColumn & column, const ISerialization & serialization, size_t row_num,
|
||||
size_t value_width, size_t pad_to_width, bool align_right)
|
||||
size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right)
|
||||
{
|
||||
String serialized_value = " ";
|
||||
{
|
||||
@ -332,7 +336,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding(
|
||||
serialization.serializeText(column, row_num, out_serialize, format_settings);
|
||||
}
|
||||
|
||||
if (value_width > format_settings.pretty.max_value_width)
|
||||
if (cut_to_width && value_width > cut_to_width)
|
||||
{
|
||||
serialized_value.resize(UTF8::computeBytesBeforeWidth(
|
||||
reinterpret_cast<const UInt8 *>(serialized_value.data()), serialized_value.size(), 0, 1 + format_settings.pretty.max_value_width));
|
||||
@ -419,16 +423,19 @@ void PrettyBlockOutputFormat::writeReadableNumberTip(const Chunk & chunk)
|
||||
auto is_single_number = readable_number_tip && chunk.getNumRows() == 1 && chunk.getNumColumns() == 1;
|
||||
if (!is_single_number)
|
||||
return;
|
||||
|
||||
auto value = columns[0]->getFloat64(0);
|
||||
auto threshold = format_settings.pretty.output_format_pretty_single_large_number_tip_threshold;
|
||||
if (threshold == 0 || value <= threshold)
|
||||
return;
|
||||
if (color)
|
||||
writeCString("\033[90m", out);
|
||||
writeCString(" -- ", out);
|
||||
formatReadableQuantity(value, out, 2);
|
||||
if (color)
|
||||
writeCString("\033[0m", out);
|
||||
|
||||
if (threshold && isFinite(value) && abs(value) >= threshold)
|
||||
{
|
||||
if (color)
|
||||
writeCString("\033[90m", out);
|
||||
writeCString(" -- ", out);
|
||||
formatReadableQuantity(value, out, 2);
|
||||
if (color)
|
||||
writeCString("\033[0m", out);
|
||||
}
|
||||
}
|
||||
|
||||
void registerOutputFormatPretty(FormatFactory & factory)
|
||||
|
@ -48,7 +48,7 @@ protected:
|
||||
|
||||
void writeValueWithPadding(
|
||||
const IColumn & column, const ISerialization & serialization, size_t row_num,
|
||||
size_t value_width, size_t pad_to_width, bool align_right);
|
||||
size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right);
|
||||
|
||||
void resetFormatterImpl() override
|
||||
{
|
||||
|
@ -160,6 +160,10 @@ void PrettyCompactBlockOutputFormat::writeRow(
|
||||
size_t num_columns = max_widths.size();
|
||||
const auto & columns = chunk.getColumns();
|
||||
|
||||
size_t cut_to_width = format_settings.pretty.max_value_width;
|
||||
if (!format_settings.pretty.max_value_width_apply_for_single_value && chunk.getNumRows() == 1 && num_columns == 1 && total_rows == 0)
|
||||
cut_to_width = 0;
|
||||
|
||||
writeCString(grid_symbols.bar, out);
|
||||
|
||||
for (size_t j = 0; j < num_columns; ++j)
|
||||
@ -169,7 +173,7 @@ void PrettyCompactBlockOutputFormat::writeRow(
|
||||
|
||||
const auto & type = *header.getByPosition(j).type;
|
||||
const auto & cur_widths = widths[j].empty() ? max_widths[j] : widths[j][row_num];
|
||||
writeValueWithPadding(*columns[j], *serializations[j], row_num, cur_widths, max_widths[j], type.shouldAlignRightInPrettyFormats());
|
||||
writeValueWithPadding(*columns[j], *serializations[j], row_num, cur_widths, max_widths[j], cut_to_width, type.shouldAlignRightInPrettyFormats());
|
||||
}
|
||||
|
||||
writeCString(grid_symbols.bar, out);
|
||||
|
@ -24,6 +24,10 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port
|
||||
const auto & header = getPort(port_kind).getHeader();
|
||||
const auto & columns = chunk.getColumns();
|
||||
|
||||
size_t cut_to_width = format_settings.pretty.max_value_width;
|
||||
if (!format_settings.pretty.max_value_width_apply_for_single_value && num_rows == 1 && num_columns == 1 && total_rows == 0)
|
||||
cut_to_width = 0;
|
||||
|
||||
WidthsPerColumn widths;
|
||||
Widths max_widths;
|
||||
Widths name_widths;
|
||||
@ -84,7 +88,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port
|
||||
const auto & type = *header.getByPosition(column).type;
|
||||
auto & cur_width = widths[column].empty() ? max_widths[column] : widths[column][row];
|
||||
writeValueWithPadding(
|
||||
*columns[column], *serializations[column], row, cur_width, max_widths[column], type.shouldAlignRightInPrettyFormats());
|
||||
*columns[column], *serializations[column], row, cur_width, max_widths[column], cut_to_width, type.shouldAlignRightInPrettyFormats());
|
||||
}
|
||||
|
||||
writeReadableNumberTip(chunk);
|
||||
|
@ -0,0 +1,4 @@
|
||||
COMMENT 'Contains history of metrics values from tables system.metrics and system.events, periodically flushed to disk.' │
|
||||
COMMENT 'Contains history of metrics values from tables system.metrics and system.events, periodically flushed to disk.' │
|
||||
COMMENT 'Contains history of metrics values from tables system.metrics and system.events, periodically flushed to disk.'
|
||||
\nCOMMENT \'Contains history of metrics values from tables system.metrics and system.events, periodically flushed to disk.\'
|
17
tests/queries/0_stateless/03020_long_values_pretty_are_not_cut_if_single.sh
Executable file
17
tests/queries/0_stateless/03020_long_values_pretty_are_not_cut_if_single.sh
Executable file
@ -0,0 +1,17 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
# The metric_log table has a large number of columns, so its definition is also large.
|
||||
# And it is larger than the default value of `output_format_pretty_max_value_width`.
|
||||
# But cutting it in the result of SHOW CREATE TABLE will be bad for a user.
|
||||
# That's why we control it with the setting `output_format_pretty_max_value_width_apply_for_single_value`.
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE system.metric_log" --format Pretty | grep -P '^COMMENT'
|
||||
${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE system.metric_log" --format PrettyCompact | grep -P '^COMMENT'
|
||||
${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE system.metric_log" --format PrettySpace | grep -P '^COMMENT'
|
||||
|
||||
# Just in case, non-Pretty format:
|
||||
${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE system.metric_log" --format TSV | grep -o -P '\\nCOMMENT.+$'
|
Loading…
Reference in New Issue
Block a user