Merge branch 'master' into clang-18-ci

This commit is contained in:
Alexey Milovidov 2024-05-10 02:38:20 +02:00
commit 42710158e4
34 changed files with 136 additions and 41 deletions

View File

@ -1114,6 +1114,7 @@ class IColumn;
\
M(String, format_json_object_each_row_column_for_object_name, "", "The name of column that will be used as object names in JSONObjectEachRow format. Column type should be String", 0) \
\
M(Bool, output_format_pretty_preserve_border_for_multiline_string, true, "Applies better rendering for multiline strings.", 0) \
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) \

View File

@ -92,6 +92,7 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"cross_join_min_bytes_to_compress", 0, 1_GiB, "A new setting."},
{"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."},
{"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"},
{"output_format_pretty_preserve_border_for_multiline_string", 1, 1, "Applies better rendering for multiline strings."},
}},
{"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"},
{"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"},

View File

@ -174,6 +174,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
format_settings.pretty.max_value_width_apply_for_single_value = settings.output_format_pretty_max_value_width_apply_for_single_value;
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.preserve_border_for_multiline_string = settings.output_format_pretty_preserve_border_for_multiline_string;
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;
format_settings.protobuf.output_nullables_with_google_wrappers = settings.output_format_protobuf_nullables_with_google_wrappers;

View File

@ -283,6 +283,7 @@ struct FormatSettings
SettingFieldUInt64Auto color{"auto"};
bool output_format_pretty_row_numbers = false;
bool preserve_border_for_multiline_string = true;
UInt64 output_format_pretty_single_large_number_tip_threshold = 1'000'000;
enum class Charset : uint8_t

View File

@ -2499,10 +2499,13 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
max_block_size = std::max<UInt64>(1, max_block_limited);
max_threads_execute_query = max_streams = 1;
}
if (local_limits.local_limits.size_limits.max_rows != 0)
{
if (max_block_limited < local_limits.local_limits.size_limits.max_rows)
query_info.limit = max_block_limited;
else if (local_limits.local_limits.size_limits.max_rows < std::numeric_limits<UInt64>::max()) /// Ask to read just enough rows to make the max_rows limit effective (so it has a chance to be triggered).
query_info.limit = 1 + local_limits.local_limits.size_limits.max_rows;
}
else
{

View File

@ -691,6 +691,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
{
if (max_block_size_limited < select_query_info.local_storage_limits.local_limits.size_limits.max_rows)
table_expression_query_info.limit = max_block_size_limited;
/// Ask to read just enough rows to make the max_rows limit effective (so it has a chance to be triggered).
else if (select_query_info.local_storage_limits.local_limits.size_limits.max_rows < std::numeric_limits<UInt64>::max())
table_expression_query_info.limit = 1 + select_query_info.local_storage_limits.local_limits.size_limits.max_rows;
}
else
{

View File

@ -38,7 +38,7 @@ void PrettyBlockOutputFormat::calculateWidths(
max_padded_widths.resize_fill(num_columns);
name_widths.resize(num_columns);
const bool need_cut_to_width = format_settings.pretty.max_value_width_apply_for_single_value || num_rows != 1 || num_columns != 1 || total_rows != 0;
const bool need_cut_to_width = format_settings.pretty.preserve_border_for_multiline_string && (format_settings.pretty.max_value_width_apply_for_single_value || num_rows != 1 || num_columns != 1 || total_rows != 0);
/// Calculate widths of all values.
String serialized_value;
@ -333,7 +333,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind
WriteBufferFromString out_serialize(serialized_value, AppendModeTag());
serializations[j]->serializeText(*columns[j], i, out_serialize, format_settings);
}
if (cut_to_width)
if (cut_to_width && format_settings.pretty.preserve_border_for_multiline_string)
splitValueAtBreakLine(serialized_value, transferred_row[j], cur_width);
has_transferred_row |= !transferred_row[j].empty() && cur_width <= cut_to_width;

View File

@ -182,7 +182,7 @@ void PrettyCompactBlockOutputFormat::writeRow(
WriteBufferFromString out_serialize(serialized_value, AppendModeTag());
serializations[j]->serializeText(*columns[j], row_num, out_serialize, format_settings);
}
if (cut_to_width)
if (cut_to_width && format_settings.pretty.preserve_border_for_multiline_string)
splitValueAtBreakLine(serialized_value, transferred_row[j], cur_width);
has_transferred_row |= !transferred_row[j].empty() && cur_width <= cut_to_width;

View File

@ -100,7 +100,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port
WriteBufferFromString out_serialize(serialized_value, AppendModeTag());
serializations[column]->serializeText(*columns[column], row, out_serialize, format_settings);
}
if (cut_to_width)
if (cut_to_width && format_settings.pretty.preserve_border_for_multiline_string)
splitValueAtBreakLine(serialized_value, transferred_row[column], cur_width);
has_transferred_row |= !transferred_row[column].empty() && cur_width <= cut_to_width;

View File

@ -12,8 +12,8 @@
#include <fmt/format.h>
#include <Common/iota.h>
#include <Common/typeid_cast.h>
#include "Core/Types.h"
#include "base/types.h"
#include <Core/Types.h>
namespace DB
{
@ -443,7 +443,6 @@ Pipe ReadFromSystemNumbersStep::makePipe()
/// Build rpn of query filters
KeyCondition condition(filter_actions_dag, context, column_names, key_expression);
if (condition.extractPlainRanges(ranges))
{
/// Intersect ranges with table range
@ -505,7 +504,6 @@ Pipe ReadFromSystemNumbersStep::makePipe()
}
}
/// ranges is blank, return a source who has no data
if (intersected_ranges.empty())
{

View File

@ -8,6 +8,7 @@
#include <Storages/SelectQueryInfo.h>
#include <Storages/StorageSnapshot.h>
namespace DB
{
@ -43,4 +44,5 @@ private:
UInt64 limit;
std::shared_ptr<const StorageLimitsList> storage_limits;
};
}

View File

@ -28,7 +28,7 @@ namespace ErrorCodes
template <typename Distance>
AnnoyIndexWithSerialization<Distance>::AnnoyIndexWithSerialization(size_t dimensions)
: Base::AnnoyIndex(dimensions)
: Base::AnnoyIndex(static_cast<int>(dimensions))
{
}

View File

@ -3,6 +3,7 @@
#include <Storages/StorageGenerateRandom.h>
#include <Storages/StorageFactory.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include <Storages/SelectQueryInfo.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <QueryPipeline/Pipe.h>
#include <Parsers/ASTLiteral.h>
@ -30,12 +31,9 @@
#include <Common/SipHash.h>
#include <Common/randomSeed.h>
#include <Interpreters/Context.h>
#include <base/unaligned.h>
#include <Functions/FunctionFactory.h>
#include <pcg_random.hpp>
namespace DB
{
@ -639,7 +637,7 @@ void registerStorageGenerateRandom(StorageFactory & factory)
Pipe StorageGenerateRandom::read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & /*query_info*/,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size,
@ -682,7 +680,14 @@ Pipe StorageGenerateRandom::read(
pcg64 generate(random_seed);
for (UInt64 i = 0; i < num_streams; ++i)
pipes.emplace_back(std::make_shared<GenerateSource>(max_block_size, max_array_length, max_string_length, generate(), block_header, context));
{
auto source = std::make_shared<GenerateSource>(max_block_size, max_array_length, max_string_length, generate(), block_header, context);
if (i == 0 && query_info.limit)
source->addTotalRowsApprox(query_info.limit);
pipes.emplace_back(std::move(source));
}
return Pipe::unitePipes(std::move(pipes));
}

View File

@ -1,17 +1,14 @@
#include <Storages/System/StorageSystemNumbers.h>
#include <mutex>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Processors/ISource.h>
#include <Processors/LimitTransform.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromSystemNumbersStep.h>
#include <Processors/Sources/NullSource.h>
#include <QueryPipeline/Pipe.h>
#include <Storages/SelectQueryInfo.h>
namespace DB
{

View File

@ -1,4 +1,5 @@
#include <Storages/System/StorageSystemZeros.h>
#include <Storages/SelectQueryInfo.h>
#include <Processors/ISource.h>
#include <QueryPipeline/Pipe.h>
@ -6,6 +7,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
namespace DB
{
@ -93,7 +95,7 @@ StorageSystemZeros::StorageSystemZeros(const StorageID & table_id_, bool multith
Pipe StorageSystemZeros::read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo &,
SelectQueryInfo & query_info,
ContextPtr /*context*/,
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size,
@ -123,8 +125,13 @@ Pipe StorageSystemZeros::read(
{
auto source = std::make_shared<ZerosSource>(max_block_size, limit ? *limit : 0, state);
if (limit && i == 0)
source->addTotalRowsApprox(*limit);
if (i == 0)
{
if (limit)
source->addTotalRowsApprox(*limit);
else if (query_info.limit)
source->addTotalRowsApprox(query_info.limit);
}
res.addSource(std::move(source));
}

View File

@ -908,7 +908,7 @@ class MergeTreeSettingsRandomizer:
),
"cache_populated_by_fetch": lambda: random.randint(0, 1),
"concurrent_part_removal_threshold": threshold_generator(0.2, 0.3, 0, 100),
"old_parts_lifetime": threshold_generator(0.2, 0.3, 30, 8 * 60),
"old_parts_lifetime": threshold_generator(0.2, 0.3, 10, 8 * 60),
}
@staticmethod

View File

@ -181,11 +181,8 @@ elif [[ "$USE_AZURE_STORAGE_FOR_MERGE_TREE" == "1" ]]; then
ln -sf $SRC_PATH/config.d/azure_storage_policy_by_default.xml $DEST_SERVER_PATH/config.d/
fi
ARM="aarch64"
OS="$(uname -m)"
if [[ -n "$EXPORT_S3_STORAGE_POLICIES" ]]; then
echo "$OS"
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] || [[ "$OS" == "$ARM" ]]; then
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
echo "Azure configuration will not be added"
else
echo "Adding azure configuration"

View File

@ -1,4 +1,4 @@
-- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-cpu-aarch64
-- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug
-- This file contains tests for the event_time_microseconds field for various tables.
-- Note: Only event_time_microseconds for asynchronous_metric_log table is tested via

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings, no-cpu-aarch64, no-replicated-database
# Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings, no-replicated-database
# set -x

View File

@ -9,7 +9,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
for STORAGE_POLICY in 's3_cache' 'local_cache'; do
echo "Using storage policy: $STORAGE_POLICY"
${CLICKHOUSE_CLIENT} --query "SYSTEM STOP MERGES"
${CLICKHOUSE_CLIENT} --query "SYSTEM DROP FILESYSTEM CACHE"
${CLICKHOUSE_CLIENT} --query "SYSTEM DROP MARK CACHE"
${CLICKHOUSE_CLIENT} --query "SELECT count() FROM system.filesystem_cache"

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long, no-fasttest, no-parallel, no-s3-storage, no-random-settings, no-cpu-aarch64
# Tags: long, no-fasttest, no-parallel, no-s3-storage, no-random-settings
# set -x

View File

@ -1,4 +1,4 @@
-- Tags: no-fasttest, no-parallel, no-cpu-aarch64
-- Tags: no-fasttest, no-parallel
-- Tag no-fasttest: Depends on Java
insert into table function hdfs('hdfs://localhost:12222/test_1.tsv', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') select 1, 2, 3 settings hdfs_truncate_on_insert=1;

View File

@ -1,5 +1,5 @@
-- Tags: no-fasttest, no-cpu-aarch64
-- Tags: no-fasttest
SELECT * FROM hdfsCluster('test_shard_localhost', '', 'TSV'); -- { serverError BAD_ARGUMENTS }
SELECT * FROM hdfsCluster('test_shard_localhost', ' ', 'TSV'); -- { serverError BAD_ARGUMENTS }
SELECT * FROM hdfsCluster('test_shard_localhost', '/', 'TSV'); -- { serverError BAD_ARGUMENTS }
SELECT * FROM hdfsCluster('test_shard_localhost', 'http/', 'TSV'); -- { serverError BAD_ARGUMENTS }
SELECT * FROM hdfsCluster('test_shard_localhost', 'http/', 'TSV'); -- { serverError BAD_ARGUMENTS }

View File

@ -1,4 +1,4 @@
-- Tags: no-fasttest, no-parallel, no-cpu-aarch64
-- Tags: no-fasttest, no-parallel
-- Tag no-fasttest: Depends on Java
insert into table function hdfs('hdfs://localhost:12222/test_02458_1.tsv', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') select 1, 2, 3 settings hdfs_truncate_on_insert=1;
@ -9,4 +9,3 @@ desc hdfsCluster('test_cluster_one_shard_three_replicas_localhost', 'hdfs://loca
select * from hdfsCluster('test_cluster_one_shard_three_replicas_localhost', 'hdfs://localhost:12222/test_02458_{1,2}.tsv') order by c1, c2, c3;
select * from hdfsCluster('test_cluster_one_shard_three_replicas_localhost', 'hdfs://localhost:12222/test_02458_{1,2}.tsv', 'TSV') order by c1, c2, c3;

View File

@ -1,4 +1,4 @@
-- Tags: no-fasttest, no-parallel, no-cpu-aarch64
-- Tags: no-fasttest, no-parallel
-- Tag no-fasttest: Depends on Java
insert into table function hdfs('hdfs://localhost:12222/test_02536.jsonl', 'TSV') select '{"x" : {"a" : 1, "b" : 2}}' settings hdfs_truncate_on_insert=1;
@ -9,4 +9,3 @@ insert into test select * from hdfsCluster('test_cluster_two_shards_localhost',
insert into test select * from hdfsCluster('test_cluster_two_shards_localhost', 'hdfs://localhost:12222/test_02536.jsonl') settings use_structure_from_insertion_table_in_table_functions=1;
select * from test;
drop table test;

View File

@ -1,4 +1,4 @@
-- Tags: no-fasttest, no-cpu-aarch64, no-msan
-- Tags: no-fasttest, no-msan
drop table if exists dummy;
CREATE TABLE dummy ( num1 Int32, num2 Enum8('foo' = 0, 'bar' = 1, 'tar' = 2) )

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-fasttest, no-cpu-aarch64
# Tags: no-fasttest
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
@ -20,4 +20,3 @@ $CLICKHOUSE_LOCAL -q "select count() from file('$ERRORS_FILE', CSV)"
rm $ERRORS_FILE
rm $FILE

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-random-settings, no-replicated-database, no-shared-merge-tree
# Tags: no-random-settings, no-s3-storage, no-replicated-database, no-shared-merge-tree
# Tag no-random-settings: enable after root causing flakiness
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)

View File

@ -0,0 +1,49 @@
#!/usr/bin/expect -f
set basedir [file dirname $argv0]
set basename [file tail $argv0]
if {[info exists env(CLICKHOUSE_TMP)]} {
set CLICKHOUSE_TMP $env(CLICKHOUSE_TMP)
} else {
set CLICKHOUSE_TMP "."
}
exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0
log_user 0
set timeout 60
match_max 100000
set stty_init "rows 25 cols 120"
expect_after {
-i $any_spawn_id eof { exp_continue }
-i $any_spawn_id timeout { exit 1 }
}
spawn clickhouse-local
expect ":) "
# Trivial SELECT with LIMIT from system.zeros shows progress bar.
send "SELECT * FROM system.zeros LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r"
expect "Progress: "
expect "█"
send "\3"
expect "Query was cancelled."
expect ":) "
send "SELECT * FROM system.zeros_mt LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r"
expect "Progress: "
expect "█"
send "\3"
expect "Query was cancelled."
expect ":) "
# As well as from generateRandom
send "SELECT * FROM generateRandom() LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r"
expect "Progress: "
expect "█"
send "\3"
expect "Query was cancelled."
expect ":) "
send "exit\r"
expect eof

View File

@ -0,0 +1,9 @@
SET max_rows_to_read = 1e11;
SELECT * FROM system.numbers LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS }
SELECT * FROM system.numbers_mt LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS }
SELECT * FROM system.zeros LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS }
SELECT * FROM system.zeros_mt LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS }
SELECT * FROM generateRandom() LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS }

View File

@ -86,3 +86,21 @@
2. │ 1 │ hello world │ hellow …│
│ │ │…мир │
└────┴─────────────┴─────────────┘
┏━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓
┃ id ┃ value ┃ value1 ┃
┡━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩
│ 0 │ привет
world │ hello world │
├────┼─────────────┼─────────────┤
│ 1 │ hello world │ hellow
мир │
└────┴─────────────┴─────────────┘
┏━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓
┃ id ┃ value ┃ value1 ┃
┡━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩
1. │ 0 │ привет
world │ hello world │
├────┼─────────────┼─────────────┤
2. │ 1 │ hello world │ hellow
мир │
└────┴─────────────┴─────────────┘

View File

@ -1,5 +1,7 @@
DROP TABLE IF EXISTS t_break_line;
SET output_format_pretty_preserve_border_for_multiline_string=1;
CREATE TABLE t_break_line (id UInt64, value String, value1 String) ENGINE=MergeTree ORDER BY id;
INSERT INTO t_break_line VALUES(0, 'hello\nworld', 'hello world');
@ -30,4 +32,9 @@ INSERT INTO t_break_line VALUES(1, 'hello world', 'hellow\nмир');
SELECT * FROM t_break_line ORDER BY id FORMAT PrettyMonoBlock SETTINGS output_format_pretty_row_numbers = 0;
SELECT * FROM t_break_line ORDER BY id FORMAT PrettyMonoBlock;
SET output_format_pretty_preserve_border_for_multiline_string=0;
SELECT * FROM t_break_line ORDER BY id FORMAT PrettyMonoBlock SETTINGS output_format_pretty_row_numbers = 0;
SELECT * FROM t_break_line ORDER BY id FORMAT PrettyMonoBlock;
DROP TABLE t_break_line;