From 5c67c5fd2640521121c7ebcf782d368f194cc6e1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 4 May 2023 14:40:22 +0800 Subject: [PATCH 001/634] add tests for hive null_as_default --- .../Formats/Impl/HiveTextRowInputFormat.cpp | 1 + .../test_hive_query/data/prepare_hive_data.sh | 6 ++-- tests/integration/test_hive_query/test.py | 28 +++++++++++++++++++ 3 files changed, 33 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp index 1b73e0131f6..3f9d0bfc7a2 100644 --- a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp @@ -17,6 +17,7 @@ static FormatSettings updateFormatSettings(const FormatSettings & settings, cons updated.skip_unknown_fields = true; updated.with_names_use_header = true; updated.date_time_input_format = FormatSettings::DateTimeInputFormat::BestEffort; + updated.defaults_for_omitted_fields = true; updated.csv.delimiter = updated.hive_text.fields_delimiter; if (settings.hive_text.input_field_names.empty()) updated.hive_text.input_field_names = header.getNames(); diff --git a/tests/integration/test_hive_query/data/prepare_hive_data.sh b/tests/integration/test_hive_query/data/prepare_hive_data.sh index 495ea201870..ca3f04c7a68 100755 --- a/tests/integration/test_hive_query/data/prepare_hive_data.sh +++ b/tests/integration/test_hive_query/data/prepare_hive_data.sh @@ -5,11 +5,13 @@ hive -e "drop table if exists test.demo; create table test.demo(id string, score hive -e "drop table if exists test.parquet_demo; create table test.parquet_demo(id string, score int) PARTITIONED BY(day string, hour string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'" hive -e "drop table if exists test.demo_text; create table test.demo_text(id string, score int, day string)row format delimited fields terminated by ','; load data local inpath '/demo_data.txt' into table test.demo_text " hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo partition(day) select * from test.demo_text; insert into test.demo_orc partition(day) select * from test.demo_text" - hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '00' as hour from test.demo;" hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '01' as hour from test.demo;" hive -e "drop table if exists test.test_hive_types; CREATE TABLE test.test_hive_types( f_tinyint tinyint, f_smallint smallint, f_int int, f_integer int, f_bigint bigint, f_float float, f_double double, f_decimal decimal(10,0), f_timestamp timestamp, f_date date, f_string string, f_varchar varchar(100), f_char char(100), f_bool boolean, f_array_int array, f_array_string array, f_array_float array, f_map_int map, f_map_string map, f_map_float map, f_struct struct>) PARTITIONED BY( day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat';" - hive -e "insert into test.test_hive_types partition(day='2022-02-20') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-20 14:47:04', '2022-02-20', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3), 'd', named_struct('x', 10, 'y', 'xyz')); insert into test.test_hive_types partition(day='2022-02-19') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-19 14:47:04', '2022-02-19', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3), 'd', named_struct('x', 11, 'y', 'abc'));" +hive -e "drop table if exists test.null_as_default_orc; create table test.null_as_default_orc (x string, y string) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'" +hive -e "insert into test.null_as_default_orc partition(day='2023-05-01') select null, null;" +hive -e "drop table if exists test.null_as_default_parquet; create table test.null_as_default_parquet (x string, y string) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'" +hive -e "set hive.exec.dynamic.partition.mode=nonstrict; insert into test.null_as_default_parquet partition(day) select x, y, day from test.null_as_default_orc;" diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 791ae03f9f6..c8f87ece6be 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -496,3 +496,31 @@ CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") expected_result = """id\tNullable(String)\t\t\tText comment\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)""" assert result.strip() == expected_result + + +@pytest.mark.parametrize( + "table", + [ + pytest.param( + "null_as_default_orc", + id="test_null_as_default_orc", + ), + pytest.param( + "null_as_default_parquet", + id="test_null_as_default_parquet", + ), + ], +) +def test_null_as_default(started_cluster, table): + node = started_cluster.instances["h0_0_0"] + node.query("set input_format_null_as_default = true") + result = node.query( + """ +DROP TABLE IF EXISTS default.${table}; +CREATE TABLE default.${table} (`x` String, `y` String DEFAULT 'world', `day` String) ENGINE = Hive('thrift://hivetest:9083', 'test', '${table}') PARTITION BY(day); +select x, y from default.${table}; +""".format( + table=table + ) + ) + assert result.strip("\n") == "\tworld" From d35142ba654f9171e52d960b9c869c414c6c9054 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 4 May 2023 14:50:26 +0000 Subject: [PATCH 002/634] Allow parameters in queries with partitions --- src/Parsers/ASTPartition.h | 4 +- src/Parsers/ParserPartition.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 62 +++++++++++++++++++----- 3 files changed, 54 insertions(+), 16 deletions(-) diff --git a/src/Parsers/ASTPartition.h b/src/Parsers/ASTPartition.h index fbe05ce3a8e..d17941a9bb6 100644 --- a/src/Parsers/ASTPartition.h +++ b/src/Parsers/ASTPartition.h @@ -1,7 +1,7 @@ #pragma once #include - +#include namespace DB { @@ -11,7 +11,7 @@ class ASTPartition : public IAST { public: ASTPtr value; - size_t fields_count = 0; + std::optional fields_count; String id; bool all = false; diff --git a/src/Parsers/ParserPartition.cpp b/src/Parsers/ParserPartition.cpp index 9f1d4d4e889..ba55847593a 100644 --- a/src/Parsers/ParserPartition.cpp +++ b/src/Parsers/ParserPartition.cpp @@ -40,7 +40,7 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!parser_expr.parse(pos, value, expected)) return false; - size_t fields_count; + std::optional fields_count; const auto * tuple_ast = value->as(); bool surrounded_by_parens = false; @@ -65,7 +65,7 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) fields_count = 1; } } - else + else if (!value->as()) return false; if (surrounded_by_parens) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index afde1cd2fca..9ea8074c65b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -35,6 +35,7 @@ #include #include #include +#include #include #include #include @@ -59,6 +60,7 @@ #include #include #include +#include #include #include #include @@ -5206,20 +5208,57 @@ void MergeTreeData::restorePartFromBackup(std::shared_ptr r String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr local_context, DataPartsLock * acquired_lock) const { const auto & partition_ast = ast->as(); + ASTPtr partition_value_ast = partition_ast.value; if (partition_ast.all) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only Support DETACH PARTITION ALL currently"); - if (!partition_ast.value) + if (!partition_value_ast) { MergeTreePartInfo::validatePartitionID(partition_ast.id, format_version); return partition_ast.id; } + size_t partition_ast_fields_count; + if (partition_value_ast->as()) + { + assert(!partition_ast.fields_count); + + ReplaceQueryParameterVisitor param_visitor(local_context->getQueryParameters()); + param_visitor.visit(partition_value_ast); + + if (partition_value_ast->as()) + { + partition_ast_fields_count = 1; + } + else if (const auto * tuple_ast = partition_value_ast->as()) + { + if (tuple_ast->name != "tuple") + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, + "Expected tuple for complex partition key, got {}", tuple_ast->name); + + const auto * arguments_ast = tuple_ast->arguments->as(); + if (arguments_ast) + partition_ast_fields_count = arguments_ast->children.size(); + else + partition_ast_fields_count = 0; + } + else + { + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, + "Expected literal or tuple for partition key, got {}", partition_value_ast->getID()); + } + } + else + { + assert(partition_ast.fields_count); + partition_ast_fields_count = partition_ast.fields_count.value(); + } + if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { /// Month-partitioning specific - partition ID can be passed in the partition value. - const auto * partition_lit = partition_ast.value->as(); + const auto * partition_lit = partition_value_ast->as(); if (partition_lit && partition_lit->value.getType() == Field::Types::String) { String partition_id = partition_lit->value.get(); @@ -5232,29 +5271,28 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc auto metadata_snapshot = getInMemoryMetadataPtr(); const Block & key_sample_block = metadata_snapshot->getPartitionKey().sample_block; size_t fields_count = key_sample_block.columns(); - if (partition_ast.fields_count != fields_count) + if (partition_ast_fields_count != fields_count) throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Wrong number of fields in the partition expression: {}, must be: {}", - partition_ast.fields_count, fields_count); + partition_ast_fields_count, fields_count); Row partition_row(fields_count); if (fields_count == 0) { /// Function tuple(...) requires at least one argument, so empty key is a special case - assert(!partition_ast.fields_count); - assert(typeid_cast(partition_ast.value.get())); - assert(partition_ast.value->as()->name == "tuple"); - assert(partition_ast.value->as()->arguments); - auto args = partition_ast.value->as()->arguments; + assert(!partition_ast_fields_count); + assert(typeid_cast(partition_value_ast.get())); + assert(partition_value_ast->as()->name == "tuple"); + assert(partition_value_ast->as()->arguments); + auto args = partition_value_ast->as()->arguments; if (!args) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected at least one argument in partition AST"); - bool empty_tuple = partition_ast.value->as()->arguments->children.empty(); + bool empty_tuple = partition_value_ast->as()->arguments->children.empty(); if (!empty_tuple) throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Partition key is empty, expected 'tuple()' as partition key"); } else if (fields_count == 1) { - ASTPtr partition_value_ast = partition_ast.value; if (auto * tuple = partition_value_ast->as()) { assert(tuple->name == "tuple"); @@ -5269,7 +5307,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc else { /// Complex key, need to evaluate, untuple and cast - Field partition_key_value = evaluateConstantExpression(partition_ast.value, local_context).first; + Field partition_key_value = evaluateConstantExpression(partition_value_ast, local_context).first; if (partition_key_value.getType() != Field::Types::Tuple) throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", partition_key_value.getTypeName()); From 0477db85c1a8975ae70d93fcdd3e66dfb646fe53 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 24 May 2023 13:04:53 +0200 Subject: [PATCH 003/634] Repro for #49887 --- ...tion_materialize_and_replication.reference | 0 ...projection_materialize_and_replication.sql | 42 +++++++++++++++++++ 2 files changed, 42 insertions(+) create mode 100644 tests/queries/0_stateless/02597_projection_materialize_and_replication.reference create mode 100644 tests/queries/0_stateless/02597_projection_materialize_and_replication.sql diff --git a/tests/queries/0_stateless/02597_projection_materialize_and_replication.reference b/tests/queries/0_stateless/02597_projection_materialize_and_replication.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql b/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql new file mode 100644 index 00000000000..c3f86ddc014 --- /dev/null +++ b/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql @@ -0,0 +1,42 @@ +CREATE TABLE test ( + `c_id` String, + `p_id` String, + `d` String +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1') +ORDER BY (c_id, p_id); + + +---CREATE TABLE test_r2 ( +--- `c_id` String, +--- `p_id` String, +--- `d` String +---) +---ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '2') +---ORDER BY (c_id, p_id); + + +INSERT INTO test SELECT '1', '11', '111' FROM numbers(3); + +INSERT INTO test SELECT '2', '22', '22' FROM numbers(3); + +select * from test format Null; +select min(c_id) from test group by d format Null; + + + +ALTER TABLE test ADD PROJECTION d_order ( SELECT min(c_id) GROUP BY `d`); + +ALTER TABLE test MATERIALIZE PROJECTION d_order; + +ALTER TABLE test DROP PROJECTION d_order SETTINGS mutations_sync = 2; + +SELECT * FROM system.mutations WHERE database=currentDatabase() AND table='test' AND NOT is_done; + + + +select * from test format Null; + + +DROP TABLE test; +--DROP TABLE test_r2; From cffc1e058cbbfdff965d8089a25fc726947fbf3b Mon Sep 17 00:00:00 2001 From: Alexander Nikolaev Date: Thu, 8 Jun 2023 19:51:50 +0300 Subject: [PATCH 004/634] Initialization of the database only once. --- docker/server/entrypoint.sh | 118 +++++++++++++++++++----------------- 1 file changed, 64 insertions(+), 54 deletions(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index d4596a68f1e..8419c1887d6 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -104,66 +104,76 @@ if [ -n "$CLICKHOUSE_USER" ] && [ "$CLICKHOUSE_USER" != "default" ] || [ -n "$CL EOT fi -if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then - # port is needed to check if clickhouse-server is ready for connections - HTTP_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=http_port --try)" - HTTPS_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=https_port --try)" +# checking $DATA_DIR for initialization +if [ -d "${DATA_DIR}data" ]; then + DATABASE_ALREADY_EXISTS='true' +fi - if [ -n "$HTTP_PORT" ]; then - URL="http://127.0.0.1:$HTTP_PORT/ping" - else - URL="https://127.0.0.1:$HTTPS_PORT/ping" - fi +# only run initialization on an empty data directory +if [ -z "${DATABASE_ALREADY_EXISTS}" ]; then + if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then + # port is needed to check if clickhouse-server is ready for connections + HTTP_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=http_port --try)" + HTTPS_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=https_port --try)" - # Listen only on localhost until the initialization is done - /usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" -- --listen_host=127.0.0.1 & - pid="$!" + if [ -n "$HTTP_PORT" ]; then + URL="http://127.0.0.1:$HTTP_PORT/ping" + else + URL="https://127.0.0.1:$HTTPS_PORT/ping" + fi - # check if clickhouse is ready to accept connections - # will try to send ping clickhouse via http_port (max 1000 retries by default, with 1 sec timeout and 1 sec delay between retries) - tries=${CLICKHOUSE_INIT_TIMEOUT:-1000} - while ! wget --spider --no-check-certificate -T 1 -q "$URL" 2>/dev/null; do - if [ "$tries" -le "0" ]; then - echo >&2 'ClickHouse init process failed.' + # Listen only on localhost until the initialization is done + /usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" -- --listen_host=127.0.0.1 & + pid="$!" + + # check if clickhouse is ready to accept connections + # will try to send ping clickhouse via http_port (max 1000 retries by default, with 1 sec timeout and 1 sec delay between retries) + tries=${CLICKHOUSE_INIT_TIMEOUT:-1000} + while ! wget --spider --no-check-certificate -T 1 -q "$URL" 2>/dev/null; do + if [ "$tries" -le "0" ]; then + echo >&2 'ClickHouse init process failed.' + exit 1 + fi + tries=$(( tries-1 )) + sleep 1 + done + + clickhouseclient=( clickhouse-client --multiquery --host "127.0.0.1" -u "$CLICKHOUSE_USER" --password "$CLICKHOUSE_PASSWORD" ) + + echo + + # create default database, if defined + if [ -n "$CLICKHOUSE_DB" ]; then + echo "$0: create database '$CLICKHOUSE_DB'" + "${clickhouseclient[@]}" -q "CREATE DATABASE IF NOT EXISTS $CLICKHOUSE_DB"; + fi + + for f in /docker-entrypoint-initdb.d/*; do + case "$f" in + *.sh) + if [ -x "$f" ]; then + echo "$0: running $f" + "$f" + else + echo "$0: sourcing $f" + # shellcheck source=/dev/null + . "$f" + fi + ;; + *.sql) echo "$0: running $f"; "${clickhouseclient[@]}" < "$f" ; echo ;; + *.sql.gz) echo "$0: running $f"; gunzip -c "$f" | "${clickhouseclient[@]}"; echo ;; + *) echo "$0: ignoring $f" ;; + esac + echo + done + + if ! kill -s TERM "$pid" || ! wait "$pid"; then + echo >&2 'Finishing of ClickHouse init process failed.' exit 1 fi - tries=$(( tries-1 )) - sleep 1 - done - - clickhouseclient=( clickhouse-client --multiquery --host "127.0.0.1" -u "$CLICKHOUSE_USER" --password "$CLICKHOUSE_PASSWORD" ) - - echo - - # create default database, if defined - if [ -n "$CLICKHOUSE_DB" ]; then - echo "$0: create database '$CLICKHOUSE_DB'" - "${clickhouseclient[@]}" -q "CREATE DATABASE IF NOT EXISTS $CLICKHOUSE_DB"; - fi - - for f in /docker-entrypoint-initdb.d/*; do - case "$f" in - *.sh) - if [ -x "$f" ]; then - echo "$0: running $f" - "$f" - else - echo "$0: sourcing $f" - # shellcheck source=/dev/null - . "$f" - fi - ;; - *.sql) echo "$0: running $f"; "${clickhouseclient[@]}" < "$f" ; echo ;; - *.sql.gz) echo "$0: running $f"; gunzip -c "$f" | "${clickhouseclient[@]}"; echo ;; - *) echo "$0: ignoring $f" ;; - esac - echo - done - - if ! kill -s TERM "$pid" || ! wait "$pid"; then - echo >&2 'Finishing of ClickHouse init process failed.' - exit 1 fi +else + echo "ClickHouse Database directory appears to contain a database; Skipping initialization" fi # if no args passed to `docker run` or first argument start with `--`, then the user is passing clickhouse-server arguments From 885136fce0fd80aa65a20a5c8892397909b6195f Mon Sep 17 00:00:00 2001 From: Alexander Nikolaev Date: Thu, 29 Jun 2023 15:59:22 +0300 Subject: [PATCH 005/634] Fix missing / --- docker/server/entrypoint.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index 8419c1887d6..d94ffb893e1 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -105,7 +105,7 @@ EOT fi # checking $DATA_DIR for initialization -if [ -d "${DATA_DIR}data" ]; then +if [ -d "${DATA_DIR%/}/data" ]; then DATABASE_ALREADY_EXISTS='true' fi From 9a5f357412939b741ce76a40c7ab1af4cf3619af Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jul 2023 10:10:09 +0200 Subject: [PATCH 006/634] Revert "Merge pull request #51805 from ClickHouse/fix_for_parallel_replicas_and_empty_header" This reverts commit 2c592a6a2c11e5ffb6b2f7be24153d28e26fd80f, reversing changes made to 7a593ed9a59139aba3ea133c3170488ac7818e64. --- src/Storages/MergeTree/MergeTreeData.cpp | 3 --- src/Storages/SelectQueryInfo.h | 2 -- src/Storages/StorageMergeTree.cpp | 9 ++----- ...parallel_replicas_prewhere_count.reference | 4 ---- ...02811_parallel_replicas_prewhere_count.sql | 24 ------------------- 5 files changed, 2 insertions(+), 40 deletions(-) delete mode 100644 tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.reference delete mode 100644 tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ec8ce3f5e3d..116fa0eba91 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7189,10 +7189,7 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( if (query_context->canUseParallelReplicasOnInitiator() && to_stage >= QueryProcessingStage::WithMergeableState) { if (!canUseParallelReplicasBasedOnPKAnalysis(query_context, storage_snapshot, query_info)) - { - query_info.parallel_replicas_disabled = true; return QueryProcessingStage::Enum::FetchColumns; - } /// ReplicatedMergeTree if (supportsReplication()) diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 13d6909fd52..8fbc64b7a24 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -255,8 +255,6 @@ struct SelectQueryInfo Block minmax_count_projection_block; MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr; - bool parallel_replicas_disabled = false; - bool is_parameterized_view = false; NameToNameMap parameterized_view_values; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 29880f10f28..afd303cd094 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -209,9 +209,7 @@ void StorageMergeTree::read( size_t max_block_size, size_t num_streams) { - if (!query_info.parallel_replicas_disabled && - local_context->canUseParallelReplicasOnInitiator() && - local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) + if (local_context->canUseParallelReplicasOnInitiator() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) { auto table_id = getStorageID(); @@ -242,10 +240,7 @@ void StorageMergeTree::read( } else { - const bool enable_parallel_reading = - !query_info.parallel_replicas_disabled && - local_context->canUseParallelReplicasOnFollower() && - local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree; + const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree; if (auto plan = reader.read( column_names, storage_snapshot, query_info, diff --git a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.reference b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.reference deleted file mode 100644 index fe8f022b908..00000000000 --- a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.reference +++ /dev/null @@ -1,4 +0,0 @@ --- count() ------------------------------ -2 --- count() with parallel replicas ------- -2 diff --git a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql deleted file mode 100644 index 141ae947e5e..00000000000 --- a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql +++ /dev/null @@ -1,24 +0,0 @@ -DROP TABLE IF EXISTS users; -CREATE TABLE users (uid Int16, name String, age Int16) ENGINE=MergeTree() ORDER BY uid; - -INSERT INTO users VALUES (111, 'JFK', 33); -INSERT INTO users VALUES (6666, 'KLM', 48); -INSERT INTO users VALUES (88888, 'AMS', 50); - -SELECT '-- count() ------------------------------'; -SELECT count() FROM users PREWHERE uid > 2000; - --- enable parallel replicas but with high granules threshold -SET -skip_unavailable_shards=1, -allow_experimental_parallel_reading_from_replicas=1, -max_parallel_replicas=3, -use_hedged_requests=0, -cluster_for_parallel_replicas='parallel_replicas', -parallel_replicas_for_non_replicated_merge_tree=1, -parallel_replicas_min_number_of_granules_to_enable=1000; - -SELECT '-- count() with parallel replicas -------'; -SELECT count() FROM users PREWHERE uid > 2000; - -DROP TABLE users; From f5e0c1bd252d54e12bfee3fa173c007a58476f71 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 30 Jun 2023 16:08:19 +0200 Subject: [PATCH 007/634] Interpreter: Control number of parallel replicas based on row estimation --- src/Core/Settings.h | 5 +- src/Interpreters/InterpreterSelectQuery.cpp | 82 ++++++++++++++++--- .../QueryPlan/ReadFromMergeTree.cpp | 10 +++ src/Processors/QueryPlan/ReadFromMergeTree.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 27 ++---- src/Storages/MergeTree/MergeTreeData.h | 16 ++-- 6 files changed, 102 insertions(+), 39 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f1a314fed37..a4ee4bbf785 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -169,7 +169,7 @@ class IColumn; M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ - M(UInt64, parallel_replicas_min_number_of_granules_to_enable, 0, "If the number of marks to read is less than the value of this setting - parallel replicas will be disabled", 0) \ + M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \ \ M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \ \ @@ -821,13 +821,14 @@ class IColumn; MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_replicated_sends_network_bandwidth_for_server, 0) \ /* ---- */ \ MAKE_OBSOLETE(M, DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic) \ - MAKE_OBSOLETE(M, UInt64, max_pipeline_depth, 0) \ + MAKE_OBSOLETE(M, UInt64, max_pipeline_depth, 0) \ MAKE_OBSOLETE(M, Seconds, temporary_live_view_timeout, 1) \ MAKE_OBSOLETE(M, Milliseconds, async_insert_cleanup_timeout_ms, 1000) \ MAKE_OBSOLETE(M, Bool, optimize_fuse_sum_count_avg, 0) \ MAKE_OBSOLETE(M, Seconds, drain_timeout, 3) \ MAKE_OBSOLETE(M, UInt64, backup_threads, 16) \ MAKE_OBSOLETE(M, UInt64, restore_threads, 16) \ + MAKE_OBSOLETE(M, UInt64, parallel_replicas_min_number_of_granules_to_enable, 0) \ /** The section above is for obsolete settings. Do not add anything there. */ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6ea15312ec4..c32ef4349f8 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -833,17 +833,77 @@ InterpreterSelectQuery::InterpreterSelectQuery( need_analyze_again = true; } - if (can_analyze_again - && settings.max_parallel_replicas > 1 - && settings.allow_experimental_parallel_reading_from_replicas > 0 - && settings.parallel_replicas_custom_key.value.empty() - && getTrivialCount(0).has_value()) + if (storage + /// While only analyzing we don't know anything about parts, so any decision about parallel would be wrong + && !options.only_analyze && can_analyze_again && settings.max_parallel_replicas > 1 + && settings.allow_experimental_parallel_reading_from_replicas > 0 && settings.parallel_replicas_custom_key.value.empty()) { - /// The query could use trivial count if it didn't use parallel replicas, so let's disable it and reanalyze - context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); - context->setSetting("max_parallel_replicas", UInt64{0}); - need_analyze_again = true; - LOG_TRACE(log, "Disabling parallel replicas to be able to use a trivial count optimization"); + if (getTrivialCount(0).has_value()) + { + /// The query could use trivial count if it didn't use parallel replicas, so let's disable it and reanalyze + context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + context->setSetting("max_parallel_replicas", UInt64{0}); + need_analyze_again = true; + LOG_DEBUG(log, "Disabling parallel replicas to be able to use a trivial count optimization"); + } + else if (settings.parallel_replicas_min_number_of_rows_per_replica > 0) + { + std::optional rows_to_read{}; + + auto storage_merge_tree = std::dynamic_pointer_cast(storage); + if (storage_merge_tree) + { + /// TODO: Improve this block as this should only happen once, right? vvvvvvvvvvvvvvvvvvvvvv + addPrewhereAliasActions(); + auto & prewhere_info = analysis_result.prewhere_info; + if (prewhere_info) + { + query_info.prewhere_info = prewhere_info; + if (query.prewhere() && !query.where()) + query_info.prewhere_info->need_filter = true; + } + + ActionDAGNodes added_filter_nodes; + if (additional_filter_info) + added_filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name)); + + if (analysis_result.before_where) + added_filter_nodes.nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name)); + /// END OF TODO BLOCK ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + + rows_to_read.emplace( + storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info, added_filter_nodes)); + } + + if (!rows_to_read.has_value()) + rows_to_read = storage->totalRows(settings); + + /// Open question: How should we treat 0 estimated rows to read? + /// It is a real estimation of 0 rows? + size_t number_of_replicas_to_use = rows_to_read.has_value() + ? *rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica + : settings.allow_experimental_parallel_reading_from_replicas; + + LOG_TRACE( + log, + "Estimated {} rows to read, which is work enough for {} parallel replicas", + rows_to_read.has_value() ? *rows_to_read : 0, + number_of_replicas_to_use); + + if (number_of_replicas_to_use <= 1) + { + context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + context->setSetting("max_parallel_replicas", UInt64{0}); + need_analyze_again = true; + LOG_DEBUG(log, "Disabling parallel replicas because there aren't enough rows to read"); + } + else if (number_of_replicas_to_use < settings.max_parallel_replicas) + { + /// TODO: Confirm that reducing the number of parallel replicas doesn't require a re-analysis + context->setSetting("max_parallel_replicas", number_of_replicas_to_use); + LOG_DEBUG(log, "Reducing the number of replicas to use to {}", number_of_replicas_to_use); + } + } } if (need_analyze_again) @@ -2264,7 +2324,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions() } } -/// Based on the query analysis, check if optimizing the count trivial count to use totalRows is possible +/// Based on the query analysis, check if using a trivial count (storage or partition metadata) is possible std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_parallel_replicas) { const Settings & settings = context->getSettingsRef(); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 13de5d1d140..9205e1f853b 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -2109,4 +2109,14 @@ size_t MergeTreeDataSelectAnalysisResult::marks() const return index_stats.back().num_granules_after; } +UInt64 MergeTreeDataSelectAnalysisResult::rows() const +{ + if (std::holds_alternative(result)) + std::rethrow_exception(std::get(result)); + + const auto & index_stats = std::get(result).index_stats; + if (index_stats.empty()) + return 0; + return std::get(result).selected_rows; +} } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 3e3edd4dc5c..7b5c182f19e 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -330,6 +330,7 @@ struct MergeTreeDataSelectAnalysisResult bool error() const; size_t marks() const; + UInt64 rows() const; }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 116fa0eba91..927d9459835 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7188,9 +7188,6 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( /// Parallel replicas if (query_context->canUseParallelReplicasOnInitiator() && to_stage >= QueryProcessingStage::WithMergeableState) { - if (!canUseParallelReplicasBasedOnPKAnalysis(query_context, storage_snapshot, query_info)) - return QueryProcessingStage::Enum::FetchColumns; - /// ReplicatedMergeTree if (supportsReplication()) return QueryProcessingStage::Enum::WithMergeableState; @@ -7216,10 +7213,11 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( } -bool MergeTreeData::canUseParallelReplicasBasedOnPKAnalysis( +UInt64 MergeTreeData::estimateNumberOfRowsToRead( ContextPtr query_context, const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info) const + const SelectQueryInfo & query_info, + const ActionDAGNodes & added_filter_nodes) const { const auto & snapshot_data = assert_cast(*storage_snapshot->data); const auto & parts = snapshot_data.parts; @@ -7232,23 +7230,14 @@ bool MergeTreeData::canUseParallelReplicasBasedOnPKAnalysis( storage_snapshot->metadata, storage_snapshot->metadata, query_info, - /*added_filter_nodes*/ActionDAGNodes{}, + added_filter_nodes, query_context, query_context->getSettingsRef().max_threads); - if (result_ptr->error()) - std::rethrow_exception(std::get(result_ptr->result)); - - LOG_TRACE(log, "Estimated number of granules to read is {}", result_ptr->marks()); - - bool decision = result_ptr->marks() >= query_context->getSettingsRef().parallel_replicas_min_number_of_granules_to_enable; - - if (!decision) - LOG_DEBUG(log, "Parallel replicas will be disabled, because the estimated number of granules to read {} is less than the threshold which is {}", - result_ptr->marks(), - query_context->getSettingsRef().parallel_replicas_min_number_of_granules_to_enable); - - return decision; + UInt64 total_rows = result_ptr->rows(); + if (query_info.limit > 0 && query_info.limit < total_rows) + total_rows = query_info.limit; + return total_rows; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 8b5b50b1841..5bbc6758dd3 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1062,6 +1062,15 @@ public: /// TODO: make enabled by default in the next release if no problems found. bool allowRemoveStaleMovingParts() const; + /// Estimate the number of rows to read based on primary key analysis (which could be very rough) + /// It is used to make a decision whether to enable parallel replicas (distributed processing) or not and how + /// many to replicas to use + UInt64 estimateNumberOfRowsToRead( + ContextPtr query_context, + const StorageSnapshotPtr & storage_snapshot, + const SelectQueryInfo & query_info, + const ActionDAGNodes & added_filter_nodes) const; + protected: friend class IMergeTreeDataPart; friend class MergeTreeDataMergerMutator; @@ -1549,13 +1558,6 @@ private: static MutableDataPartPtr asMutableDeletingPart(const DataPartPtr & part); mutable TemporaryParts temporary_parts; - - /// Estimate the number of marks to read to make a decision whether to enable parallel replicas (distributed processing) or not - /// Note: it could be very rough. - bool canUseParallelReplicasBasedOnPKAnalysis( - ContextPtr query_context, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info) const; }; /// RAII struct to record big parts that are submerging or emerging. From b9969e87309ef12b196734e8f0a42d6947cae488 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 30 Jun 2023 17:20:10 +0200 Subject: [PATCH 008/634] =?UTF-8?q?Test=20automatic=20decision=20of=20n?= =?UTF-8?q?=C2=BA=20of=20parallel=20replicas?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- tests/analyzer_tech_debt.txt | 2 +- ...lel_replicas_trivial_count_optimization.sh | 4 +- ...llel_replicas_automatic_decision.reference | 14 ++ ...84_parallel_replicas_automatic_decision.sh | 125 ++++++++++++++++++ ...lel_replicas_automatic_disabling.reference | 2 - ..._parallel_replicas_automatic_disabling.sql | 15 --- 6 files changed, 142 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference create mode 100755 tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh delete mode 100644 tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.reference delete mode 100644 tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.sql diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e0f259306aa..739001e1a3b 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -126,7 +126,7 @@ 02721_url_cluster 02534_s3_cluster_insert_select_schema_inference 02765_parallel_replicas_final_modifier -02784_parallel_replicas_automatic_disabling +02784_parallel_replicas_automatic_decision 02581_share_big_sets_between_mutation_tasks_long 02581_share_big_sets_between_multiple_mutations_tasks_long 00992_system_parts_race_condition_zookeeper_long diff --git a/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh index 6210ef2e8b6..9cfd3a392c8 100755 --- a/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh +++ b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh @@ -5,6 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh function has_used_parallel_replicas () { + # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it $CLICKHOUSE_CLIENT --query " SELECT initial_query_id, @@ -12,7 +13,7 @@ function has_used_parallel_replicas () { sumIf(read_rows, is_initial_query) as read_rows, sumIf(read_bytes, is_initial_query) as read_bytes FROM system.query_log - WHERE event_date >= yesterday() and initial_query_id LIKE '$1%' AND current_database = '$CLICKHOUSE_DATABASE' + WHERE event_date >= yesterday() and initial_query_id LIKE '$1%' GROUP BY initial_query_id ORDER BY min(event_time_microseconds) ASC FORMAT TSV" @@ -34,7 +35,6 @@ function run_query_with_pure_parallel_replicas () { --allow_experimental_parallel_reading_from_replicas 1 \ --allow_experimental_analyzer 0 - # Not implemented yet $CLICKHOUSE_CLIENT \ --query "$2" \ --query_id "${1}_pure_analyzer" \ diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference new file mode 100644 index 00000000000..30da8c9f946 --- /dev/null +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference @@ -0,0 +1,14 @@ +02783_automatic_parallel_replicas-default_0_0_pure 3 +02783_automatic_parallel_replicas-default_0_10M_pure 0 +02783_automatic_parallel_replicas-default_0_6M_pure 0 +02783_automatic_parallel_replicas-default_0_5M_pure 2 +02783_automatic_parallel_replicas-default_0_1M_pure 3 +02783_automatic_parallel_replicas-default_1_0_pure 3 +02783_automatic_parallel_replicas-default_1_10M_pure 0 +02783_automatic_parallel_replicas-default_1_1M_pure 2 +02783_automatic_parallel_replicas-default_1_500k_pure 3 +02783_automatic_parallel_replicas-default_2_0_pure 3 +02783_automatic_parallel_replicas-default_2_1M_pure 0 +02783_automatic_parallel_replicas-default_2_300k_pure 0 +02783_automatic_parallel_replicas-default_2_200k_pure 2 +02783_automatic_parallel_replicas-default_2_100k_pure 3 diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh new file mode 100755 index 00000000000..19aed514c10 --- /dev/null +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -0,0 +1,125 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function involved_parallel_replicas () { + # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it + $CLICKHOUSE_CLIENT --query " + SELECT + initial_query_id, + (count() - 2) / 2 as number_of_parallel_replicas + FROM system.query_log + WHERE event_date >= yesterday() + AND initial_query_id LIKE '$1%' + GROUP BY initial_query_id + ORDER BY min(event_time_microseconds) ASC + FORMAT TSV" +} + +$CLICKHOUSE_CLIENT --query " + CREATE TABLE test_parallel_replicas_automatic_count + ( + number Int64, + p Int64 + ) + ENGINE=MergeTree() + ORDER BY number + PARTITION BY p + SETTINGS index_granularity = 8192 -- Don't randomize it to avoid flakiness + AS + SELECT number, number % 2 AS p FROM numbers(2_000_000) + UNION ALL + SELECT number, 3 AS p FROM numbers(10_000_000, 8_000_000) +" + +function run_query_with_pure_parallel_replicas () { + # $1 -> query_id + # $2 -> min rows per replica + # $3 -> query + $CLICKHOUSE_CLIENT \ + --query "$3" \ + --query_id "${1}_pure" \ + --max_parallel_replicas 3 \ + --prefer_localhost_replica 1 \ + --use_hedged_requests 0 \ + --cluster_for_parallel_replicas 'test_cluster_one_shard_three_replicas_localhost' \ + --allow_experimental_parallel_reading_from_replicas 1 \ + --parallel_replicas_for_non_replicated_merge_tree 1 \ + --parallel_replicas_min_number_of_rows_per_replica "$2" \ + --allow_experimental_analyzer 0 + +# Analyzer: Not implemented yet +# $CLICKHOUSE_CLIENT \ +# --query "$3" \ +# --query_id "${1}_pure_analyzer" \ +# --max_parallel_replicas 3 \ +# --prefer_localhost_replica 1 \ +# --use_hedged_requests 0 \ +# --cluster_for_parallel_replicas 'test_cluster_one_shard_three_replicas_localhost' \ +# --allow_experimental_parallel_reading_from_replicas 1 \ +# --parallel_replicas_for_non_replicated_merge_tree 1 \ +# --parallel_replicas_min_number_of_rows_per_replica "$2" \ +# --allow_experimental_analyzer 0 +} + +function run_query_with_custom_key_parallel_replicas () { + $CLICKHOUSE_CLIENT \ + --query "$3" \ + --query_id "${1}_custom_key" \ + --max_parallel_replicas 3 \ + --use_hedged_requests 0 \ + --parallel_replicas_custom_key_filter_type 'default' \ + --parallel_replicas_custom_key "$2" \ + --parallel_replicas_for_non_replicated_merge_tree 1 \ + --parallel_replicas_min_number_of_rows_per_replica "$2" \ + --allow_experimental_analyzer 0 + + $CLICKHOUSE_CLIENT \ + --query "$3" \ + --query_id "${1}_custom_key_analyzer" \ + --max_parallel_replicas 3 \ + --use_hedged_requests 0 \ + --parallel_replicas_custom_key_filter_type 'default' \ + --parallel_replicas_custom_key "sipHash64(number)" \ + --parallel_replicas_for_non_replicated_merge_tree 1 \ + --parallel_replicas_min_number_of_rows_per_replica "$2" \ + --allow_experimental_analyzer 1 +} + +query_id_base="02783_automatic_parallel_replicas-$CLICKHOUSE_DATABASE" + +#### Reading 10M rows without filters +whole_table_query="SELECT sum(number) FROM test_parallel_replicas_automatic_count format Null" +run_query_with_pure_parallel_replicas "${query_id_base}_0_0" 0 "$whole_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_0_10M" 10000000 "$whole_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_0_6M" 6000000 "$whole_table_query" # 1.6 replicas -> 1 replica -> No parallel replicas +run_query_with_pure_parallel_replicas "${query_id_base}_0_5M" 5000000 "$whole_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_0_1M" 1000000 "$whole_table_query" +# +##### Reading 2M rows without filters as partition (p=3) is pruned completely +query_with_partition_pruning="SELECT sum(number) FROM test_parallel_replicas_automatic_count WHERE p != 3 format Null" +run_query_with_pure_parallel_replicas "${query_id_base}_1_0" 0 "$query_with_partition_pruning" +run_query_with_pure_parallel_replicas "${query_id_base}_1_10M" 10000000 "$query_with_partition_pruning" +run_query_with_pure_parallel_replicas "${query_id_base}_1_1M" 1000000 "$query_with_partition_pruning" +run_query_with_pure_parallel_replicas "${query_id_base}_1_500k" 500000 "$query_with_partition_pruning" + +## Reading ~500k rows as index filter should prune granules from partition=1 and partition=2, and drop p3 completely +query_with_index="SELECT sum(number) FROM test_parallel_replicas_automatic_count WHERE number < 500_000 format Null" +run_query_with_pure_parallel_replicas "${query_id_base}_2_0" 0 "$query_with_index" +run_query_with_pure_parallel_replicas "${query_id_base}_2_1M" 1000000 "$query_with_index" +run_query_with_pure_parallel_replicas "${query_id_base}_2_300k" 300000 "$query_with_index" +run_query_with_pure_parallel_replicas "${query_id_base}_2_200k" 200000 "$query_with_index" +run_query_with_pure_parallel_replicas "${query_id_base}_2_100k" 100000 "$query_with_index" + +# Custom key parallel replicas: Not implemented +#whole_table_query="SELECT sum(number) FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), test_parallel_replicas_automatic_count) format Null" +#run_query_with_custom_key_parallel_replicas "${query_id_base}_0_0" 0 "$whole_table_query" +#run_query_with_custom_key_parallel_replicas "${query_id_base}_0_10M" 10000000 "$whole_table_query" +#run_query_with_custom_key_parallel_replicas "${query_id_base}_0_6M" 6000000 "$whole_table_query" # 1.6 replicas -> 1 replica -> No parallel replicas +#run_query_with_custom_key_parallel_replicas "${query_id_base}_0_5M" 5000000 "$whole_table_query" +#run_query_with_custom_key_parallel_replicas "${query_id_base}_0_1M" 1000000 "$whole_table_query" + +$CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" +involved_parallel_replicas "${query_id_base}" diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.reference deleted file mode 100644 index af81158ecae..00000000000 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.reference +++ /dev/null @@ -1,2 +0,0 @@ -10 -1 diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.sql b/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.sql deleted file mode 100644 index b2f674ddb64..00000000000 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.sql +++ /dev/null @@ -1,15 +0,0 @@ -DROP TABLE IF EXISTS test_parallel_replicas_automatic_disabling; -CREATE TABLE test_parallel_replicas_automatic_disabling (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); -INSERT INTO test_parallel_replicas_automatic_disabling SELECT * FROM numbers(10); - -SYSTEM FLUSH LOGS; - -SET skip_unavailable_shards=1, allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1, parallel_replicas_min_number_of_granules_to_enable=10000; -SET send_logs_level='error'; -SELECT count() FROM test_parallel_replicas_automatic_disabling WHERE NOT ignore(*); - -SYSTEM FLUSH LOGS; - -SELECT count() > 0 FROM system.text_log WHERE event_time >= now() - INTERVAL 2 MINUTE AND message LIKE '%Parallel replicas will be disabled, because the estimated number of granules to read%'; - -DROP TABLE test_parallel_replicas_automatic_disabling; From 088b0527f6e01c9270ceb1465b414e1f78077a6b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 3 Jul 2023 11:36:41 +0200 Subject: [PATCH 009/634] Consider trivial LIMIT when deciding on parallel replicas --- src/Interpreters/InterpreterSelectQuery.cpp | 25 +++++++++++++++++++ ...llel_replicas_automatic_decision.reference | 8 ++++++ ...84_parallel_replicas_automatic_decision.sh | 21 +++++++++++++--- 3 files changed, 51 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c32ef4349f8..5dc43953d82 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -108,6 +108,9 @@ namespace ProfileEvents namespace DB { +static UInt64 getLimitUIntValue(const ASTPtr & node, const ContextPtr & context, const std::string & expr); +static std::pair getLimitLengthAndOffset(const ASTSelectQuery & query, const ContextPtr & context); + namespace ErrorCodes { extern const int TOO_DEEP_SUBQUERIES; @@ -869,6 +872,28 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (analysis_result.before_where) added_filter_nodes.nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name)); + + auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); + + auto local_limits = getStorageLimits(*context, options); + + if (!query.distinct + && !query.limit_with_ties + && !query.prewhere() + && !query.where() + && query_info.filter_asts.empty() + && !query.groupBy() + && !query.having() + && !query.orderBy() + && !query.limitBy() + && !query.join() + && !query_analyzer->hasAggregation() + && !query_analyzer->hasWindow() + && query.limitLength() + && limit_length <= std::numeric_limits::max() - limit_offset) + { + query_info.limit = limit_length + limit_offset; + } /// END OF TODO BLOCK ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ rows_to_read.emplace( diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference index 30da8c9f946..739bae1e047 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference @@ -12,3 +12,11 @@ 02783_automatic_parallel_replicas-default_2_300k_pure 0 02783_automatic_parallel_replicas-default_2_200k_pure 2 02783_automatic_parallel_replicas-default_2_100k_pure 3 +02783_automatic_parallel_replicas-default_3_0_pure 3 +02783_automatic_parallel_replicas-default_3_10M_pure 0 +02783_automatic_parallel_replicas-default_3_1M_pure 0 +02783_automatic_parallel_replicas-default_3_500k_pure 2 +02783_automatic_parallel_replicas-default_4_0_pure 3 +02783_automatic_parallel_replicas-default_4_10M_pure 0 +02783_automatic_parallel_replicas-default_4_1M_pure 3 +02783_automatic_parallel_replicas-default_4_500k_pure 3 diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 19aed514c10..4a89f5731af 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -97,7 +97,7 @@ run_query_with_pure_parallel_replicas "${query_id_base}_0_10M" 10000000 "$whole_ run_query_with_pure_parallel_replicas "${query_id_base}_0_6M" 6000000 "$whole_table_query" # 1.6 replicas -> 1 replica -> No parallel replicas run_query_with_pure_parallel_replicas "${query_id_base}_0_5M" 5000000 "$whole_table_query" run_query_with_pure_parallel_replicas "${query_id_base}_0_1M" 1000000 "$whole_table_query" -# + ##### Reading 2M rows without filters as partition (p=3) is pruned completely query_with_partition_pruning="SELECT sum(number) FROM test_parallel_replicas_automatic_count WHERE p != 3 format Null" run_query_with_pure_parallel_replicas "${query_id_base}_1_0" 0 "$query_with_partition_pruning" @@ -105,7 +105,7 @@ run_query_with_pure_parallel_replicas "${query_id_base}_1_10M" 10000000 "$query_ run_query_with_pure_parallel_replicas "${query_id_base}_1_1M" 1000000 "$query_with_partition_pruning" run_query_with_pure_parallel_replicas "${query_id_base}_1_500k" 500000 "$query_with_partition_pruning" -## Reading ~500k rows as index filter should prune granules from partition=1 and partition=2, and drop p3 completely +#### Reading ~500k rows as index filter should prune granules from partition=1 and partition=2, and drop p3 completely query_with_index="SELECT sum(number) FROM test_parallel_replicas_automatic_count WHERE number < 500_000 format Null" run_query_with_pure_parallel_replicas "${query_id_base}_2_0" 0 "$query_with_index" run_query_with_pure_parallel_replicas "${query_id_base}_2_1M" 1000000 "$query_with_index" @@ -113,7 +113,22 @@ run_query_with_pure_parallel_replicas "${query_id_base}_2_300k" 300000 "$query_w run_query_with_pure_parallel_replicas "${query_id_base}_2_200k" 200000 "$query_with_index" run_query_with_pure_parallel_replicas "${query_id_base}_2_100k" 100000 "$query_with_index" -# Custom key parallel replicas: Not implemented +#### Reading 1M (because of LIMIT) +limit_table_query="SELECT sum(number) FROM (SELECT number FROM test_parallel_replicas_automatic_count LIMIT 1_000_000) format Null" +run_query_with_pure_parallel_replicas "${query_id_base}_3_0" 0 "$limit_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_3_10M" 10000000 "$limit_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_3_1M" 1000000 "$limit_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_3_500k" 500000 "$limit_table_query" + +#### Reading 10M (because of LIMIT is applied after aggregations) +limit_agg_table_query="SELECT sum(number) FROM test_parallel_replicas_automatic_count LIMIT 1_000_000 format Null" +run_query_with_pure_parallel_replicas "${query_id_base}_4_0" 0 "$limit_agg_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_4_10M" 10000000 "$limit_agg_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_4_1M" 1000000 "$limit_agg_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_4_500k" 500000 "$limit_agg_table_query" + + +#### Custom key parallel replicas: Not implemented #whole_table_query="SELECT sum(number) FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), test_parallel_replicas_automatic_count) format Null" #run_query_with_custom_key_parallel_replicas "${query_id_base}_0_0" 0 "$whole_table_query" #run_query_with_custom_key_parallel_replicas "${query_id_base}_0_10M" 10000000 "$whole_table_query" From dacc8cb4c9a41a8c1aab7427923e1b2818063ba3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 3 Jul 2023 12:46:29 +0200 Subject: [PATCH 010/634] Don't attempt to work with not mergetree tables --- src/Interpreters/InterpreterSelectQuery.cpp | 89 +++++++-------------- 1 file changed, 31 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5dc43953d82..1793299a874 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -849,71 +849,44 @@ InterpreterSelectQuery::InterpreterSelectQuery( need_analyze_again = true; LOG_DEBUG(log, "Disabling parallel replicas to be able to use a trivial count optimization"); } - else if (settings.parallel_replicas_min_number_of_rows_per_replica > 0) + else if (auto storage_merge_tree = std::dynamic_pointer_cast(storage); + storage_merge_tree && settings.parallel_replicas_min_number_of_rows_per_replica) { - std::optional rows_to_read{}; - - auto storage_merge_tree = std::dynamic_pointer_cast(storage); - if (storage_merge_tree) + /// TODO: Improve this block as this should only happen once, right? vvvvvvvvvvvvvvvvvvvvvv + addPrewhereAliasActions(); + auto & prewhere_info = analysis_result.prewhere_info; + if (prewhere_info) { - /// TODO: Improve this block as this should only happen once, right? vvvvvvvvvvvvvvvvvvvvvv - addPrewhereAliasActions(); - auto & prewhere_info = analysis_result.prewhere_info; - if (prewhere_info) - { - query_info.prewhere_info = prewhere_info; - if (query.prewhere() && !query.where()) - query_info.prewhere_info->need_filter = true; - } - - ActionDAGNodes added_filter_nodes; - if (additional_filter_info) - added_filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name)); - - if (analysis_result.before_where) - added_filter_nodes.nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name)); - - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); - - auto local_limits = getStorageLimits(*context, options); - - if (!query.distinct - && !query.limit_with_ties - && !query.prewhere() - && !query.where() - && query_info.filter_asts.empty() - && !query.groupBy() - && !query.having() - && !query.orderBy() - && !query.limitBy() - && !query.join() - && !query_analyzer->hasAggregation() - && !query_analyzer->hasWindow() - && query.limitLength() - && limit_length <= std::numeric_limits::max() - limit_offset) - { - query_info.limit = limit_length + limit_offset; - } - /// END OF TODO BLOCK ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - - rows_to_read.emplace( - storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info, added_filter_nodes)); + query_info.prewhere_info = prewhere_info; + if (query.prewhere() && !query.where()) + query_info.prewhere_info->need_filter = true; } - if (!rows_to_read.has_value()) - rows_to_read = storage->totalRows(settings); + ActionDAGNodes added_filter_nodes; + if (additional_filter_info) + added_filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name)); - /// Open question: How should we treat 0 estimated rows to read? - /// It is a real estimation of 0 rows? - size_t number_of_replicas_to_use = rows_to_read.has_value() - ? *rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica - : settings.allow_experimental_parallel_reading_from_replicas; + if (analysis_result.before_where) + added_filter_nodes.nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name)); + auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); + + auto local_limits = getStorageLimits(*context, options); + + if (!query.distinct && !query.limit_with_ties && !query.prewhere() && !query.where() && query_info.filter_asts.empty() + && !query.groupBy() && !query.having() && !query.orderBy() && !query.limitBy() && !query.join() + && !query_analyzer->hasAggregation() && !query_analyzer->hasWindow() && query.limitLength() + && limit_length <= std::numeric_limits::max() - limit_offset) + { + query_info.limit = limit_length + limit_offset; + } + /// END OF TODO BLOCK ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + + UInt64 rows_to_read = storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info, added_filter_nodes); + /// Note that we treat an estimation of 0 rows as a real estimation of no data to be read + size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; LOG_TRACE( - log, - "Estimated {} rows to read, which is work enough for {} parallel replicas", - rows_to_read.has_value() ? *rows_to_read : 0, - number_of_replicas_to_use); + log, "Estimated {} rows to read, which is work enough for {} parallel replicas", rows_to_read, number_of_replicas_to_use); if (number_of_replicas_to_use <= 1) { From 97717e0e88420fd17973d5bcbb8816d7ad811226 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 4 Jul 2023 11:52:00 +0200 Subject: [PATCH 011/634] Add tests for parallel replicas JOINs --- ...llel_replicas_automatic_decision.reference | 5 ++++ ...84_parallel_replicas_automatic_decision.sh | 24 +++++++++++++++++++ 2 files changed, 29 insertions(+) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference index 739bae1e047..e56fe63f489 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference @@ -20,3 +20,8 @@ 02783_automatic_parallel_replicas-default_4_10M_pure 0 02783_automatic_parallel_replicas-default_4_1M_pure 3 02783_automatic_parallel_replicas-default_4_500k_pure 3 +02783_automatic_parallel_replicas-default_simple_join_0_pure 6 +02783_automatic_parallel_replicas-default_simple_join_10M_pure 0 +02783_automatic_parallel_replicas-default_simple_join_5M_pure 2 +02783_automatic_parallel_replicas-default_simple_join_1M_pure 3 +02783_automatic_parallel_replicas-default_simple_join_300k_pure 5 diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 4a89f5731af..4fd4d0d59f0 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -34,6 +34,20 @@ $CLICKHOUSE_CLIENT --query " SELECT number, 3 AS p FROM numbers(10_000_000, 8_000_000) " +$CLICKHOUSE_CLIENT --query " + CREATE TABLE test_parallel_replicas_automatic_count_right_side + ( + number Int64, + value Int64 + ) + ENGINE=MergeTree() + ORDER BY number + SETTINGS index_granularity = 8192 -- Don't randomize it to avoid flakiness + AS + SELECT number, number % 2 AS v FROM numbers(1_000_000) +" + + function run_query_with_pure_parallel_replicas () { # $1 -> query_id # $2 -> min rows per replica @@ -127,6 +141,16 @@ run_query_with_pure_parallel_replicas "${query_id_base}_4_10M" 10000000 "$limit_ run_query_with_pure_parallel_replicas "${query_id_base}_4_1M" 1000000 "$limit_agg_table_query" run_query_with_pure_parallel_replicas "${query_id_base}_4_500k" 500000 "$limit_agg_table_query" +#### JOIN (left side 10M, right side 1M) +#### As the right side of the JOIN is a table, ideally it shouldn't be executed with parallel replicas and instead passed as is to the replicas +#### so each of them executes the join with the assigned granules of the left table, but that's not implemented yet +#### https://github.com/ClickHouse/ClickHouse/issues/49301#issuecomment-1619897920 +simple_join_query="SELECT sum(value) FROM test_parallel_replicas_automatic_count INNER JOIN test_parallel_replicas_automatic_count_right_side USING number format Null" +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_0" 0 "$simple_join_query" # 3 replicas for the right side first, 3 replicas for the left +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_10M" 10000000 "$simple_join_query" # Right: 0. Left: 0 +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_5M" 5000000 "$simple_join_query" # Right: 0. Left: 2 +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_1M" 1000000 "$simple_join_query" # Right: 1->0. Left: 10->3 +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_300k" 400000 "$simple_join_query" # Right: 2. Left: 3 #### Custom key parallel replicas: Not implemented #whole_table_query="SELECT sum(number) FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), test_parallel_replicas_automatic_count) format Null" From d5c332327d378e8af55e2498652896cf4cb7f08e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 4 Jul 2023 18:50:00 +0200 Subject: [PATCH 012/634] Start cleaning up things --- src/Interpreters/InterpreterSelectQuery.cpp | 21 ++++++---------- src/Storages/MergeTree/MergeTreeData.cpp | 27 +++++++++++++-------- src/Storages/MergeTree/MergeTreeData.h | 4 +++ 3 files changed, 29 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 1793299a874..03f03f13069 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -852,37 +852,32 @@ InterpreterSelectQuery::InterpreterSelectQuery( else if (auto storage_merge_tree = std::dynamic_pointer_cast(storage); storage_merge_tree && settings.parallel_replicas_min_number_of_rows_per_replica) { + auto query_info_copy = query_info; /// TODO: Improve this block as this should only happen once, right? vvvvvvvvvvvvvvvvvvvvvv addPrewhereAliasActions(); auto & prewhere_info = analysis_result.prewhere_info; if (prewhere_info) { - query_info.prewhere_info = prewhere_info; + query_info_copy.prewhere_info = prewhere_info; if (query.prewhere() && !query.where()) - query_info.prewhere_info->need_filter = true; + query_info_copy.prewhere_info->need_filter = true; } - ActionDAGNodes added_filter_nodes; - if (additional_filter_info) - added_filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name)); - - if (analysis_result.before_where) - added_filter_nodes.nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name)); + ActionDAGNodes added_filter_nodes = MergeTreeData::getFiltersForPrimaryKeyAnalysis(*this); auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); - auto local_limits = getStorageLimits(*context, options); - - if (!query.distinct && !query.limit_with_ties && !query.prewhere() && !query.where() && query_info.filter_asts.empty() + if (!query.distinct && !query.limit_with_ties && !query.prewhere() && !query.where() && query_info_copy.filter_asts.empty() && !query.groupBy() && !query.having() && !query.orderBy() && !query.limitBy() && !query.join() && !query_analyzer->hasAggregation() && !query_analyzer->hasWindow() && query.limitLength() && limit_length <= std::numeric_limits::max() - limit_offset) { - query_info.limit = limit_length + limit_offset; + query_info_copy.limit = limit_length + limit_offset; } /// END OF TODO BLOCK ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - UInt64 rows_to_read = storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info, added_filter_nodes); + UInt64 rows_to_read + = storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info_copy, added_filter_nodes); /// Note that we treat an estimation of 0 rows as a real estimation of no data to be read size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; LOG_TRACE( diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 927d9459835..ba8b89f53e6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6697,6 +6697,22 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( return res; } +ActionDAGNodes MergeTreeData::getFiltersForPrimaryKeyAnalysis(const InterpreterSelectQuery & select) +{ + const auto & analysis_result = select.getAnalysisResult(); + const auto & before_where = analysis_result.before_where; + const auto & where_column_name = analysis_result.where_column_name; + + ActionDAGNodes filter_nodes; + if (auto additional_filter_info = select.getAdditionalQueryInfo()) + filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name)); + + if (before_where) + filter_nodes.nodes.push_back(&before_where->findInOutputs(where_column_name)); + + return filter_nodes; +} + std::optional MergeTreeData::getQueryProcessingStageWithAggregateProjection( ContextPtr query_context, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info) const @@ -6778,19 +6794,10 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg query_info.prepared_sets); const auto & analysis_result = select.getAnalysisResult(); - query_info.prepared_sets = select.getQueryAnalyzer()->getPreparedSets(); - const auto & before_where = analysis_result.before_where; - const auto & where_column_name = analysis_result.where_column_name; - /// For PK analysis - ActionDAGNodes added_filter_nodes; - if (auto additional_filter_info = select.getAdditionalQueryInfo()) - added_filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name)); - - if (before_where) - added_filter_nodes.nodes.push_back(&before_where->findInOutputs(where_column_name)); + ActionDAGNodes added_filter_nodes = MergeTreeData::getFiltersForPrimaryKeyAnalysis(select); bool can_use_aggregate_projection = true; /// If the first stage of the query pipeline is more complex than Aggregating - Expression - Filter - ReadFromStorage, diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5bbc6758dd3..a78a5fcc594 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -48,6 +48,7 @@ namespace DB const size_t DEFAULT_DELAYED_STREAMS_FOR_PARALLEL_WRITE = 1000; class AlterCommands; +class InterpreterSelectQuery; class MergeTreePartsMover; class MergeTreeDataMergerMutator; class MutationCommands; @@ -1062,6 +1063,9 @@ public: /// TODO: make enabled by default in the next release if no problems found. bool allowRemoveStaleMovingParts() const; + /// Generate DAG filters based on query info (for PK analysis) + static struct ActionDAGNodes getFiltersForPrimaryKeyAnalysis(const InterpreterSelectQuery & select); + /// Estimate the number of rows to read based on primary key analysis (which could be very rough) /// It is used to make a decision whether to enable parallel replicas (distributed processing) or not and how /// many to replicas to use From 35b315beffdffeaca37e04fddd7b920ef2af7b89 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 5 Jul 2023 13:57:38 +0200 Subject: [PATCH 013/634] Add test from #51805 --- ...84_parallel_replicas_automatic_decision.sh | 2 +- ...parallel_replicas_prewhere_count.reference | 4 ++++ ...02811_parallel_replicas_prewhere_count.sql | 24 +++++++++++++++++++ 3 files changed, 29 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.reference create mode 100644 tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 4fd4d0d59f0..426c77398f3 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -58,7 +58,7 @@ function run_query_with_pure_parallel_replicas () { --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ --use_hedged_requests 0 \ - --cluster_for_parallel_replicas 'test_cluster_one_shard_three_replicas_localhost' \ + --cluster_for_parallel_replicas 'parallel_replicas' \ --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica "$2" \ diff --git a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.reference b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.reference new file mode 100644 index 00000000000..fe8f022b908 --- /dev/null +++ b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.reference @@ -0,0 +1,4 @@ +-- count() ------------------------------ +2 +-- count() with parallel replicas ------- +2 diff --git a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql new file mode 100644 index 00000000000..18c1e0a67f8 --- /dev/null +++ b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql @@ -0,0 +1,24 @@ +CREATE TABLE users (uid Int16, name String, age Int16) ENGINE=MergeTree() ORDER BY uid; + +INSERT INTO users VALUES (111, 'JFK', 33); +INSERT INTO users VALUES (6666, 'KLM', 48); +INSERT INTO users VALUES (88888, 'AMS', 50); + + +SELECT '-- count() ------------------------------'; +SELECT count() FROM users PREWHERE uid > 2000; + + +-- enable parallel replicas but with high rows threshold +SET +skip_unavailable_shards=1, +allow_experimental_parallel_reading_from_replicas=1, +max_parallel_replicas=3, +use_hedged_requests=0, +cluster_for_parallel_replicas='parallel_replicas', +parallel_replicas_for_non_replicated_merge_tree=1, +parallel_replicas_min_number_of_rows_per_replica=1000; + + +SELECT '-- count() with parallel replicas -------'; +SELECT count() FROM users PREWHERE uid > 2000; From dd3917760fcc1cbe019c683ea5b4632aefeea91d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 5 Jul 2023 16:05:40 +0200 Subject: [PATCH 014/634] Partially cleanup the TODO --- src/Interpreters/InterpreterSelectQuery.cpp | 191 +++++++++++--------- src/Interpreters/InterpreterSelectQuery.h | 4 + 2 files changed, 112 insertions(+), 83 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 03f03f13069..74beeca0845 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -836,68 +836,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( need_analyze_again = true; } - if (storage - /// While only analyzing we don't know anything about parts, so any decision about parallel would be wrong - && !options.only_analyze && can_analyze_again && settings.max_parallel_replicas > 1 - && settings.allow_experimental_parallel_reading_from_replicas > 0 && settings.parallel_replicas_custom_key.value.empty()) - { - if (getTrivialCount(0).has_value()) - { - /// The query could use trivial count if it didn't use parallel replicas, so let's disable it and reanalyze - context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); - context->setSetting("max_parallel_replicas", UInt64{0}); - need_analyze_again = true; - LOG_DEBUG(log, "Disabling parallel replicas to be able to use a trivial count optimization"); - } - else if (auto storage_merge_tree = std::dynamic_pointer_cast(storage); - storage_merge_tree && settings.parallel_replicas_min_number_of_rows_per_replica) - { - auto query_info_copy = query_info; - /// TODO: Improve this block as this should only happen once, right? vvvvvvvvvvvvvvvvvvvvvv - addPrewhereAliasActions(); - auto & prewhere_info = analysis_result.prewhere_info; - if (prewhere_info) - { - query_info_copy.prewhere_info = prewhere_info; - if (query.prewhere() && !query.where()) - query_info_copy.prewhere_info->need_filter = true; - } - - ActionDAGNodes added_filter_nodes = MergeTreeData::getFiltersForPrimaryKeyAnalysis(*this); - - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); - auto local_limits = getStorageLimits(*context, options); - if (!query.distinct && !query.limit_with_ties && !query.prewhere() && !query.where() && query_info_copy.filter_asts.empty() - && !query.groupBy() && !query.having() && !query.orderBy() && !query.limitBy() && !query.join() - && !query_analyzer->hasAggregation() && !query_analyzer->hasWindow() && query.limitLength() - && limit_length <= std::numeric_limits::max() - limit_offset) - { - query_info_copy.limit = limit_length + limit_offset; - } - /// END OF TODO BLOCK ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - - UInt64 rows_to_read - = storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info_copy, added_filter_nodes); - /// Note that we treat an estimation of 0 rows as a real estimation of no data to be read - size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; - LOG_TRACE( - log, "Estimated {} rows to read, which is work enough for {} parallel replicas", rows_to_read, number_of_replicas_to_use); - - if (number_of_replicas_to_use <= 1) - { - context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); - context->setSetting("max_parallel_replicas", UInt64{0}); - need_analyze_again = true; - LOG_DEBUG(log, "Disabling parallel replicas because there aren't enough rows to read"); - } - else if (number_of_replicas_to_use < settings.max_parallel_replicas) - { - /// TODO: Confirm that reducing the number of parallel replicas doesn't require a re-analysis - context->setSetting("max_parallel_replicas", number_of_replicas_to_use); - LOG_DEBUG(log, "Reducing the number of replicas to use to {}", number_of_replicas_to_use); - } - } - } + if (can_analyze_again) + need_analyze_again |= adjustParallelReplicasAfterAnalysis(); if (need_analyze_again) { @@ -947,6 +887,72 @@ InterpreterSelectQuery::InterpreterSelectQuery( sanitizeBlock(result_header, true); } +bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis() +{ + const Settings & settings = context->getSettingsRef(); + ASTSelectQuery & query = getSelectQuery(); + + /// While only_analyze we don't know anything about parts, so any decision about how many parallel replicas to use would be wrong + if (!storage || options.only_analyze || settings.max_parallel_replicas <= 1 + || settings.allow_experimental_parallel_reading_from_replicas == 0 || !settings.parallel_replicas_custom_key.value.empty()) + return false; + + if (getTrivialCount(0).has_value()) + { + /// The query could use trivial count if it didn't use parallel replicas, so let's disable it and reanalyze + context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + context->setSetting("max_parallel_replicas", UInt64{0}); + LOG_DEBUG(log, "Disabling parallel replicas to be able to use a trivial count optimization"); + return true; + } + + auto storage_merge_tree = std::dynamic_pointer_cast(storage); + if (!storage_merge_tree || settings.parallel_replicas_min_number_of_rows_per_replica == 0) + return false; + + auto query_info_copy = query_info; + + /// There is a couple of instances where we there might be a lower limit on the rows to read + /// * The settings.max_rows_to_read setting + /// * A LIMIT in a simple query (see maxBlockSizeByLimit) + UInt64 max_rows = maxBlockSizeByLimit(); + if (settings.max_rows_to_read) + max_rows = max_rows ? std::min(max_rows, settings.max_rows_to_read.value) : settings.max_rows_to_read; + query_info_copy.limit = max_rows; + + /// TODO: Improve this block as this should only happen once, right? vvvvvvvvvvvvvvvvvvvvvv + addPrewhereAliasActions(); + auto & prewhere_info = analysis_result.prewhere_info; + if (prewhere_info) + { + query_info_copy.prewhere_info = prewhere_info; + if (query.prewhere() && !query.where()) + query_info_copy.prewhere_info->need_filter = true; + } + /// END OF TODO BLOCK ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + + ActionDAGNodes added_filter_nodes = MergeTreeData::getFiltersForPrimaryKeyAnalysis(*this); + UInt64 rows_to_read = storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info_copy, added_filter_nodes); + /// Note that we treat an estimation of 0 rows as a real estimation of no data to be read + size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; + LOG_TRACE(log, "Estimated {} rows to read, which is work enough for {} parallel replicas", rows_to_read, number_of_replicas_to_use); + + if (number_of_replicas_to_use <= 1) + { + context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + context->setSetting("max_parallel_replicas", UInt64{0}); + LOG_DEBUG(log, "Disabling parallel replicas because there aren't enough rows to read"); + return true; + } + else if (number_of_replicas_to_use < settings.max_parallel_replicas) + { + context->setSetting("max_parallel_replicas", number_of_replicas_to_use); + LOG_DEBUG(log, "Reducing the number of replicas to use to {}", number_of_replicas_to_use); + /// No need to reanalyze + } + return false; +} + void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) { executeImpl(query_plan, std::move(input_pipe)); @@ -2354,6 +2360,41 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle } } +/** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, JOIN, LIMIT BY, WITH TIES + * but LIMIT is specified, and limit + offset < max_block_size, + * then as the block size we will use limit + offset (not to read more from the table than requested), + * and also set the number of threads to 1. + */ +UInt64 InterpreterSelectQuery::maxBlockSizeByLimit() const +{ + const auto & query = query_ptr->as(); + + auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); + + /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, JOIN, LIMIT BY, WITH TIES + * but LIMIT is specified, and limit + offset < max_block_size, + * then as the block size we will use limit + offset (not to read more from the table than requested), + * and also set the number of threads to 1. + */ + if (!query.distinct + && !query.limit_with_ties + && !query.prewhere() + && !query.where() + && query_info.filter_asts.empty() + && !query.groupBy() + && !query.having() + && !query.orderBy() + && !query.limitBy() + && !query.join() + && !query_analyzer->hasAggregation() + && !query_analyzer->hasWindow() + && query.limitLength() + && limit_length <= std::numeric_limits::max() - limit_offset) + return limit_length + limit_offset; + + return 0; +} + void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan) { auto & query = getSelectQuery(); @@ -2424,9 +2465,6 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc } UInt64 max_block_size = settings.max_block_size; - - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); - auto local_limits = getStorageLimits(*context, options); /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, JOIN, LIMIT BY, WITH TIES @@ -2434,29 +2472,16 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc * then as the block size we will use limit + offset (not to read more from the table than requested), * and also set the number of threads to 1. */ - if (!query.distinct - && !query.limit_with_ties - && !query.prewhere() - && !query.where() - && query_info.filter_asts.empty() - && !query.groupBy() - && !query.having() - && !query.orderBy() - && !query.limitBy() - && !query.join() - && !query_analyzer->hasAggregation() - && !query_analyzer->hasWindow() - && query.limitLength() - && limit_length <= std::numeric_limits::max() - limit_offset) + if (UInt64 max_block_limited = maxBlockSizeByLimit()) { - if (limit_length + limit_offset < max_block_size) + if (max_block_limited < max_block_size) { - max_block_size = std::max(1, limit_length + limit_offset); + max_block_size = std::max(1, max_block_limited); max_threads_execute_query = max_streams = 1; } - if (limit_length + limit_offset < local_limits.local_limits.size_limits.max_rows) + if (max_block_limited < local_limits.local_limits.size_limits.max_rows) { - query_info.limit = limit_length + limit_offset; + query_info.limit = max_block_limited; } } diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 0739e818cd6..28063f2b0ee 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -189,6 +189,10 @@ private: void executeSubqueriesInSetsAndJoins(QueryPlan & query_plan); bool autoFinalOnQuery(ASTSelectQuery & select_query); std::optional getTrivialCount(UInt64 max_parallel_replicas); + /// Check if we can limit block size to read based on LIMIT clause + UInt64 maxBlockSizeByLimit() const; + /// Adjust the parallel replicas settings (enabled, disabled) based on the query analysis + bool adjustParallelReplicasAfterAnalysis(); enum class Modificator { From 0d9888e183f8f5942c56a10d4f9298bfe795c72d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jul 2023 10:08:27 +0200 Subject: [PATCH 015/634] Cleanup parallel replica adjustement --- src/Interpreters/InterpreterSelectQuery.cpp | 80 +++++++++++---------- src/Interpreters/InterpreterSelectQuery.h | 3 +- 2 files changed, 44 insertions(+), 39 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 74beeca0845..a26e34fb936 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -907,35 +907,34 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis() } auto storage_merge_tree = std::dynamic_pointer_cast(storage); - if (!storage_merge_tree || settings.parallel_replicas_min_number_of_rows_per_replica == 0) + if (!storage_merge_tree || settings.parallel_replicas_min_number_of_rows_per_replica <= 1) return false; auto query_info_copy = query_info; + auto analysis_copy = analysis_result; - /// There is a couple of instances where we there might be a lower limit on the rows to read - /// * The settings.max_rows_to_read setting - /// * A LIMIT in a simple query (see maxBlockSizeByLimit) + /// There is a couple of instances where there might be a lower limit on the rows to be read + /// * The max_rows_to_read setting + /// * A LIMIT in a simple query (see maxBlockSizeByLimit()) UInt64 max_rows = maxBlockSizeByLimit(); if (settings.max_rows_to_read) max_rows = max_rows ? std::min(max_rows, settings.max_rows_to_read.value) : settings.max_rows_to_read; query_info_copy.limit = max_rows; - /// TODO: Improve this block as this should only happen once, right? vvvvvvvvvvvvvvvvvvvvvv - addPrewhereAliasActions(); - auto & prewhere_info = analysis_result.prewhere_info; - if (prewhere_info) + /// Apply filters to prewhere and add them to the query_info so we can filter out parts efficiently during row estimation + applyFiltersToPrewhereInAnalysis(analysis_copy); + if (analysis_copy.prewhere_info) { - query_info_copy.prewhere_info = prewhere_info; + query_info_copy.prewhere_info = analysis_copy.prewhere_info; if (query.prewhere() && !query.where()) query_info_copy.prewhere_info->need_filter = true; } - /// END OF TODO BLOCK ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ActionDAGNodes added_filter_nodes = MergeTreeData::getFiltersForPrimaryKeyAnalysis(*this); UInt64 rows_to_read = storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info_copy, added_filter_nodes); - /// Note that we treat an estimation of 0 rows as a real estimation of no data to be read + /// Note that we treat an estimation of 0 rows as a real estimation size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; - LOG_TRACE(log, "Estimated {} rows to read, which is work enough for {} parallel replicas", rows_to_read, number_of_replicas_to_use); + LOG_TRACE(log, "Estimated {} rows to read. It is enough work for {} parallel replicas", rows_to_read, number_of_replicas_to_use); if (number_of_replicas_to_use <= 1) { @@ -2128,43 +2127,48 @@ void InterpreterSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, cons } } -bool InterpreterSelectQuery::shouldMoveToPrewhere() +bool InterpreterSelectQuery::shouldMoveToPrewhere() const { const Settings & settings = context->getSettingsRef(); - const ASTSelectQuery & query = getSelectQuery(); + const ASTSelectQuery & query = query_ptr->as(); return settings.optimize_move_to_prewhere && (!query.final() || settings.optimize_move_to_prewhere_if_final); } -void InterpreterSelectQuery::addPrewhereAliasActions() +/// Note that this is const and accepts the analysis ref to be able to use it to do analysis for parallel replicas +/// without affecting the final analysis multiple times +void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysisResult & analysis) const { - auto & expressions = analysis_result; - if (expressions.filter_info) + if (!analysis.filter_info) + return; + + if (!analysis.prewhere_info) { - if (!expressions.prewhere_info) + const bool does_storage_support_prewhere = !input_pipe && storage && storage->supportsPrewhere(); + if (does_storage_support_prewhere && shouldMoveToPrewhere()) { - const bool does_storage_support_prewhere = !input_pipe && storage && storage->supportsPrewhere(); - if (does_storage_support_prewhere && shouldMoveToPrewhere()) - { - /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. - expressions.prewhere_info = std::make_shared( - std::move(expressions.filter_info->actions), - std::move(expressions.filter_info->column_name)); - expressions.prewhere_info->prewhere_actions->projectInput(false); - expressions.prewhere_info->remove_prewhere_column = expressions.filter_info->do_remove_column; - expressions.prewhere_info->need_filter = true; - expressions.filter_info = nullptr; - } - } - else - { - /// Add row level security actions to prewhere. - expressions.prewhere_info->row_level_filter = std::move(expressions.filter_info->actions); - expressions.prewhere_info->row_level_column_name = std::move(expressions.filter_info->column_name); - expressions.prewhere_info->row_level_filter->projectInput(false); - expressions.filter_info = nullptr; + /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. + analysis.prewhere_info + = std::make_shared(std::move(analysis.filter_info->actions), std::move(analysis.filter_info->column_name)); + analysis.prewhere_info->prewhere_actions->projectInput(false); + analysis.prewhere_info->remove_prewhere_column = analysis.filter_info->do_remove_column; + analysis.prewhere_info->need_filter = true; + analysis.filter_info = nullptr; } } + else + { + /// Add row level security actions to prewhere. + analysis.prewhere_info->row_level_filter = std::move(analysis.filter_info->actions); + analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); + analysis.prewhere_info->row_level_filter->projectInput(false); + analysis.filter_info = nullptr; + } +} + +void InterpreterSelectQuery::addPrewhereAliasActions() +{ + applyFiltersToPrewhereInAnalysis(analysis_result); auto & prewhere_info = analysis_result.prewhere_info; auto & columns_to_remove_after_prewhere = analysis_result.columns_to_remove_after_prewhere; diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 28063f2b0ee..9b937071834 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -158,7 +158,8 @@ private: ASTSelectQuery & getSelectQuery() { return query_ptr->as(); } void addPrewhereAliasActions(); - bool shouldMoveToPrewhere(); + void applyFiltersToPrewhereInAnalysis(ExpressionAnalysisResult & analysis) const; + bool shouldMoveToPrewhere() const; Block getSampleBlockImpl(); From 9d0cc5652f839b12c7506d7ea3554e0ac376637e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jul 2023 12:58:55 +0200 Subject: [PATCH 016/634] More tests and better names --- ...llel_replicas_automatic_decision.reference | 48 +++++++++-------- ...84_parallel_replicas_automatic_decision.sh | 51 +++++++++++-------- 2 files changed, 55 insertions(+), 44 deletions(-) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference index e56fe63f489..c052a5cc67e 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference @@ -1,27 +1,31 @@ -02783_automatic_parallel_replicas-default_0_0_pure 3 -02783_automatic_parallel_replicas-default_0_10M_pure 0 -02783_automatic_parallel_replicas-default_0_6M_pure 0 -02783_automatic_parallel_replicas-default_0_5M_pure 2 -02783_automatic_parallel_replicas-default_0_1M_pure 3 -02783_automatic_parallel_replicas-default_1_0_pure 3 -02783_automatic_parallel_replicas-default_1_10M_pure 0 -02783_automatic_parallel_replicas-default_1_1M_pure 2 -02783_automatic_parallel_replicas-default_1_500k_pure 3 -02783_automatic_parallel_replicas-default_2_0_pure 3 -02783_automatic_parallel_replicas-default_2_1M_pure 0 -02783_automatic_parallel_replicas-default_2_300k_pure 0 -02783_automatic_parallel_replicas-default_2_200k_pure 2 -02783_automatic_parallel_replicas-default_2_100k_pure 3 -02783_automatic_parallel_replicas-default_3_0_pure 3 -02783_automatic_parallel_replicas-default_3_10M_pure 0 -02783_automatic_parallel_replicas-default_3_1M_pure 0 -02783_automatic_parallel_replicas-default_3_500k_pure 2 -02783_automatic_parallel_replicas-default_4_0_pure 3 -02783_automatic_parallel_replicas-default_4_10M_pure 0 -02783_automatic_parallel_replicas-default_4_1M_pure 3 -02783_automatic_parallel_replicas-default_4_500k_pure 3 +02783_automatic_parallel_replicas-default_whole_table_0_pure 3 +02783_automatic_parallel_replicas-default_whole_table_10M_pure 0 +02783_automatic_parallel_replicas-default_whole_table_6M_pure 0 +02783_automatic_parallel_replicas-default_whole_table_5M_pure 2 +02783_automatic_parallel_replicas-default_whole_table_1M_pure 3 +02783_automatic_parallel_replicas-default_pruning_0_pure 3 +02783_automatic_parallel_replicas-default_pruning_10M_pure 0 +02783_automatic_parallel_replicas-default_pruning_1M_pure 2 +02783_automatic_parallel_replicas-default_pruning_500k_pure 3 +02783_automatic_parallel_replicas-default_index_0_pure 3 +02783_automatic_parallel_replicas-default_index_1M_pure 0 +02783_automatic_parallel_replicas-default_index_300k_pure 0 +02783_automatic_parallel_replicas-default_index_200k_pure 2 +02783_automatic_parallel_replicas-default_index_100k_pure 3 +02783_automatic_parallel_replicas-default_limit_0_pure 3 +02783_automatic_parallel_replicas-default_limit_10M_pure 0 +02783_automatic_parallel_replicas-default_limit_1M_pure 0 +02783_automatic_parallel_replicas-default_limit_500k_pure 2 +02783_automatic_parallel_replicas-default_useless_limit_0_pure 3 +02783_automatic_parallel_replicas-default_useless_limit_10M_pure 0 +02783_automatic_parallel_replicas-default_useless_limit_1M_pure 3 +02783_automatic_parallel_replicas-default_useless_limit_500k_pure 3 02783_automatic_parallel_replicas-default_simple_join_0_pure 6 02783_automatic_parallel_replicas-default_simple_join_10M_pure 0 02783_automatic_parallel_replicas-default_simple_join_5M_pure 2 02783_automatic_parallel_replicas-default_simple_join_1M_pure 3 02783_automatic_parallel_replicas-default_simple_join_300k_pure 5 +02783_automatic_parallel_replicas-default_helpless_filter_0_pure 3 +02783_automatic_parallel_replicas-default_helpless_filter_2M_pure 0 +02783_automatic_parallel_replicas-default_helpless_filter_500000_pure 2 +02783_automatic_parallel_replicas-default_helpless_filter_100000_pure 3 diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 426c77398f3..5307c137a93 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -106,40 +106,40 @@ query_id_base="02783_automatic_parallel_replicas-$CLICKHOUSE_DATABASE" #### Reading 10M rows without filters whole_table_query="SELECT sum(number) FROM test_parallel_replicas_automatic_count format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_0_0" 0 "$whole_table_query" -run_query_with_pure_parallel_replicas "${query_id_base}_0_10M" 10000000 "$whole_table_query" -run_query_with_pure_parallel_replicas "${query_id_base}_0_6M" 6000000 "$whole_table_query" # 1.6 replicas -> 1 replica -> No parallel replicas -run_query_with_pure_parallel_replicas "${query_id_base}_0_5M" 5000000 "$whole_table_query" -run_query_with_pure_parallel_replicas "${query_id_base}_0_1M" 1000000 "$whole_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_whole_table_0" 0 "$whole_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_whole_table_10M" 10000000 "$whole_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_whole_table_6M" 6000000 "$whole_table_query" # 1.6 replicas -> 1 replica -> No parallel replicas +run_query_with_pure_parallel_replicas "${query_id_base}_whole_table_5M" 5000000 "$whole_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_whole_table_1M" 1000000 "$whole_table_query" ##### Reading 2M rows without filters as partition (p=3) is pruned completely query_with_partition_pruning="SELECT sum(number) FROM test_parallel_replicas_automatic_count WHERE p != 3 format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_1_0" 0 "$query_with_partition_pruning" -run_query_with_pure_parallel_replicas "${query_id_base}_1_10M" 10000000 "$query_with_partition_pruning" -run_query_with_pure_parallel_replicas "${query_id_base}_1_1M" 1000000 "$query_with_partition_pruning" -run_query_with_pure_parallel_replicas "${query_id_base}_1_500k" 500000 "$query_with_partition_pruning" +run_query_with_pure_parallel_replicas "${query_id_base}_pruning_0" 0 "$query_with_partition_pruning" +run_query_with_pure_parallel_replicas "${query_id_base}_pruning_10M" 10000000 "$query_with_partition_pruning" +run_query_with_pure_parallel_replicas "${query_id_base}_pruning_1M" 1000000 "$query_with_partition_pruning" +run_query_with_pure_parallel_replicas "${query_id_base}_pruning_500k" 500000 "$query_with_partition_pruning" #### Reading ~500k rows as index filter should prune granules from partition=1 and partition=2, and drop p3 completely query_with_index="SELECT sum(number) FROM test_parallel_replicas_automatic_count WHERE number < 500_000 format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_2_0" 0 "$query_with_index" -run_query_with_pure_parallel_replicas "${query_id_base}_2_1M" 1000000 "$query_with_index" -run_query_with_pure_parallel_replicas "${query_id_base}_2_300k" 300000 "$query_with_index" -run_query_with_pure_parallel_replicas "${query_id_base}_2_200k" 200000 "$query_with_index" -run_query_with_pure_parallel_replicas "${query_id_base}_2_100k" 100000 "$query_with_index" +run_query_with_pure_parallel_replicas "${query_id_base}_index_0" 0 "$query_with_index" +run_query_with_pure_parallel_replicas "${query_id_base}_index_1M" 1000000 "$query_with_index" +run_query_with_pure_parallel_replicas "${query_id_base}_index_300k" 300000 "$query_with_index" +run_query_with_pure_parallel_replicas "${query_id_base}_index_200k" 200000 "$query_with_index" +run_query_with_pure_parallel_replicas "${query_id_base}_index_100k" 100000 "$query_with_index" #### Reading 1M (because of LIMIT) limit_table_query="SELECT sum(number) FROM (SELECT number FROM test_parallel_replicas_automatic_count LIMIT 1_000_000) format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_3_0" 0 "$limit_table_query" -run_query_with_pure_parallel_replicas "${query_id_base}_3_10M" 10000000 "$limit_table_query" -run_query_with_pure_parallel_replicas "${query_id_base}_3_1M" 1000000 "$limit_table_query" -run_query_with_pure_parallel_replicas "${query_id_base}_3_500k" 500000 "$limit_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_limit_0" 0 "$limit_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_limit_10M" 10000000 "$limit_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_limit_1M" 1000000 "$limit_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_limit_500k" 500000 "$limit_table_query" #### Reading 10M (because of LIMIT is applied after aggregations) limit_agg_table_query="SELECT sum(number) FROM test_parallel_replicas_automatic_count LIMIT 1_000_000 format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_4_0" 0 "$limit_agg_table_query" -run_query_with_pure_parallel_replicas "${query_id_base}_4_10M" 10000000 "$limit_agg_table_query" -run_query_with_pure_parallel_replicas "${query_id_base}_4_1M" 1000000 "$limit_agg_table_query" -run_query_with_pure_parallel_replicas "${query_id_base}_4_500k" 500000 "$limit_agg_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_useless_limit_0" 0 "$limit_agg_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_useless_limit_10M" 10000000 "$limit_agg_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_useless_limit_1M" 1000000 "$limit_agg_table_query" +run_query_with_pure_parallel_replicas "${query_id_base}_useless_limit_500k" 500000 "$limit_agg_table_query" #### JOIN (left side 10M, right side 1M) #### As the right side of the JOIN is a table, ideally it shouldn't be executed with parallel replicas and instead passed as is to the replicas @@ -152,6 +152,13 @@ run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_5M" 5000000 run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_1M" 1000000 "$simple_join_query" # Right: 1->0. Left: 10->3 run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_300k" 400000 "$simple_join_query" # Right: 2. Left: 3 +#### If the filter does not help, it shouldn't disable parallel replicas. Table has 1M rows, filter removes all rows +helpless_filter_query="SELECT sum(number) FROM test_parallel_replicas_automatic_count_right_side WHERE value = 42 format Null" +run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_0" 0 "$helpless_filter_query" +run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_2M" 2000000 "$helpless_filter_query" +run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_500000" 500000 "$helpless_filter_query" +run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_100000" 100000 "$helpless_filter_query" + #### Custom key parallel replicas: Not implemented #whole_table_query="SELECT sum(number) FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), test_parallel_replicas_automatic_count) format Null" #run_query_with_custom_key_parallel_replicas "${query_id_base}_0_0" 0 "$whole_table_query" From b6900c851753004bae99a3d3165f475fadfa2e11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 11 Jul 2023 13:57:19 +0200 Subject: [PATCH 017/634] Add support for the analyzer --- src/Interpreters/InterpreterSelectQuery.cpp | 5 +- src/Interpreters/InterpreterSelectQuery.h | 5 +- .../InterpreterSelectQueryAnalyzer.h | 2 - src/Planner/Planner.cpp | 2 +- src/Planner/PlannerJoinTree.cpp | 42 +++++++++++-- ...llel_replicas_automatic_decision.reference | 62 +++++++++---------- ...84_parallel_replicas_automatic_decision.sh | 48 +------------- 7 files changed, 76 insertions(+), 90 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a26e34fb936..4461143362e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -893,8 +893,7 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis() ASTSelectQuery & query = getSelectQuery(); /// While only_analyze we don't know anything about parts, so any decision about how many parallel replicas to use would be wrong - if (!storage || options.only_analyze || settings.max_parallel_replicas <= 1 - || settings.allow_experimental_parallel_reading_from_replicas == 0 || !settings.parallel_replicas_custom_key.value.empty()) + if (!storage || options.only_analyze || !context->canUseParallelReplicasOnInitiator()) return false; if (getTrivialCount(0).has_value()) @@ -907,7 +906,7 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis() } auto storage_merge_tree = std::dynamic_pointer_cast(storage); - if (!storage_merge_tree || settings.parallel_replicas_min_number_of_rows_per_replica <= 1) + if (!storage_merge_tree || settings.parallel_replicas_min_number_of_rows_per_replica == 0) return false; auto query_info_copy = query_info; diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 9b937071834..7c2555deaa7 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -134,6 +134,9 @@ public: static bool isQueryWithFinal(const SelectQueryInfo & info); + /// Adjust the parallel replicas settings (enabled, disabled) based on the query analysis + bool adjustParallelReplicasAfterAnalysis(); + private: InterpreterSelectQuery( const ASTPtr & query_ptr_, @@ -192,8 +195,6 @@ private: std::optional getTrivialCount(UInt64 max_parallel_replicas); /// Check if we can limit block size to read based on LIMIT clause UInt64 maxBlockSizeByLimit() const; - /// Adjust the parallel replicas settings (enabled, disabled) based on the query analysis - bool adjustParallelReplicasAfterAnalysis(); enum class Modificator { diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 4434fabe746..37ec04abecd 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -63,8 +63,6 @@ public: bool ignoreQuota() const override { return select_query_options.ignore_quota; } - /// Set number_of_current_replica and count_participating_replicas in client_info - void setProperClientInfo(size_t replica_number, size_t count_participating_replicas); const Planner & getPlanner() const { return planner; } Planner & getPlanner() { return planner; } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 9f6c22f90f3..8117ee637a3 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1233,7 +1233,7 @@ void Planner::buildPlanForQueryNode() { if (settings.allow_experimental_parallel_reading_from_replicas == 1 || !settings.parallel_replicas_custom_key.value.empty()) { - LOG_WARNING( + LOG_DEBUG( &Poco::Logger::get("Planner"), "JOINs are not supported with parallel replicas. Query will be executed without using them."); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 5d8f8ca8741..862e7bccb69 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -46,11 +47,12 @@ #include #include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include #include #include @@ -61,7 +63,6 @@ #include #include - namespace DB { @@ -644,6 +645,37 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres { if (!select_query_options.only_analyze) { + auto storage_merge_tree = std::dynamic_pointer_cast(storage); + if (storage_merge_tree && query_context->canUseParallelReplicasOnInitiator() + && settings.parallel_replicas_min_number_of_rows_per_replica > 0) + { + /// This is trash + /// It uses the old InterpreterSelectQuery to do the estimation of how many rows will be read + /// Ideally we should be able to use estimateNumberOfRowsToRead over the storage, but to do this + /// properly we need all the actions/filters, which aren't available yet + /// If we instead delay this check for later several things will happen: + /// * The header might be different (updatePrewhereOutputsIfNeeded) + /// * The storage will have been initiated (thus already preparing parallel replicas) + auto query_options = SelectQueryOptions( + QueryProcessingStage::WithMergeableState, + /* depth */ 1, + /* is_subquery_= */ true) + .ignoreProjections() + .ignoreAlias(); + InterpreterSelectQuery select( + table_expression_query_info.original_query, + query_context, + query_options, + table_expression_query_info.prepared_sets); + select.adjustParallelReplicasAfterAnalysis(); + planner_context->getMutableQueryContext()->setSetting( + "allow_experimental_parallel_reading_from_replicas", + select.getContext()->getSettingsRef().allow_experimental_parallel_reading_from_replicas.operator Field()); + planner_context->getMutableQueryContext()->setSetting( + "max_parallel_replicas", select.getContext()->getSettingsRef().max_parallel_replicas.operator Field()); + } + + const auto & prewhere_actions = table_expression_data.getPrewhereFilterActions(); if (prewhere_actions) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference index c052a5cc67e..ae317665bd3 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference @@ -1,31 +1,31 @@ -02783_automatic_parallel_replicas-default_whole_table_0_pure 3 -02783_automatic_parallel_replicas-default_whole_table_10M_pure 0 -02783_automatic_parallel_replicas-default_whole_table_6M_pure 0 -02783_automatic_parallel_replicas-default_whole_table_5M_pure 2 -02783_automatic_parallel_replicas-default_whole_table_1M_pure 3 -02783_automatic_parallel_replicas-default_pruning_0_pure 3 -02783_automatic_parallel_replicas-default_pruning_10M_pure 0 -02783_automatic_parallel_replicas-default_pruning_1M_pure 2 -02783_automatic_parallel_replicas-default_pruning_500k_pure 3 -02783_automatic_parallel_replicas-default_index_0_pure 3 -02783_automatic_parallel_replicas-default_index_1M_pure 0 -02783_automatic_parallel_replicas-default_index_300k_pure 0 -02783_automatic_parallel_replicas-default_index_200k_pure 2 -02783_automatic_parallel_replicas-default_index_100k_pure 3 -02783_automatic_parallel_replicas-default_limit_0_pure 3 -02783_automatic_parallel_replicas-default_limit_10M_pure 0 -02783_automatic_parallel_replicas-default_limit_1M_pure 0 -02783_automatic_parallel_replicas-default_limit_500k_pure 2 -02783_automatic_parallel_replicas-default_useless_limit_0_pure 3 -02783_automatic_parallel_replicas-default_useless_limit_10M_pure 0 -02783_automatic_parallel_replicas-default_useless_limit_1M_pure 3 -02783_automatic_parallel_replicas-default_useless_limit_500k_pure 3 -02783_automatic_parallel_replicas-default_simple_join_0_pure 6 -02783_automatic_parallel_replicas-default_simple_join_10M_pure 0 -02783_automatic_parallel_replicas-default_simple_join_5M_pure 2 -02783_automatic_parallel_replicas-default_simple_join_1M_pure 3 -02783_automatic_parallel_replicas-default_simple_join_300k_pure 5 -02783_automatic_parallel_replicas-default_helpless_filter_0_pure 3 -02783_automatic_parallel_replicas-default_helpless_filter_2M_pure 0 -02783_automatic_parallel_replicas-default_helpless_filter_500000_pure 2 -02783_automatic_parallel_replicas-default_helpless_filter_100000_pure 3 +02784_automatic_parallel_replicas-default_whole_table_0_pure 3 +02784_automatic_parallel_replicas-default_whole_table_10M_pure 0 +02784_automatic_parallel_replicas-default_whole_table_6M_pure 0 +02784_automatic_parallel_replicas-default_whole_table_5M_pure 2 +02784_automatic_parallel_replicas-default_whole_table_1M_pure 3 +02784_automatic_parallel_replicas-default_pruning_0_pure 3 +02784_automatic_parallel_replicas-default_pruning_10M_pure 0 +02784_automatic_parallel_replicas-default_pruning_1M_pure 2 +02784_automatic_parallel_replicas-default_pruning_500k_pure 3 +02784_automatic_parallel_replicas-default_index_0_pure 3 +02784_automatic_parallel_replicas-default_index_1M_pure 0 +02784_automatic_parallel_replicas-default_index_300k_pure 0 +02784_automatic_parallel_replicas-default_index_200k_pure 2 +02784_automatic_parallel_replicas-default_index_100k_pure 3 +02784_automatic_parallel_replicas-default_limit_0_pure 3 +02784_automatic_parallel_replicas-default_limit_10M_pure 0 +02784_automatic_parallel_replicas-default_limit_1M_pure 0 +02784_automatic_parallel_replicas-default_limit_500k_pure 2 +02784_automatic_parallel_replicas-default_useless_limit_0_pure 3 +02784_automatic_parallel_replicas-default_useless_limit_10M_pure 0 +02784_automatic_parallel_replicas-default_useless_limit_1M_pure 3 +02784_automatic_parallel_replicas-default_useless_limit_500k_pure 3 +02784_automatic_parallel_replicas-default_simple_join_0_pure 6 +02784_automatic_parallel_replicas-default_simple_join_10M_pure 0 +02784_automatic_parallel_replicas-default_simple_join_5M_pure 2 +02784_automatic_parallel_replicas-default_simple_join_1M_pure 3 +02784_automatic_parallel_replicas-default_simple_join_300k_pure 5 +02784_automatic_parallel_replicas-default_helpless_filter_0_pure 3 +02784_automatic_parallel_replicas-default_helpless_filter_2M_pure 0 +02784_automatic_parallel_replicas-default_helpless_filter_500000_pure 2 +02784_automatic_parallel_replicas-default_helpless_filter_100000_pure 3 diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 5307c137a93..5d5dcbe7ef0 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -62,47 +62,10 @@ function run_query_with_pure_parallel_replicas () { --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica "$2" \ - --allow_experimental_analyzer 0 - -# Analyzer: Not implemented yet -# $CLICKHOUSE_CLIENT \ -# --query "$3" \ -# --query_id "${1}_pure_analyzer" \ -# --max_parallel_replicas 3 \ -# --prefer_localhost_replica 1 \ -# --use_hedged_requests 0 \ -# --cluster_for_parallel_replicas 'test_cluster_one_shard_three_replicas_localhost' \ -# --allow_experimental_parallel_reading_from_replicas 1 \ -# --parallel_replicas_for_non_replicated_merge_tree 1 \ -# --parallel_replicas_min_number_of_rows_per_replica "$2" \ -# --allow_experimental_analyzer 0 -} - -function run_query_with_custom_key_parallel_replicas () { - $CLICKHOUSE_CLIENT \ - --query "$3" \ - --query_id "${1}_custom_key" \ - --max_parallel_replicas 3 \ - --use_hedged_requests 0 \ - --parallel_replicas_custom_key_filter_type 'default' \ - --parallel_replicas_custom_key "$2" \ - --parallel_replicas_for_non_replicated_merge_tree 1 \ - --parallel_replicas_min_number_of_rows_per_replica "$2" \ - --allow_experimental_analyzer 0 - - $CLICKHOUSE_CLIENT \ - --query "$3" \ - --query_id "${1}_custom_key_analyzer" \ - --max_parallel_replicas 3 \ - --use_hedged_requests 0 \ - --parallel_replicas_custom_key_filter_type 'default' \ - --parallel_replicas_custom_key "sipHash64(number)" \ - --parallel_replicas_for_non_replicated_merge_tree 1 \ - --parallel_replicas_min_number_of_rows_per_replica "$2" \ --allow_experimental_analyzer 1 } -query_id_base="02783_automatic_parallel_replicas-$CLICKHOUSE_DATABASE" +query_id_base="02784_automatic_parallel_replicas-$CLICKHOUSE_DATABASE" #### Reading 10M rows without filters whole_table_query="SELECT sum(number) FROM test_parallel_replicas_automatic_count format Null" @@ -145,6 +108,7 @@ run_query_with_pure_parallel_replicas "${query_id_base}_useless_limit_500k" 5000 #### As the right side of the JOIN is a table, ideally it shouldn't be executed with parallel replicas and instead passed as is to the replicas #### so each of them executes the join with the assigned granules of the left table, but that's not implemented yet #### https://github.com/ClickHouse/ClickHouse/issues/49301#issuecomment-1619897920 +#### Note that this currently fails with the analyzer since it doesn't support JOIN with parallel replicas simple_join_query="SELECT sum(value) FROM test_parallel_replicas_automatic_count INNER JOIN test_parallel_replicas_automatic_count_right_side USING number format Null" run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_0" 0 "$simple_join_query" # 3 replicas for the right side first, 3 replicas for the left run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_10M" 10000000 "$simple_join_query" # Right: 0. Left: 0 @@ -159,13 +123,5 @@ run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_2M" 2000 run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_500000" 500000 "$helpless_filter_query" run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_100000" 100000 "$helpless_filter_query" -#### Custom key parallel replicas: Not implemented -#whole_table_query="SELECT sum(number) FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), test_parallel_replicas_automatic_count) format Null" -#run_query_with_custom_key_parallel_replicas "${query_id_base}_0_0" 0 "$whole_table_query" -#run_query_with_custom_key_parallel_replicas "${query_id_base}_0_10M" 10000000 "$whole_table_query" -#run_query_with_custom_key_parallel_replicas "${query_id_base}_0_6M" 6000000 "$whole_table_query" # 1.6 replicas -> 1 replica -> No parallel replicas -#run_query_with_custom_key_parallel_replicas "${query_id_base}_0_5M" 5000000 "$whole_table_query" -#run_query_with_custom_key_parallel_replicas "${query_id_base}_0_1M" 1000000 "$whole_table_query" - $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" involved_parallel_replicas "${query_id_base}" From 897023c1b3e649480f271fbca4b903c3f637fdaf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 12 Jul 2023 11:01:46 +0200 Subject: [PATCH 018/634] Make the test independent of analyzer --- .../0_stateless/02784_parallel_replicas_automatic_decision.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 5d5dcbe7ef0..a4b1a295c26 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -61,8 +61,7 @@ function run_query_with_pure_parallel_replicas () { --cluster_for_parallel_replicas 'parallel_replicas' \ --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ - --parallel_replicas_min_number_of_rows_per_replica "$2" \ - --allow_experimental_analyzer 1 + --parallel_replicas_min_number_of_rows_per_replica "$2" } query_id_base="02784_automatic_parallel_replicas-$CLICKHOUSE_DATABASE" From 42a7ec77895c3b3def77192ab7ecb8b3e7692beb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 12 Jul 2023 17:47:31 +0200 Subject: [PATCH 019/634] Reduce number of tests to avoid taking too much time --- .../02784_parallel_replicas_automatic_decision.reference | 9 --------- .../02784_parallel_replicas_automatic_decision.sh | 9 --------- 2 files changed, 18 deletions(-) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference index ae317665bd3..518b8d635a3 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference @@ -1,31 +1,22 @@ -02784_automatic_parallel_replicas-default_whole_table_0_pure 3 02784_automatic_parallel_replicas-default_whole_table_10M_pure 0 02784_automatic_parallel_replicas-default_whole_table_6M_pure 0 02784_automatic_parallel_replicas-default_whole_table_5M_pure 2 02784_automatic_parallel_replicas-default_whole_table_1M_pure 3 -02784_automatic_parallel_replicas-default_pruning_0_pure 3 02784_automatic_parallel_replicas-default_pruning_10M_pure 0 02784_automatic_parallel_replicas-default_pruning_1M_pure 2 02784_automatic_parallel_replicas-default_pruning_500k_pure 3 -02784_automatic_parallel_replicas-default_index_0_pure 3 02784_automatic_parallel_replicas-default_index_1M_pure 0 -02784_automatic_parallel_replicas-default_index_300k_pure 0 02784_automatic_parallel_replicas-default_index_200k_pure 2 02784_automatic_parallel_replicas-default_index_100k_pure 3 -02784_automatic_parallel_replicas-default_limit_0_pure 3 02784_automatic_parallel_replicas-default_limit_10M_pure 0 02784_automatic_parallel_replicas-default_limit_1M_pure 0 02784_automatic_parallel_replicas-default_limit_500k_pure 2 -02784_automatic_parallel_replicas-default_useless_limit_0_pure 3 -02784_automatic_parallel_replicas-default_useless_limit_10M_pure 0 -02784_automatic_parallel_replicas-default_useless_limit_1M_pure 3 02784_automatic_parallel_replicas-default_useless_limit_500k_pure 3 02784_automatic_parallel_replicas-default_simple_join_0_pure 6 02784_automatic_parallel_replicas-default_simple_join_10M_pure 0 02784_automatic_parallel_replicas-default_simple_join_5M_pure 2 02784_automatic_parallel_replicas-default_simple_join_1M_pure 3 02784_automatic_parallel_replicas-default_simple_join_300k_pure 5 -02784_automatic_parallel_replicas-default_helpless_filter_0_pure 3 02784_automatic_parallel_replicas-default_helpless_filter_2M_pure 0 02784_automatic_parallel_replicas-default_helpless_filter_500000_pure 2 02784_automatic_parallel_replicas-default_helpless_filter_100000_pure 3 diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index a4b1a295c26..d964bea6406 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -68,7 +68,6 @@ query_id_base="02784_automatic_parallel_replicas-$CLICKHOUSE_DATABASE" #### Reading 10M rows without filters whole_table_query="SELECT sum(number) FROM test_parallel_replicas_automatic_count format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_whole_table_0" 0 "$whole_table_query" run_query_with_pure_parallel_replicas "${query_id_base}_whole_table_10M" 10000000 "$whole_table_query" run_query_with_pure_parallel_replicas "${query_id_base}_whole_table_6M" 6000000 "$whole_table_query" # 1.6 replicas -> 1 replica -> No parallel replicas run_query_with_pure_parallel_replicas "${query_id_base}_whole_table_5M" 5000000 "$whole_table_query" @@ -76,31 +75,24 @@ run_query_with_pure_parallel_replicas "${query_id_base}_whole_table_1M" 1000000 ##### Reading 2M rows without filters as partition (p=3) is pruned completely query_with_partition_pruning="SELECT sum(number) FROM test_parallel_replicas_automatic_count WHERE p != 3 format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_pruning_0" 0 "$query_with_partition_pruning" run_query_with_pure_parallel_replicas "${query_id_base}_pruning_10M" 10000000 "$query_with_partition_pruning" run_query_with_pure_parallel_replicas "${query_id_base}_pruning_1M" 1000000 "$query_with_partition_pruning" run_query_with_pure_parallel_replicas "${query_id_base}_pruning_500k" 500000 "$query_with_partition_pruning" #### Reading ~500k rows as index filter should prune granules from partition=1 and partition=2, and drop p3 completely query_with_index="SELECT sum(number) FROM test_parallel_replicas_automatic_count WHERE number < 500_000 format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_index_0" 0 "$query_with_index" run_query_with_pure_parallel_replicas "${query_id_base}_index_1M" 1000000 "$query_with_index" -run_query_with_pure_parallel_replicas "${query_id_base}_index_300k" 300000 "$query_with_index" run_query_with_pure_parallel_replicas "${query_id_base}_index_200k" 200000 "$query_with_index" run_query_with_pure_parallel_replicas "${query_id_base}_index_100k" 100000 "$query_with_index" #### Reading 1M (because of LIMIT) limit_table_query="SELECT sum(number) FROM (SELECT number FROM test_parallel_replicas_automatic_count LIMIT 1_000_000) format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_limit_0" 0 "$limit_table_query" run_query_with_pure_parallel_replicas "${query_id_base}_limit_10M" 10000000 "$limit_table_query" run_query_with_pure_parallel_replicas "${query_id_base}_limit_1M" 1000000 "$limit_table_query" run_query_with_pure_parallel_replicas "${query_id_base}_limit_500k" 500000 "$limit_table_query" #### Reading 10M (because of LIMIT is applied after aggregations) limit_agg_table_query="SELECT sum(number) FROM test_parallel_replicas_automatic_count LIMIT 1_000_000 format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_useless_limit_0" 0 "$limit_agg_table_query" -run_query_with_pure_parallel_replicas "${query_id_base}_useless_limit_10M" 10000000 "$limit_agg_table_query" -run_query_with_pure_parallel_replicas "${query_id_base}_useless_limit_1M" 1000000 "$limit_agg_table_query" run_query_with_pure_parallel_replicas "${query_id_base}_useless_limit_500k" 500000 "$limit_agg_table_query" #### JOIN (left side 10M, right side 1M) @@ -117,7 +109,6 @@ run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_300k" 400000 #### If the filter does not help, it shouldn't disable parallel replicas. Table has 1M rows, filter removes all rows helpless_filter_query="SELECT sum(number) FROM test_parallel_replicas_automatic_count_right_side WHERE value = 42 format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_0" 0 "$helpless_filter_query" run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_2M" 2000000 "$helpless_filter_query" run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_500000" 500000 "$helpless_filter_query" run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_100000" 100000 "$helpless_filter_query" From b836ddb4dcc50b4f1bf9af26b296ad4465646b85 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 14 Jul 2023 11:52:36 +0200 Subject: [PATCH 020/634] Divide tests into join and no join --- tests/analyzer_tech_debt.txt | 2 +- ...llel_replicas_automatic_decision.reference | 10 +-- ...84_parallel_replicas_automatic_decision.sh | 41 ++------- ...replicas_automatic_decision_join.reference | 5 ++ ...rallel_replicas_automatic_decision_join.sh | 85 +++++++++++++++++++ 5 files changed, 101 insertions(+), 42 deletions(-) create mode 100644 tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference create mode 100755 tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 739001e1a3b..5ac294d193a 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -126,7 +126,7 @@ 02721_url_cluster 02534_s3_cluster_insert_select_schema_inference 02765_parallel_replicas_final_modifier -02784_parallel_replicas_automatic_decision +02784_parallel_replicas_automatic_decision_join 02581_share_big_sets_between_mutation_tasks_long 02581_share_big_sets_between_multiple_mutations_tasks_long 00992_system_parts_race_condition_zookeeper_long diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference index 518b8d635a3..10c63626c90 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference @@ -12,11 +12,5 @@ 02784_automatic_parallel_replicas-default_limit_1M_pure 0 02784_automatic_parallel_replicas-default_limit_500k_pure 2 02784_automatic_parallel_replicas-default_useless_limit_500k_pure 3 -02784_automatic_parallel_replicas-default_simple_join_0_pure 6 -02784_automatic_parallel_replicas-default_simple_join_10M_pure 0 -02784_automatic_parallel_replicas-default_simple_join_5M_pure 2 -02784_automatic_parallel_replicas-default_simple_join_1M_pure 3 -02784_automatic_parallel_replicas-default_simple_join_300k_pure 5 -02784_automatic_parallel_replicas-default_helpless_filter_2M_pure 0 -02784_automatic_parallel_replicas-default_helpless_filter_500000_pure 2 -02784_automatic_parallel_replicas-default_helpless_filter_100000_pure 3 +02784_automatic_parallel_replicas-default_helpless_filter_10M_pure 0 +02784_automatic_parallel_replicas-default_helpless_filter_5M_pure 2 diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index d964bea6406..57613d3cc8b 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -19,7 +19,7 @@ function involved_parallel_replicas () { } $CLICKHOUSE_CLIENT --query " - CREATE TABLE test_parallel_replicas_automatic_count + CREATE TABLE IF NOT EXISTS test_parallel_replicas_automatic_count ( number Int64, p Int64 @@ -34,20 +34,6 @@ $CLICKHOUSE_CLIENT --query " SELECT number, 3 AS p FROM numbers(10_000_000, 8_000_000) " -$CLICKHOUSE_CLIENT --query " - CREATE TABLE test_parallel_replicas_automatic_count_right_side - ( - number Int64, - value Int64 - ) - ENGINE=MergeTree() - ORDER BY number - SETTINGS index_granularity = 8192 -- Don't randomize it to avoid flakiness - AS - SELECT number, number % 2 AS v FROM numbers(1_000_000) -" - - function run_query_with_pure_parallel_replicas () { # $1 -> query_id # $2 -> min rows per replica @@ -61,7 +47,7 @@ function run_query_with_pure_parallel_replicas () { --cluster_for_parallel_replicas 'parallel_replicas' \ --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ - --parallel_replicas_min_number_of_rows_per_replica "$2" + --parallel_replicas_min_number_of_rows_per_replica "$2" -- } query_id_base="02784_automatic_parallel_replicas-$CLICKHOUSE_DATABASE" @@ -95,23 +81,12 @@ run_query_with_pure_parallel_replicas "${query_id_base}_limit_500k" 500000 "$lim limit_agg_table_query="SELECT sum(number) FROM test_parallel_replicas_automatic_count LIMIT 1_000_000 format Null" run_query_with_pure_parallel_replicas "${query_id_base}_useless_limit_500k" 500000 "$limit_agg_table_query" -#### JOIN (left side 10M, right side 1M) -#### As the right side of the JOIN is a table, ideally it shouldn't be executed with parallel replicas and instead passed as is to the replicas -#### so each of them executes the join with the assigned granules of the left table, but that's not implemented yet -#### https://github.com/ClickHouse/ClickHouse/issues/49301#issuecomment-1619897920 -#### Note that this currently fails with the analyzer since it doesn't support JOIN with parallel replicas -simple_join_query="SELECT sum(value) FROM test_parallel_replicas_automatic_count INNER JOIN test_parallel_replicas_automatic_count_right_side USING number format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_0" 0 "$simple_join_query" # 3 replicas for the right side first, 3 replicas for the left -run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_10M" 10000000 "$simple_join_query" # Right: 0. Left: 0 -run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_5M" 5000000 "$simple_join_query" # Right: 0. Left: 2 -run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_1M" 1000000 "$simple_join_query" # Right: 1->0. Left: 10->3 -run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_300k" 400000 "$simple_join_query" # Right: 2. Left: 3 - -#### If the filter does not help, it shouldn't disable parallel replicas. Table has 1M rows, filter removes all rows -helpless_filter_query="SELECT sum(number) FROM test_parallel_replicas_automatic_count_right_side WHERE value = 42 format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_2M" 2000000 "$helpless_filter_query" -run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_500000" 500000 "$helpless_filter_query" -run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_100000" 100000 "$helpless_filter_query" +#### If the filter does not help, it shouldn't disable parallel replicas. Table has 10M rows, filter removes all rows +helpless_filter_query="SELECT sum(number) FROM test_parallel_replicas_automatic_count WHERE number + p = 42 format Null" +run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_10M" 10000000 "$helpless_filter_query" +run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_5M" 5000000 "$helpless_filter_query" $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" involved_parallel_replicas "${query_id_base}" + +$CLICKHOUSE_CLIENT --query "DROP TABLE test_parallel_replicas_automatic_count" diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference new file mode 100644 index 00000000000..a4b89a975d1 --- /dev/null +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference @@ -0,0 +1,5 @@ +02784_automatic_parallel_replicas_join-default_simple_join_0_pure 6 +02784_automatic_parallel_replicas_join-default_simple_join_10M_pure 0 +02784_automatic_parallel_replicas_join-default_simple_join_5M_pure 2 +02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 3 +02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 5 diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh new file mode 100755 index 00000000000..f5a534892c8 --- /dev/null +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh @@ -0,0 +1,85 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function involved_parallel_replicas () { + # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it + $CLICKHOUSE_CLIENT --query " + SELECT + initial_query_id, + (count() - 2) / 2 as number_of_parallel_replicas + FROM system.query_log + WHERE event_date >= yesterday() + AND initial_query_id LIKE '$1%' + GROUP BY initial_query_id + ORDER BY min(event_time_microseconds) ASC + FORMAT TSV" +} + +$CLICKHOUSE_CLIENT --query " + CREATE TABLE IF NOT EXISTS test_parallel_replicas_automatic_left_side + ( + number Int64, + p Int64 + ) + ENGINE=MergeTree() + ORDER BY number + PARTITION BY p + SETTINGS index_granularity = 8192 -- Don't randomize it to avoid flakiness + AS + SELECT number, number % 2 AS p FROM numbers(2_000_000) + UNION ALL + SELECT number, 3 AS p FROM numbers(10_000_000, 8_000_000) +" + +$CLICKHOUSE_CLIENT --query " + CREATE TABLE IF NOT EXISTS test_parallel_replicas_automatic_count_right_side + ( + number Int64, + value Int64 + ) + ENGINE=MergeTree() + ORDER BY number + SETTINGS index_granularity = 8192 -- Don't randomize it to avoid flakiness + AS + SELECT number, number % 2 AS v FROM numbers(1_000_000) +" + +function run_query_with_pure_parallel_replicas () { + # $1 -> query_id + # $2 -> min rows per replica + # $3 -> query + $CLICKHOUSE_CLIENT \ + --query "$3" \ + --query_id "${1}_pure" \ + --max_parallel_replicas 3 \ + --prefer_localhost_replica 1 \ + --use_hedged_requests 0 \ + --cluster_for_parallel_replicas 'parallel_replicas' \ + --allow_experimental_parallel_reading_from_replicas 1 \ + --parallel_replicas_for_non_replicated_merge_tree 1 \ + --parallel_replicas_min_number_of_rows_per_replica "$2" +} + +query_id_base="02784_automatic_parallel_replicas_join-$CLICKHOUSE_DATABASE" + + +#### JOIN (left side 10M, right side 1M) +#### As the right side of the JOIN is a table, ideally it shouldn't be executed with parallel replicas and instead passed as is to the replicas +#### so each of them executes the join with the assigned granules of the left table, but that's not implemented yet +#### https://github.com/ClickHouse/ClickHouse/issues/49301#issuecomment-1619897920 +#### Note that this currently fails with the analyzer since it doesn't support JOIN with parallel replicas +simple_join_query="SELECT sum(value) FROM test_parallel_replicas_automatic_left_side INNER JOIN test_parallel_replicas_automatic_count_right_side USING number format Null" +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_0" 0 "$simple_join_query" # 3 replicas for the right side first, 3 replicas for the left +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_10M" 10000000 "$simple_join_query" # Right: 0. Left: 0 +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_5M" 5000000 "$simple_join_query" # Right: 0. Left: 2 +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_1M" 1000000 "$simple_join_query" # Right: 1->0. Left: 10->3 +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_300k" 400000 "$simple_join_query" # Right: 2. Left: 3 + +$CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" +involved_parallel_replicas "${query_id_base}" + +$CLICKHOUSE_CLIENT --query "DROP TABLE test_parallel_replicas_automatic_left_side" +$CLICKHOUSE_CLIENT --query "DROP TABLE test_parallel_replicas_automatic_count_right_side" From 10c008e5c7bd635a6937d9cbbb0fee5ed45d4197 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 26 Jul 2023 10:49:24 +0200 Subject: [PATCH 021/634] Remove duplicated comment --- src/Interpreters/InterpreterSelectQuery.cpp | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index fa79de147e8..b895b90a2b5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2373,11 +2373,6 @@ UInt64 InterpreterSelectQuery::maxBlockSizeByLimit() const auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); - /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, JOIN, LIMIT BY, WITH TIES - * but LIMIT is specified, and limit + offset < max_block_size, - * then as the block size we will use limit + offset (not to read more from the table than requested), - * and also set the number of threads to 1. - */ if (!query.distinct && !query.limit_with_ties && !query.prewhere() @@ -2469,11 +2464,6 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc UInt64 max_block_size = settings.max_block_size; auto local_limits = getStorageLimits(*context, options); - /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, JOIN, LIMIT BY, WITH TIES - * but LIMIT is specified, and limit + offset < max_block_size, - * then as the block size we will use limit + offset (not to read more from the table than requested), - * and also set the number of threads to 1. - */ if (UInt64 max_block_limited = maxBlockSizeByLimit()) { if (max_block_limited < max_block_size) From adfdba4b83eee834beb69d4c0323128ca27db77c Mon Sep 17 00:00:00 2001 From: helifu Date: Thu, 3 Aug 2023 14:33:33 +0800 Subject: [PATCH 022/634] Display reloaded values of server settings --- src/Common/MemoryTracker.h | 4 ++ src/Interpreters/Context.cpp | 8 ++++ src/Interpreters/Context.h | 2 + src/Interpreters/ProcessList.h | 18 +++++++++ .../MergeTree/MergeTreeBackgroundExecutor.cpp | 7 ++++ .../MergeTree/MergeTreeBackgroundExecutor.h | 2 + .../System/StorageSystemServerSettings.cpp | 39 ++++++++++++++++++- 7 files changed, 79 insertions(+), 1 deletion(-) diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 5041dc2af41..7c303c7f2c2 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -158,6 +158,10 @@ public: { allow_use_jemalloc_memory.store(value, std::memory_order_relaxed); } + bool getAllowUseJemallocMmemory() const + { + return allow_use_jemalloc_memory.load(std::memory_order_relaxed); + } /** Set limit if it was not set. * Otherwise, set limit to new value, if new value is greater than previous limit. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index cece3a6e738..0ab1c637fcb 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3696,6 +3696,10 @@ void Context::setMaxTableSizeToDrop(size_t max_size) shared->max_table_size_to_drop.store(max_size, std::memory_order_relaxed); } +size_t Context::getMaxTableSizeToDrop() const +{ + return shared->max_table_size_to_drop.load(std::memory_order_relaxed); +} void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const { @@ -3711,6 +3715,10 @@ void Context::setMaxPartitionSizeToDrop(size_t max_size) shared->max_partition_size_to_drop.store(max_size, std::memory_order_relaxed); } +size_t Context::getMaxPartitionSizeToDrop() const +{ + return shared->max_partition_size_to_drop.load(std::memory_order_relaxed); +} void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 9acb489e2d8..6932dfb1843 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1041,10 +1041,12 @@ public: /// Prevents DROP TABLE if its size is greater than max_size (50GB by default, max_size=0 turn off this check) void setMaxTableSizeToDrop(size_t max_size); + size_t getMaxTableSizeToDrop() const; void checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const; /// Prevents DROP PARTITION if its size is greater than max_size (50GB by default, max_size=0 turn off this check) void setMaxPartitionSizeToDrop(size_t max_size); + size_t getMaxPartitionSizeToDrop() const; void checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const; /// Lets you select the compression codec according to the conditions described in the configuration file. diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 2eea49e1267..690c7661d8d 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -412,18 +412,36 @@ public: max_size = max_size_; } + size_t getMaxSize() const + { + auto lock = unsafeLock(); + return max_size; + } + void setMaxInsertQueriesAmount(size_t max_insert_queries_amount_) { auto lock = unsafeLock(); max_insert_queries_amount = max_insert_queries_amount_; } + size_t getMaxInsertQueriesAmount() const + { + auto lock = unsafeLock(); + return max_insert_queries_amount; + } + void setMaxSelectQueriesAmount(size_t max_select_queries_amount_) { auto lock = unsafeLock(); max_select_queries_amount = max_select_queries_amount_; } + size_t getMaxSelectQueriesAmount() const + { + auto lock = unsafeLock(); + return max_select_queries_amount; + } + /// Try call cancel() for input and output streams of query with specified id and user CancellationCode sendCancelToQuery(const String & current_query_id, const String & current_user, bool kill = false); diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index e497a799274..2df0a6c1c1f 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -112,6 +112,13 @@ void MergeTreeBackgroundExecutor::increaseThreadsAndMaxTasksCount(size_t threads_count = new_threads_count; } +template +size_t MergeTreeBackgroundExecutor::getMaxThreads() const +{ + std::lock_guard lock(mutex); + return threads_count; +} + template size_t MergeTreeBackgroundExecutor::getMaxTasksCount() const { diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index 552ed32e2d2..63f75ffc8d9 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -270,6 +270,8 @@ public: /// implementing tasks eviction will definitely be too error-prone and buggy. void increaseThreadsAndMaxTasksCount(size_t new_threads_count, size_t new_max_tasks_count); + size_t getMaxThreads() const; + /// This method can return stale value of max_tasks_count (no mutex locking). /// It's okay because amount of tasks can be only increased and getting stale value /// can lead only to some postponing, not logical error. diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index 290b575465c..7a8b45de736 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -1,8 +1,20 @@ #include +#include #include #include #include +#include #include +#include + + +namespace CurrentMetrics +{ + extern const Metric BackgroundSchedulePoolSize; + extern const Metric BackgroundBufferFlushSchedulePoolSize; + extern const Metric BackgroundDistributedSchedulePoolSize; + extern const Metric BackgroundMessageBrokerSchedulePoolSize; +} namespace DB { @@ -21,6 +33,29 @@ NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() void StorageSystemServerSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { + // Server settings that have been reloaded from the config file. + std::unordered_map updated = { + {"max_server_memory_usage", std::to_string(total_memory_tracker.getHardLimit())}, + {"allow_use_jemalloc_memory", std::to_string(total_memory_tracker.getAllowUseJemallocMmemory())}, + + {"max_table_size_to_drop", std::to_string(context->getMaxTableSizeToDrop())}, + {"max_partition_size_to_drop", std::to_string(context->getMaxPartitionSizeToDrop())}, + + {"max_concurrent_queries", std::to_string(context->getProcessList().getMaxSize())}, + {"max_concurrent_insert_queries", std::to_string(context->getProcessList().getMaxInsertQueriesAmount())}, + {"max_concurrent_select_queries", std::to_string(context->getProcessList().getMaxSelectQueriesAmount())}, + + {"background_pool_size", std::to_string(context->getMergeMutateExecutor()->getMaxThreads())}, + {"background_move_pool_size", std::to_string(context->getMovesExecutor()->getMaxThreads())}, + {"background_fetches_pool_size", std::to_string(context->getFetchesExecutor()->getMaxThreads())}, + {"background_common_pool_size", std::to_string(context->getCommonExecutor()->getMaxThreads())}, + + {"background_buffer_flush_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize))}, + {"background_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize))}, + {"background_message_broker_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize))}, + {"background_distributed_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize))} + }; + const auto & config = context->getConfigRef(); ServerSettings settings; settings.loadSettingsFromConfig(config); @@ -28,8 +63,10 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context for (const auto & setting : settings.all()) { const auto & setting_name = setting.getName(); + const auto & it = updated.find(setting_name); + res_columns[0]->insert(setting_name); - res_columns[1]->insert(setting.getValueString()); + res_columns[1]->insert((it != updated.end()) ? it->second : setting.getValueString()); res_columns[2]->insert(setting.getDefaultValueString()); res_columns[3]->insert(setting.isValueChanged()); res_columns[4]->insert(setting.getDescription()); From 93ab53163ac3529b0dbb3af239f02728c1040d89 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 24 Aug 2023 12:32:56 +0200 Subject: [PATCH 023/634] Revert "Revert "Planner prepare filters for analysis"" --- src/Planner/Planner.cpp | 82 +++++++++++++++++++ src/Planner/PlannerJoinTree.cpp | 1 + src/Planner/Utils.cpp | 40 +++++++-- src/Planner/Utils.h | 4 +- src/Storages/ColumnsDescription.cpp | 11 +++ src/Storages/ColumnsDescription.h | 5 ++ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 +- src/Storages/StorageDummy.h | 4 +- src/Storages/StorageMergeTree.cpp | 7 -- src/Storages/StorageReplicatedMergeTree.cpp | 9 -- src/Storages/StorageSnapshot.cpp | 10 +++ src/Storages/StorageSnapshot.h | 2 + 12 files changed, 154 insertions(+), 27 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 7cce495dfb8..734458ed086 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -42,6 +42,7 @@ #include #include #include +#include #include #include @@ -138,6 +139,84 @@ void checkStoragesSupportTransactions(const PlannerContextPtr & planner_context) } } +/** Storages can rely that filters that for storage will be available for analysis before + * getQueryProcessingStage method will be called. + * + * StorageDistributed skip unused shards optimization relies on this. + * + * To collect filters that will be applied to specific table in case we have JOINs requires + * to run query plan optimization pipeline. + * + * Algorithm: + * 1. Replace all table expressions in query tree with dummy tables. + * 2. Build query plan. + * 3. Optimize query plan. + * 4. Extract filters from ReadFromDummy query plan steps from query plan leaf nodes. + */ +void collectFiltersForAnalysis(const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context) +{ + bool collect_filters = false; + + for (auto & [table_expression, table_expression_data] : planner_context->getTableExpressionNodeToData()) + { + auto * table_node = table_expression->as(); + auto * table_function_node = table_expression->as(); + if (!table_node && !table_function_node) + continue; + + const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); + if (typeid_cast(storage.get())) + { + collect_filters = true; + break; + } + } + + if (!collect_filters) + return; + + ResultReplacementMap replacement_map; + auto updated_query_tree = replaceTableExpressionsWithDummyTables(query_tree, planner_context->getQueryContext(), &replacement_map); + + std::unordered_map dummy_storage_to_table_expression_data; + + for (auto & [from_table_expression, dummy_table_expression] : replacement_map) + { + auto * dummy_storage = dummy_table_expression->as().getStorage().get(); + auto * table_expression_data = &planner_context->getTableExpressionDataOrThrow(from_table_expression); + dummy_storage_to_table_expression_data.emplace(dummy_storage, table_expression_data); + } + + const auto & query_context = planner_context->getQueryContext(); + + SelectQueryOptions select_query_options; + Planner planner(updated_query_tree, select_query_options); + planner.buildQueryPlanIfNeeded(); + + auto & result_query_plan = planner.getQueryPlan(); + + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(query_context); + result_query_plan.optimize(optimization_settings); + + std::vector nodes_to_process; + nodes_to_process.push_back(result_query_plan.getRootNode()); + + while (!nodes_to_process.empty()) + { + const auto * node_to_process = nodes_to_process.back(); + nodes_to_process.pop_back(); + nodes_to_process.insert(nodes_to_process.end(), node_to_process->children.begin(), node_to_process->children.end()); + + auto * read_from_dummy = typeid_cast(node_to_process->step.get()); + if (!read_from_dummy) + continue; + + auto filter_actions = ActionsDAG::buildFilterActionsDAG(read_from_dummy->getFilterNodes().nodes, {}, query_context); + auto & table_expression_data = dummy_storage_to_table_expression_data.at(&read_from_dummy->getStorage()); + table_expression_data->setFilterActions(std::move(filter_actions)); + } +} + /// Extend lifetime of query context, storages, and table locks void extendQueryContextAndStoragesLifetime(QueryPlan & query_plan, const PlannerContextPtr & planner_context) { @@ -1226,6 +1305,9 @@ void Planner::buildPlanForQueryNode() collectSets(query_tree, *planner_context); collectTableExpressionData(query_tree, planner_context); + if (!select_query_options.only_analyze) + collectFiltersForAnalysis(query_tree, planner_context); + const auto & settings = query_context->getSettingsRef(); /// Check support for JOIN for parallel replicas with custom key diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 70ff32c1d31..f9833a9fbd4 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -544,6 +544,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto table_expression_query_info = select_query_info; table_expression_query_info.table_expression = table_expression; + table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions(); size_t max_streams = settings.max_threads; size_t max_threads_execute_query = settings.max_threads; diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 733db0f00bc..9c46622f578 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -355,24 +355,52 @@ QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, con return function_node; } -QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNodePtr & query_node, +QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr & query_node, const ContextPtr & context, ResultReplacementMap * result_replacement_map) { auto & query_node_typed = query_node->as(); auto table_expressions = extractTableExpressions(query_node_typed.getJoinTree()); std::unordered_map replacement_map; + size_t subquery_index = 0; for (auto & table_expression : table_expressions) { auto * table_node = table_expression->as(); auto * table_function_node = table_expression->as(); - if (!table_node && !table_function_node) - continue; + auto * subquery_node = table_expression->as(); + auto * union_node = table_expression->as(); + + StoragePtr storage_dummy; + + if (table_node || table_function_node) + { + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); + + storage_dummy + = std::make_shared(storage_snapshot->storage.getStorageID(), ColumnsDescription(storage_snapshot->getColumns(get_column_options))); + } + else if (subquery_node || union_node) + { + const auto & subquery_projection_columns + = subquery_node ? subquery_node->getProjectionColumns() : union_node->computeProjectionColumns(); + + NameSet unique_column_names; + NamesAndTypes storage_dummy_columns; + storage_dummy_columns.reserve(subquery_projection_columns.size()); + + for (const auto & projection_column : subquery_projection_columns) + { + auto [_, inserted] = unique_column_names.insert(projection_column.name); + if (inserted) + storage_dummy_columns.emplace_back(projection_column); + } + + storage_dummy = std::make_shared(StorageID{"dummy", "subquery_" + std::to_string(subquery_index)}, ColumnsDescription(storage_dummy_columns)); + ++subquery_index; + } - const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); - auto storage_dummy = std::make_shared(storage_snapshot->storage.getStorageID(), - storage_snapshot->metadata->getColumns()); auto dummy_table_node = std::make_shared(std::move(storage_dummy), context); if (result_replacement_map) diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index d9412800e61..1b8397f47cc 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -65,9 +65,9 @@ bool queryHasWithTotalsInAnySubqueryInJoinTree(const QueryTreeNodePtr & query_no /// Returns `and` function node that has condition nodes as its arguments QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, const ContextPtr & context); -/// Replace tables nodes and table function nodes with dummy table nodes +/// Replace table expressions from query JOIN TREE with dummy tables using ResultReplacementMap = std::unordered_map; -QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNodePtr & query_node, +QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr & query_node, const ContextPtr & context, ResultReplacementMap * result_replacement_map = nullptr); diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 0c918bda5fd..9b721280add 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -140,6 +140,17 @@ void ColumnDescription::readText(ReadBuffer & buf) } } +ColumnsDescription::ColumnsDescription(std::initializer_list ordinary) +{ + for (const auto & elem : ordinary) + add(ColumnDescription(elem.name, elem.type)); +} + +ColumnsDescription::ColumnsDescription(NamesAndTypes ordinary) +{ + for (auto & elem : ordinary) + add(ColumnDescription(std::move(elem.name), std::move(elem.type))); +} ColumnsDescription::ColumnsDescription(NamesAndTypesList ordinary) { diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index fb1eeed3127..276968ffe4e 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -102,6 +102,11 @@ class ColumnsDescription : public IHints<1, ColumnsDescription> { public: ColumnsDescription() = default; + + ColumnsDescription(std::initializer_list ordinary); + + explicit ColumnsDescription(NamesAndTypes ordinary); + explicit ColumnsDescription(NamesAndTypesList ordinary); explicit ColumnsDescription(NamesAndTypesList ordinary, NamesAndAliases aliases); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 62547ff8786..5e9aefb7fb5 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1304,6 +1304,10 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); + /// Do not keep data parts in snapshot. + /// They are stored separately, and some could be released after PK analysis. + auto storage_snapshot_copy = storage_snapshot->clone(std::make_unique()); + return std::make_unique( std::move(parts), std::move(alter_conversions), @@ -1311,7 +1315,7 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( virt_column_names, data, query_info, - storage_snapshot, + storage_snapshot_copy, context, max_block_size, num_streams, diff --git a/src/Storages/StorageDummy.h b/src/Storages/StorageDummy.h index 2f9a8beb4d0..aa2201a196b 100644 --- a/src/Storages/StorageDummy.h +++ b/src/Storages/StorageDummy.h @@ -8,7 +8,7 @@ namespace DB { -class StorageDummy : public IStorage +class StorageDummy final : public IStorage { public: StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_, ColumnsDescription object_columns_ = {}); @@ -46,7 +46,7 @@ private: const ColumnsDescription object_columns; }; -class ReadFromDummy : public SourceStepWithFilter +class ReadFromDummy final : public SourceStepWithFilter { public: explicit ReadFromDummy(const StorageDummy & storage_, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9d7f6903b46..c7f27c00899 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -255,13 +255,6 @@ void StorageMergeTree::read( processed_stage, nullptr, enable_parallel_reading)) query_plan = std::move(*plan); } - - /// Now, copy of parts that is required for the query, stored in the processors, - /// while snapshot_data.parts includes all parts, even one that had been filtered out with partition pruning, - /// reset them to avoid holding them. - auto & snapshot_data = assert_cast(*storage_snapshot->data); - snapshot_data.parts = {}; - snapshot_data.alter_conversions = {}; } std::optional StorageMergeTree::totalRows(const Settings &) const diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 76a2ad9883c..6f50c9c773c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5103,15 +5103,6 @@ void StorageReplicatedMergeTree::read( const size_t max_block_size, const size_t num_streams) { - SCOPE_EXIT({ - /// Now, copy of parts that is required for the query, stored in the processors, - /// while snapshot_data.parts includes all parts, even one that had been filtered out with partition pruning, - /// reset them to avoid holding them. - auto & snapshot_data = assert_cast(*storage_snapshot->data); - snapshot_data.parts = {}; - snapshot_data.alter_conversions = {}; - }); - const auto & settings = local_context->getSettingsRef(); /// The `select_sequential_consistency` setting has two meanings: diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index c0e85900794..5de60f4decd 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -17,6 +17,16 @@ namespace ErrorCodes extern const int COLUMN_QUERIED_MORE_THAN_ONCE; } +std::shared_ptr StorageSnapshot::clone(DataPtr data_) const +{ + auto res = std::make_shared(storage, metadata, object_columns); + + res->projection = projection; + res->data = std::move(data_); + + return res; +} + void StorageSnapshot::init() { for (const auto & [name, type] : storage.getVirtuals()) diff --git a/src/Storages/StorageSnapshot.h b/src/Storages/StorageSnapshot.h index a69f9b95955..d62e118e1f2 100644 --- a/src/Storages/StorageSnapshot.h +++ b/src/Storages/StorageSnapshot.h @@ -60,6 +60,8 @@ struct StorageSnapshot init(); } + std::shared_ptr clone(DataPtr data_) const; + /// Get all available columns with types according to options. NamesAndTypesList getColumns(const GetColumnsOptions & options) const; From 6128146371bfa02384636f915b9b98f8a39bfe53 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 24 Aug 2023 10:32:13 +0000 Subject: [PATCH 024/634] Fix 02834_remote_session_log (cherry picked from commit 40a84e48aae09aff08850e19c5527fd9bcf4ab8e) --- src/Planner/PlannerJoinTree.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index f9833a9fbd4..935a1c8384a 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -46,6 +46,8 @@ #include #include +#include + #include #include #include @@ -84,6 +86,10 @@ namespace /// Check if current user has privileges to SELECT columns from table void checkAccessRights(const TableNode & table_node, const Names & column_names, const ContextPtr & query_context) { + /// StorageDummy is created on preliminary stage, igore access check for it. + if (typeid_cast(table_node.getStorage().get())) + return; + const auto & storage_id = table_node.getStorageID(); const auto & storage_snapshot = table_node.getStorageSnapshot(); From 15182772cc860639f059a4d58d9be34ed27a8385 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 24 Aug 2023 12:54:11 +0200 Subject: [PATCH 025/634] Update src/Planner/PlannerJoinTree.cpp MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Raúl Marín --- src/Planner/PlannerJoinTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 935a1c8384a..8f87b90e949 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -86,7 +86,7 @@ namespace /// Check if current user has privileges to SELECT columns from table void checkAccessRights(const TableNode & table_node, const Names & column_names, const ContextPtr & query_context) { - /// StorageDummy is created on preliminary stage, igore access check for it. + /// StorageDummy is created on preliminary stage, ignore access check for it. if (typeid_cast(table_node.getStorage().get())) return; From 6962070baad4e36a1aa0e15acc879ea459f2979c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 31 Aug 2023 04:56:47 +0200 Subject: [PATCH 026/634] Add suport for mathematical minus character --- src/Parsers/ExpressionListParsers.cpp | 4 +++- src/Parsers/Lexer.cpp | 13 +++++++++++++ .../0_stateless/02869_unicode_minus.reference | 2 ++ tests/queries/0_stateless/02869_unicode_minus.sql | 2 ++ 4 files changed, 20 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02869_unicode_minus.reference create mode 100644 tests/queries/0_stateless/02869_unicode_minus.sql diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 61cac0480a9..95736dba307 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2360,6 +2360,7 @@ const std::vector> ParserExpressionImpl::o {"||", Operator("concat", 10, 2, OperatorType::Mergeable)}, {"+", Operator("plus", 11, 2)}, {"-", Operator("minus", 11, 2)}, + {"−", Operator("minus", 11, 2)}, {"*", Operator("multiply", 12, 2)}, {"/", Operator("divide", 12, 2)}, {"%", Operator("modulo", 12, 2)}, @@ -2373,7 +2374,8 @@ const std::vector> ParserExpressionImpl::o const std::vector> ParserExpressionImpl::unary_operators_table { {"NOT", Operator("not", 5, 1)}, - {"-", Operator("negate", 13, 1)} + {"-", Operator("negate", 13, 1)}, + {"−", Operator("negate", 13, 1)} }; const Operator ParserExpressionImpl::finish_between_operator("", 8, 0, OperatorType::FinishBetween); diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index be67807ad8f..a22d04955b5 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -303,6 +303,19 @@ Token Lexer::nextTokenImpl() return Token(TokenType::Minus, token_begin, pos); } + case '\xE2': + { + /// Mathematical minus symbol, UTF-8 + if (pos + 3 <= end && pos[1] == '\x88' && pos[2] == '\x92') + { + pos += 3; + return Token(TokenType::Minus, token_begin, pos); + } + else + { + return Token(TokenType::Error, token_begin, ++pos); + } + } case '*': ++pos; return Token(TokenType::Asterisk, token_begin, pos); diff --git a/tests/queries/0_stateless/02869_unicode_minus.reference b/tests/queries/0_stateless/02869_unicode_minus.reference new file mode 100644 index 00000000000..343ee5c2f6c --- /dev/null +++ b/tests/queries/0_stateless/02869_unicode_minus.reference @@ -0,0 +1,2 @@ +-1 +-1 diff --git a/tests/queries/0_stateless/02869_unicode_minus.sql b/tests/queries/0_stateless/02869_unicode_minus.sql new file mode 100644 index 00000000000..15fe7b83993 --- /dev/null +++ b/tests/queries/0_stateless/02869_unicode_minus.sql @@ -0,0 +1,2 @@ +SELECT 1 − 2; +SELECT −1; From 8c28e51425367c1b01ee1edc75cea1e42043db25 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 31 Aug 2023 04:59:58 +0200 Subject: [PATCH 027/634] Add suport for mathematical minus character --- src/Parsers/Lexer.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index a22d04955b5..cf646f86a74 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -311,10 +311,8 @@ Token Lexer::nextTokenImpl() pos += 3; return Token(TokenType::Minus, token_begin, pos); } - else - { - return Token(TokenType::Error, token_begin, ++pos); - } + /// Other characters starting at E2 can be parsed, see skipWhitespacesUTF8 + [[fallthrough]]; } case '*': ++pos; From 2755fc045e988089faec9c6f41469d8109002ae0 Mon Sep 17 00:00:00 2001 From: Joey Wang Date: Thu, 7 Sep 2023 00:11:16 +0800 Subject: [PATCH 028/634] add function jsonMerge --- src/Common/ErrorCodes.cpp | 1 + src/Functions/jsonMerge.cpp | 117 ++++++++++++++++++ .../02874_json_merge_function_test.reference | 11 ++ .../02874_json_merge_function_test.sql | 9 ++ 4 files changed, 138 insertions(+) create mode 100644 src/Functions/jsonMerge.cpp create mode 100644 tests/queries/0_stateless/02874_json_merge_function_test.reference create mode 100644 tests/queries/0_stateless/02874_json_merge_function_test.sql diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f23685c37d1..a9f4f12c9f1 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -585,6 +585,7 @@ M(700, USER_SESSION_LIMIT_EXCEEDED) \ M(701, CLUSTER_DOESNT_EXIST) \ M(702, CLIENT_INFO_DOES_NOT_MATCH) \ + M(710, ILLEGAL_JSON_OBJECT_FORMAT) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Functions/jsonMerge.cpp b/src/Functions/jsonMerge.cpp new file mode 100644 index 00000000000..d9900d2203d --- /dev/null +++ b/src/Functions/jsonMerge.cpp @@ -0,0 +1,117 @@ +#include "config.h" + +#if USE_RAPIDJSON + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "rapidjson/document.h" +#include "rapidjson/writer.h" +#include "rapidjson/stringbuffer.h" +#include "rapidjson/filewritestream.h" +#include "rapidjson/prettywriter.h" +#include "rapidjson/filereadstream.h" + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_JSON_OBJECT_FORMAT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +namespace +{ + // select jsonMerge('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); + // || + // \/ + // ┌───────────────────────┐ + // │ {"a":1,"name":"zoey"} │ + // └───────────────────────┘ + class FunctionjsonMerge : public IFunction + { + public: + static constexpr auto name = "jsonMerge"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.empty()) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least one argument.", getName()); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + rapidjson::Document merged_json; + merged_json.SetObject(); + rapidjson::Document::AllocatorType& allocator = merged_json.GetAllocator(); + + for (const auto & arg : arguments) + { + const ColumnPtr column = arg.column; + const ColumnString * col = typeid_cast(column.get()); + if (!col) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "First argument of function {} must be string", getName()); + + for (size_t i = 0; i < input_rows_count; ++i) + { + auto str_ref = col->getDataAt(i); + const char* json = str_ref.data; + rapidjson::Document document; + document.Parse(json); + if (!document.IsObject()) + throw Exception(ErrorCodes::ILLEGAL_JSON_OBJECT_FORMAT, "Wrong input Json object format"); + + for (auto it = document.MemberBegin(); it != document.MemberEnd(); ++it) + { + rapidjson::Value key(it->name, allocator); + rapidjson::Value value(it->value, allocator); + if (merged_json.HasMember(key)) + merged_json[key] = value; + else + merged_json.AddMember(key, value, allocator); + } + } + } + + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + merged_json.Accept(writer); + std::string json_string = buffer.GetString(); + + auto res = ColumnString::create(); + res->insertData(json_string.c_str(), json_string.size()); + + return res; + } + }; + +} + +REGISTER_FUNCTION(jsonMerge) +{ + factory.registerFunction(FunctionDocumentation{ + .description="Return the merged JSON object, which is formed by merging multiple JSON objects."}); +} + +} + +#endif diff --git a/tests/queries/0_stateless/02874_json_merge_function_test.reference b/tests/queries/0_stateless/02874_json_merge_function_test.reference new file mode 100644 index 00000000000..84912310d19 --- /dev/null +++ b/tests/queries/0_stateless/02874_json_merge_function_test.reference @@ -0,0 +1,11 @@ +-- { echoOn } +select jsonMerge(null); +\N +select jsonMerge('[1]'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } +select jsonMerge('{"a":1}'); +{"a":1} +select jsonMerge('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); +{"a":1,"name":"zoey"} +select jsonMerge('{"a": "1","b": 2,"c": [true,{"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); +{"a":"1","b":2,"c":"1"} +select jsonMerge('{"a": "1","b": 2,"c": [true,"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } diff --git a/tests/queries/0_stateless/02874_json_merge_function_test.sql b/tests/queries/0_stateless/02874_json_merge_function_test.sql new file mode 100644 index 00000000000..fa2a4c721c1 --- /dev/null +++ b/tests/queries/0_stateless/02874_json_merge_function_test.sql @@ -0,0 +1,9 @@ +-- { echoOn } +select jsonMerge(null); +select jsonMerge('[1]'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } +select jsonMerge('{"a":1}'); +select jsonMerge('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); + +select jsonMerge('{"a": "1","b": 2,"c": [true,{"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); + +select jsonMerge('{"a": "1","b": 2,"c": [true,"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } From 872f98141603ffa2f03c9a94c9589d587575ea15 Mon Sep 17 00:00:00 2001 From: Joey Wang Date: Thu, 7 Sep 2023 10:10:33 +0800 Subject: [PATCH 029/634] add doc --- .../sql-reference/functions/json-functions.md | 31 +++++++++++++++++++ src/Functions/jsonMerge.cpp | 9 +++--- .../02874_json_merge_function_test.reference | 10 ++---- .../02874_json_merge_function_test.sql | 8 +++-- .../aspell-ignore/en/aspell-dict.txt | 1 + 5 files changed, 44 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index 31d53ba0359..e5ff13763ad 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -509,3 +509,34 @@ SELECT │ ᴺᵁᴸᴸ │ 3 │ └─────────────────────┴────────────────────────────┘ ``` + + +## jsonMerge + +Return the merged JSON object string which is formed by merging multiple JSON objects. + +**Syntax** + +``` sql +jsonMerge(json1, json2, ...) +``` + +**Arguments** + +- `json` — [String](../../sql-reference/data-types/string.md) with valid JSON. + +**Returned value** + +- If JSON object strings are valid, return the merged JSON object string. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +``` sql +SELECT jsonMerge('{"a":1}', '{"name": "joey"}', '{"name": "tom"}', '{"name": "zoey"}') AS res + +┌─res───────────────────┐ +│ {"a":1,"name":"zoey"} │ +└───────────────────────┘ +``` diff --git a/src/Functions/jsonMerge.cpp b/src/Functions/jsonMerge.cpp index d9900d2203d..e57b56d73a2 100644 --- a/src/Functions/jsonMerge.cpp +++ b/src/Functions/jsonMerge.cpp @@ -1,7 +1,3 @@ -#include "config.h" - -#if USE_RAPIDJSON - #include #include #include @@ -10,6 +6,9 @@ #include #include #include +#include "config.h" + +#if USE_RAPIDJSON #include "rapidjson/document.h" #include "rapidjson/writer.h" @@ -109,7 +108,7 @@ namespace REGISTER_FUNCTION(jsonMerge) { factory.registerFunction(FunctionDocumentation{ - .description="Return the merged JSON object, which is formed by merging multiple JSON objects."}); + .description="Return the merged JSON object string, which is formed by merging multiple JSON objects."}); } } diff --git a/tests/queries/0_stateless/02874_json_merge_function_test.reference b/tests/queries/0_stateless/02874_json_merge_function_test.reference index 84912310d19..11bc968e6c6 100644 --- a/tests/queries/0_stateless/02874_json_merge_function_test.reference +++ b/tests/queries/0_stateless/02874_json_merge_function_test.reference @@ -1,11 +1,7 @@ --- { echoOn } -select jsonMerge(null); \N -select jsonMerge('[1]'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } -select jsonMerge('{"a":1}'); {"a":1} -select jsonMerge('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); +{"a":1,"b":1} +{"a":1,"b":1,"c":[1,2]} +{"a":1,"b":1,"c":[{"d":1},2]} {"a":1,"name":"zoey"} -select jsonMerge('{"a": "1","b": 2,"c": [true,{"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); {"a":"1","b":2,"c":"1"} -select jsonMerge('{"a": "1","b": 2,"c": [true,"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } diff --git a/tests/queries/0_stateless/02874_json_merge_function_test.sql b/tests/queries/0_stateless/02874_json_merge_function_test.sql index fa2a4c721c1..b6287d13778 100644 --- a/tests/queries/0_stateless/02874_json_merge_function_test.sql +++ b/tests/queries/0_stateless/02874_json_merge_function_test.sql @@ -1,9 +1,11 @@ --- { echoOn } +-- Tags: no-fasttest select jsonMerge(null); -select jsonMerge('[1]'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } select jsonMerge('{"a":1}'); +select jsonMerge('{"a":1}', '{"b":1}'); +select jsonMerge('{"a":1}', '{"b":1}', '{"c":[1,2]}'); +select jsonMerge('{"a":1}', '{"b":1}', '{"c":[{"d":1},2]}'); select jsonMerge('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); - select jsonMerge('{"a": "1","b": 2,"c": [true,{"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); +select jsonMerge('[1]'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } select jsonMerge('{"a": "1","b": 2,"c": [true,"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2e45b885afd..26ac93d56b7 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1660,6 +1660,7 @@ jsoncompactstringseachrowwithnames jsoncompactstringseachrowwithnamesandtypes jsoneachrow jsoneachrowwithprogress +jsonMerge jsonobjecteachrow jsonstrings jsonstringseachrow From 483184cbdbd4b415568e215fcd848074e4ca1adf Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 7 Sep 2023 16:02:11 +0000 Subject: [PATCH 030/634] Add cluster groups to Replicated database engine --- src/Databases/DatabaseReplicated.cpp | 35 ++++-- src/Databases/DatabaseReplicated.h | 2 + .../__init__.py | 0 .../configs/backup_group.xml | 3 + .../configs/settings.xml | 14 +++ .../test.py | 116 ++++++++++++++++++ 6 files changed, 163 insertions(+), 7 deletions(-) create mode 100644 tests/integration/test_replicated_database_cluster_groups/__init__.py create mode 100644 tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml create mode 100644 tests/integration/test_replicated_database_cluster_groups/configs/settings.xml create mode 100644 tests/integration/test_replicated_database_cluster_groups/test.py diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index ed56edd7503..8e4460a58bd 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -116,6 +116,8 @@ DatabaseReplicated::DatabaseReplicated( if (!db_settings.collection_name.value.empty()) fillClusterAuthInfo(db_settings.collection_name.value, context_->getConfigRef()); + + cluster_path = zookeeper_path + "/" + getClusterGroup(context_->getConfigRef()); } String DatabaseReplicated::getFullReplicaName(const String & shard, const String & replica) @@ -186,7 +188,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const { host_ids.resize(0); Coordination::Stat stat; - hosts = zookeeper->getChildren(zookeeper_path + "/replicas", &stat); + hosts = zookeeper->getChildren(cluster_path, &stat); if (hosts.empty()) throw Exception(ErrorCodes::NO_ACTIVE_REPLICAS, "No replicas of database {} found. " "It's possible if the first replica is not fully created yet " @@ -198,7 +200,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const futures.reserve(hosts.size()); host_ids.reserve(hosts.size()); for (const auto & host : hosts) - futures.emplace_back(zookeeper->asyncTryGet(zookeeper_path + "/replicas/" + host)); + futures.emplace_back(zookeeper->asyncTryGet(cluster_path + "/" + host)); success = true; for (auto & future : futures) @@ -209,7 +211,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const host_ids.emplace_back(res.data); } - zookeeper->get(zookeeper_path + "/replicas", &stat); + zookeeper->get(cluster_path, &stat); if (cversion != stat.cversion) success = false; if (success) @@ -270,7 +272,7 @@ std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr for (const auto & replica : addresses_with_failover[shard_index]) { String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); - paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); + paths.emplace_back(fs::path(cluster_path) / full_name / "active"); } } @@ -309,6 +311,16 @@ void DatabaseReplicated::fillClusterAuthInfo(String collection_name, const Poco: cluster_auth_info.cluster_secure_connection = config_ref.getBool(config_prefix + ".cluster_secure_connection", false); } +String DatabaseReplicated::getClusterGroup(const Poco::Util::AbstractConfiguration & config_ref) +{ + const auto cluster_group = config_ref.getString("database_replicated_cluster_group", ""); + + if (cluster_group.empty()) + return "replicas"; + + return "replicas_" + cluster_group; +} + void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(LoadingStrictnessLevel mode) { try @@ -326,7 +338,16 @@ void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(LoadingStrictnessL createDatabaseNodesInZooKeeper(current_zookeeper); } - replica_path = fs::path(zookeeper_path) / "replicas" / getFullReplicaName(); + if (!current_zookeeper->exists(cluster_path)) + { + /// Create new cluster group, multiple nodes can execute it concurrently + auto code = current_zookeeper->tryCreate(cluster_path, "", zkutil::CreateMode::Persistent); + + if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) + throw Coordination::Exception(code); + } + + replica_path = fs::path(cluster_path) / getFullReplicaName(); bool is_create_query = mode == LoadingStrictnessLevel::CREATE; String replica_host_id; @@ -704,7 +725,7 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex entry.tracing_context = OpenTelemetry::CurrentContext(); String node_path = ddl_worker->tryEnqueueAndExecuteEntry(entry, query_context); - Strings hosts_to_wait = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); + Strings hosts_to_wait = getZooKeeper()->getChildren(cluster_path); return getDistributedDDLStatus(node_path, entry, query_context, &hosts_to_wait); } @@ -1140,7 +1161,7 @@ void DatabaseReplicated::drop(ContextPtr context_) current_zookeeper->tryRemoveRecursive(replica_path); /// TODO it may leave garbage in ZooKeeper if the last node lost connection here - if (current_zookeeper->tryRemove(zookeeper_path + "/replicas") == Coordination::Error::ZOK) + if (current_zookeeper->tryRemove(cluster_path) == Coordination::Error::ZOK) { /// It was the last replica, remove all metadata current_zookeeper->tryRemoveRecursive(zookeeper_path); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 7ba91e48085..6cb3c98afe3 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -101,6 +101,7 @@ private: } cluster_auth_info; void fillClusterAuthInfo(String collection_name, const Poco::Util::AbstractConfiguration & config); + String getClusterGroup(const Poco::Util::AbstractConfiguration & config); void checkQueryValid(const ASTPtr & query, ContextPtr query_context) const; @@ -127,6 +128,7 @@ private: String shard_name; String replica_name; String replica_path; + String cluster_path; DatabaseReplicatedSettings db_settings; zkutil::ZooKeeperPtr getZooKeeper() const; diff --git a/tests/integration/test_replicated_database_cluster_groups/__init__.py b/tests/integration/test_replicated_database_cluster_groups/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml b/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml new file mode 100644 index 00000000000..8e15851a39d --- /dev/null +++ b/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml @@ -0,0 +1,3 @@ + + backups + diff --git a/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml b/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml new file mode 100644 index 00000000000..5666ffeace8 --- /dev/null +++ b/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml @@ -0,0 +1,14 @@ + + + + 1 + 1 + 1 + + + + + default + + + diff --git a/tests/integration/test_replicated_database_cluster_groups/test.py b/tests/integration/test_replicated_database_cluster_groups/test.py new file mode 100644 index 00000000000..13e51dd6e5f --- /dev/null +++ b/tests/integration/test_replicated_database_cluster_groups/test.py @@ -0,0 +1,116 @@ +import re +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) + +main_node_1 = cluster.add_instance( + "main_node_1", + user_configs=["configs/settings.xml"], + with_zookeeper=True, + stay_alive=True, + macros={"shard": 1, "replica": 1}, +) +main_node_2 = cluster.add_instance( + "main_node_2", + user_configs=["configs/settings.xml"], + with_zookeeper=True, + stay_alive=True, + macros={"shard": 1, "replica": 2}, +) +backup_node_1 = cluster.add_instance( + "backup_node_1", + main_configs=["configs/backup_group.xml"], + user_configs=["configs/settings.xml"], + with_zookeeper=True, + stay_alive=True, + macros={"shard": 1, "replica": 3}, +) +backup_node_2 = cluster.add_instance( + "backup_node_2", + main_configs=["configs/backup_group.xml"], + user_configs=["configs/settings.xml"], + with_zookeeper=True, + stay_alive=True, + macros={"shard": 1, "replica": 4}, +) + +all_nodes = [ + main_node_1, + main_node_2, + backup_node_1, + backup_node_2, +] + +uuid_regex = re.compile("[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}") + + +def assert_create_query(nodes, table_name, expected): + replace_uuid = lambda x: re.sub(uuid_regex, "uuid", x) + query = "show create table {}".format(table_name) + for node in nodes: + assert_eq_with_retry(node, query, expected, get_result=replace_uuid) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_cluster_groups(started_cluster): + for node in all_nodes: + node.query( + f"CREATE DATABASE cluster_groups ENGINE = Replicated('/test/cluster_groups', '{node.macros['shard']}', '{node.macros['replica']}');" + ) + + # 1. system.clusters + + query = "SELECT host_name from system.clusters WHERE cluster = 'cluster_groups' ORDER BY host_name" + expected_main = "main_node_1\nmain_node_2\n" + expected_backup = "backup_node_1\nbackup_node_2\n" + + for node in [main_node_1, main_node_2]: + assert_eq_with_retry(node, query, expected_main) + + for node in [backup_node_1, backup_node_2]: + assert_eq_with_retry(node, query, expected_backup) + + # 2. Query execution depends only on your cluster group + + backup_node_1.stop_clickhouse() + backup_node_2.stop_clickhouse() + + # OK + main_node_1.query( + "CREATE TABLE cluster_groups.table_1 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);" + ) + + # Exception + main_node_2.stop_clickhouse() + settings = {"distributed_ddl_task_timeout": 5} + assert ( + "There are 1 unfinished hosts (0 of them are currently active)" + in main_node_1.query_and_get_error( + "CREATE TABLE cluster_groups.table_2 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);", + settings=settings, + ) + ) + + # 3. After start both groups are synced + + backup_node_1.start_clickhouse() + backup_node_2.start_clickhouse() + main_node_2.start_clickhouse() + + expected_1 = "CREATE TABLE cluster_groups.table_1\\n(\\n `d` Date,\\n `k` UInt64\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\nPARTITION BY toYYYYMM(d)\\nORDER BY k\\nSETTINGS index_granularity = 8192" + expected_2 = "CREATE TABLE cluster_groups.table_2\\n(\\n `d` Date,\\n `k` UInt64\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\nPARTITION BY toYYYYMM(d)\\nORDER BY k\\nSETTINGS index_granularity = 8192" + + assert_create_query(all_nodes, "cluster_groups.table_1", expected_1) + assert_create_query(all_nodes, "cluster_groups.table_2", expected_2) From 7338b560a82be77ccd4c2a880899798906c9b0ca Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 7 Sep 2023 14:37:24 +0200 Subject: [PATCH 031/634] S3Queue fixes --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 535 ++++++++---------- src/Storages/S3Queue/S3QueueFilesMetadata.h | 125 ++-- src/Storages/S3Queue/S3QueueSettings.cpp | 4 +- src/Storages/S3Queue/S3QueueSource.cpp | 246 ++------ src/Storages/S3Queue/S3QueueSource.h | 108 +--- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 20 +- src/Storages/S3Queue/S3QueueTableMetadata.h | 9 +- src/Storages/S3Queue/StorageS3Queue.cpp | 470 ++++++--------- src/Storages/S3Queue/StorageS3Queue.h | 107 ++-- .../integration/test_storage_s3_queue/test.py | 48 +- 10 files changed, 619 insertions(+), 1053 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 4624566a517..d0e4d0f88cc 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -1,18 +1,18 @@ -#include "IO/VarInt.h" #include "config.h" #if USE_AWS_S3 -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include - +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB { @@ -30,151 +30,6 @@ namespace } } -void S3QueueFilesMetadata::S3QueueCollection::read(ReadBuffer & in) -{ - files = {}; - if (in.eof()) - return; - - size_t files_num; - in >> files_num >> "\n"; - while (files_num--) - { - TrackedCollectionItem item; - in >> item.file_path >> "\n"; - in >> item.timestamp >> "\n"; - in >> item.retries_count >> "\n"; - in >> item.last_exception >> "\n"; - files.push_back(item); - } -} - -void S3QueueFilesMetadata::S3QueueCollection::write(WriteBuffer & out) const -{ - out << files.size() << "\n"; - for (const auto & processed_file : files) - { - out << processed_file.file_path << "\n"; - out << processed_file.timestamp << "\n"; - out << processed_file.retries_count << "\n"; - out << processed_file.last_exception << "\n"; - } -} - -String S3QueueFilesMetadata::S3QueueCollection::toString() const -{ - WriteBufferFromOwnString out; - write(out); - return out.str(); -} - -S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::S3QueueCollection::getFileNames() -{ - S3FilesCollection keys = {}; - for (const auto & pair : files) - keys.insert(pair.file_path); - return keys; -} - - -S3QueueFilesMetadata::S3QueueProcessedCollection::S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_) - : max_size(max_size_), max_age(max_age_) -{ -} - -void S3QueueFilesMetadata::S3QueueProcessedCollection::parse(const String & collection_str) -{ - ReadBufferFromString buf(collection_str); - read(buf); - if (max_age > 0) // Remove old items - { - std::erase_if( - files, - [timestamp = getCurrentTime(), this](const TrackedCollectionItem & processed_file) - { return (timestamp - processed_file.timestamp) > max_age; }); - } -} - - -void S3QueueFilesMetadata::S3QueueProcessedCollection::add(const String & file_name) -{ - TrackedCollectionItem processed_file; - processed_file.file_path = file_name; - processed_file.timestamp = getCurrentTime(); - files.push_back(processed_file); - - /// TODO: it is strange that in parse() we take into account only max_age, but here only max_size. - while (files.size() > max_size) - { - files.pop_front(); - } -} - - -S3QueueFilesMetadata::S3QueueFailedCollection::S3QueueFailedCollection(const UInt64 & max_retries_count_) - : max_retries_count(max_retries_count_) -{ -} - -void S3QueueFilesMetadata::S3QueueFailedCollection::parse(const String & collection_str) -{ - ReadBufferFromString buf(collection_str); - read(buf); -} - - -bool S3QueueFilesMetadata::S3QueueFailedCollection::add(const String & file_name, const String & exception_message) -{ - auto failed_it = std::find_if( - files.begin(), files.end(), - [&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; }); - - if (failed_it == files.end()) - { - files.emplace_back(file_name, 0, max_retries_count, exception_message); - } - else if (failed_it->retries_count == 0 || --failed_it->retries_count == 0) - { - return false; - } - return true; -} - -S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::S3QueueFailedCollection::getFileNames() -{ - S3FilesCollection failed_keys; - for (const auto & pair : files) - { - if (pair.retries_count == 0) - failed_keys.insert(pair.file_path); - } - return failed_keys; -} - -void S3QueueFilesMetadata::S3QueueProcessingCollection::parse(const String & collection_str) -{ - ReadBufferFromString rb(collection_str); - Strings result; - readQuoted(result, rb); - files = S3FilesCollection(result.begin(), result.end()); -} - -void S3QueueFilesMetadata::S3QueueProcessingCollection::add(const Strings & file_names) -{ - files.insert(file_names.begin(), file_names.end()); -} - -void S3QueueFilesMetadata::S3QueueProcessingCollection::remove(const String & file_name) -{ - files.erase(file_name); -} - -String S3QueueFilesMetadata::S3QueueProcessingCollection::toString() const -{ - return DB::toString(Strings(files.begin(), files.end())); -} - - S3QueueFilesMetadata::S3QueueFilesMetadata( const StorageS3Queue * storage_, const S3QueueSettings & settings_) @@ -183,171 +38,273 @@ S3QueueFilesMetadata::S3QueueFilesMetadata( , max_set_size(settings_.s3queue_tracked_files_limit.value) , max_set_age_sec(settings_.s3queue_tracked_file_ttl_sec.value) , max_loading_retries(settings_.s3queue_loading_retries.value) - , zookeeper_processing_path(fs::path(storage->getZooKeeperPath()) / "processing") - , zookeeper_processed_path(fs::path(storage->getZooKeeperPath()) / "processed") - , zookeeper_failed_path(fs::path(storage->getZooKeeperPath()) / "failed") - , zookeeper_lock_path(fs::path(storage->getZooKeeperPath()) / "lock") + , zookeeper_processing_path(storage->getZooKeeperPath() / "processing") + , zookeeper_processed_path(storage->getZooKeeperPath() / "processed") + , zookeeper_failed_path(storage->getZooKeeperPath() / "failed") , log(&Poco::Logger::get("S3QueueFilesMetadata")) { } -void S3QueueFilesMetadata::setFileProcessed(const String & file_path) +std::string S3QueueFilesMetadata::NodeMetadata::toString() const { - auto zookeeper = storage->getZooKeeper(); - auto lock = acquireLock(zookeeper); + Poco::JSON::Object json; + json.set("file_path", file_path); + json.set("last_processed_timestamp", getCurrentTime()); + json.set("last_exception", last_exception); + json.set("retries", retries); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); +} + +S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::NodeMetadata::fromString(const std::string & metadata_str) +{ + Poco::JSON::Parser parser; + auto json = parser.parse(metadata_str).extract(); + + NodeMetadata metadata; + metadata.file_path = json->getValue("file_path"); + metadata.last_processed_timestamp = json->getValue("last_processed_timestamp"); + metadata.last_exception = json->getValue("last_exception"); + metadata.retries = json->getValue("retries"); + return metadata; +} + +std::string S3QueueFilesMetadata::getNodeName(const std::string & path) +{ + SipHash path_hash; + path_hash.update(path); + return toString(path_hash.get64()); +} + +S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( + const std::string & path, + const std::string & exception, + size_t retries) +{ + NodeMetadata metadata; + metadata.file_path = path; + metadata.last_processed_timestamp = getCurrentTime(); + metadata.last_exception = exception; + metadata.retries = retries; + return metadata; +} + +bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) +{ switch (mode) { - case S3QueueMode::UNORDERED: - { - S3QueueProcessedCollection processed_files(max_set_size, max_set_age_sec); - processed_files.parse(zookeeper->get(zookeeper_processed_path)); - processed_files.add(file_path); - zookeeper->set(zookeeper_processed_path, processed_files.toString()); - break; - } case S3QueueMode::ORDERED: { - // Check that we set in ZooKeeper node only maximum processed file path. - // This check can be useful, when multiple table engines consume in ordered mode. - String max_file = getMaxProcessedFile(); - if (max_file.compare(file_path) <= 0) - zookeeper->set(zookeeper_processed_path, file_path); - break; + return trySetFileAsProcessingForOrderedMode(path); } - } - removeProcessingFile(file_path); -} - - -bool S3QueueFilesMetadata::setFileFailed(const String & file_path, const String & exception_message) -{ - auto zookeeper = storage->getZooKeeper(); - auto lock = acquireLock(zookeeper); - - S3QueueFailedCollection failed_collection(max_loading_retries); - failed_collection.parse(zookeeper->get(zookeeper_failed_path)); - const bool can_be_retried = failed_collection.add(file_path, exception_message); - zookeeper->set(zookeeper_failed_path, failed_collection.toString()); - removeProcessingFile(file_path); - return can_be_retried; -} - -S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getFailedFiles() -{ - auto zookeeper = storage->getZooKeeper(); - String failed_files = zookeeper->get(zookeeper_failed_path); - - S3QueueFailedCollection failed_collection(max_loading_retries); - failed_collection.parse(failed_files); - return failed_collection.getFileNames(); -} - -String S3QueueFilesMetadata::getMaxProcessedFile() -{ - auto zookeeper = storage->getZooKeeper(); - return zookeeper->get(zookeeper_processed_path); -} - -S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getProcessingFiles() -{ - auto zookeeper = storage->getZooKeeper(); - String processing_files; - if (!zookeeper->tryGet(zookeeper_processing_path, processing_files)) - return {}; - - S3QueueProcessingCollection processing_collection; - if (!processing_files.empty()) - processing_collection.parse(processing_files); - return processing_collection.getFileNames(); -} - -void S3QueueFilesMetadata::setFilesProcessing(const Strings & file_paths) -{ - auto zookeeper = storage->getZooKeeper(); - String processing_files; - zookeeper->tryGet(zookeeper_processing_path, processing_files); - - S3QueueProcessingCollection processing_collection; - if (!processing_files.empty()) - processing_collection.parse(processing_files); - processing_collection.add(file_paths); - - if (zookeeper->exists(zookeeper_processing_path)) - zookeeper->set(zookeeper_processing_path, processing_collection.toString()); - else - zookeeper->create(zookeeper_processing_path, processing_collection.toString(), zkutil::CreateMode::Ephemeral); -} - -void S3QueueFilesMetadata::removeProcessingFile(const String & file_path) -{ - auto zookeeper = storage->getZooKeeper(); - String processing_files; - zookeeper->tryGet(zookeeper_processing_path, processing_files); - - S3QueueProcessingCollection processing_collection; - processing_collection.parse(processing_files); - processing_collection.remove(file_path); - zookeeper->set(zookeeper_processing_path, processing_collection.toString()); -} - -S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getUnorderedProcessedFiles() -{ - auto zookeeper = storage->getZooKeeper(); - S3QueueProcessedCollection processed_collection(max_set_size, max_set_age_sec); - processed_collection.parse(zookeeper->get(zookeeper_processed_path)); - return processed_collection.getFileNames(); -} - -S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getProcessedFailedAndProcessingFiles() -{ - S3FilesCollection processed_and_failed_files = getFailedFiles(); - switch (mode) - { case S3QueueMode::UNORDERED: { - processed_and_failed_files.merge(getUnorderedProcessedFiles()); - break; - } - case S3QueueMode::ORDERED: - { - processed_and_failed_files.insert(getMaxProcessedFile()); - break; + return trySetFileAsProcessingForUnorderedMode(path); } } - processed_and_failed_files.merge(getProcessingFiles()); - return processed_and_failed_files; } -std::shared_ptr S3QueueFilesMetadata::acquireLock(zkutil::ZooKeeperPtr zookeeper) +bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) { - UInt32 retry_count = 200; - UInt32 sleep_ms = 100; - UInt32 retries = 0; + const auto node_name = getNodeName(path); + const auto node_metadata = createNodeMetadata(path).toString(); + const auto zk_client = storage->getZooKeeper(); + + /// The following requests to the following: + /// If !exists(processed_node) && !exists(failed_node) && !exists(processing_node) => create(processing_node) + Coordination::Requests requests; + /// Check that processed node does not appear. + requests.push_back(zkutil::makeCreateRequest(zookeeper_processed_path / node_name, "", zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processed_path / node_name, -1)); + /// Check that failed node does not appear. + requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, "", zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeRemoveRequest(zookeeper_failed_path / node_name, -1)); + /// Check that processing node does not exist and create if not. + requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata, zkutil::CreateMode::Ephemeral)); + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + return code == Coordination::Error::ZOK; +} + +bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) +{ + const auto node_name = getNodeName(path); + const auto node_metadata = createNodeMetadata(path).toString(); + const auto zk_client = storage->getZooKeeper(); while (true) { - Coordination::Error code = zookeeper->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral); - if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + Coordination::Requests requests; + zkutil::addCheckNotExistsRequest(requests, zk_client, zookeeper_failed_path / node_name); + zkutil::addCheckNotExistsRequest(requests, zk_client, zookeeper_processing_path / node_name); + requests.push_back(zkutil::makeGetRequest(zookeeper_processed_path)); + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + + if (code != Coordination::Error::ZOK) { - retries++; - if (retries > retry_count) + if (responses[0]->error != Coordination::Error::ZOK + || responses[1]->error != Coordination::Error::ZOK) { - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Can't acquire zookeeper lock"); + /// Path is already in Failed or Processing. + return false; } - sleepForMilliseconds(sleep_ms); + /// GetRequest for zookeeper_processed_path should never fail, + /// because this is persistent node created at the creation of S3Queue storage. + throw zkutil::KeeperException::fromPath(code, requests.back()->getPath()); } - else if (code != Coordination::Error::ZOK) + + Coordination::Stat processed_node_stat; + NodeMetadata processed_node_metadata; + if (const auto * get_response = dynamic_cast(responses.back().get())) { - throw Coordination::Exception::fromPath(code, zookeeper_lock_path); + processed_node_stat = get_response->stat; + if (!get_response->data.empty()) + processed_node_metadata = NodeMetadata::fromString(get_response->data); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected response type with error: {}", responses.back()->error); + + auto max_processed_file_path = processed_node_metadata.file_path; + if (!max_processed_file_path.empty() && path <= max_processed_file_path) + return false; + + requests.clear(); + zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); + requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata, zkutil::CreateMode::Ephemeral)); + requests.push_back(zkutil::makeCheckRequest(zookeeper_processed_path, processed_node_stat.version)); + + code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return true; + + if (responses[0]->error != Coordination::Error::ZOK + || responses[1]->error != Coordination::Error::ZOK) + { + /// Path is already in Failed or Processing. + return false; + } + /// Max processed path changed. Retry. + } +} + +void S3QueueFilesMetadata::setFileProcessed(const String & path) +{ + switch (mode) + { + case S3QueueMode::ORDERED: + { + return setFileProcessedForOrderedMode(path); + } + case S3QueueMode::UNORDERED: + { + return setFileProcessedForUnorderedMode(path); + } + } +} + +void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(const String & path) +{ + /// List results in s3 are always returned in UTF-8 binary order. + /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) + + const auto node_name = getNodeName(path); + const auto node_metadata = createNodeMetadata(path).toString(); + const auto zk_client = storage->getZooKeeper(); + + Coordination::Requests requests; + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); + requests.push_back(zkutil::makeCreateRequest(zookeeper_processed_path / node_name, node_metadata, zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; + + /// TODO this could be because of the expired session. + if (responses[0]->error != Coordination::Error::ZOK) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attemp to set file as processed but it is not processing"); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attemp to set file as processed but it is already processed"); +} + +void S3QueueFilesMetadata::setFileProcessedForOrderedMode(const String & path) +{ + const auto node_name = getNodeName(path); + const auto node_metadata = createNodeMetadata(path).toString(); + const auto zk_client = storage->getZooKeeper(); + + while (true) + { + std::string res; + Coordination::Stat stat; + bool exists = zk_client->tryGet(zookeeper_processed_path, res, &stat); + Coordination::Requests requests; + if (exists) + { + if (!res.empty()) + { + auto metadata = NodeMetadata::fromString(res); + if (metadata.file_path >= path) + return; + } + requests.push_back(zkutil::makeSetRequest(zookeeper_processed_path, node_metadata, stat.version)); } else { - return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zookeeper); + requests.push_back(zkutil::makeCreateRequest(zookeeper_processed_path, node_metadata, zkutil::CreateMode::Persistent)); } + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; } } +void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exception_message) +{ + const auto node_name = getNodeName(path); + auto node_metadata = createNodeMetadata(path, exception_message); + const auto zk_client = storage->getZooKeeper(); + + Coordination::Requests requests; + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); + requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, node_metadata.toString(), zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; + + if (responses[0]->error != Coordination::Error::ZOK) + { + /// TODO this could be because of the expired session. + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attemp to set file as filed but it is not processing"); + } + + Coordination::Stat stat; + auto failed_node_metadata = NodeMetadata::fromString(zk_client->get(zookeeper_failed_path / node_name, &stat)); + node_metadata.retries = failed_node_metadata.retries + 1; + + /// Failed node already exists, update it. + requests.clear(); + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); + requests.push_back(zkutil::makeSetRequest(zookeeper_failed_path / node_name, node_metadata.toString(), stat.version)); + + responses.clear(); + code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); +} + } #endif diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index 577c71b2227..302feab6028 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -1,102 +1,29 @@ #pragma once +#include "config.h" #if USE_AWS_S3 +#include +#include +#include -# include -# include -# include -# include +namespace fs = std::filesystem; +namespace Poco { class Logger; } namespace DB { -class StorageS3Queue; struct S3QueueSettings; +class StorageS3Queue; class S3QueueFilesMetadata { public: - struct TrackedCollectionItem - { - TrackedCollectionItem() = default; - TrackedCollectionItem(const String & file_path_, UInt64 timestamp_, UInt64 retries_count_, const String & last_exception_) - : file_path(file_path_), timestamp(timestamp_), retries_count(retries_count_), last_exception(last_exception_) {} - String file_path; - UInt64 timestamp = 0; - UInt64 retries_count = 0; - String last_exception; - }; - - using S3FilesCollection = std::unordered_set; - using TrackedFiles = std::deque; - S3QueueFilesMetadata(const StorageS3Queue * storage_, const S3QueueSettings & settings_); - void setFilesProcessing(const Strings & file_paths); - void setFileProcessed(const String & file_path); - bool setFileFailed(const String & file_path, const String & exception_message); + bool trySetFileAsProcessing(const std::string & path); - S3FilesCollection getProcessedFailedAndProcessingFiles(); - String getMaxProcessedFile(); - std::shared_ptr acquireLock(zkutil::ZooKeeperPtr zookeeper); + void setFileProcessed(const std::string & path); - struct S3QueueCollection - { - public: - virtual ~S3QueueCollection() = default; - virtual String toString() const; - S3FilesCollection getFileNames(); - - virtual void parse(const String & collection_str) = 0; - - protected: - TrackedFiles files; - - void read(ReadBuffer & in); - void write(WriteBuffer & out) const; - }; - - struct S3QueueProcessedCollection : public S3QueueCollection - { - public: - S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_); - - void parse(const String & collection_str) override; - void add(const String & file_name); - - private: - const UInt64 max_size; - const UInt64 max_age; - }; - - struct S3QueueFailedCollection : S3QueueCollection - { - public: - S3QueueFailedCollection(const UInt64 & max_retries_count_); - - void parse(const String & collection_str) override; - bool add(const String & file_name, const String & exception_message); - - S3FilesCollection getFileNames(); - - private: - UInt64 max_retries_count; - }; - - struct S3QueueProcessingCollection - { - public: - S3QueueProcessingCollection() = default; - - void parse(const String & collection_str); - void add(const Strings & file_names); - void remove(const String & file_name); - - String toString() const; - const S3FilesCollection & getFileNames() const { return files; } - - private: - S3FilesCollection files; - }; + void setFileFailed(const std::string & path, const std::string & exception_message); private: const StorageS3Queue * storage; @@ -105,23 +32,35 @@ private: const UInt64 max_set_age_sec; const UInt64 max_loading_retries; - const String zookeeper_processing_path; - const String zookeeper_processed_path; - const String zookeeper_failed_path; - const String zookeeper_lock_path; + const fs::path zookeeper_processing_path; + const fs::path zookeeper_processed_path; + const fs::path zookeeper_failed_path; mutable std::mutex mutex; Poco::Logger * log; - S3FilesCollection getFailedFiles(); - S3FilesCollection getProcessingFiles(); - S3FilesCollection getUnorderedProcessedFiles(); + bool trySetFileAsProcessingForOrderedMode(const std::string & path); + bool trySetFileAsProcessingForUnorderedMode(const std::string & path); - void removeProcessingFile(const String & file_path); + void setFileProcessedForOrderedMode(const std::string & path); + void setFileProcessedForUnorderedMode(const std::string & path); + + std::string getNodeName(const std::string & path); + + struct NodeMetadata + { + std::string file_path; + UInt64 last_processed_timestamp = 0; + std::string last_exception; + UInt64 retries = 0; + + std::string toString() const; + static NodeMetadata fromString(const std::string & metadata_str); + }; + + NodeMetadata createNodeMetadata(const std::string & path, const std::string & exception = "", size_t retries = 0); }; - } - #endif diff --git a/src/Storages/S3Queue/S3QueueSettings.cpp b/src/Storages/S3Queue/S3QueueSettings.cpp index b74cf8d39bb..cb312adc5d9 100644 --- a/src/Storages/S3Queue/S3QueueSettings.cpp +++ b/src/Storages/S3Queue/S3QueueSettings.cpp @@ -1,8 +1,8 @@ +#include +#include #include #include #include -#include -#include namespace DB diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 54a863aeb2c..6704345ea59 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -1,47 +1,12 @@ -#include -#include -#include -#include "IO/ParallelReadBuffer.h" -#include "Parsers/ASTCreateQuery.h" #include "config.h" #if USE_AWS_S3 - -# include - -# include - -# include -# include - -# include - -# include -# include - -# include -# include -# include -# include -# include -# include - -# include - -# include -# include -# include - -# include - -# include - -# include -# include -# include - -# include -# include +#include +#include +#include +#include +#include +#include namespace CurrentMetrics @@ -64,138 +29,43 @@ namespace ErrorCodes extern const int S3_ERROR; } - -StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( - const S3::Client & client_, - const S3::URI & globbed_uri_, - ASTPtr query, - const NamesAndTypesList & virtual_columns, - ContextPtr context, - UInt64 & max_poll_size_, - const S3Settings::RequestSettings & request_settings_) - : max_poll_size(max_poll_size_) - , glob_iterator(std::make_unique( - client_, globbed_uri_, query, virtual_columns, context, nullptr, request_settings_)) +StorageS3QueueSource::FileIterator::FileIterator( + std::shared_ptr metadata_, std::unique_ptr glob_iterator_) + : metadata(metadata_) , glob_iterator(std::move(glob_iterator_)) +{ +} + +StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::FileIterator::next() { - /// todo(kssenii): remove this loop, it should not be here while (true) { KeyWithInfo val = glob_iterator->next(); if (val.key.empty()) - break; - keys_buf.push_back(val); + return {}; + if (metadata->trySetFileAsProcessing(val.key)) + return val; } } -Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( - const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file) -{ - for (const KeyWithInfo & val : keys_buf) - { - auto full_path = val.key; - if (exclude_keys.find(full_path) != exclude_keys.end()) - { - LOG_TEST(log, "File {} will be skipped, because it was found in exclude files list " - "(either already processed or failed to be processed)", val.key); - continue; - } - - if ((engine_mode == S3QueueMode::ORDERED) && (full_path.compare(max_file) <= 0)) - continue; - - if ((processing_keys.size() < max_poll_size) || (engine_mode == S3QueueMode::ORDERED)) - { - processing_keys.push_back(val); - } - else - { - break; - } - } - - if (engine_mode == S3QueueMode::ORDERED) - { - std::sort( - processing_keys.begin(), - processing_keys.end(), - [](const KeyWithInfo & lhs, const KeyWithInfo & rhs) { return lhs.key.compare(rhs.key) < 0; }); - - if (processing_keys.size() > max_poll_size) - { - processing_keys.erase(processing_keys.begin() + max_poll_size, processing_keys.end()); - } - } - - Strings keys; - for (const auto & key_info : processing_keys) - keys.push_back(key_info.key); - - processing_keys.push_back(KeyWithInfo()); - processing_iterator = processing_keys.begin(); - return keys; -} - - -StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next() -{ - std::lock_guard lock(mutex); - if (processing_iterator != processing_keys.end()) - { - return *processing_iterator++; - } - - return KeyWithInfo(); -} - StorageS3QueueSource::StorageS3QueueSource( - const ReadFromFormatInfo & info, - const String & format_, String name_, - ContextPtr context_, - std::optional format_settings_, - UInt64 max_block_size_, - const S3Settings::RequestSettings & request_settings_, - String compression_hint_, - const std::shared_ptr & client_, - const String & bucket_, - const String & version_id_, - const String & url_host_and_port, - std::shared_ptr file_iterator_, + const Block & header_, + std::unique_ptr internal_source_, std::shared_ptr files_metadata_, const S3QueueAction & action_, - const size_t download_thread_num_) - : ISource(info.source_header) + RemoveFileFunc remove_file_func_, + const NamesAndTypesList & requested_virtual_columns_, + ContextPtr context_) + : ISource(header_) , WithContext(context_) , name(std::move(name_)) - , bucket(bucket_) - , version_id(version_id_) - , format(format_) - , columns_desc(info.columns_description) - , request_settings(request_settings_) - , client(client_) - , files_metadata(files_metadata_) - , requested_virtual_columns(info.requested_virtual_columns) - , requested_columns(info.requested_columns) - , file_iterator(file_iterator_) , action(action_) + , files_metadata(files_metadata_) + , internal_source(std::move(internal_source_)) + , requested_virtual_columns(requested_virtual_columns_) + , remove_file_func(remove_file_func_) + , log(&Poco::Logger::get("StorageS3QueueSource")) { - internal_source = std::make_shared( - info, - format_, - name_, - context_, - format_settings_, - max_block_size_, - request_settings_, - compression_hint_, - client_, - bucket_, - version_id_, - url_host_and_port, - file_iterator, - download_thread_num_, - false, - /* query_info */ std::nullopt); reader = std::move(internal_source->reader); if (reader) reader_future = std::move(internal_source->reader_future); @@ -213,7 +83,6 @@ String StorageS3QueueSource::getName() const Chunk StorageS3QueueSource::generate() { - auto file_progress = getContext()->getFileProgressCallback(); while (true) { if (isCancelled() || !reader) @@ -223,46 +92,27 @@ Chunk StorageS3QueueSource::generate() break; } - Chunk chunk; - bool success_in_pulling = false; try { + Chunk chunk; if (reader->pull(chunk)) { - UInt64 num_rows = chunk.getNumRows(); - auto file_path = reader.getPath(); - - for (const auto & virtual_column : requested_virtual_columns) - { - if (virtual_column.name == "_path") - { - chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_path)->convertToFullColumnIfConst()); - } - else if (virtual_column.name == "_file") - { - size_t last_slash_pos = file_path.find_last_of('/'); - auto column = virtual_column.type->createColumnConst(num_rows, file_path.substr(last_slash_pos + 1)); - chunk.addColumn(column->convertToFullColumnIfConst()); - } - } - success_in_pulling = true; + LOG_TEST(log, "Read {} rows from file: {}", chunk.getNumRows(), reader.getPath()); + VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath()); + return chunk; } } catch (const Exception & e) { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); files_metadata->setFileFailed(reader.getFile(), e.message()); - success_in_pulling = false; - } - if (success_in_pulling) - { - applyActionAfterProcessing(reader.getFile()); - files_metadata->setFileProcessed(reader.getFile()); - return chunk; + throw; } + files_metadata->setFileProcessed(reader.getFile()); + applyActionAfterProcessing(reader.getFile()); - assert(reader_future.valid()); + chassert(reader_future.valid()); reader = reader_future.get(); if (!reader) @@ -277,37 +127,21 @@ Chunk StorageS3QueueSource::generate() return {}; } - -void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) +void StorageS3QueueSource::applyActionAfterProcessing(const String & path) { switch (action) { case S3QueueAction::DELETE: - deleteProcessedObject(file_path); + { + assert(remove_file_func); + remove_file_func(path); break; + } case S3QueueAction::KEEP: break; } } -void StorageS3QueueSource::deleteProcessedObject(const String & file_path) -{ - LOG_INFO(log, "Delete processed file {} from bucket {}", file_path, bucket); - - S3::DeleteObjectRequest request; - request.WithKey(file_path).WithBucket(bucket); - auto outcome = client->DeleteObject(request); - if (!outcome.IsSuccess()) - { - const auto & err = outcome.GetError(); - LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); - } - else - { - LOG_TRACE(log, "Object with path {} was removed from S3", file_path); - } -} - } #endif diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index f89384fb096..1ec762d6477 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -2,29 +2,13 @@ #include "config.h" #if USE_AWS_S3 +#include +#include +#include +#include -# include - -# include - -# include -# include -# include -# include -# include - -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +namespace Poco { class Logger; } namespace DB { @@ -34,56 +18,37 @@ class StorageS3QueueSource : public ISource, WithContext { public: using IIterator = StorageS3Source::IIterator; - using DisclosedGlobIterator = StorageS3Source::DisclosedGlobIterator; - using KeysWithInfo = StorageS3Source::KeysWithInfo; + using GlobIterator = StorageS3Source::DisclosedGlobIterator; using KeyWithInfo = StorageS3Source::KeyWithInfo; - class QueueGlobIterator : public IIterator + using ZooKeeperGetter = std::function; + using RemoveFileFunc = std::function; + + class FileIterator : public IIterator { public: - QueueGlobIterator( - const S3::Client & client_, - const S3::URI & globbed_uri_, - ASTPtr query, - const NamesAndTypesList & virtual_columns, - ContextPtr context, - UInt64 & max_poll_size_, - const S3Settings::RequestSettings & request_settings_ = {}); + FileIterator( + std::shared_ptr metadata_, + std::unique_ptr glob_iterator_); KeyWithInfo next() override; - Strings - filterProcessingFiles(const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file = ""); - private: - UInt64 max_poll_size; - KeysWithInfo keys_buf; - KeysWithInfo processing_keys; - mutable std::mutex mutex; - std::unique_ptr glob_iterator; - std::vector::iterator processing_iterator; - - Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSourceIterator"); + const std::shared_ptr metadata; + const std::unique_ptr glob_iterator; + std::mutex mutex; }; static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); StorageS3QueueSource( - const ReadFromFormatInfo & info, - const String & format, String name_, - ContextPtr context_, - std::optional format_settings_, - UInt64 max_block_size_, - const S3Settings::RequestSettings & request_settings_, - String compression_hint_, - const std::shared_ptr & client_, - const String & bucket, - const String & version_id, - const String & url_host_and_port, - std::shared_ptr file_iterator_, + const Block & header_, + std::unique_ptr internal_source_, std::shared_ptr files_metadata_, const S3QueueAction & action_, - size_t download_thread_num); + RemoveFileFunc remove_file_func_, + const NamesAndTypesList & requested_virtual_columns_, + ContextPtr context_); ~StorageS3QueueSource() override; @@ -91,34 +56,21 @@ public: Chunk generate() override; - private: - String name; - String bucket; - String version_id; - String format; - ColumnsDescription columns_desc; - S3Settings::RequestSettings request_settings; - std::shared_ptr client; + const String name; + const S3QueueAction action; + const std::shared_ptr files_metadata; + const std::shared_ptr internal_source; + const NamesAndTypesList requested_virtual_columns; + + RemoveFileFunc remove_file_func; + Poco::Logger * log; - std::shared_ptr files_metadata; using ReaderHolder = StorageS3Source::ReaderHolder; ReaderHolder reader; - - NamesAndTypesList requested_virtual_columns; - NamesAndTypesList requested_columns; - std::shared_ptr file_iterator; - const S3QueueAction action; - - Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSource"); - std::future reader_future; - mutable std::mutex mutex; - - std::shared_ptr internal_source; - void deleteProcessedObject(const String & file_path); - void applyActionAfterProcessing(const String & file_path); + void applyActionAfterProcessing(const String & path); }; } diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index 23eebb6ded9..f9c89f4d87d 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -2,12 +2,12 @@ #if USE_AWS_S3 -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include namespace DB @@ -18,13 +18,17 @@ namespace ErrorCodes extern const int METADATA_MISMATCH; } -S3QueueTableMetadata::S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings) +S3QueueTableMetadata::S3QueueTableMetadata( + const StorageS3::Configuration & configuration, + const S3QueueSettings & engine_settings, + const StorageInMemoryMetadata & storage_metadata) { format_name = configuration.format; after_processing = engine_settings.after_processing.toString(); mode = engine_settings.mode.toString(); s3queue_tracked_files_limit = engine_settings.s3queue_tracked_files_limit; s3queue_tracked_file_ttl_sec = engine_settings.s3queue_tracked_file_ttl_sec; + columns = storage_metadata.getColumns().toString(); } @@ -36,6 +40,7 @@ String S3QueueTableMetadata::toString() const json.set("s3queue_tracked_files_limit", s3queue_tracked_files_limit); json.set("s3queue_tracked_file_ttl_sec", s3queue_tracked_file_ttl_sec); json.set("format_name", format_name); + json.set("columns", columns); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); @@ -52,6 +57,7 @@ void S3QueueTableMetadata::read(const String & metadata_str) s3queue_tracked_files_limit = json->getValue("s3queue_tracked_files_limit"); s3queue_tracked_file_ttl_sec = json->getValue("s3queue_tracked_file_ttl_sec"); format_name = json->getValue("format_name"); + columns = json->getValue("columns"); } S3QueueTableMetadata S3QueueTableMetadata::parse(const String & metadata_str) diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index 4b6fbc54825..f15665692c4 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -2,9 +2,9 @@ #if USE_AWS_S3 -# include -# include -# include +#include +#include +#include namespace DB { @@ -18,13 +18,14 @@ class ReadBuffer; struct S3QueueTableMetadata { String format_name; + String columns; String after_processing; String mode; UInt64 s3queue_tracked_files_limit; UInt64 s3queue_tracked_file_ttl_sec; S3QueueTableMetadata() = default; - S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings); + S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings, const StorageInMemoryMetadata & storage_metadata); void read(const String & metadata_str); static S3QueueTableMetadata parse(const String & metadata_str); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 08cbff96cd0..be71af24601 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -2,69 +2,38 @@ #if USE_AWS_S3 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include -# include -# include -# include -# include -# include -# include -# include -# include -# include "IO/ParallelReadBuffer.h" - -# include - -# include - -# include - -# include -# include - -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include - - -# include - -# include -# include -# include - -# include - -# include - -# include - -# include -# include -# include -# include namespace fs = std::filesystem; namespace ProfileEvents { -extern const Event S3DeleteObjects; -extern const Event S3ListObjects; + extern const Event S3DeleteObjects; + extern const Event S3ListObjects; } namespace DB { -static const String PARTITION_ID_WILDCARD = "{_partition_id}"; static const auto MAX_THREAD_WORK_DURATION_MS = 60000; namespace ErrorCodes @@ -78,6 +47,33 @@ namespace ErrorCodes extern const int INCOMPATIBLE_COLUMNS; } +namespace +{ + bool containsGlobs(const S3::URI & url) + { + return url.key.find_first_of("*?{") != std::string::npos; + } + + std::string chooseZooKeeperPath(const StorageID & table_id, const Settings & settings, const S3QueueSettings & s3queue_settings) + { + std::string zk_path_prefix = settings.s3queue_default_zookeeper_path.value; + if (zk_path_prefix.empty()) + zk_path_prefix = "/"; + + std::string result_zk_path; + if (s3queue_settings.keeper_path.changed) + { + /// We do not add table uuid here on purpose. + result_zk_path = fs::path(zk_path_prefix) / s3queue_settings.keeper_path.value; + } + else + { + auto database_uuid = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getUUID(); + result_zk_path = fs::path(zk_path_prefix) / toString(database_uuid) / toString(table_id.uuid); + } + return zkutil::extractZooKeeperPath(result_zk_path, true); + } +} StorageS3Queue::StorageS3Queue( std::unique_ptr s3queue_settings_, @@ -87,79 +83,64 @@ StorageS3Queue::StorageS3Queue( const ConstraintsDescription & constraints_, const String & comment, ContextPtr context_, - std::optional format_settings_, - ASTPtr partition_by_) + std::optional format_settings_) : IStorage(table_id_) , WithContext(context_) , s3queue_settings(std::move(s3queue_settings_)) + , zk_path(chooseZooKeeperPath(table_id_, context_->getSettingsRef(), *s3queue_settings)) , after_processing(s3queue_settings->after_processing) + , files_metadata(std::make_shared(this, *s3queue_settings)) , configuration{configuration_} - , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) , format_settings(format_settings_) - , partition_by(partition_by_) + , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { if (configuration.url.key.ends_with('/')) + { configuration.url.key += '*'; - - if (!withGlobs()) + } + else if (!containsGlobs(configuration.url)) + { throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs"); - - std::string zk_path_prefix = getContext()->getSettingsRef().s3queue_default_zookeeper_path.value; - if (zk_path_prefix.empty()) - zk_path_prefix = "/"; - - std::string result_zk_path; - if (s3queue_settings->keeper_path.changed) - { - /// We do not add table uuid here on purpose. - result_zk_path = fs::path(zk_path_prefix) / s3queue_settings->keeper_path.value; - } - else - { - auto database_uuid = DatabaseCatalog::instance().getDatabase(table_id_.database_name)->getUUID(); - result_zk_path = fs::path(zk_path_prefix) / toString(database_uuid) / toString(table_id_.uuid); } - zk_path = zkutil::extractZooKeeperPath(result_zk_path, true/* check_starts_with_slash */, log); - LOG_INFO(log, "Using zookeeper path: {}", zk_path); - - FormatFactory::instance().checkFormatName(configuration.format); - context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.url.uri); - StorageInMemoryMetadata storage_metadata; configuration.update(context_); + FormatFactory::instance().checkFormatName(configuration.format); + context_->getRemoteHostFilter().checkURL(configuration.url.uri); + StorageInMemoryMetadata storage_metadata; if (columns_.empty()) { auto columns = StorageS3::getTableStructureFromDataImpl(configuration, format_settings, context_); storage_metadata.setColumns(columns); } else + { storage_metadata.setColumns(columns_); - + } storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); + + createOrCheckMetadata(storage_metadata); setInMemoryMetadata(storage_metadata); - auto metadata_snapshot = getInMemoryMetadataPtr(); - const bool is_first_replica = createTableIfNotExists(metadata_snapshot); - - if (!is_first_replica) - { - checkTableStructure(zk_path, metadata_snapshot); - } - - files_metadata = std::make_shared(this, *s3queue_settings); virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); + task = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); - auto poll_thread = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); - task = std::make_shared(std::move(poll_thread)); + LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); } - -bool StorageS3Queue::supportsSubcolumns() const +void StorageS3Queue::startup() { - return true; + if (task) + task->activateAndSchedule(); +} + +void StorageS3Queue::shutdown() +{ + shutdown_called = true; + if (task) + task->deactivate(); } bool StorageS3Queue::supportsSubsetOfColumns() const @@ -177,80 +158,62 @@ Pipe StorageS3Queue::read( size_t /* num_streams */) { if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) - throw Exception( - ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + { + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. " + "To enable use setting `stream_like_engine_allow_direct_select`"); + } if (mv_attached) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageS3Queue with attached materialized views"); + { + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, + "Cannot read from {} with attached materialized views", getName()); + } - auto query_configuration = updateConfigurationAndGetCopy(local_context); - - std::shared_ptr iterator_wrapper = createFileIterator(local_context, query_info.query); + Pipes pipes; + pipes.emplace_back(createSource(column_names, storage_snapshot, query_info.query, max_block_size, local_context)); + return Pipe::unitePipes(std::move(pipes)); +} +std::shared_ptr StorageS3Queue::createSource( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + ASTPtr query, + size_t max_block_size, + ContextPtr local_context) +{ + auto configuration_snapshot = updateConfigurationAndGetCopy(local_context); + auto file_iterator = createFileIterator(local_context, query); auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); - const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; - - return Pipe(std::make_shared( - read_from_format_info, - configuration.format, - getName(), - local_context, - format_settings, + auto internal_source = std::make_unique( + read_from_format_info, configuration.format, getName(), local_context, format_settings, max_block_size, - query_configuration.request_settings, - configuration.compression_method, - query_configuration.client, - query_configuration.url.bucket, - query_configuration.url.version_id, - query_configuration.url.uri.getHost() + std::to_string(query_configuration.url.uri.getPort()), - iterator_wrapper, - files_metadata, - after_processing, - max_download_threads)); -} + configuration_snapshot.request_settings, + configuration_snapshot.compression_method, + configuration_snapshot.client, + configuration_snapshot.url.bucket, + configuration_snapshot.url.version_id, + configuration_snapshot.url.uri.getHost() + std::to_string(configuration_snapshot.url.uri.getPort()), + file_iterator, local_context->getSettingsRef().max_download_threads, false, /* query_info */ std::nullopt); -SinkToStoragePtr StorageS3Queue::write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, bool) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName()); -} - -void StorageS3Queue::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); -} - -NamesAndTypesList StorageS3Queue::getVirtuals() const -{ - return virtual_columns; -} - -bool StorageS3Queue::supportsPartitionBy() const -{ - return true; -} - -void StorageS3Queue::startup() -{ - if (task) - task->holder->activateAndSchedule(); -} - -void StorageS3Queue::shutdown() -{ - shutdown_called = true; - if (task) + auto file_deleter = [this, bucket = configuration_snapshot.url.bucket, client = configuration_snapshot.client](const std::string & path) { - task->stream_cancelled = true; - task->holder->deactivate(); - } -} - -size_t StorageS3Queue::getTableDependentCount() const -{ - auto table_id = getStorageID(); - // Check if at least one direct dependency is attached - return DatabaseCatalog::instance().getDependentViews(table_id).size(); + S3::DeleteObjectRequest request; + request.WithKey(path).WithBucket(bucket); + auto outcome = client->DeleteObject(request); + if (!outcome.IsSuccess()) + { + const auto & err = outcome.GetError(); + LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); + } + else + { + LOG_TRACE(log, "Object with path {} was removed from S3", path); + } + }; + return std::make_shared( + getName(), read_from_format_info.source_header, std::move(internal_source), + files_metadata, after_processing, file_deleter, read_from_format_info.requested_virtual_columns, local_context); } bool StorageS3Queue::hasDependencies(const StorageID & table_id) @@ -280,40 +243,33 @@ bool StorageS3Queue::hasDependencies(const StorageID & table_id) void StorageS3Queue::threadFunc() { - bool reschedule = true; + SCOPE_EXIT({ mv_attached.store(false); }); try { auto table_id = getStorageID(); - - auto dependencies_count = getTableDependentCount(); + size_t dependencies_count = DatabaseCatalog::instance().getDependentViews(table_id).size(); if (dependencies_count) { auto start_time = std::chrono::steady_clock::now(); - + /// Reset reschedule interval. + reschedule_processing_interval_ms = s3queue_settings->s3queue_polling_min_timeout_ms; + /// Disallow parallel selects while streaming to mv. mv_attached.store(true); - // Keep streaming as long as there are attached views and streaming is not cancelled - while (!task->stream_cancelled) - { - if (!hasDependencies(table_id)) - { - /// For this case, we can not wait for watch thread to wake up - reschedule = true; - break; - } + /// Keep streaming as long as there are attached views and streaming is not cancelled + while (!shutdown_called && hasDependencies(table_id)) + { LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + streamToViews(); - auto ts = std::chrono::steady_clock::now(); - auto duration = std::chrono::duration_cast(ts - start_time); + auto now = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(now - start_time); if (duration.count() > MAX_THREAD_WORK_DURATION_MS) { LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); - reschedule = true; break; } - - reschedule_processing_interval_ms = s3queue_settings->s3queue_polling_min_timeout_ms; } } } @@ -322,19 +278,16 @@ void StorageS3Queue::threadFunc() tryLogCurrentException(__PRETTY_FUNCTION__); } - mv_attached.store(false); - - if (reschedule && !shutdown_called) + if (!shutdown_called) { - LOG_TRACE(log, "Reschedule S3 Queue thread func."); - /// Reschedule with backoff. if (reschedule_processing_interval_ms < s3queue_settings->s3queue_polling_max_timeout_ms) reschedule_processing_interval_ms += s3queue_settings->s3queue_polling_backoff_ms; - task->holder->scheduleAfter(reschedule_processing_interval_ms); + + LOG_TRACE(log, "Reschedule S3 Queue processing thread in {} ms", reschedule_processing_interval_ms); + task->scheduleAfter(reschedule_processing_interval_ms); } } - void StorageS3Queue::streamToViews() { auto table_id = getStorageID(); @@ -348,8 +301,6 @@ void StorageS3Queue::streamToViews() auto insert = std::make_shared(); insert->table_id = table_id; - size_t block_size = 100; - auto s3queue_context = Context::createCopy(getContext()); s3queue_context->makeQueryContext(); auto query_configuration = updateConfigurationAndGetCopy(s3queue_context); @@ -358,40 +309,14 @@ void StorageS3Queue::streamToViews() // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true); auto block_io = interpreter.execute(); - auto column_names = block_io.pipeline.getHeader().getNames(); - - // Create a stream for each consumer and join them in a union stream - - std::shared_ptr iterator_wrapper = createFileIterator(s3queue_context, nullptr); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); - const size_t max_download_threads = s3queue_context->getSettingsRef().max_download_threads; - - auto pipe = Pipe(std::make_shared( - read_from_format_info, - configuration.format, - getName(), - s3queue_context, - format_settings, - block_size, - query_configuration.request_settings, - configuration.compression_method, - query_configuration.client, - query_configuration.url.bucket, - query_configuration.url.version_id, - query_configuration.url.uri.getHost() + std::to_string(query_configuration.url.uri.getPort()), - iterator_wrapper, - files_metadata, - after_processing, - max_download_threads)); + auto pipe = Pipe(createSource(block_io.pipeline.getHeader().getNames(), storage_snapshot, nullptr, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context)); std::atomic_size_t rows = 0; - { - block_io.pipeline.complete(std::move(pipe)); - block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); - CompletedPipelineExecutor executor(block_io.pipeline); - executor.execute(); - } + block_io.pipeline.complete(std::move(pipe)); + block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); + CompletedPipelineExecutor executor(block_io.pipeline); + executor.execute(); } StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context) @@ -411,49 +336,40 @@ zkutil::ZooKeeperPtr StorageS3Queue::getZooKeeper() const return zk_client; } - -bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot) +void StorageS3Queue::createOrCheckMetadata(const StorageInMemoryMetadata & storage_metadata) { auto zookeeper = getZooKeeper(); zookeeper->createAncestors(zk_path); - for (size_t i = 0; i < zk_create_table_retries; ++i) + for (size_t i = 0; i < 1000; ++i) { - Coordination::Requests ops; - bool is_first_replica = true; - if (zookeeper->exists(zk_path + "/metadata")) + Coordination::Requests requests; + if (zookeeper->exists(zk_path / "metadata")) { - if (!zookeeper->exists(zk_path + "/processing")) - ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processing", "", zkutil::CreateMode::Ephemeral)); - LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zk_path); - is_first_replica = false; + checkTableStructure(zk_path, storage_metadata); } else { - String metadata_str = S3QueueTableMetadata(configuration, *s3queue_settings).toString(); - ops.emplace_back(zkutil::makeCreateRequest(zk_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processed", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/failed", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processing", "", zkutil::CreateMode::Ephemeral)); - ops.emplace_back(zkutil::makeCreateRequest( - zk_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); - - ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/metadata", metadata_str, zkutil::CreateMode::Persistent)); + std::string metadata = S3QueueTableMetadata(configuration, *s3queue_settings, storage_metadata).toString(); + requests.emplace_back(zkutil::makeCreateRequest(zk_path, "", zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(zk_path / "processed", "", zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(zk_path / "failed", "", zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(zk_path / "processing", "", zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(zk_path / "metadata", metadata, zkutil::CreateMode::Persistent)); } Coordination::Responses responses; - auto code = zookeeper->tryMulti(ops, responses); + auto code = zookeeper->tryMulti(requests, responses); if (code == Coordination::Error::ZNODEEXISTS) { - LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zk_path); + LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zk_path.string()); continue; } else if (code != Coordination::Error::ZOK) { - zkutil::KeeperMultiException::check(code, ops, responses); + zkutil::KeeperMultiException::check(code, requests, responses); } - - return is_first_replica; + return; } throw Exception( @@ -463,24 +379,20 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ } -/** Verify that list of columns and table settings match those specified in ZK (/metadata). - * If not, throw an exception. - */ -void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot) +void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const StorageInMemoryMetadata & storage_metadata) { + // Verify that list of columns and table settings match those specified in ZK (/metadata). + // If not, throw an exception. + auto zookeeper = getZooKeeper(); - - S3QueueTableMetadata old_metadata(configuration, *s3queue_settings); - - Coordination::Stat metadata_stat; - String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata", &metadata_stat); + String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata"); auto metadata_from_zk = S3QueueTableMetadata::parse(metadata_str); + + S3QueueTableMetadata old_metadata(configuration, *s3queue_settings, storage_metadata); old_metadata.checkEquals(metadata_from_zk); - Coordination::Stat columns_stat; - auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns", &columns_stat)); - - const ColumnsDescription & old_columns = metadata_snapshot->getColumns(); + auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns")); + const ColumnsDescription & old_columns = storage_metadata.getColumns(); if (columns_from_zk != old_columns) { throw Exception( @@ -492,38 +404,12 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const } } - -std::shared_ptr -StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query) +std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query) { - auto it = std::make_shared( - *configuration.client, - configuration.url, - query, - virtual_columns, - local_context, - s3queue_settings->s3queue_polling_size.value, - configuration.request_settings); - - auto zookeeper = getZooKeeper(); - auto lock = files_metadata->acquireLock(zookeeper); - S3QueueFilesMetadata::S3FilesCollection files_to_skip = files_metadata->getProcessedFailedAndProcessingFiles(); - - Strings files_to_process; - if (s3queue_settings->mode == S3QueueMode::UNORDERED) - { - files_to_process = it->filterProcessingFiles(s3queue_settings->mode, files_to_skip); - } - else - { - String max_processed_file = files_metadata->getMaxProcessedFile(); - files_to_process = it->filterProcessingFiles(s3queue_settings->mode, files_to_skip, max_processed_file); - } - - LOG_TEST(log, "Found files to process: {}", fmt::join(files_to_process, ", ")); - - files_metadata->setFilesProcessing(files_to_process); - return it; + auto glob_iterator = std::make_unique( + *configuration.client, configuration.url, query, virtual_columns, local_context, + /* read_keys */nullptr, configuration.request_settings); + return std::make_shared(files_metadata, std::move(glob_iterator)); } void StorageS3Queue::drop() @@ -540,11 +426,15 @@ void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) [](const StorageFactory::Arguments & args) { if (!args.attach && !args.getLocalContext()->getSettingsRef().allow_experimental_s3queue) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3Queue is experimental. You can enable it with the `allow_experimental_s3queue` setting."); + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3Queue is experimental. " + "You can enable it with the `allow_experimental_s3queue` setting."); + } auto & engine_args = args.engine_args; if (engine_args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); // Use format settings from global server context + settings from @@ -582,10 +472,6 @@ void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) format_settings = getFormatSettings(args.getContext()); } - ASTPtr partition_by; - if (args.storage_def->partition_by) - partition_by = args.storage_def->partition_by->clone(); - return std::make_shared( std::move(s3queue_settings), std::move(configuration), @@ -594,12 +480,10 @@ void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) args.constraints, args.comment, args.getContext(), - format_settings, - partition_by); + format_settings); }, { .supports_settings = true, - .supports_sort_order = true, // for partition by .supports_schema_inference = true, .source_access_type = AccessType::S3, }); diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 712fe9e530b..9151473a9bc 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -4,29 +4,14 @@ #if USE_AWS_S3 -# include - -# include -# include - -# include -# include -# include -# include -# include -# include - -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include +#include namespace Aws::S3 { @@ -35,7 +20,7 @@ class Client; namespace DB { - +class S3QueueFilesMetadata; class StorageS3Queue : public IStorage, WithContext { @@ -50,8 +35,7 @@ public: const ConstraintsDescription & constraints_, const String & comment, ContextPtr context_, - std::optional format_settings_, - ASTPtr partition_by_ = nullptr); + std::optional format_settings_); String getName() const override { return "S3Queue"; } @@ -64,79 +48,58 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write( - const ASTPtr & query, - const StorageMetadataPtr & metadata_snapshot, - ContextPtr context, - bool async_insert) override; - - void truncate( - const ASTPtr & /*query*/, - const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr /*local_context*/, - TableExclusiveLockHolder &) override; - - NamesAndTypesList getVirtuals() const override; - - bool supportsPartitionBy() const override; + NamesAndTypesList getVirtuals() const override { return virtual_columns; } const auto & getFormatName() const { return configuration.format; } - const String & getZooKeeperPath() const { return zk_path; } + const fs::path & getZooKeeperPath() const { return zk_path; } zkutil::ZooKeeperPtr getZooKeeper() const; private: + using FileIterator = StorageS3QueueSource::FileIterator; + const std::unique_ptr s3queue_settings; + const fs::path zk_path; const S3QueueAction after_processing; std::shared_ptr files_metadata; Configuration configuration; + + const std::optional format_settings; NamesAndTypesList virtual_columns; - UInt64 reschedule_processing_interval_ms; - std::optional format_settings; - ASTPtr partition_by; - - String zk_path; mutable zkutil::ZooKeeperPtr zk_client; mutable std::mutex zk_mutex; + BackgroundSchedulePool::TaskHolder task; + std::atomic stream_cancelled{false}; + UInt64 reschedule_processing_interval_ms; + std::atomic mv_attached = false; - std::atomic shutdown_called{false}; + std::atomic shutdown_called = false; Poco::Logger * log; - bool supportsSubcolumns() const override; - bool withGlobs() const { return configuration.url.key.find_first_of("*?{") != std::string::npos; } - - void threadFunc(); - size_t getTableDependentCount() const; - bool hasDependencies(const StorageID & table_id); - void startup() override; void shutdown() override; void drop() override; - - struct TaskContext - { - BackgroundSchedulePool::TaskHolder holder; - std::atomic stream_cancelled{false}; - explicit TaskContext(BackgroundSchedulePool::TaskHolder && task_) : holder(std::move(task_)) { } - }; - std::shared_ptr task; - bool supportsSubsetOfColumns() const override; + bool supportsSubcolumns() const override { return true; } - const UInt32 zk_create_table_retries = 1000; - bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); - void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); - - using KeysWithInfo = StorageS3QueueSource::KeysWithInfo; - - std::shared_ptr - createFileIterator(ContextPtr local_context, ASTPtr query); + std::shared_ptr createFileIterator(ContextPtr local_context, ASTPtr query); + std::shared_ptr createSource( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + ASTPtr query, + size_t max_block_size, + ContextPtr local_context); + bool hasDependencies(const StorageID & table_id); void streamToViews(); + void threadFunc(); + + void createOrCheckMetadata(const StorageInMemoryMetadata & storage_metadata); + void checkTableStructure(const String & zookeeper_prefix, const StorageInMemoryMetadata & storage_metadata); Configuration updateConfigurationAndGetCopy(ContextPtr local_context); }; diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index c11bbd43dc6..65c31acb5d8 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -111,6 +111,7 @@ def generate_random_files( to_generate = [ (f"{prefix}/test_{i}.csv", i) for i in range(start_ind, start_ind + count) ] + print(f"Generating files: {to_generate}") to_generate.sort(key=lambda x: x[0]) for filename, i in to_generate: @@ -179,29 +180,58 @@ def run_query(instance, query, stdin=None, settings=None): @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_delete_after_processing(started_cluster, mode): - prefix = "delete" bucket = started_cluster.minio_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + node = started_cluster.instances["instance"] - total_values = generate_random_files(5, prefix, started_cluster, bucket) - instance.query( + table_name = "test.delete_after_processing" + dst_table_name = "test.delete_after_processing_dst" + mv_name = "test.delete_after_processing_mv" + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + files_num = 5 + row_num = 10 + + prefix = "delete" + total_values = generate_random_files( + files_num, prefix, started_cluster, bucket, row_num=row_num + ) + node.query( f""" - DROP TABLE IF EXISTS test.s3_queue; - CREATE TABLE test.s3_queue ({table_format}) + DROP TABLE IF EXISTS {table_name}; + DROP TABLE IF EXISTS {dst_table_name}; + DROP TABLE IF EXISTS {mv_name}; + CREATE TABLE {table_name} ({table_format}) ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', keeper_path = '/clickhouse/test_delete_{mode}', s3queue_loading_retries = 3, after_processing='delete'; + + CREATE TABLE {dst_table_name} ({table_format}, _path String) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE MATERIALIZED VIEW {mv_name} TO {dst_table_name} AS SELECT *, _path FROM {table_name}; """ ) - get_query = f"SELECT * FROM test.s3_queue ORDER BY column1, column2, column3" + expected_count = files_num * row_num + for _ in range(100): + count = int(node.query(f"SELECT count() FROM {dst_table_name}")) + print(f"{count}/{expected_count}") + if count == expected_count: + break + time.sleep(1) + + assert int(node.query(f"SELECT count() FROM {dst_table_name}")) == expected_count + assert int(node.query(f"SELECT uniq(_path) FROM {dst_table_name}")) == files_num assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + list(map(int, l.split())) + for l in node.query( + f"SELECT column1, column2, column3 FROM {dst_table_name} ORDER BY column1, column2, column3" + ).splitlines() ] == sorted(total_values, key=lambda x: (x[0], x[1], x[2])) + minio = started_cluster.minio_client objects = list(minio.list_objects(started_cluster.minio_bucket, recursive=True)) assert len(objects) == 0 From 220a67eca7c47b211590de6187152428c856f19d Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Sep 2023 15:49:34 +0200 Subject: [PATCH 032/634] Better --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 148 +++-- src/Storages/S3Queue/StorageS3Queue.cpp | 3 +- .../configs/zookeeper.xml | 16 + .../integration/test_storage_s3_queue/test.py | 593 +++++++++--------- 4 files changed, 391 insertions(+), 369 deletions(-) create mode 100644 tests/integration/test_storage_s3_queue/configs/zookeeper.xml diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index d0e4d0f88cc..99c0924968c 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -19,7 +19,7 @@ namespace DB namespace ErrorCodes { - extern const int TIMEOUT_EXCEEDED; + extern const int LOGICAL_ERROR; } namespace @@ -113,16 +113,9 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::str const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = storage->getZooKeeper(); - /// The following requests to the following: - /// If !exists(processed_node) && !exists(failed_node) && !exists(processing_node) => create(processing_node) Coordination::Requests requests; - /// Check that processed node does not appear. - requests.push_back(zkutil::makeCreateRequest(zookeeper_processed_path / node_name, "", zkutil::CreateMode::Persistent)); - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processed_path / node_name, -1)); - /// Check that failed node does not appear. - requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, "", zkutil::CreateMode::Persistent)); - requests.push_back(zkutil::makeRemoveRequest(zookeeper_failed_path / node_name, -1)); - /// Check that processing node does not exist and create if not. + zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_processed_path / node_name); + zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata, zkutil::CreateMode::Ephemeral)); Coordination::Responses responses; @@ -139,42 +132,30 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin while (true) { Coordination::Requests requests; - zkutil::addCheckNotExistsRequest(requests, zk_client, zookeeper_failed_path / node_name); - zkutil::addCheckNotExistsRequest(requests, zk_client, zookeeper_processing_path / node_name); - requests.push_back(zkutil::makeGetRequest(zookeeper_processed_path)); + zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); + zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_processing_path / node_name); Coordination::Responses responses; auto code = zk_client->tryMulti(requests, responses); - if (code != Coordination::Error::ZOK) { - if (responses[0]->error != Coordination::Error::ZOK - || responses[1]->error != Coordination::Error::ZOK) - { - /// Path is already in Failed or Processing. - return false; - } - /// GetRequest for zookeeper_processed_path should never fail, - /// because this is persistent node created at the creation of S3Queue storage. - throw zkutil::KeeperException::fromPath(code, requests.back()->getPath()); + LOG_TEST(log, "Skipping file `{}`: {}", + path, responses[0]->error != Coordination::Error::ZOK ? "failed" : "processing"); + return false; } Coordination::Stat processed_node_stat; + auto data = zk_client->get(zookeeper_processed_path, &processed_node_stat); NodeMetadata processed_node_metadata; - if (const auto * get_response = dynamic_cast(responses.back().get())) - { - processed_node_stat = get_response->stat; - if (!get_response->data.empty()) - processed_node_metadata = NodeMetadata::fromString(get_response->data); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected response type with error: {}", responses.back()->error); + if (!data.empty()) + processed_node_metadata = NodeMetadata::fromString(data); auto max_processed_file_path = processed_node_metadata.file_path; if (!max_processed_file_path.empty() && path <= max_processed_file_path) return false; requests.clear(); + responses.clear(); zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata, zkutil::CreateMode::Ephemeral)); requests.push_back(zkutil::makeCheckRequest(zookeeper_processed_path, processed_node_stat.version)); @@ -186,10 +167,14 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin if (responses[0]->error != Coordination::Error::ZOK || responses[1]->error != Coordination::Error::ZOK) { - /// Path is already in Failed or Processing. + LOG_TEST(log, "Skipping file `{}`: {}", + path, responses[0]->error != Coordination::Error::ZOK ? "failed" : "processing"); return false; } - /// Max processed path changed. Retry. + else + { + LOG_TEST(log, "Version of max processed file changed. Retring the check for file `{}`", path); + } } } @@ -228,9 +213,9 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(const String & path) /// TODO this could be because of the expired session. if (responses[0]->error != Coordination::Error::ZOK) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attemp to set file as processed but it is not processing"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is not processing"); else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attemp to set file as processed but it is already processed"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is already processed"); } void S3QueueFilesMetadata::setFileProcessedForOrderedMode(const String & path) @@ -273,36 +258,83 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc auto node_metadata = createNodeMetadata(path, exception_message); const auto zk_client = storage->getZooKeeper(); - Coordination::Requests requests; - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); - requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, node_metadata.toString(), zkutil::CreateMode::Persistent)); - - Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) - return; - - if (responses[0]->error != Coordination::Error::ZOK) + if (max_loading_retries == 0) { - /// TODO this could be because of the expired session. - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attemp to set file as filed but it is not processing"); + Coordination::Requests requests; + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); + requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, + node_metadata.toString(), + zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + { + LOG_TEST(log, "File `{}` failed to process and will not be retried. " + "Error: {}", path, exception_message); + return; + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); } + const auto node_name_with_retriable_suffix = node_name + ".retriable"; + Coordination::Stat stat; - auto failed_node_metadata = NodeMetadata::fromString(zk_client->get(zookeeper_failed_path / node_name, &stat)); - node_metadata.retries = failed_node_metadata.retries + 1; + std::string res; + if (zk_client->tryGet(zookeeper_failed_path / node_name_with_retriable_suffix, res, &stat)) + { + auto failed_node_metadata = NodeMetadata::fromString(res); + node_metadata.retries = failed_node_metadata.retries + 1; + } - /// Failed node already exists, update it. - requests.clear(); - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); - requests.push_back(zkutil::makeSetRequest(zookeeper_failed_path / node_name, node_metadata.toString(), stat.version)); + LOG_TEST(log, "File `{}` failed to process, try {}/{} (Error: {})", + path, node_metadata.retries, max_loading_retries, exception_message); - responses.clear(); - code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) - return; + if (node_metadata.retries >= max_loading_retries) + { + /// File is no longer retriable. + /// Make a failed/node_name node and remove failed/node_name.retriable node. + /// TODO always add version for processing node. - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); + Coordination::Requests requests; + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); + requests.push_back(zkutil::makeRemoveRequest(zookeeper_failed_path / node_name_with_retriable_suffix, + stat.version)); + requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, + node_metadata.toString(), + zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); + } + else + { + Coordination::Requests requests; + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); + if (node_metadata.retries == 0) + { + requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name_with_retriable_suffix, + node_metadata.toString(), + zkutil::CreateMode::Persistent)); + } + else + { + requests.push_back(zkutil::makeSetRequest(zookeeper_failed_path / node_name_with_retriable_suffix, + node_metadata.toString(), + stat.version)); + } + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); + } } } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index be71af24601..9aa83a1aa97 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -41,7 +41,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int S3_ERROR; - extern const int NOT_IMPLEMENTED; extern const int QUERY_NOT_ALLOWED; extern const int REPLICA_ALREADY_EXISTS; extern const int INCOMPATIBLE_COLUMNS; @@ -391,7 +390,7 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const S3QueueTableMetadata old_metadata(configuration, *s3queue_settings, storage_metadata); old_metadata.checkEquals(metadata_from_zk); - auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns")); + auto columns_from_zk = ColumnsDescription::parse(metadata_from_zk.columns); const ColumnsDescription & old_columns = storage_metadata.getColumns(); if (columns_from_zk != old_columns) { diff --git a/tests/integration/test_storage_s3_queue/configs/zookeeper.xml b/tests/integration/test_storage_s3_queue/configs/zookeeper.xml new file mode 100644 index 00000000000..27334dca590 --- /dev/null +++ b/tests/integration/test_storage_s3_queue/configs/zookeeper.xml @@ -0,0 +1,16 @@ + + + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + + diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 65c31acb5d8..9793f2b7191 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -17,6 +17,11 @@ export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/home/sergey/vkr/ClickHouse/programs/ser """ +MINIO_INTERNAL_PORT = 9001 +AVAILABLE_MODES = ["unordered", "ordered"] +AUTH = "'minio','minio123'," +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. @@ -92,53 +97,6 @@ def s3_queue_setup_teardown(started_cluster): yield # run test -MINIO_INTERNAL_PORT = 9001 -AVAILABLE_MODES = ["unordered", "ordered"] -AUTH = "'minio','minio123'," - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - - -def put_s3_file_content(started_cluster, bucket, filename, data): - buf = io.BytesIO(data) - started_cluster.minio_client.put_object(bucket, filename, buf, len(data)) - - -def generate_random_files( - count, prefix, cluster, bucket, column_num=3, row_num=10, start_ind=0 -): - total_values = [] - to_generate = [ - (f"{prefix}/test_{i}.csv", i) for i in range(start_ind, start_ind + count) - ] - print(f"Generating files: {to_generate}") - to_generate.sort(key=lambda x: x[0]) - - for filename, i in to_generate: - rand_values = [ - [random.randint(0, 50) for _ in range(column_num)] for _ in range(row_num) - ] - total_values += rand_values - values_csv = ( - "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" - ).encode() - put_s3_file_content(cluster, bucket, filename, values_csv) - return total_values - - -# Returns content of given S3 file as string. -def get_s3_file_content(started_cluster, bucket, filename, decode=True): - # type: (ClickHouseCluster, str, str, bool) -> str - - data = started_cluster.minio_client.get_object(bucket, filename) - data_str = b"" - for chunk in data.stream(): - data_str += chunk - if decode: - return data_str.decode() - return data_str - - @pytest.fixture(scope="module") def started_cluster(): try: @@ -148,7 +106,11 @@ def started_cluster(): user_configs=["configs/users.xml"], with_minio=True, with_zookeeper=True, - main_configs=["configs/defaultS3.xml", "configs/named_collections.xml"], + main_configs=[ + "configs/defaultS3.xml", + "configs/named_collections.xml", + "configs/zookeeper.xml", + ], ) cluster.add_instance( "instance2", @@ -178,43 +140,119 @@ def run_query(instance, query, stdin=None, settings=None): return result -@pytest.mark.parametrize("mode", AVAILABLE_MODES) -def test_delete_after_processing(started_cluster, mode): - bucket = started_cluster.minio_bucket - node = started_cluster.instances["instance"] +def generate_random_files( + started_cluster, files_path, count, column_num=3, row_num=10, start_ind=0 +): + files = [ + (f"{files_path}/test_{i}.csv", i) for i in range(start_ind, start_ind + count) + ] + files.sort(key=lambda x: x[0]) - table_name = "test.delete_after_processing" - dst_table_name = "test.delete_after_processing_dst" - mv_name = "test.delete_after_processing_mv" - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - files_num = 5 - row_num = 10 + print(f"Generating files: {files}") - prefix = "delete" - total_values = generate_random_files( - files_num, prefix, started_cluster, bucket, row_num=row_num + total_values = [] + for filename, i in files: + rand_values = [ + [random.randint(0, 1000) for _ in range(column_num)] for _ in range(row_num) + ] + total_values += rand_values + values_csv = ( + "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" + ).encode() + put_s3_file_content(started_cluster, filename, values_csv) + return total_values + + +def put_s3_file_content(started_cluster, filename, data): + buf = io.BytesIO(data) + started_cluster.minio_client.put_object( + started_cluster.minio_bucket, filename, buf, len(data) ) + + +def get_s3_file_content(started_cluster, bucket, filename, decode=True): + # type: (ClickHouseCluster, str, str, bool) -> str + # Returns content of given S3 file as string. + + data = started_cluster.minio_client.get_object(bucket, filename) + data_str = b"" + for chunk in data.stream(): + data_str += chunk + if decode: + return data_str.decode() + return data_str + + +def create_table( + started_cluster, + node, + table_name, + mode, + files_path, + format="column1 UInt32, column2 UInt32, column3 UInt32", + additional_settings={}, +): + settings = { + "s3queue_loading_retries": 0, + "after_processing": "keep", + "keeper_path": f"/clickhouse/test_{table_name}", + "mode": f"{mode}", + } + settings.update(additional_settings) + + url = f"http://{started_cluster.minio_host}:{started_cluster.minio_port}/{started_cluster.minio_bucket}/{files_path}/" + node.query(f"DROP TABLE IF EXISTS {table_name}") + create_query = f""" + CREATE TABLE {table_name} ({format}) + ENGINE = S3Queue('{url}', {AUTH}'CSV') + SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))} + """ + node.query(create_query) + + +def create_mv( + node, + src_table_name, + dst_table_name, + format="column1 UInt32, column2 UInt32, column3 UInt32", +): + mv_name = f"{dst_table_name}_mv" node.query( f""" - DROP TABLE IF EXISTS {table_name}; DROP TABLE IF EXISTS {dst_table_name}; DROP TABLE IF EXISTS {mv_name}; - CREATE TABLE {table_name} ({table_format}) - ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/test_delete_{mode}', - s3queue_loading_retries = 3, - after_processing='delete'; - CREATE TABLE {dst_table_name} ({table_format}, _path String) + CREATE TABLE {dst_table_name} ({format}, _path String) ENGINE = MergeTree() ORDER BY column1; - CREATE MATERIALIZED VIEW {mv_name} TO {dst_table_name} AS SELECT *, _path FROM {table_name}; + CREATE MATERIALIZED VIEW {mv_name} TO {dst_table_name} AS SELECT *, _path FROM {src_table_name}; """ ) + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_delete_after_processing(started_cluster, mode): + node = started_cluster.instances["instance"] + table_name = f"test.delete_after_processing_{mode}" + dst_table_name = f"{table_name}_dst" + files_path = f"{table_name}_data" + files_num = 5 + row_num = 10 + + total_values = generate_random_files( + started_cluster, files_path, files_num, row_num=row_num + ) + create_table( + started_cluster, + node, + table_name, + mode, + files_path, + additional_settings={"after_processing": "delete"}, + ) + create_mv(node, table_name, dst_table_name) + expected_count = files_num * row_num for _ in range(100): count = int(node.query(f"SELECT count() FROM {dst_table_name}")) @@ -239,9 +277,13 @@ def test_delete_after_processing(started_cluster, mode): @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_failed_retry(started_cluster, mode): - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + node = started_cluster.instances["instance"] + table_name = f"test.failed_retry_{mode}" + dst_table_name = f"{table_name}_dst" + files_path = f"{table_name}_data" + file_path = f"{files_path}/trash_test.csv" + keeper_path = f"/clickhouse/test_{table_name}" + retries_num = 3 values = [ ["failed", 1, 1], @@ -249,54 +291,55 @@ def test_failed_retry(started_cluster, mode): values_csv = ( "\n".join((",".join(map(str, row)) for row in values)) + "\n" ).encode() - filename = f"test.csv" - put_s3_file_content(started_cluster, bucket, filename, values_csv) + put_s3_file_content(started_cluster, file_path, values_csv) - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue; - CREATE TABLE test.s3_queue ({table_format}) - ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/select_failed_retry_{mode}', - s3queue_loading_retries = 3; - """ + create_table( + started_cluster, + node, + table_name, + mode, + files_path, + additional_settings={ + "s3queue_loading_retries": retries_num, + "keeper_path": keeper_path, + }, ) + create_mv(node, table_name, dst_table_name) - # first try - get_query = f"SELECT * FROM test.s3_queue" - assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == [] - # second try - assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == [] - # upload correct file - values = [ - [1, 1, 1], - ] - values_csv = ( - "\n".join((",".join(map(str, row)) for row in values)) + "\n" - ).encode() - put_s3_file_content(started_cluster, bucket, filename, values_csv) + failed_node_path = "" + for _ in range(20): + zk = started_cluster.get_kazoo_client("zoo1") + failed_nodes = zk.get_children(f"{keeper_path}/failed/") + if len(failed_nodes) > 0: + assert len(failed_nodes) == 1 + failed_node_path = f"{keeper_path}/failed/{failed_nodes[0]}" + time.sleep(1) - assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == values + assert failed_node_path != "" - assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == [] + retries = 0 + for _ in range(20): + data, stat = zk.get(failed_node_path) + json_data = json.loads(data) + print(f"Failed node metadata: {json_data}") + assert json_data["file_path"] == file_path + retries = int(json_data["retries"]) + if retries == retries_num: + break + time.sleep(1) + + assert retries == retries_num + assert 0 == int(node.query(f"SELECT count() FROM {dst_table_name}")) @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_direct_select_file(started_cluster, mode): - auth = "'minio','minio123'," - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + node = started_cluster.instances["instance"] + table_name = f"test.direct_select_file_{mode}" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" + file_path = f"{files_path}/test.csv" + values = [ [12549, 2463, 19893], [64021, 38652, 66703], @@ -305,63 +348,68 @@ def test_direct_select_file(started_cluster, mode): values_csv = ( "\n".join((",".join(map(str, row)) for row in values)) + "\n" ).encode() - filename = f"test.csv" - put_s3_file_content(started_cluster, bucket, filename, values_csv) - instance.query( - """ - DROP TABLE IF EXISTS test.s3_queue; - DROP TABLE IF EXISTS test.s3_queue_2; - DROP TABLE IF EXISTS test.s3_queue_3; - """ - ) + put_s3_file_content(started_cluster, file_path, values_csv) - instance.query( - f""" - CREATE TABLE test.s3_queue ({table_format}) - ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/select_{mode}' - """ - ) + for i in range(3): + create_table( + started_cluster, + node, + f"{table_name}_{i + 1}", + mode, + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, + ) - get_query = f"SELECT * FROM test.s3_queue" assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + list(map(int, l.split())) + for l in node.query(f"SELECT * FROM {table_name}_1").splitlines() ] == values - instance.query( - f""" - CREATE TABLE test.s3_queue_2 ({table_format}) - ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/select_{mode}' - """ + assert [ + list(map(int, l.split())) + for l in node.query(f"SELECT * FROM {table_name}_2").splitlines() + ] == [] + + assert [ + list(map(int, l.split())) + for l in node.query(f"SELECT * FROM {table_name}_3").splitlines() + ] == [] + + # New table with same zookeeper path + create_table( + started_cluster, + node, + f"{table_name}_4", + mode, + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, ) - get_query = f"SELECT * FROM test.s3_queue" assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == [] - # New table with same zookeeper path - get_query = f"SELECT * FROM test.s3_queue_2" - assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + list(map(int, l.split())) + for l in node.query(f"SELECT * FROM {table_name}_4").splitlines() ] == [] + # New table with different zookeeper path - instance.query( - f""" - CREATE TABLE test.s3_queue_3 ({table_format}) - ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') - SETTINGS - mode = '{mode}', - keeper_path='/clickhouse/select_{mode}_2' - """ + keeper_path = f"/clickhouse/test_{table_name}_{mode}_2" + create_table( + started_cluster, + node, + f"{table_name}_4", + mode, + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, ) - get_query = f"SELECT * FROM test.s3_queue_3" + assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + list(map(int, l.split())) + for l in node.query(f"SELECT * FROM {table_name}_4").splitlines() ] == values values = [ @@ -370,189 +418,116 @@ def test_direct_select_file(started_cluster, mode): values_csv = ( "\n".join((",".join(map(str, row)) for row in values)) + "\n" ).encode() - filename = f"t.csv" - put_s3_file_content(started_cluster, bucket, filename, values_csv) + file_path = f"{files_path}/t.csv" + put_s3_file_content(started_cluster, file_path, values_csv) - get_query = f"SELECT * FROM test.s3_queue_3" if mode == "unordered": assert [ list(map(int, l.split())) - for l in run_query(instance, get_query).splitlines() + for l in node.query(f"SELECT * FROM {table_name}_4").splitlines() ] == values elif mode == "ordered": assert [ list(map(int, l.split())) - for l in run_query(instance, get_query).splitlines() + for l in node.query(f"SELECT * FROM {table_name}_4").splitlines() ] == [] @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_direct_select_multiple_files(started_cluster, mode): - prefix = f"multiple_files_{mode}" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - instance.query("drop table if exists test.s3_queue") - instance.query( - f""" - CREATE TABLE test.s3_queue ({table_format}) - ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/select_multiple_{mode}' - """ - ) + node = started_cluster.instances["instance"] + table_name = f"direct_select_multiple_files_{mode}" + files_path = f"{table_name}_data" + create_table(started_cluster, node, table_name, mode, files_path) for i in range(5): rand_values = [[random.randint(0, 50) for _ in range(3)] for _ in range(10)] - values_csv = ( "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" ).encode() - filename = f"{prefix}/test_{i}.csv" - put_s3_file_content(started_cluster, bucket, filename, values_csv) - get_query = f"SELECT * FROM test.s3_queue" + file_path = f"{files_path}/test_{i}.csv" + put_s3_file_content(started_cluster, file_path, values_csv) + assert [ list(map(int, l.split())) - for l in run_query(instance, get_query).splitlines() + for l in node.query(f"SELECT * FROM {table_name}").splitlines() ] == rand_values - total_values = generate_random_files( - 4, prefix, started_cluster, bucket, start_ind=5 - ) - get_query = f"SELECT * FROM test.s3_queue" + total_values = generate_random_files(started_cluster, files_path, 4, start_ind=5) assert { - tuple(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + tuple(map(int, l.split())) + for l in node.query(f"SELECT * FROM {table_name}").splitlines() } == set([tuple(i) for i in total_values]) @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_streaming_to_view_(started_cluster, mode): - prefix = f"streaming_files_{mode}" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + node = started_cluster.instances["instance"] + table_name = f"streaming_to_view_{mode}" + dst_table_name = f"{table_name}_dst" + files_path = f"{table_name}_data" - total_values = generate_random_files(10, prefix, started_cluster, bucket) - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue_persistent; - DROP TABLE IF EXISTS test.s3_queue; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv; + total_values = generate_random_files(started_cluster, files_path, 10) + create_table(started_cluster, node, table_name, mode, files_path) + create_mv(node, table_name, dst_table_name) - CREATE TABLE test.s3_queue_persistent ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/view_{mode}'; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS - SELECT - * - FROM test.s3_queue; - """ - ) expected_values = set([tuple(i) for i in total_values]) for i in range(10): - get_query = f"SELECT * FROM test.persistent_s3_queue_mv" - selected_values = { tuple(map(int, l.split())) - for l in run_query(instance, get_query).splitlines() + for l in node.query( + f"SELECT column1, column2, column3 FROM {dst_table_name}" + ).splitlines() } - if selected_values != expected_values: - time.sleep(1) - else: + if selected_values == expected_values: break - + time.sleep(1) assert selected_values == expected_values @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_streaming_to_many_views(started_cluster, mode): - prefix = f"streaming_files_{mode}" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - retry_cnt = 10 + node = started_cluster.instances["instance"] + table_name = f"streaming_to_many_views_{mode}" + dst_table_name = f"{table_name}_dst" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue_persistent; - DROP TABLE IF EXISTS test.s3_queue_persistent_2; - DROP TABLE IF EXISTS test.s3_queue_persistent_3; - DROP TABLE IF EXISTS test.s3_queue; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv_2; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv_3; + for i in range(3): + table = f"{table_name}_{i + 1}" + create_table( + started_cluster, + node, + table, + mode, + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, + ) + create_mv(node, table, dst_table_name) - - CREATE TABLE test.s3_queue_persistent ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE TABLE test.s3_queue_persistent_2 ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE TABLE test.s3_queue_persistent_3 ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/multiple_view_{mode}'; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS - SELECT - * - FROM test.s3_queue; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_2 TO test.s3_queue_persistent_2 AS - SELECT - * - FROM test.s3_queue; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_3 TO test.s3_queue_persistent_3 AS - SELECT - * - FROM test.s3_queue; - """ - ) - total_values = generate_random_files(5, prefix, started_cluster, bucket) + total_values = generate_random_files(started_cluster, files_path, 5) expected_values = set([tuple(i) for i in total_values]) - for i in range(retry_cnt): - retry = False - for get_query in [ - f"SELECT * FROM test.s3_queue_persistent", - f"SELECT * FROM test.s3_queue_persistent_2", - f"SELECT * FROM test.s3_queue_persistent_3", - ]: - selected_values = { - tuple(map(int, l.split())) - for l in run_query(instance, get_query).splitlines() - } - if i == retry_cnt - 1: - assert selected_values == expected_values - if selected_values != expected_values: - retry = True - break - if retry: - time.sleep(1) - else: + def select(): + return { + tuple(map(int, l.split())) + for l in node.query( + f"SELECT column1, column2, column3 FROM {dst_table_name}" + ).splitlines() + } + + for _ in range(20): + if select() == expected_values: break + time.sleep(1) + assert select() == expected_values def test_multiple_tables_meta_mismatch(started_cluster): - prefix = f"test_meta" + files_path = f"test_meta" bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -562,7 +537,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): DROP TABLE IF EXISTS test.s3_queue; CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = 'ordered', keeper_path = '/clickhouse/test_meta'; @@ -574,7 +549,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): instance.query( f""" CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = 'unordered', keeper_path = '/clickhouse/test_meta'; @@ -591,7 +566,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): instance.query( f""" CREATE TABLE test.s3_queue_copy ({table_format_copy}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = 'ordered', keeper_path = '/clickhouse/test_meta'; @@ -611,7 +586,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): instance.query( f""" CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'TSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'TSV') SETTINGS mode = 'ordered', keeper_path = '/clickhouse/test_meta'; @@ -626,7 +601,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): instance.query( f""" CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = 'ordered', keeper_path = '/clickhouse/test_meta'; @@ -637,7 +612,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): def test_max_set_age(started_cluster): files_to_generate = 10 max_age = 1 - prefix = f"test_multiple" + files_path = f"test_multiple" bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -647,7 +622,7 @@ def test_max_set_age(started_cluster): DROP TABLE IF EXISTS test.s3_queue; CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = 'unordered', keeper_path = '/clickhouse/test_set_age', @@ -657,7 +632,7 @@ def test_max_set_age(started_cluster): ) total_values = generate_random_files( - files_to_generate, prefix, started_cluster, bucket, row_num=1 + files_to_generate, files_path, started_cluster, bucket, row_num=1 ) get_query = f"SELECT * FROM test.s3_queue" res1 = [ @@ -677,7 +652,7 @@ def test_max_set_age(started_cluster): def test_multiple_tables_streaming_sync(started_cluster, mode): files_to_generate = 300 poll_size = 30 - prefix = f"test_multiple_{mode}" + files_path = f"test_multiple_{mode}" bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -697,21 +672,21 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): DROP TABLE IF EXISTS test.persistent_s3_queue_mv_copy_2; CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', s3queue_polling_size = {poll_size}; CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', s3queue_polling_size = {poll_size}; CREATE TABLE test.s3_queue_copy_2 ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', @@ -746,7 +721,7 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): """ ) total_values = generate_random_files( - files_to_generate, prefix, started_cluster, bucket, row_num=1 + files_to_generate, files_path, started_cluster, bucket, row_num=1 ) def get_count(table_name): @@ -792,7 +767,7 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): files_to_generate = 100 poll_size = 2 - prefix = f"test_multiple_{mode}" + files_path = f"test_multiple_{mode}" bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] instance_2 = started_cluster.instances["instance2"] @@ -806,7 +781,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): DROP TABLE IF EXISTS test.persistent_s3_queue_mv; CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', keeper_path = '/clickhouse/test_multiple_consumers_{mode}', @@ -829,7 +804,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): ) total_values = generate_random_files( - files_to_generate, prefix, started_cluster, bucket, row_num=1 + files_to_generate, files_path, started_cluster, bucket, row_num=1 ) def get_count(node, table_name): @@ -869,7 +844,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): def test_max_set_size(started_cluster): files_to_generate = 10 - prefix = f"test_multiple" + files_path = f"test_multiple" bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -879,7 +854,7 @@ def test_max_set_size(started_cluster): DROP TABLE IF EXISTS test.s3_queue; CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = 'unordered', keeper_path = '/clickhouse/test_set_size', @@ -888,7 +863,7 @@ def test_max_set_size(started_cluster): ) total_values = generate_random_files( - files_to_generate, prefix, started_cluster, bucket, start_ind=0, row_num=1 + files_to_generate, files_path, started_cluster, bucket, start_ind=0, row_num=1 ) get_query = f"SELECT * FROM test.s3_queue" res1 = [ From 342755d35e9ecef55df306007f6f5b95b8d2b8db Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 14 Sep 2023 18:41:31 +0200 Subject: [PATCH 033/634] Better --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 173 +++++++- src/Storages/S3Queue/S3QueueFilesMetadata.h | 13 +- src/Storages/S3Queue/S3QueueSource.cpp | 3 + src/Storages/S3Queue/StorageS3Queue.cpp | 13 +- src/Storages/S3Queue/StorageS3Queue.h | 2 - .../integration/test_storage_s3_queue/test.py | 371 ++++++++---------- 6 files changed, 357 insertions(+), 218 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 99c0924968c..d4c2c116a47 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -1,8 +1,13 @@ +#include "Common/Exception.h" +#include "Common/ZooKeeper/Types.h" +#include "Interpreters/Context_fwd.h" +#include "Storages/S3Queue/S3QueueSettings.h" #include "config.h" #if USE_AWS_S3 #include #include +#include #include #include #include @@ -28,11 +33,22 @@ namespace { return std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); } + + size_t generateRescheduleInterval() + { + /// Use more or less random interval for unordered mode cleanup task. + /// So that distributed processing cleanup tasks would not schedule cleanup at the same time. + /// TODO: make lower and upper boundary configurable by settings + pcg64 rng(randomSeed()); + //return 5000 + rng() % 30000; + return rng() % 100; + } } S3QueueFilesMetadata::S3QueueFilesMetadata( const StorageS3Queue * storage_, - const S3QueueSettings & settings_) + const S3QueueSettings & settings_, + ContextPtr context) : storage(storage_) , mode(settings_.mode) , max_set_size(settings_.s3queue_tracked_files_limit.value) @@ -43,6 +59,27 @@ S3QueueFilesMetadata::S3QueueFilesMetadata( , zookeeper_failed_path(storage->getZooKeeperPath() / "failed") , log(&Poco::Logger::get("S3QueueFilesMetadata")) { + if (mode == S3QueueMode::UNORDERED && (max_set_size || max_set_age_sec)) + { + task = context->getSchedulePool().createTask("S3QueueCleanupFunc", [this] { cleanupThreadFunc(); }); + task->activate(); + + auto schedule_ms = generateRescheduleInterval(); + LOG_TEST(log, "Scheduling a cleanup task in {} ms", schedule_ms); + task->scheduleAfter(schedule_ms); + } +} + +S3QueueFilesMetadata::~S3QueueFilesMetadata() +{ + deactivateCleanupTask(); +} + +void S3QueueFilesMetadata::deactivateCleanupTask() +{ + shutdown = true; + if (task) + task->deactivate(); } std::string S3QueueFilesMetadata::NodeMetadata::toString() const @@ -109,6 +146,10 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) { + /// Create an ephemenral node in /processing + /// if corresponding node does not exist in failed/, processed/ and processing/. + /// Return false otherwise. + const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = storage->getZooKeeper(); @@ -125,6 +166,10 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::str bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) { + /// Create an ephemenral node in /processing + /// if corresponding it does not exist in failed/, processing/ and satisfied max processed file check. + /// Return false otherwise. + const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = storage->getZooKeeper(); @@ -195,8 +240,7 @@ void S3QueueFilesMetadata::setFileProcessed(const String & path) void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(const String & path) { - /// List results in s3 are always returned in UTF-8 binary order. - /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) + /// Create a persistent node in /processed and remove ephemeral node from /processing. const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); @@ -337,6 +381,129 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc } } +void S3QueueFilesMetadata::cleanupThreadFunc() +{ + /// A background task is responsible for maintaining + /// max_set_size and max_set_age settings for `unordered` processing mode. + + if (shutdown) + return; + + try + { + cleanupThreadFuncImpl(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + if (shutdown) + return; + + task->scheduleAfter(generateRescheduleInterval()); +} + +void S3QueueFilesMetadata::cleanupThreadFuncImpl() +{ + chassert(max_set_size || max_set_age_sec); + + const bool check_nodes_limit = max_set_size > 0; + const bool check_nodes_ttl = max_set_age_sec > 0; + + const auto zk_client = storage->getZooKeeper(); + auto nodes = zk_client->getChildren(zookeeper_processed_path); + if (nodes.empty()) + { + LOG_TEST(log, "A set of nodes is empty"); + return; + } + + const bool nodes_limit_exceeded = nodes.size() > max_set_size; + if (!nodes_limit_exceeded && check_nodes_limit && !check_nodes_ttl) + { + LOG_TEST(log, "No limit exceeded"); + return; + } + + struct Node + { + std::string name; + NodeMetadata metadata; + }; + auto node_cmp = [](const Node & a, const Node & b) + { + return a.metadata.last_processed_timestamp < b.metadata.last_processed_timestamp; + }; + + /// Ordered in ascending order of timestamps. + std::set sorted_nodes(node_cmp); + + for (const auto & node : nodes) + { + try + { + std::string metadata_str; + if (zk_client->tryGet(zookeeper_processed_path / node, metadata_str)) + { + bool inserted = sorted_nodes.emplace(node, NodeMetadata::fromString(metadata_str)).second; + chassert(inserted); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + /// TODO add a zookeeper lock for cleanup + + LOG_TRACE(log, "Checking node limits"); + + size_t nodes_to_remove = check_nodes_limit && nodes_limit_exceeded ? nodes.size() - max_set_size : 0; + for (const auto & node : sorted_nodes) + { + if (nodes_to_remove) + { + auto path = zookeeper_processed_path / node.name; + LOG_TEST(log, "Removing node at path `{}` because max files limit is reached", path.string()); + + auto code = zk_client->tryRemove(path); + if (code == Coordination::Error::ZOK) + --nodes_to_remove; + else + LOG_ERROR(log, "Failed to remove a node `{}`", path.string()); + } + else if (check_nodes_ttl) + { + UInt64 node_age = getCurrentTime() - node.metadata.last_processed_timestamp; + if (node_age >= max_set_age_sec) + { + auto path = zookeeper_processed_path / node.name; + LOG_TEST(log, "Removing node at path `{}` because file ttl is reached", path.string()); + + auto code = zk_client->tryRemove(path); + if (code != Coordination::Error::ZOK) + LOG_ERROR(log, "Failed to remove a node `{}`", path.string()); + } + else if (!nodes_to_remove) + { + /// Nodes limit satisfied. + /// Nodes ttl satisfied as well as if current node is under tll, then all remaining as well + /// (because we are iterating in timestamp ascending order). + break; + } + } + else + { + /// Nodes limit and ttl are satisfied. + break; + } + } + + LOG_TRACE(log, "Node limits check finished"); +} + } #endif diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index 302feab6028..b8e172bcd88 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace fs = std::filesystem; namespace Poco { class Logger; } @@ -17,7 +18,9 @@ class StorageS3Queue; class S3QueueFilesMetadata { public: - S3QueueFilesMetadata(const StorageS3Queue * storage_, const S3QueueSettings & settings_); + S3QueueFilesMetadata(const StorageS3Queue * storage_, const S3QueueSettings & settings_, ContextPtr context); + + ~S3QueueFilesMetadata(); bool trySetFileAsProcessing(const std::string & path); @@ -25,6 +28,8 @@ public: void setFileFailed(const std::string & path, const std::string & exception_message); + void deactivateCleanupTask(); + private: const StorageS3Queue * storage; const S3QueueMode mode; @@ -39,6 +44,9 @@ private: mutable std::mutex mutex; Poco::Logger * log; + std::atomic_bool shutdown = false; + BackgroundSchedulePool::TaskHolder task; + bool trySetFileAsProcessingForOrderedMode(const std::string & path); bool trySetFileAsProcessingForUnorderedMode(const std::string & path); @@ -59,6 +67,9 @@ private: }; NodeMetadata createNodeMetadata(const std::string & path, const std::string & exception = "", size_t retries = 0); + + void cleanupThreadFunc(); + void cleanupThreadFuncImpl(); }; } diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 6704345ea59..3a834dae8d0 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -37,6 +37,9 @@ StorageS3QueueSource::FileIterator::FileIterator( StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::FileIterator::next() { + /// List results in s3 are always returned in UTF-8 binary order. + /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) + while (true) { KeyWithInfo val = glob_iterator->next(); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 53a79aa9cff..bbaf5ae5311 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -88,7 +88,7 @@ StorageS3Queue::StorageS3Queue( , s3queue_settings(std::move(s3queue_settings_)) , zk_path(chooseZooKeeperPath(table_id_, context_->getSettingsRef(), *s3queue_settings)) , after_processing(s3queue_settings->after_processing) - , files_metadata(std::make_shared(this, *s3queue_settings)) + , files_metadata(std::make_shared(this, *s3queue_settings, context_)) , configuration{configuration_} , format_settings(format_settings_) , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) @@ -138,8 +138,17 @@ void StorageS3Queue::startup() void StorageS3Queue::shutdown() { shutdown_called = true; + if (task) + { task->deactivate(); + } + + if (files_metadata) + { + files_metadata->deactivateCleanupTask(); + files_metadata.reset(); + } } bool StorageS3Queue::supportsSubsetOfColumns(const ContextPtr & context_) const @@ -182,7 +191,7 @@ std::shared_ptr StorageS3Queue::createSource( { auto configuration_snapshot = updateConfigurationAndGetCopy(local_context); auto file_iterator = createFileIterator(local_context, query); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); auto internal_source = std::make_unique( read_from_format_info, configuration.format, getName(), local_context, format_settings, diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 5cdac607645..07f52c434de 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -1,9 +1,7 @@ #pragma once - #include "config.h" #if USE_AWS_S3 - #include #include #include diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 9793f2b7191..dd59138d935 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -191,6 +191,7 @@ def create_table( files_path, format="column1 UInt32, column2 UInt32, column3 UInt32", additional_settings={}, + file_format="CSV", ): settings = { "s3queue_loading_retries": 0, @@ -204,7 +205,7 @@ def create_table( node.query(f"DROP TABLE IF EXISTS {table_name}") create_query = f""" CREATE TABLE {table_name} ({format}) - ENGINE = S3Queue('{url}', {AUTH}'CSV') + ENGINE = S3Queue('{url}', {AUTH}'{file_format}') SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))} """ node.query(create_query) @@ -527,50 +528,52 @@ def test_streaming_to_many_views(started_cluster, mode): def test_multiple_tables_meta_mismatch(started_cluster): - files_path = f"test_meta" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + node = started_cluster.instances["instance"] + table_name = f"multiple_tables_meta_mismatch" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = 'ordered', - keeper_path = '/clickhouse/test_meta'; - """ + create_table( + started_cluster, + node, + table_name, + "ordered", + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, ) # check mode failed = False try: - instance.query( - f""" - CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = 'unordered', - keeper_path = '/clickhouse/test_meta'; - """ + create_table( + started_cluster, + node, + f"{table_name}_copy", + "unordered", + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, ) except QueryRuntimeException as e: assert "Existing table metadata in ZooKeeper differs in engine mode" in str(e) failed = True + assert failed is True # check columns - table_format_copy = table_format + ", column4 UInt32" try: - instance.query( - f""" - CREATE TABLE test.s3_queue_copy ({table_format_copy}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = 'ordered', - keeper_path = '/clickhouse/test_meta'; - """ + create_table( + started_cluster, + node, + f"{table_name}_copy", + "ordered", + files_path, + format="column1 UInt32, column2 UInt32, column3 UInt32, column4 UInt32", + additional_settings={ + "keeper_path": keeper_path, + }, ) except QueryRuntimeException as e: assert ( @@ -583,172 +586,96 @@ def test_multiple_tables_meta_mismatch(started_cluster): # check format try: - instance.query( - f""" - CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'TSV') - SETTINGS - mode = 'ordered', - keeper_path = '/clickhouse/test_meta'; - """ + create_table( + started_cluster, + node, + f"{table_name}_copy", + "ordered", + files_path, + format="column1 UInt32, column2 UInt32, column3 UInt32, column4 UInt32", + additional_settings={ + "keeper_path": keeper_path, + }, + file_format="TSV", ) except QueryRuntimeException as e: assert "Existing table metadata in ZooKeeper differs in format name" in str(e) failed = True + assert failed is True # create working engine - instance.query( - f""" - CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = 'ordered', - keeper_path = '/clickhouse/test_meta'; - """ + create_table( + started_cluster, + node, + f"{table_name}_copy", + "ordered", + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, ) -def test_max_set_age(started_cluster): - files_to_generate = 10 - max_age = 1 - files_path = f"test_multiple" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = 'unordered', - keeper_path = '/clickhouse/test_set_age', - s3queue_tracked_files_limit = 10, - s3queue_tracked_file_ttl_sec = {max_age}; - """ - ) - - total_values = generate_random_files( - files_to_generate, files_path, started_cluster, bucket, row_num=1 - ) - get_query = f"SELECT * FROM test.s3_queue" - res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] - assert res1 == total_values - time.sleep(max_age + 1) - - get_query = f"SELECT * FROM test.s3_queue" - res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] - assert res1 == total_values - - @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_multiple_tables_streaming_sync(started_cluster, mode): + node = started_cluster.instances["instance"] + table_name = f"multiple_tables_streaming_sync_{mode}" + dst_table_name = f"{table_name}_dst" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" files_to_generate = 300 poll_size = 30 - files_path = f"test_multiple_{mode}" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue; - DROP TABLE IF EXISTS test.s3_queue_copy; - DROP TABLE IF EXISTS test.s3_queue_copy_2; + for i in range(3): + table = f"{table_name}_{i + 1}" + dst_table = f"{dst_table_name}_{i + 1}" + create_table( + started_cluster, + node, + table, + mode, + files_path, + additional_settings={ + "s3queue_polling_size": poll_size, + "keeper_path": keeper_path, + }, + ) + create_mv(node, table, dst_table) - DROP TABLE IF EXISTS test.s3_queue_persistent; - DROP TABLE IF EXISTS test.s3_queue_persistent_copy; - DROP TABLE IF EXISTS test.s3_queue_persistent_copy_2; - - DROP TABLE IF EXISTS test.persistent_s3_queue_mv; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv_copy; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv_copy_2; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', - s3queue_polling_size = {poll_size}; - - CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', - s3queue_polling_size = {poll_size}; - - CREATE TABLE test.s3_queue_copy_2 ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', - s3queue_polling_size = {poll_size}; - - CREATE TABLE test.s3_queue_persistent ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE TABLE test.s3_queue_persistent_copy ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE TABLE test.s3_queue_persistent_copy_2 ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS - SELECT - * - FROM test.s3_queue; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_copy TO test.s3_queue_persistent_copy AS - SELECT - * - FROM test.s3_queue_copy; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_copy_2 TO test.s3_queue_persistent_copy_2 AS - SELECT - * - FROM test.s3_queue_copy_2; - """ - ) total_values = generate_random_files( - files_to_generate, files_path, started_cluster, bucket, row_num=1 + started_cluster, files_path, files_to_generate, row_num=1 ) def get_count(table_name): - return int(run_query(instance, f"SELECT count() FROM {table_name}")) + return int(run_query(node, f"SELECT count() FROM {table_name}")) for _ in range(100): if ( - get_count("test.s3_queue_persistent") - + get_count("test.s3_queue_persistent_copy") - + get_count("test.s3_queue_persistent_copy_2") + get_count(f"{dst_table_name}_1") + + get_count(f"{dst_table_name}_2") + + get_count(f"{dst_table_name}_3") ) == files_to_generate: break time.sleep(1) - get_query = f"SELECT * FROM test.s3_queue_persistent" res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + list(map(int, l.split())) + for l in node.query( + f"SELECT column1, column2, column3 FROM {dst_table_name}_1" + ).splitlines() ] - get_query_copy = f"SELECT * FROM test.s3_queue_persistent_copy" res2 = [ list(map(int, l.split())) - for l in run_query(instance, get_query_copy).splitlines() + for l in node.query( + f"SELECT column1, column2, column3 FROM {dst_table_name}_2" + ).splitlines() ] - get_query_copy_2 = f"SELECT * FROM test.s3_queue_persistent_copy_2" res3 = [ list(map(int, l.split())) - for l in run_query(instance, get_query_copy_2).splitlines() + for l in node.query( + f"SELECT column1, column2, column3 FROM {dst_table_name}_3" + ).splitlines() ] assert {tuple(v) for v in res1 + res2 + res3} == set( [tuple(i) for i in total_values] @@ -757,54 +684,41 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): # Checking that all files were processed only once time.sleep(10) assert ( - get_count("test.s3_queue_persistent") - + get_count("test.s3_queue_persistent_copy") - + get_count("test.s3_queue_persistent_copy_2") + get_count(f"{dst_table_name}_1") + + get_count(f"{dst_table_name}_2") + + get_count(f"{dst_table_name}_3") ) == files_to_generate @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): - files_to_generate = 100 + node = started_cluster.instances["instance"] + node_2 = started_cluster.instances["instance2"] + table_name = f"multiple_tables_streaming_sync_distributed_{mode}" + dst_table_name = f"{table_name}_dst" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" + files_to_generate = 300 poll_size = 2 - files_path = f"test_multiple_{mode}" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - instance_2 = started_cluster.instances["instance2"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - for inst in [instance, instance_2]: - inst.query( - f""" - DROP TABLE IF EXISTS test.s3_queue; - DROP TABLE IF EXISTS test.s3_queue_persistent; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/test_multiple_consumers_{mode}', - s3queue_polling_size = {poll_size}; - - CREATE TABLE test.s3_queue_persistent ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - """ + for instance in [node, node_2]: + create_table( + started_cluster, + instance, + table_name, + mode, + files_path, + additional_settings={ + "s3queue_polling_size": poll_size, + "keeper_path": keeper_path, + }, ) - for inst in [instance, instance_2]: - inst.query( - f""" - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS - SELECT - * - FROM test.s3_queue; - """ - ) + for instance in [node, node_2]: + create_mv(instance, table_name, dst_table_name) total_values = generate_random_files( - files_to_generate, files_path, started_cluster, bucket, row_num=1 + started_cluster, files_path, files_to_generate, row_num=1 ) def get_count(node, table_name): @@ -812,18 +726,15 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): for _ in range(150): if ( - get_count(instance, "test.s3_queue_persistent") - + get_count(instance_2, "test.s3_queue_persistent") + get_count(node, dst_table_name) + get_count(node_2, dst_table_name) ) == files_to_generate: break time.sleep(1) - get_query = f"SELECT * FROM test.s3_queue_persistent" - res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] + get_query = f"SELECT column1, column2, column3 FROM {dst_table_name}" + res1 = [list(map(int, l.split())) for l in run_query(node, get_query).splitlines()] res2 = [ - list(map(int, l.split())) for l in run_query(instance_2, get_query).splitlines() + list(map(int, l.split())) for l in run_query(node_2, get_query).splitlines() ] assert len(res1) + len(res2) == files_to_generate @@ -837,11 +748,51 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): # Checking that all files were processed only once time.sleep(10) assert ( - get_count(instance, "test.s3_queue_persistent") - + get_count(instance_2, "test.s3_queue_persistent") + get_count(node, dst_table_name) + get_count(node_2, dst_table_name) ) == files_to_generate +def test_max_set_age(started_cluster): + node = started_cluster.instances["instance"] + table_name = f"max_set_age" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" + max_age = 1 + files_to_generate = 10 + + create_table( + started_cluster, + node, + table_name, + "unordered", + files_path, + additional_settings={ + "keeper_path": keeper_path, + "s3queue_tracked_files_limit": 10, + "s3queue_tracked_file_ttl_sec": max_age, + }, + ) + + node.wait_for_log_line("Checking node limits") + node.wait_for_log_line("Node limits check finished") + + total_values = generate_random_files( + started_cluster, files_path, files_to_generate, row_num=1 + ) + res1 = [ + list(map(int, l.split())) + for l in run_query(node, f"SELECT * FROM {table_name}").splitlines() + ] + assert res1 == total_values + time.sleep(max_age + 1) + + res1 = [ + list(map(int, l.split())) + for l in run_query(node, f"SELECT * FROM {table_name}").splitlines() + ] + assert res1 == total_values + + def test_max_set_size(started_cluster): files_to_generate = 10 files_path = f"test_multiple" From 434e2d4b57d8fb428f4764f857885aca50bc70be Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Sep 2023 18:57:41 +0000 Subject: [PATCH 034/634] Add blind test. --- .../test_merge_session_expired/__init__.py | 0 .../configs/keeper_config.xml | 5 +++ .../test_merge_session_expired/test.py | 41 +++++++++++++++++++ 3 files changed, 46 insertions(+) create mode 100644 tests/integration/test_merge_session_expired/__init__.py create mode 100644 tests/integration/test_merge_session_expired/configs/keeper_config.xml create mode 100644 tests/integration/test_merge_session_expired/test.py diff --git a/tests/integration/test_merge_session_expired/__init__.py b/tests/integration/test_merge_session_expired/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_merge_session_expired/configs/keeper_config.xml b/tests/integration/test_merge_session_expired/configs/keeper_config.xml new file mode 100644 index 00000000000..4258475f7b5 --- /dev/null +++ b/tests/integration/test_merge_session_expired/configs/keeper_config.xml @@ -0,0 +1,5 @@ + + + 3000 + + diff --git a/tests/integration/test_merge_session_expired/test.py b/tests/integration/test_merge_session_expired/test.py new file mode 100644 index 00000000000..90a8f940e6b --- /dev/null +++ b/tests/integration/test_merge_session_expired/test.py @@ -0,0 +1,41 @@ +import logging +import pytest +import time +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager +from helpers.client import QueryRuntimeException + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", main_configs=["configs/keeper_config.xml"], user_configs=["configs/timeouts.xml"], stay_alive=True, with_zookeeper=True +) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_merge_session_expired(started_cluster): + node1.query("create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple()") + node1.query("insert into tab select number from numbers(10)") + node1.query("alter table tab delete where x = 2 settings mutations_sync=2") + node1.query("alter table tab delete where x = 4 settings mutations_sync=2") + node1.query("alter table tab delete where x = 6 settings mutations_sync=2") + node1.query("system stop merges") + node1.query("insert into tab select number + 10 from numbers(10)") + node1.query("optimize table tab final settings alter_sync=0") + + with PartitionManager() as pm: + #logging.info(pm.dump_rules()) + pm.drop_instance_zk_connections(node1) + node1.query("system start merges") + node1.query("system sync replica tab") + node1.restart_clickhouse() + + node1.query("system sync replica tab") + assert node1.query("select count() from tab") == '17' From 0b661188e7d9e51e7e040ee065a559487539bb9a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Sep 2023 19:00:37 +0000 Subject: [PATCH 035/634] Add blind test. --- tests/integration/test_merge_session_expired/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_merge_session_expired/test.py b/tests/integration/test_merge_session_expired/test.py index 90a8f940e6b..e6208203584 100644 --- a/tests/integration/test_merge_session_expired/test.py +++ b/tests/integration/test_merge_session_expired/test.py @@ -34,7 +34,7 @@ def test_merge_session_expired(started_cluster): #logging.info(pm.dump_rules()) pm.drop_instance_zk_connections(node1) node1.query("system start merges") - node1.query("system sync replica tab") + node1.query("select sleep(1)") node1.restart_clickhouse() node1.query("system sync replica tab") From 466fee196bd1b8bc31046f8de9ee27f0ad32a655 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Sep 2023 11:41:42 +0000 Subject: [PATCH 036/634] Update test. --- .../test_merge_session_expired/test.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_merge_session_expired/test.py b/tests/integration/test_merge_session_expired/test.py index e6208203584..f2122016365 100644 --- a/tests/integration/test_merge_session_expired/test.py +++ b/tests/integration/test_merge_session_expired/test.py @@ -7,7 +7,7 @@ from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/keeper_config.xml"], user_configs=["configs/timeouts.xml"], stay_alive=True, with_zookeeper=True + "node1", main_configs=["configs/keeper_config.xml"], stay_alive=True, with_zookeeper=True ) @pytest.fixture(scope="module") @@ -21,13 +21,14 @@ def started_cluster(): def test_merge_session_expired(started_cluster): - node1.query("create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple()") + node1.query("create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple() settings old_parts_lifetime=3") node1.query("insert into tab select number from numbers(10)") - node1.query("alter table tab delete where x = 2 settings mutations_sync=2") - node1.query("alter table tab delete where x = 4 settings mutations_sync=2") - node1.query("alter table tab delete where x = 6 settings mutations_sync=2") - node1.query("system stop merges") node1.query("insert into tab select number + 10 from numbers(10)") + node1.query("alter table tab delete where x = 12 settings mutations_sync=2") + node1.query("alter table tab delete where x = 14 settings mutations_sync=2") + node1.query("alter table tab delete where x = 16 settings mutations_sync=2") + node1.query("system stop merges") + # node1.query("insert into tab select number + 20 from numbers(10)") node1.query("optimize table tab final settings alter_sync=0") with PartitionManager() as pm: @@ -36,6 +37,8 @@ def test_merge_session_expired(started_cluster): node1.query("system start merges") node1.query("select sleep(1)") node1.restart_clickhouse() + pm.restore_instance_zk_connections(node1) + node1.query("system restart replica tab") node1.query("system sync replica tab") - assert node1.query("select count() from tab") == '17' + assert node1.query("select count() from tab") == '17\n' From 6846fe3c589b0388a039e668e494350ecdff94ab Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 15 Sep 2023 14:21:08 +0200 Subject: [PATCH 037/634] Fxi --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 34 ++++-- .../integration/test_storage_s3_queue/test.py | 114 +++++++++++------- 2 files changed, 97 insertions(+), 51 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index d4c2c116a47..0e9a121339f 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -1,3 +1,4 @@ +#include #include "Common/Exception.h" #include "Common/ZooKeeper/Types.h" #include "Interpreters/Context_fwd.h" @@ -253,7 +254,10 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(const String & path) Coordination::Responses responses; auto code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) + { + LOG_TEST(log, "Moved file `{}` to processed", path); return; + } /// TODO this could be because of the expired session. if (responses[0]->error != Coordination::Error::ZOK) @@ -433,11 +437,16 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() }; auto node_cmp = [](const Node & a, const Node & b) { - return a.metadata.last_processed_timestamp < b.metadata.last_processed_timestamp; + if (a.metadata.last_processed_timestamp == b.metadata.last_processed_timestamp) + return a.metadata.file_path < b.metadata.file_path; + else + return a.metadata.last_processed_timestamp < b.metadata.last_processed_timestamp; }; /// Ordered in ascending order of timestamps. - std::set sorted_nodes(node_cmp); + std::multiset sorted_nodes(node_cmp); + + LOG_TRACE(log, "Found {} nodes", nodes.size()); for (const auto & node : nodes) { @@ -446,9 +455,11 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() std::string metadata_str; if (zk_client->tryGet(zookeeper_processed_path / node, metadata_str)) { - bool inserted = sorted_nodes.emplace(node, NodeMetadata::fromString(metadata_str)).second; - chassert(inserted); + sorted_nodes.emplace(node, NodeMetadata::fromString(metadata_str)); + LOG_TEST(log, "Fetched metadata for node {}", node); } + else + LOG_TEST(log, "Failed to fetch node metadata {}", node); } catch (...) { @@ -458,7 +469,14 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() /// TODO add a zookeeper lock for cleanup - LOG_TRACE(log, "Checking node limits"); + auto get_nodes_str = [&]() + { + WriteBufferFromOwnString wb; + for (const auto & [node, metadata] : sorted_nodes) + wb << fmt::format("Node: {}, path: {}, timestamp: {};\n", node, metadata.file_path, metadata.last_processed_timestamp); + return wb.str(); + }; + LOG_TEST(log, "Checking node limits (max size: {}, max age: {}) for {}", max_set_size, max_set_age_sec, get_nodes_str()); size_t nodes_to_remove = check_nodes_limit && nodes_limit_exceeded ? nodes.size() - max_set_size : 0; for (const auto & node : sorted_nodes) @@ -466,7 +484,8 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() if (nodes_to_remove) { auto path = zookeeper_processed_path / node.name; - LOG_TEST(log, "Removing node at path `{}` because max files limit is reached", path.string()); + LOG_TEST(log, "Removing node at path {} ({}) because max files limit is reached", + node.metadata.file_path, path.string()); auto code = zk_client->tryRemove(path); if (code == Coordination::Error::ZOK) @@ -480,7 +499,8 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() if (node_age >= max_set_age_sec) { auto path = zookeeper_processed_path / node.name; - LOG_TEST(log, "Removing node at path `{}` because file ttl is reached", path.string()); + LOG_TEST(log, "Removing node at path {} ({}) because file is reached", + node.metadata.file_path, path.string()); auto code = zk_client->tryRemove(path); if (code != Coordination::Error::ZOK) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index dd59138d935..d26afcc3aa3 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -159,6 +159,7 @@ def generate_random_files( values_csv = ( "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" ).encode() + print(f"File {filename}, content: {total_values}") put_s3_file_content(started_cluster, filename, values_csv) return total_values @@ -755,9 +756,10 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): def test_max_set_age(started_cluster): node = started_cluster.instances["instance"] table_name = f"max_set_age" + dst_table_name = f"{table_name}_dst" keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" - max_age = 1 + max_age = 10 files_to_generate = 10 create_table( @@ -768,68 +770,92 @@ def test_max_set_age(started_cluster): files_path, additional_settings={ "keeper_path": keeper_path, - "s3queue_tracked_files_limit": 10, "s3queue_tracked_file_ttl_sec": max_age, }, ) - - node.wait_for_log_line("Checking node limits") - node.wait_for_log_line("Node limits check finished") + create_mv(node, table_name, dst_table_name) total_values = generate_random_files( started_cluster, files_path, files_to_generate, row_num=1 ) - res1 = [ - list(map(int, l.split())) - for l in run_query(node, f"SELECT * FROM {table_name}").splitlines() - ] - assert res1 == total_values + + expected_rows = 10 + + node.wait_for_log_line("Checking node limits") + node.wait_for_log_line("Node limits check finished") + + def get_count(): + return int(node.query(f"SELECT count() FROM {dst_table_name}")) + + for _ in range(20): + if expected_rows == get_count(): + break + time.sleep(1) + + assert expected_rows == get_count() + assert 10 == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) + time.sleep(max_age + 1) - res1 = [ - list(map(int, l.split())) - for l in run_query(node, f"SELECT * FROM {table_name}").splitlines() + expected_rows = 20 + + for _ in range(20): + if expected_rows == get_count(): + break + time.sleep(1) + + assert expected_rows == get_count() + assert 10 == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) + + paths_count = [ + int(x) + for x in node.query( + f"SELECT count() from {dst_table_name} GROUP BY _path" + ).splitlines() ] - assert res1 == total_values + assert 10 == len(paths_count) + for path_count in paths_count: + assert 2 == path_count def test_max_set_size(started_cluster): + node = started_cluster.instances["instance"] + table_name = f"max_set_size" + dst_table_name = f"{table_name}_dst" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" + max_age = 10 files_to_generate = 10 - files_path = f"test_multiple" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = 'unordered', - keeper_path = '/clickhouse/test_set_size', - s3queue_tracked_files_limit = {files_to_generate - 1}; - """ + create_table( + started_cluster, + node, + table_name, + "unordered", + files_path, + additional_settings={ + "keeper_path": keeper_path, + "s3queue_tracked_files_limit": 9, + }, ) - total_values = generate_random_files( - files_to_generate, files_path, started_cluster, bucket, start_ind=0, row_num=1 + started_cluster, files_path, files_to_generate, start_ind=0, row_num=1 ) - get_query = f"SELECT * FROM test.s3_queue" - res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] - assert res1 == total_values - get_query = f"SELECT * FROM test.s3_queue" - res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] + get_query = f"SELECT * FROM {table_name}" + res1 = [list(map(int, l.split())) for l in run_query(node, get_query).splitlines()] + assert res1 == total_values + print(total_values) + + time.sleep(10) + + zk = started_cluster.get_kazoo_client("zoo1") + processed_nodes = zk.get_children(f"{keeper_path}/processed/") + assert len(processed_nodes) == 9 + + res1 = [list(map(int, l.split())) for l in run_query(node, get_query).splitlines()] assert res1 == [total_values[0]] - get_query = f"SELECT * FROM test.s3_queue" - res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] + time.sleep(10) + res1 = [list(map(int, l.split())) for l in run_query(node, get_query).splitlines()] assert res1 == [total_values[1]] From d4027d835e0d12afcd751ceb82d7636a3f5c2069 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Sep 2023 18:40:05 +0000 Subject: [PATCH 038/634] Add test with zero copy replication. --- .../configs/disks.xml | 21 ++++++++++ .../test_merge_session_expired/test.py | 41 +++++++++++++++++-- 2 files changed, 58 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_merge_session_expired/configs/disks.xml diff --git a/tests/integration/test_merge_session_expired/configs/disks.xml b/tests/integration/test_merge_session_expired/configs/disks.xml new file mode 100644 index 00000000000..94ac83b32ac --- /dev/null +++ b/tests/integration/test_merge_session_expired/configs/disks.xml @@ -0,0 +1,21 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + +
+ s3 +
+
+
+
+
+
diff --git a/tests/integration/test_merge_session_expired/test.py b/tests/integration/test_merge_session_expired/test.py index f2122016365..db8eb954d3a 100644 --- a/tests/integration/test_merge_session_expired/test.py +++ b/tests/integration/test_merge_session_expired/test.py @@ -7,7 +7,7 @@ from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/keeper_config.xml"], stay_alive=True, with_zookeeper=True + "node1", main_configs=["configs/keeper_config.xml", "configs/disks.xml"], stay_alive=True, with_zookeeper=True, with_minio=True ) @pytest.fixture(scope="module") @@ -21,6 +21,7 @@ def started_cluster(): def test_merge_session_expired(started_cluster): + node1.query("drop table if exists tab") node1.query("create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple() settings old_parts_lifetime=3") node1.query("insert into tab select number from numbers(10)") node1.query("insert into tab select number + 10 from numbers(10)") @@ -28,11 +29,9 @@ def test_merge_session_expired(started_cluster): node1.query("alter table tab delete where x = 14 settings mutations_sync=2") node1.query("alter table tab delete where x = 16 settings mutations_sync=2") node1.query("system stop merges") - # node1.query("insert into tab select number + 20 from numbers(10)") node1.query("optimize table tab final settings alter_sync=0") with PartitionManager() as pm: - #logging.info(pm.dump_rules()) pm.drop_instance_zk_connections(node1) node1.query("system start merges") node1.query("select sleep(1)") @@ -40,5 +39,39 @@ def test_merge_session_expired(started_cluster): pm.restore_instance_zk_connections(node1) node1.query("system restart replica tab") - node1.query("system sync replica tab") assert node1.query("select count() from tab") == '17\n' + + +def test_merge_session_expired_zero_copy(started_cluster): + node1.query("drop table if exists tab") + node1.query(""" + create table tab (x UInt64, y UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab2/', '0') order by tuple() + settings old_parts_lifetime=1, storage_policy='s3', allow_remote_fs_zero_copy_replication=1, replicated_max_ratio_of_wrong_parts=1, min_bytes_for_wide_part=1 + """) + + node1.query("insert into tab select number, number from numbers(10)") + node1.query("insert into tab select number + 10, number + 10 from numbers(10)") + node1.query("alter table tab update x = x + 1 where 1 settings mutations_sync=2") + node1.query("select * from tab") + node1.query("alter table tab update x = x + 1, y = y + 1 where 1 settings mutations_sync=2") + node1.query("select * from tab") + node1.query("alter table tab update x = x + 1 where 1 settings mutations_sync=2") + node1.query("select * from tab") + + node1.query("alter table tab add column z UInt64 materialized x + sleepEachRow(0.05) settings mutations_sync=2") + node1.query("optimize table tab final settings alter_sync=0") + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node1) + # Wait some time for merge to start + # Part should be merged and stayed on disk, but not commited into zk + node1.query("select sleep(2)") + node1.restart_clickhouse() + pm.restore_instance_zk_connections(node1) + + node1.query("system restart replica tab") + # Wait for outdated parts to be removed + node1.query("select sleep(3)") + node1.query("select * from tab") + node1.query("system sync replica tab") + assert node1.query("select count() from tab") == '20\n' From 9460cd2503ae291e6e57c0206b3621d12e83ca68 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 15 Sep 2023 18:55:55 +0000 Subject: [PATCH 039/634] Automatic style fix --- .../test_merge_session_expired/test.py | 29 ++++++++++++++----- 1 file changed, 21 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_merge_session_expired/test.py b/tests/integration/test_merge_session_expired/test.py index db8eb954d3a..61e8ff3c627 100644 --- a/tests/integration/test_merge_session_expired/test.py +++ b/tests/integration/test_merge_session_expired/test.py @@ -7,9 +7,14 @@ from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/keeper_config.xml", "configs/disks.xml"], stay_alive=True, with_zookeeper=True, with_minio=True + "node1", + main_configs=["configs/keeper_config.xml", "configs/disks.xml"], + stay_alive=True, + with_zookeeper=True, + with_minio=True, ) + @pytest.fixture(scope="module") def started_cluster(): try: @@ -22,7 +27,9 @@ def started_cluster(): def test_merge_session_expired(started_cluster): node1.query("drop table if exists tab") - node1.query("create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple() settings old_parts_lifetime=3") + node1.query( + "create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple() settings old_parts_lifetime=3" + ) node1.query("insert into tab select number from numbers(10)") node1.query("insert into tab select number + 10 from numbers(10)") node1.query("alter table tab delete where x = 12 settings mutations_sync=2") @@ -39,26 +46,32 @@ def test_merge_session_expired(started_cluster): pm.restore_instance_zk_connections(node1) node1.query("system restart replica tab") - assert node1.query("select count() from tab") == '17\n' + assert node1.query("select count() from tab") == "17\n" def test_merge_session_expired_zero_copy(started_cluster): node1.query("drop table if exists tab") - node1.query(""" + node1.query( + """ create table tab (x UInt64, y UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab2/', '0') order by tuple() settings old_parts_lifetime=1, storage_policy='s3', allow_remote_fs_zero_copy_replication=1, replicated_max_ratio_of_wrong_parts=1, min_bytes_for_wide_part=1 - """) + """ + ) node1.query("insert into tab select number, number from numbers(10)") node1.query("insert into tab select number + 10, number + 10 from numbers(10)") node1.query("alter table tab update x = x + 1 where 1 settings mutations_sync=2") node1.query("select * from tab") - node1.query("alter table tab update x = x + 1, y = y + 1 where 1 settings mutations_sync=2") + node1.query( + "alter table tab update x = x + 1, y = y + 1 where 1 settings mutations_sync=2" + ) node1.query("select * from tab") node1.query("alter table tab update x = x + 1 where 1 settings mutations_sync=2") node1.query("select * from tab") - node1.query("alter table tab add column z UInt64 materialized x + sleepEachRow(0.05) settings mutations_sync=2") + node1.query( + "alter table tab add column z UInt64 materialized x + sleepEachRow(0.05) settings mutations_sync=2" + ) node1.query("optimize table tab final settings alter_sync=0") with PartitionManager() as pm: @@ -74,4 +87,4 @@ def test_merge_session_expired_zero_copy(started_cluster): node1.query("select sleep(3)") node1.query("select * from tab") node1.query("system sync replica tab") - assert node1.query("select count() from tab") == '20\n' + assert node1.query("select count() from tab") == "20\n" From b9781f0da11ecb709252436aac68865bf9852487 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 18 Sep 2023 11:17:57 +0200 Subject: [PATCH 040/634] Updated to use unique path names for different azure tests to avoid collision --- tests/integration/test_storage_azure_blob_storage/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 3ec0506c525..919405e49b8 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -448,7 +448,7 @@ def test_schema_inference_from_globs(cluster): max_path = "" for i in range(10): for j in range(10): - path = "{}/{}_{}/{}.csv".format( + path = "{}/{}_{}/{}_schema.csv".format( unique_prefix, i, random.choice(["a", "b", "c", "d"]), j ) max_path = max(path, max_path) @@ -536,7 +536,7 @@ def test_put_get_with_globs_tf(cluster): max_path = "" for i in range(10): for j in range(10): - path = "{}/{}_{}/{}.csv".format( + path = "{}/{}_{}/{}_tf.csv".format( unique_prefix, i, random.choice(["a", "b", "c", "d"]), j ) max_path = max(path, max_path) @@ -576,7 +576,7 @@ def test_schema_inference_from_globs_tf(cluster): for i in range(10): for j in range(10): - path = "{}/{}_{}/{}.csv".format( + path = "{}/{}_{}/{}_schema_tf.csv".format( unique_prefix, i, random.choice(["a", "b", "c", "d"]), j ) max_path = max(path, max_path) From 5cac9f6ac5000292dd3c55693d89fc1b7a708287 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 18 Sep 2023 21:47:39 +0200 Subject: [PATCH 041/634] Fixed tests --- .../test_storage_azure_blob_storage/test.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 919405e49b8..19465dbe256 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -464,14 +464,14 @@ def test_schema_inference_from_globs(cluster): azure_query( node, - f"CREATE TABLE test_glob_select_inference Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')", + f"CREATE TABLE test_glob_select_inference Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/*_schema.csv')", ) print(node.query("SHOW CREATE TABLE test_glob_select_inference")) query = "select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from test_glob_select_inference" assert azure_query(node, query).splitlines() == [ - "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( + "450\t450\t900\t0_schema.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path ) ] @@ -546,9 +546,9 @@ def test_put_get_with_globs_tf(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", ) - query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV', structure='{table_format}')" + query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/*_tf.csv', format='CSV', structure='{table_format}')" assert azure_query(node, query).splitlines() == [ - "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( + "450\t450\t900\t0_tf.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path ) ] @@ -585,9 +585,9 @@ def test_schema_inference_from_globs_tf(cluster): query = f"insert into table function azureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" azure_query(node, query) - query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')" + query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/*_schema_tf.csv')" assert azure_query(node, query).splitlines() == [ - "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( + "450\t450\t900\t0_schema_tf.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path ) ] From 9e3c54ddb9e15d08849b3d5b75b0d4d897c35abf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Wed, 20 Sep 2023 10:07:02 +0800 Subject: [PATCH 042/634] add function getHttpHeader --- src/Functions/getHttpHeader.cpp | 95 +++++++++++++++++++ src/Interpreters/ClientInfo.h | 3 + src/Interpreters/Context.cpp | 10 +- src/Interpreters/Context.h | 8 +- src/Interpreters/Session.cpp | 6 +- src/Interpreters/Session.h | 3 +- src/Server/HTTPHandler.cpp | 11 ++- .../02884_getHttpHeaderFunction.reference | 2 + .../02884_getHttpHeaderFunction.sh | 21 ++++ 9 files changed, 153 insertions(+), 6 deletions(-) create mode 100644 src/Functions/getHttpHeader.cpp create mode 100644 tests/queries/0_stateless/02884_getHttpHeaderFunction.reference create mode 100755 tests/queries/0_stateless/02884_getHttpHeaderFunction.sh diff --git a/src/Functions/getHttpHeader.cpp b/src/Functions/getHttpHeader.cpp new file mode 100644 index 00000000000..ee914dd40b9 --- /dev/null +++ b/src/Functions/getHttpHeader.cpp @@ -0,0 +1,95 @@ +#include +#include +#include +#include +#include +#include +#include "Common/CurrentThread.h" +#include +#include "Interpreters/ClientInfo.h" +#include "Interpreters/Context_fwd.h" +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int FUNCTION_NOT_ALLOWED; +} + +namespace +{ + +/** Get the value of parameter in http headers. + * If there no such parameter or the method of request is not + * http, the function will return empty string. + */ +class FunctionGetHttpHeader : public IFunction +{ +private: + +public: + FunctionGetHttpHeader() = default; + + static constexpr auto name = "getHttpHeader"; + + static FunctionPtr create(ContextPtr /*context*/) + { + return std::make_shared(); + } + + + String getName() const override { return name; } + + bool isDeterministic() const override { return false; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The argument of function {} must have String type", getName()); + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + const auto & query_context = DB::CurrentThread::getQueryContext(); + const auto & method = query_context->getClientInfo().http_method; + + const auto & headers = DB::CurrentThread::getQueryContext()->getClientInfo().headers; + + const IColumn * arg_column = arguments[0].column.get(); + const ColumnString * arg_string = checkAndGetColumnConstData(arg_column); + + if (!arg_string) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The argument of function {} must be constant String", getName()); + + if (method != ClientInfo::HTTPMethod::GET && method != ClientInfo::HTTPMethod::POST) + return result_type->createColumnConst(input_rows_count, ""); + + if (!headers.has(arg_string->getDataAt(0).toString())) + return result_type->createColumnConst(input_rows_count, ""); + + return result_type->createColumnConst(input_rows_count, headers[arg_string->getDataAt(0).toString()]); + } +}; + +} + +REGISTER_FUNCTION(GetHttpHeader) +{ + factory.registerFunction(); +} + +} diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 70524333047..5878f0b424e 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -96,6 +97,8 @@ public: /// For mysql and postgresql UInt64 connection_id = 0; + Poco::Net::NameValueCollection headers; + void setHttpHeaders(const Poco::Net::NameValueCollection & _headers) { headers = _headers; } /// Comma separated list of forwarded IP addresses (from X-Forwarded-For for HTTP interface). /// It's expected that proxy appends the forwarded address to the end of the list. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 8695669a7de..113d862e5d6 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -85,6 +86,7 @@ #include #include #include +#include "Disks/ObjectStorages/S3/diskSettings.h" #include #include #include @@ -4008,12 +4010,18 @@ void Context::setClientConnectionId(uint32_t connection_id_) client_info.connection_id = connection_id_; } -void Context::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) +void Context::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer, const Poco::Net::NameValueCollection & http_headers) { client_info.http_method = http_method; client_info.http_user_agent = http_user_agent; client_info.http_referer = http_referer; need_recalculate_access = true; + + if (!http_headers.empty()) + { + for (const auto & http_header : http_headers) + client_info.headers.set(http_header.first, http_header.second); + } } void Context::setForwardedFor(const String & forwarded_for) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b4a5b3d8c85..b985f45a091 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,5 +1,7 @@ #pragma once +#include +#include "Core/Types.h" #ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD #include @@ -609,7 +611,7 @@ public: void setClientInterface(ClientInfo::Interface interface); void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); void setClientConnectionId(uint32_t connection_id); - void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); + void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer, const Poco::Net::NameValueCollection & http_headers = {}); void setForwardedFor(const String & forwarded_for); void setQueryKind(ClientInfo::QueryKind query_kind); void setQueryKindInitial(); @@ -804,6 +806,10 @@ public: /// Storage of forbidden HTTP headers from config.xml void setHTTPHeaderFilter(const Poco::Util::AbstractConfiguration & config); const HTTPHeaderFilter & getHTTPHeaderFilter() const; + const Poco::Net::NameValueCollection & getHttpHeaders() const + { + return client_info.headers; + } /// The port that the server listens for executing SQL queries. UInt16 getTCPPort() const; diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 439bf6056ba..7105e18ce18 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -15,6 +15,7 @@ #include #include +#include #include #include @@ -428,17 +429,18 @@ void Session::setClientConnectionId(uint32_t connection_id) prepared_client_info->connection_id = connection_id; } -void Session::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) +void Session::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer, const Poco::Net::NameValueCollection & http_headers) { if (session_context) { - session_context->setHttpClientInfo(http_method, http_user_agent, http_referer); + session_context->setHttpClientInfo(http_method, http_user_agent, http_referer, http_headers); } else { prepared_client_info->http_method = http_method; prepared_client_info->http_user_agent = http_user_agent; prepared_client_info->http_referer = http_referer; + prepared_client_info->headers = http_headers; } } diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 81ef987b428..43e54474bbd 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -64,7 +65,7 @@ public: void setClientInterface(ClientInfo::Interface interface); void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); void setClientConnectionId(uint32_t connection_id); - void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); + void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer, const Poco::Net::NameValueCollection & http_headers = {}); void setForwardedFor(const String & forwarded_for); void setQuotaClientKey(const String & quota_key); void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index ebb7f0d3490..f787d2abe45 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -40,6 +40,7 @@ #include #include #include +#include #include #include #include @@ -500,7 +501,11 @@ bool HTTPHandler::authenticateUser( else if (request.getMethod() == HTTPServerRequest::HTTP_POST) http_method = ClientInfo::HTTPMethod::POST; - session->setHttpClientInfo(http_method, request.get("User-Agent", ""), request.get("Referer", "")); + NameValueCollection http_headers; + for (const auto & it : request) + http_headers.set(it.first, it.second); + + session->setHttpClientInfo(http_method, request.get("User-Agent", ""), request.get("Referer", ""), http_headers); session->setForwardedFor(request.get("X-Forwarded-For", "")); session->setQuotaClientKey(quota_key); @@ -580,6 +585,10 @@ void HTTPHandler::processQuery( session->makeSessionContext(); } + NameValueCollection headers; + for (auto it = request.begin(); it != request.end(); ++it) + headers.set(it->first, it->second); + auto context = session->makeQueryContext(); /// This parameter is used to tune the behavior of output formats (such as Native) for compatibility. diff --git a/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference b/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference new file mode 100644 index 00000000000..564a057086f --- /dev/null +++ b/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference @@ -0,0 +1,2 @@ +Code: 81. DB::Exception: Database `02884_getHttpHeaderFunction` does not exist. (UNKNOWN_DATABASE) (version 23.9.1.1) +default diff --git a/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh b/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh new file mode 100755 index 00000000000..b03b05f7cdb --- /dev/null +++ b/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh +db="02884_getHttpHeaderFunction" + +$CLICKHOUSE_CLIENT -q "CREATE DATABASE IF NOT EXISTS ${db}" +$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS ${db}.get_http_header (id UInt32, header_value String DEFAULT getHttpHeader('X-Clickhouse-User')) Engine=Memory()" + +#Insert data via tcp client +$CLICKHOUSE_CLIENT -q "INSERT INTO ${db}.get_http_header (id) values (1), (2)" + +#Insert data via http request +echo "INSERT INTO ${db}.get_http_header (id) values (3), (4)" | curl -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' 'http://localhost:8123/' -d @- + +$CLICKHOUSE_CLIENT -q "SELECT * FROM ${db}.get_http_header ORDER BY id;" +$CLICKHOUSE_CLIENT -q "DROP DATABASE ${db}" + +echo "SELECT getHttpHeader('X-Clickhouse-User')" | curl -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' 'http://localhost:8123/' -d @- + From f81f0b6e3d96843205c1422cb0902f6e8bfb87c5 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 20 Sep 2023 17:42:25 +0800 Subject: [PATCH 043/634] Use levenshteinDistance for neareast hostname load balancer --- src/Common/NamePrompter.h | 27 +------------------------ src/Common/isLocalAddress.cpp | 8 ++------ src/Common/levenshteinDistance.cpp | 32 ++++++++++++++++++++++++++++++ src/Common/levenshteinDistance.h | 12 +++++++++++ 4 files changed, 47 insertions(+), 32 deletions(-) create mode 100644 src/Common/levenshteinDistance.cpp create mode 100644 src/Common/levenshteinDistance.h diff --git a/src/Common/NamePrompter.h b/src/Common/NamePrompter.h index 1c2d4a2706e..cc72554657f 100644 --- a/src/Common/NamePrompter.h +++ b/src/Common/NamePrompter.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include @@ -29,31 +29,6 @@ public: } private: - static size_t levenshteinDistance(const String & lhs, const String & rhs) - { - size_t m = lhs.size(); - size_t n = rhs.size(); - - PODArrayWithStackMemory row(n + 1); - - for (size_t i = 1; i <= n; ++i) - row[i] = i; - - for (size_t j = 1; j <= m; ++j) - { - row[0] = j; - size_t prev = j - 1; - for (size_t i = 1; i <= n; ++i) - { - size_t old = row[i]; - row[i] = std::min(prev + (std::tolower(lhs[j - 1]) != std::tolower(rhs[i - 1])), - std::min(row[i - 1], row[i]) + 1); - prev = old; - } - } - return row[n]; - } - static void appendToQueue(size_t ind, const String & name, DistanceIndexQueue & queue, const std::vector & prompting_strings) { const String & prompt = prompting_strings[ind]; diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index 7569c6fc14e..ab830e9bc6b 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -124,12 +125,7 @@ bool isLocalAddress(const Poco::Net::SocketAddress & address, UInt16 clickhouse_ size_t getHostNameDifference(const std::string & local_hostname, const std::string & host) { - /// FIXME should we replace it with Levenstein distance? (we already have it in NamePrompter) - size_t hostname_difference = 0; - for (size_t i = 0; i < std::min(local_hostname.length(), host.length()); ++i) - if (local_hostname[i] != host[i]) - ++hostname_difference; - return hostname_difference; + return levenshteinDistance(local_hostname, host); } } diff --git a/src/Common/levenshteinDistance.cpp b/src/Common/levenshteinDistance.cpp new file mode 100644 index 00000000000..9eb6c0f9050 --- /dev/null +++ b/src/Common/levenshteinDistance.cpp @@ -0,0 +1,32 @@ +#include +#include + +namespace DB +{ + +size_t levenshteinDistance(const String & lhs, const String & rhs) +{ + size_t m = lhs.size(); + size_t n = rhs.size(); + + PODArrayWithStackMemory row(n + 1); + + for (size_t i = 1; i <= n; ++i) + row[i] = i; + + for (size_t j = 1; j <= m; ++j) + { + row[0] = j; + size_t prev = j - 1; + for (size_t i = 1; i <= n; ++i) + { + size_t old = row[i]; + row[i] = std::min(prev + (std::tolower(lhs[j - 1]) != std::tolower(rhs[i - 1])), + std::min(row[i - 1], row[i]) + 1); + prev = old; + } + } + return row[n]; +} + +} diff --git a/src/Common/levenshteinDistance.h b/src/Common/levenshteinDistance.h new file mode 100644 index 00000000000..b062f7c1d73 --- /dev/null +++ b/src/Common/levenshteinDistance.h @@ -0,0 +1,12 @@ +#pragma once + +#include + +namespace DB +{ + +/// How many steps if we want to change lhs to rhs. +/// Details in https://en.wikipedia.org/wiki/Levenshtein_distance +size_t levenshteinDistance(const String & lhs, const String & rhs); + +} From 3646c9fa587739f687fd0030629089320d13c756 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 20 Sep 2023 10:57:12 +0000 Subject: [PATCH 044/634] caching skip-able entries in DDLWorker Signed-off-by: Duc Canh Le --- src/Interpreters/DDLWorker.cpp | 18 ++++++++++++++---- src/Interpreters/DDLWorker.h | 32 ++++++++++++++++++++++++++++++++ 2 files changed, 46 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 8be334d6223..da46aad0329 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -167,6 +167,9 @@ ZooKeeperPtr DDLWorker::getAndSetZooKeeper() DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper) { + if (entries_to_skip.contains(entry_name)) + return {}; + String node_data; String entry_path = fs::path(queue_dir) / entry_name; @@ -186,6 +189,12 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r zookeeper->tryCreate(fs::path(entry_path) / "finished" / host_id, status.serializeText(), zkutil::CreateMode::Persistent); }; + auto add_to_skip_set = [&]() + { + entries_to_skip.insert(entry_name); + return nullptr; + }; + try { /// Stage 1: parse entry @@ -198,7 +207,7 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r /// Otherwise, that node will be ignored by DDLQueryStatusSource. out_reason = "Incorrect task format"; write_error_status(host_fqdn_id, ExecutionStatus::fromCurrentException(), out_reason); - return {}; + return add_to_skip_set(); } /// Stage 2: resolve host_id and check if we should execute query or not @@ -207,7 +216,7 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r if (!task->findCurrentHostID(context, log)) { out_reason = "There is no a local address in host list"; - return {}; + return add_to_skip_set(); } try @@ -223,13 +232,13 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r { out_reason = "Cannot parse query or obtain cluster info"; write_error_status(task->host_id_str, ExecutionStatus::fromCurrentException(), out_reason); - return {}; + return add_to_skip_set(); } if (zookeeper->exists(task->getFinishedNodePath())) { out_reason = TASK_PROCESSED_OUT_REASON; - return {}; + return add_to_skip_set(); } /// Now task is ready for execution @@ -955,6 +964,7 @@ void DDLWorker::cleanupQueue(Int64, const ZooKeeperPtr & zookeeper) continue; } zkutil::KeeperMultiException::check(rm_entry_res, ops, res); + entries_to_skip.remove(node_name); } catch (...) { diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 01ed89907a1..d34a4135199 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -13,7 +13,9 @@ #include #include #include +#include #include +#include namespace zkutil { @@ -79,6 +81,33 @@ public: ZooKeeperPtr getAndSetZooKeeper(); protected: + + class ConcurrentSet + { + public: + bool contains(const String & key) const + { + std::shared_lock lock(mtx); + return set.contains(key); + } + + bool insert(const String & key) + { + std::unique_lock lock(mtx); + return set.emplace(key).second; + } + + bool remove(const String & key) + { + std::unique_lock lock(mtx); + return set.erase(key); + } + + private: + std::unordered_set set; + mutable std::shared_mutex mtx; + }; + /// Iterates through queue tasks in ZooKeeper, runs execution of new tasks void scheduleTasks(bool reinitialized); @@ -160,6 +189,9 @@ protected: size_t max_tasks_in_queue = 1000; std::atomic max_id = 0; + + ConcurrentSet entries_to_skip; + const CurrentMetrics::Metric * max_entry_metric; const CurrentMetrics::Metric * max_pushed_entry_metric; }; From be07a7e3e9097de1594f49d793d0532c0b168e9b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Sep 2023 20:31:53 +0200 Subject: [PATCH 045/634] Update 02597_projection_materialize_and_replication.sql --- .../02597_projection_materialize_and_replication.sql | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql b/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql index c3f86ddc014..6e78e04a9e7 100644 --- a/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql +++ b/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql @@ -25,11 +25,7 @@ select min(c_id) from test group by d format Null; -ALTER TABLE test ADD PROJECTION d_order ( SELECT min(c_id) GROUP BY `d`); - -ALTER TABLE test MATERIALIZE PROJECTION d_order; - -ALTER TABLE test DROP PROJECTION d_order SETTINGS mutations_sync = 2; +ALTER TABLE test ADD PROJECTION d_order ( SELECT min(c_id) GROUP BY `d`), MATERIALIZE PROJECTION d_order, DROP PROJECTION d_order SETTINGS mutations_sync = 2; SELECT * FROM system.mutations WHERE database=currentDatabase() AND table='test' AND NOT is_done; From f27a76b92317c2f44ac863135d8fcf02cb8ced42 Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Sun, 24 Sep 2023 20:58:00 +0100 Subject: [PATCH 046/634] Add function `byteSwap`. byteSwap accepts an integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. Issue: #54734 --- .../sql-reference/functions/math-functions.md | 32 +++++++ src/Functions/byteSwap.cpp | 86 +++++++++++++++++++ ...new_functions_must_be_documented.reference | 1 + .../aspell-ignore/en/aspell-dict.txt | 1 + 4 files changed, 120 insertions(+) create mode 100644 src/Functions/byteSwap.cpp diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 9eab2274210..81603cba869 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -711,3 +711,35 @@ Result: │ 11 │ └──────────────────────────────────┘ ``` + +## byteSwap + +Accepts an integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. + +**Syntax** + +```sql +byteSwap(operand) +``` + +**Example** + +```sql +byteSwap(3351772109) +``` + +Result: + +```result +┌─byteSwap(3351772109)─┐ +│ 3455829959 │ +└──────────────────────┘ +``` + +The above example can be understood in the following manner: +1. First, the integer operand (base 10) is converted to bytes (base 2) in little-endian i.e. 3351772109 -> CD FB C7 C7 +2. Then, the bytes are reversed i.e CD FB C7 C7 -> C7 C7 FB CD +3. Finally, the bytes are interpreted back to an integer assuming little-endian i.e. C7 C7 FB CD -> 3455829959 + +Note that, in step#1, we can also choose to convert the operand to bytes in big-endian as long as we also assume big-endian when +converting back to integer in step#3. diff --git a/src/Functions/byteSwap.cpp b/src/Functions/byteSwap.cpp new file mode 100644 index 00000000000..071e50bb786 --- /dev/null +++ b/src/Functions/byteSwap.cpp @@ -0,0 +1,86 @@ +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ +extern const int NOT_IMPLEMENTED; +} + +namespace +{ + +template +requires std::is_integral_v && (sizeof(T) <= sizeof(UInt32)) +inline T roundDownToPowerOfTwo(T x) +{ + return x <= 0 ? 0 : (T(1) << (31 - __builtin_clz(x))); +} + +template +requires std::is_integral_v && (sizeof(T) == sizeof(UInt64)) +inline T roundDownToPowerOfTwo(T x) +{ + return x <= 0 ? 0 : (T(1) << (63 - __builtin_clzll(x))); +} + +template +requires std::is_same_v +inline T roundDownToPowerOfTwo(T x) +{ + return bit_cast(bit_cast(x) & ~((1ULL << 23) - 1)); +} + +template +requires std::is_same_v +inline T roundDownToPowerOfTwo(T x) +{ + return bit_cast(bit_cast(x) & ~((1ULL << 52) - 1)); +} + +template +requires is_big_int_v +inline T roundDownToPowerOfTwo(T) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "roundToExp2() for big integers is not implemented"); +} + + +template +struct ByteSwapImpl +{ + using ResultType = T; + + static inline T apply(T x) + { + // return roundDownToPowerOfTwo(x); + return x; + } + +#if USE_EMBEDDED_COMPILER + static constexpr bool compilable = false; +#endif +}; + +struct NameByteSwap +{ + static constexpr auto name = "byteSwap"; +}; +using FunctionByteSwap = FunctionUnaryArithmetic; + +} + +template <> +struct FunctionUnaryArithmeticMonotonicity : PositiveMonotonicity +{ +}; + +REGISTER_FUNCTION(ByteSwap) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 69f455773b0..16cab29649b 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -192,6 +192,7 @@ blockSerializedSize blockSize buildId byteSize +byteSwap caseWithExpr caseWithExpression caseWithoutExpr diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index e5a031dabe3..a418676ad6d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1176,6 +1176,7 @@ buildId buildable builtins byteSize +byteSwap bytebase bytesToCutForIPv cLoki From e4256eb268ed1307a00c86dfc644750dc3411566 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Sep 2023 13:44:45 +0200 Subject: [PATCH 047/634] Better --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 56 +++++++++++++------ src/Storages/S3Queue/S3QueueFilesMetadata.h | 3 + src/Storages/S3Queue/S3QueueSettings.h | 16 +++--- src/Storages/S3Queue/StorageS3Queue.cpp | 8 +-- src/Storages/S3Queue/StorageS3Queue.h | 3 - .../integration/test_storage_s3_queue/test.py | 4 ++ 6 files changed, 57 insertions(+), 33 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 0e9a121339f..da520d9155a 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -1,6 +1,7 @@ #include #include "Common/Exception.h" #include "Common/ZooKeeper/Types.h" +#include "Common/scope_guard_safe.h" #include "Interpreters/Context_fwd.h" #include "Storages/S3Queue/S3QueueSettings.h" #include "config.h" @@ -35,14 +36,12 @@ namespace return std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); } - size_t generateRescheduleInterval() + size_t generateRescheduleInterval(size_t min, size_t max) { /// Use more or less random interval for unordered mode cleanup task. /// So that distributed processing cleanup tasks would not schedule cleanup at the same time. - /// TODO: make lower and upper boundary configurable by settings pcg64 rng(randomSeed()); - //return 5000 + rng() % 30000; - return rng() % 100; + return min + rng() % (max - min + 1); } } @@ -55,19 +54,19 @@ S3QueueFilesMetadata::S3QueueFilesMetadata( , max_set_size(settings_.s3queue_tracked_files_limit.value) , max_set_age_sec(settings_.s3queue_tracked_file_ttl_sec.value) , max_loading_retries(settings_.s3queue_loading_retries.value) + , min_cleanup_interval_ms(settings_.s3queue_cleanup_interval_min_ms.value) + , max_cleanup_interval_ms(settings_.s3queue_cleanup_interval_max_ms.value) , zookeeper_processing_path(storage->getZooKeeperPath() / "processing") , zookeeper_processed_path(storage->getZooKeeperPath() / "processed") , zookeeper_failed_path(storage->getZooKeeperPath() / "failed") + , zookeeper_cleanup_lock_path(storage->getZooKeeperPath() / "cleanup_lock") , log(&Poco::Logger::get("S3QueueFilesMetadata")) { if (mode == S3QueueMode::UNORDERED && (max_set_size || max_set_age_sec)) { task = context->getSchedulePool().createTask("S3QueueCleanupFunc", [this] { cleanupThreadFunc(); }); task->activate(); - - auto schedule_ms = generateRescheduleInterval(); - LOG_TEST(log, "Scheduling a cleanup task in {} ms", schedule_ms); - task->scheduleAfter(schedule_ms); + task->scheduleAfter(generateRescheduleInterval(min_cleanup_interval_ms, max_cleanup_interval_ms)); } } @@ -343,7 +342,7 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc { /// File is no longer retriable. /// Make a failed/node_name node and remove failed/node_name.retriable node. - /// TODO always add version for processing node. + /// TODO: always add version for processing node. Coordination::Requests requests; requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); @@ -405,7 +404,7 @@ void S3QueueFilesMetadata::cleanupThreadFunc() if (shutdown) return; - task->scheduleAfter(generateRescheduleInterval()); + task->scheduleAfter(generateRescheduleInterval(min_cleanup_interval_ms, max_cleanup_interval_ms)); } void S3QueueFilesMetadata::cleanupThreadFuncImpl() @@ -430,6 +429,33 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() return; } + /// Create a lock so that with distributed processing + /// multiple nodes do not execute cleanup in parallel. + Coordination::Error code = zk_client->tryCreate(zookeeper_cleanup_lock_path, + toString(getCurrentTime()), + zkutil::CreateMode::Ephemeral); + if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_TEST(log, "Cleanup is already being executed by another node"); + return; + } + else if (code != Coordination::Error::ZOK) + { + throw Coordination::Exception::fromPath(code, zookeeper_cleanup_lock_path); + } + + SCOPE_EXIT_SAFE({ + try + { + zk_client->remove(zookeeper_cleanup_lock_path); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); + } + }); + struct Node { std::string name; @@ -467,8 +493,6 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() } } - /// TODO add a zookeeper lock for cleanup - auto get_nodes_str = [&]() { WriteBufferFromOwnString wb; @@ -487,11 +511,11 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TEST(log, "Removing node at path {} ({}) because max files limit is reached", node.metadata.file_path, path.string()); - auto code = zk_client->tryRemove(path); + code = zk_client->tryRemove(path); if (code == Coordination::Error::ZOK) --nodes_to_remove; else - LOG_ERROR(log, "Failed to remove a node `{}`", path.string()); + LOG_ERROR(log, "Failed to remove a node `{}` (code: {})", path.string(), code); } else if (check_nodes_ttl) { @@ -502,9 +526,9 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TEST(log, "Removing node at path {} ({}) because file is reached", node.metadata.file_path, path.string()); - auto code = zk_client->tryRemove(path); + code = zk_client->tryRemove(path); if (code != Coordination::Error::ZOK) - LOG_ERROR(log, "Failed to remove a node `{}`", path.string()); + LOG_ERROR(log, "Failed to remove a node `{}` (code: {})", path.string(), code); } else if (!nodes_to_remove) { diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index b8e172bcd88..d794adc03c5 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -36,10 +36,13 @@ private: const UInt64 max_set_size; const UInt64 max_set_age_sec; const UInt64 max_loading_retries; + const size_t min_cleanup_interval_ms; + const size_t max_cleanup_interval_ms; const fs::path zookeeper_processing_path; const fs::path zookeeper_processed_path; const fs::path zookeeper_failed_path; + const fs::path zookeeper_cleanup_lock_path; mutable std::mutex mutex; Poco::Logger * log; diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 75defc4a57f..f2c9cd8ac19 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -19,17 +19,19 @@ class ASTStorage; 0) \ M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete or keep file in S3 after successful processing", 0) \ M(String, keeper_path, "", "Zookeeper node path", 0) \ - M(UInt64, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \ - M(UInt64, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ - M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ - M(UInt64, s3queue_polling_backoff_ms, 0, "Polling backoff", 0) \ - M(UInt64, s3queue_tracked_files_limit, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ - M(UInt64, \ + M(UInt32, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \ + M(UInt32, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ + M(UInt32, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ + M(UInt32, s3queue_polling_backoff_ms, 0, "Polling backoff", 0) \ + M(UInt32, s3queue_tracked_files_limit, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ + M(UInt32, \ s3queue_tracked_file_ttl_sec, \ 0, \ "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", \ 0) \ - M(UInt64, s3queue_polling_size, 50, "Maximum files to fetch from S3 with SELECT", 0) + M(UInt32, s3queue_polling_size, 50, "Maximum files to fetch from S3 with SELECT", 0) \ + M(UInt32, s3queue_cleanup_interval_min_ms, 10000, "Polling backoff min for cleanup for `unordered` mode", 0) \ + M(UInt32, s3queue_cleanup_interval_max_ms, 30000, "Polling backoff max for cleanup for `unordered` mode", 0) \ #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index bbaf5ae5311..3e662946b24 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -334,13 +334,7 @@ StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(Cont zkutil::ZooKeeperPtr StorageS3Queue::getZooKeeper() const { - std::lock_guard lock{zk_mutex}; - if (!zk_client || zk_client->expired()) - { - zk_client = getContext()->getZooKeeper(); - zk_client->sync(zk_path); - } - return zk_client; + return getContext()->getZooKeeper(); } void StorageS3Queue::createOrCheckMetadata(const StorageInMemoryMetadata & storage_metadata) diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 07f52c434de..fde3c699142 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -67,9 +67,6 @@ private: const std::optional format_settings; NamesAndTypesList virtual_columns; - mutable zkutil::ZooKeeperPtr zk_client; - mutable std::mutex zk_mutex; - BackgroundSchedulePool::TaskHolder task; std::atomic stream_cancelled{false}; UInt64 reschedule_processing_interval_ms; diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index d26afcc3aa3..ffcbdebde16 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -771,6 +771,8 @@ def test_max_set_age(started_cluster): additional_settings={ "keeper_path": keeper_path, "s3queue_tracked_file_ttl_sec": max_age, + "s3queue_cleanup_interval_min_ms": 0, + "s3queue_cleanup_interval_max_ms": 0, }, ) create_mv(node, table_name, dst_table_name) @@ -836,6 +838,8 @@ def test_max_set_size(started_cluster): additional_settings={ "keeper_path": keeper_path, "s3queue_tracked_files_limit": 9, + "s3queue_cleanup_interval_min_ms": 0, + "s3queue_cleanup_interval_max_ms": 0, }, ) total_values = generate_random_files( From b142756c5de12c8f2823c19be488d702ac311b85 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 25 Sep 2023 14:08:05 +0200 Subject: [PATCH 048/634] Added fixture to delete all files --- .../test_storage_azure_blob_storage/test.py | 34 ++++++++++++++----- 1 file changed, 25 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 19465dbe256..5df4708f381 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -92,6 +92,22 @@ def put_azure_file_content(filename, data): buf = io.BytesIO(data) blob_client.upload_blob(buf) +@pytest.fixture(autouse=True, scope="function") +def delete_all_files(): + connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" + blob_service_client = BlobServiceClient.from_connection_string(connection_string) + containers = blob_service_client.list_containers() + for container in containers: + container_client = blob_service_client.get_container_client(container) + blob_list = container_client.list_blobs() + for blob in blob_list: + print(blob) + blob_client = container_client.get_blob_client(blob) + blob_client.delete_blob() + + assert (len(list(container_client.list_blobs())) == 0) + + yield def test_create_table_connection_string(cluster): node = cluster.instances["node"] @@ -448,7 +464,7 @@ def test_schema_inference_from_globs(cluster): max_path = "" for i in range(10): for j in range(10): - path = "{}/{}_{}/{}_schema.csv".format( + path = "{}/{}_{}/{}.csv".format( unique_prefix, i, random.choice(["a", "b", "c", "d"]), j ) max_path = max(path, max_path) @@ -464,14 +480,14 @@ def test_schema_inference_from_globs(cluster): azure_query( node, - f"CREATE TABLE test_glob_select_inference Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/*_schema.csv')", + f"CREATE TABLE test_glob_select_inference Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')", ) print(node.query("SHOW CREATE TABLE test_glob_select_inference")) query = "select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from test_glob_select_inference" assert azure_query(node, query).splitlines() == [ - "450\t450\t900\t0_schema.csv\t{bucket}/{max_path}".format( + "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path ) ] @@ -536,7 +552,7 @@ def test_put_get_with_globs_tf(cluster): max_path = "" for i in range(10): for j in range(10): - path = "{}/{}_{}/{}_tf.csv".format( + path = "{}/{}_{}/{}.csv".format( unique_prefix, i, random.choice(["a", "b", "c", "d"]), j ) max_path = max(path, max_path) @@ -546,9 +562,9 @@ def test_put_get_with_globs_tf(cluster): node, f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", ) - query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/*_tf.csv', format='CSV', structure='{table_format}')" + query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV', structure='{table_format}')" assert azure_query(node, query).splitlines() == [ - "450\t450\t900\t0_tf.csv\t{bucket}/{max_path}".format( + "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path ) ] @@ -576,7 +592,7 @@ def test_schema_inference_from_globs_tf(cluster): for i in range(10): for j in range(10): - path = "{}/{}_{}/{}_schema_tf.csv".format( + path = "{}/{}_{}/{}.csv".format( unique_prefix, i, random.choice(["a", "b", "c", "d"]), j ) max_path = max(path, max_path) @@ -585,9 +601,9 @@ def test_schema_inference_from_globs_tf(cluster): query = f"insert into table function azureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" azure_query(node, query) - query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/*_schema_tf.csv')" + query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')" assert azure_query(node, query).splitlines() == [ - "450\t450\t900\t0_schema_tf.csv\t{bucket}/{max_path}".format( + "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path ) ] From 6224a4fcfa8badec08594624ad9c4484133897f7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 25 Sep 2023 12:21:14 +0000 Subject: [PATCH 049/634] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 5df4708f381..e24ba0d0f1f 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -92,6 +92,7 @@ def put_azure_file_content(filename, data): buf = io.BytesIO(data) blob_client.upload_blob(buf) + @pytest.fixture(autouse=True, scope="function") def delete_all_files(): connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" @@ -105,10 +106,11 @@ def delete_all_files(): blob_client = container_client.get_blob_client(blob) blob_client.delete_blob() - assert (len(list(container_client.list_blobs())) == 0) + assert len(list(container_client.list_blobs())) == 0 yield + def test_create_table_connection_string(cluster): node = cluster.instances["node"] azure_query( From 3b54b6be88b83b6f1903ea375579c1e89989d1ad Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Sep 2023 19:23:46 +0200 Subject: [PATCH 050/634] Add system tables --- .../table-engines/integrations/s3queue.md | 32 +++++++ src/Common/ProfileEvents.cpp | 13 ++- src/Common/ProfileEvents.h | 1 + src/Common/SystemLogBase.cpp | 1 + src/Common/SystemLogBase.h | 1 + src/Interpreters/Context.cpp | 9 ++ src/Interpreters/Context.h | 2 + src/Interpreters/S3QueueLog.cpp | 43 ++++++++++ src/Interpreters/S3QueueLog.h | 38 +++++++++ src/Interpreters/SystemLog.cpp | 2 + src/Interpreters/SystemLog.h | 2 + src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 65 +++++++++++++- src/Storages/S3Queue/S3QueueFilesMetadata.h | 28 ++++++- src/Storages/S3Queue/S3QueueSettings.h | 17 ++-- src/Storages/S3Queue/S3QueueSource.cpp | 68 +++++++++++++-- src/Storages/S3Queue/S3QueueSource.h | 20 ++++- src/Storages/S3Queue/StorageS3Queue.cpp | 21 ++++- src/Storages/S3Queue/StorageS3Queue.h | 2 + src/Storages/System/StorageSystemS3Queue.cpp | 84 +++++++++++++++++++ src/Storages/System/StorageSystemS3Queue.h | 22 +++++ src/Storages/System/attachSystemTables.cpp | 2 + .../integration/test_storage_s3_queue/test.py | 4 - 22 files changed, 443 insertions(+), 34 deletions(-) create mode 100644 src/Interpreters/S3QueueLog.cpp create mode 100644 src/Interpreters/S3QueueLog.h create mode 100644 src/Storages/System/StorageSystemS3Queue.cpp create mode 100644 src/Storages/System/StorageSystemS3Queue.h diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 50330962b86..76cc60ad4fd 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -200,6 +200,38 @@ Example: SELECT * FROM stats ORDER BY name; ``` +## Introspection {#introspection} + +For introspection use `system.s3queue` stateless table and `system.s3_queue_log` persistent table. + +In order to use `system.s3_queue_log` define its configuration in server config file: + +``` xml + + system + s3_queue_log
+
+``` + +Example: + +``` sql +:) select * from system.s3queue + +SELECT * +FROM system.s3queue + +Query id: bb41964e-c947-4112-be3a-0f01770a1e84 + +┌─database─┬─table───┬─file_name──────────────────────────────────────────┬─rows_processed─┬─status─────┬─processing_start_time─┬─processing_end_time─┬─ProfileEvents──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ default │ s3queue │ hits_compatible/athena_partitioned/hits_60.parquet │ 0 │ Processing │ 2023-09-25 19:56:51 │ ᴺᵁᴸᴸ │ {'S3QueueSetFileProcessingMicroseconds':516} │ +│ default │ s3queue │ hits_compatible/athena_partitioned/hits_54.parquet │ 1000000 │ Processing │ 2023-09-25 19:56:50 │ ᴺᵁᴸᴸ │ {'SelectedRows':1000000,'SelectedBytes':1284181126,'S3ReadMicroseconds':385274,'S3ReadRequestsCount':3,'S3ReadRequestsErrors':1,'S3GetObject':1,'ReadBufferFromS3Microseconds':405970,'ReadBufferFromS3InitMicroseconds':385790,'ReadBufferFromS3Bytes':65536,'ReadBufferFromS3PreservedSessions':1,'S3QueueSetFileProcessingMicroseconds':567,'S3QueuePullMicroseconds':2475045} │ +└──────────┴─────────┴────────────────────────────────────────────────────┴────────────────┴────────────┴───────────────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + + +SELECT * FROM system.s3_queue_log; +``` + ## Virtual columns {#virtual-columns} - `_path` — Path to the file. diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index fd7b47b4f87..dc6a3108971 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -529,6 +529,12 @@ The server successfully detected this situation and will download merged part fr M(OverflowThrow, "Number of times, data processing was cancelled by query complexity limitation with setting '*_overflow_mode' = 'throw' and exception was thrown.") \ M(OverflowAny, "Number of times approximate GROUP BY was in effect: when aggregation was performed only on top of first 'max_rows_to_group_by' unique keys and other keys were ignored due to 'group_by_overflow_mode' = 'any'.") \ \ + M(S3QueueSetFileProcessingMicroseconds, "Time spent to set file as processing")\ + M(S3QueueSetFileProcessedMicroseconds, "Time spent to set file as processed")\ + M(S3QueueSetFileFailedMicroseconds, "Time spent to set file as failed")\ + M(S3QueueCleanupMaxSetSizeOrTTLMicroseconds, "Time spent to set file as failed")\ + M(S3QueuePullMicroseconds, "Time spent to read file data")\ + \ M(ServerStartupMilliseconds, "Time elapsed from starting server to listening to sockets in milliseconds")\ M(IOUringSQEsSubmitted, "Total number of io_uring SQEs submitted") \ M(IOUringSQEsResubmits, "Total number of io_uring SQE resubmits performed") \ @@ -588,9 +594,14 @@ Timer::Timer(Counters & counters_, Event timer_event_, Event counter_event, Reso counters.increment(counter_event); } +UInt64 Timer::get() +{ + return watch.elapsedNanoseconds() / static_cast(resolution); +} + void Timer::end() { - counters.increment(timer_event, watch.elapsedNanoseconds() / static_cast(resolution)); + counters.increment(timer_event, get()); watch.reset(); } diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index 97c5ccd3731..adf2adb9808 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -41,6 +41,7 @@ namespace ProfileEvents ~Timer() { end(); } void cancel() { watch.reset(); } void end(); + UInt64 get(); private: Counters & counters; diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 611e14fd9b3..45e08d28f2d 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 9770629e96a..6efab699b75 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -27,6 +27,7 @@ M(ZooKeeperLogElement) \ M(ProcessorProfileLogElement) \ M(TextLogElement) \ + M(S3QueueLogElement) \ M(FilesystemCacheLogElement) \ M(FilesystemReadPrefetchesLogElement) \ M(AsynchronousInsertLogElement) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 86651f31e7c..b1663e3b2b4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3413,6 +3413,15 @@ std::shared_ptr Context::getFilesystemCacheLog() const return shared->system_logs->filesystem_cache_log; } +std::shared_ptr Context::getS3QueueLog() const +{ + auto lock = getLock(); + if (!shared->system_logs) + return {}; + + return shared->system_logs->s3_queue_log; +} + std::shared_ptr Context::getFilesystemReadPrefetchesLog() const { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 24441ff9ab8..c705067806a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -103,6 +103,7 @@ class TransactionsInfoLog; class ProcessorsProfileLog; class FilesystemCacheLog; class FilesystemReadPrefetchesLog; +class S3QueueLog; class AsynchronousInsertLog; class BackupLog; class IAsynchronousReader; @@ -1026,6 +1027,7 @@ public: std::shared_ptr getTransactionsInfoLog() const; std::shared_ptr getProcessorsProfileLog() const; std::shared_ptr getFilesystemCacheLog() const; + std::shared_ptr getS3QueueLog() const; std::shared_ptr getFilesystemReadPrefetchesLog() const; std::shared_ptr getAsynchronousInsertLog() const; std::shared_ptr getBackupLog() const; diff --git a/src/Interpreters/S3QueueLog.cpp b/src/Interpreters/S3QueueLog.cpp new file mode 100644 index 00000000000..963a4789d35 --- /dev/null +++ b/src/Interpreters/S3QueueLog.cpp @@ -0,0 +1,43 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +NamesAndTypesList S3QueueLogElement::getNamesAndTypes() +{ + auto status_datatype = std::make_shared( + DataTypeEnum8::Values + { + {"Processed", static_cast(S3QueueLogElement::S3QueueStatus::Processed)}, + {"Failed", static_cast(S3QueueLogElement::S3QueueStatus::Failed)}, + }); + return { + {"event_date", std::make_shared()}, + {"event_time", std::make_shared()}, + {"table_uuid", std::make_shared()}, + {"file_name", std::make_shared()}, + {"rows_processed", std::make_shared()}, + {"status", status_datatype}, + }; +} + +void S3QueueLogElement::appendToBlock(MutableColumns & columns) const +{ + size_t i = 0; + columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType()); + columns[i++]->insert(event_time); + columns[i++]->insert(table_uuid); + columns[i++]->insert(file_name); + columns[i++]->insert(rows_processed); + columns[i++]->insert(magic_enum::enum_name(status)); +} + +} diff --git a/src/Interpreters/S3QueueLog.h b/src/Interpreters/S3QueueLog.h new file mode 100644 index 00000000000..3c99221026c --- /dev/null +++ b/src/Interpreters/S3QueueLog.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +struct S3QueueLogElement +{ + time_t event_time{}; + std::string table_uuid; + std::string file_name; + size_t rows_processed = 0; + + enum class S3QueueStatus + { + Processed, + Failed, + }; + S3QueueStatus status; + + static std::string name() { return "S3QueueLog"; } + + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases() { return {}; } + + void appendToBlock(MutableColumns & columns) const; + static const char * getCustomColumnList() { return nullptr; } +}; + +class S3QueueLog : public SystemLog +{ + using SystemLog::SystemLog; +}; + +} diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 07ef6c33d29..d2dd4eabaec 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -289,6 +290,7 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf processors_profile_log = createSystemLog(global_context, "system", "processors_profile_log", config, "processors_profile_log"); asynchronous_insert_log = createSystemLog(global_context, "system", "asynchronous_insert_log", config, "asynchronous_insert_log"); backup_log = createSystemLog(global_context, "system", "backup_log", config, "backup_log"); + s3_queue_log = createSystemLog(global_context, "system", "s3queue_log", config, "s3queue_log"); if (query_log) logs.emplace_back(query_log.get()); diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index ec04e1f4162..932afec3bad 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -50,6 +50,7 @@ class FilesystemCacheLog; class FilesystemReadPrefetchesLog; class AsynchronousInsertLog; class BackupLog; +class S3QueueLog; /// System logs should be destroyed in destructor of the last Context and before tables, /// because SystemLog destruction makes insert query while flushing data into underlying tables @@ -70,6 +71,7 @@ struct SystemLogs std::shared_ptr metric_log; /// Used to log all metrics. std::shared_ptr filesystem_cache_log; std::shared_ptr filesystem_read_prefetches_log; + std::shared_ptr s3_queue_log; /// Metrics from system.asynchronous_metrics. std::shared_ptr asynchronous_metric_log; /// OpenTelemetry trace spans. diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index da520d9155a..837d270f8fd 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -21,12 +21,21 @@ #include #include +namespace ProfileEvents +{ + extern const Event S3QueueSetFileProcessingMicroseconds; + extern const Event S3QueueSetFileProcessedMicroseconds; + extern const Event S3QueueSetFileFailedMicroseconds; + extern const Event S3QueueCleanupMaxSetSizeOrTTLMicroseconds; +}; + namespace DB { namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int FILE_DOESNT_EXIST; } namespace @@ -129,19 +138,48 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( return metadata; } +std::shared_ptr S3QueueFilesMetadata::getFileStatus(const std::string & path) +{ + std::lock_guard lock(file_statuses_mutex); + return file_statuses.at(path); +} + +S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::getFileStateses() const +{ + std::lock_guard lock(file_statuses_mutex); + return file_statuses; +} + bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) { + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessingMicroseconds); + + bool result; switch (mode) { case S3QueueMode::ORDERED: { - return trySetFileAsProcessingForOrderedMode(path); + result = trySetFileAsProcessingForOrderedMode(path); + break; } case S3QueueMode::UNORDERED: { - return trySetFileAsProcessingForUnorderedMode(path); + result = trySetFileAsProcessingForUnorderedMode(path); + break; } } + if (result) + { + std::lock_guard lock(file_statuses_mutex); + auto it = file_statuses.emplace(path, std::make_shared()).first; + auto & file_status = it->second; + file_status->state = FileStatus::State::Processing; + file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); + timer.cancel(); + if (!file_status->processing_start_time) + file_status->processing_start_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + } + return result; } bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) @@ -225,6 +263,16 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin void S3QueueFilesMetadata::setFileProcessed(const String & path) { + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessedMicroseconds); + SCOPE_EXIT({ + std::lock_guard lock(file_statuses_mutex); + auto & file_status = file_statuses.at(path); + file_status->state = FileStatus::State::Processed; + file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessedMicroseconds, timer.get()); + timer.cancel(); + file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + }); + switch (mode) { case S3QueueMode::ORDERED: @@ -301,6 +349,17 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(const String & path) void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exception_message) { + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileFailedMicroseconds); + + SCOPE_EXIT_SAFE({ + std::lock_guard lock(file_statuses_mutex); + auto & file_status = file_statuses.at(path); + file_status->state = FileStatus::State::Failed; + file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileFailedMicroseconds, timer.get()); + timer.cancel(); + file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + }); + const auto node_name = getNodeName(path); auto node_metadata = createNodeMetadata(path, exception_message); const auto zk_client = storage->getZooKeeper(); @@ -409,6 +468,8 @@ void S3QueueFilesMetadata::cleanupThreadFunc() void S3QueueFilesMetadata::cleanupThreadFuncImpl() { + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueCleanupMaxSetSizeOrTTLMicroseconds); + chassert(max_set_size || max_set_age_sec); const bool check_nodes_limit = max_set_size > 0; diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index d794adc03c5..079020514c6 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -28,8 +28,32 @@ public: void setFileFailed(const std::string & path, const std::string & exception_message); + using OnProgress = std::function; + void deactivateCleanupTask(); + struct FileStatus + { + size_t processed_rows = 0; + enum class State + { + Processing, + Processed, + Failed, + None + }; + State state = State::None; + ProfileEvents::Counters profile_counters; + + time_t processing_start_time = 0; + time_t processing_end_time = 0; + }; + using FileStatuses = std::unordered_map>; + + std::shared_ptr getFileStatus(const std::string & path); + + FileStatuses getFileStateses() const; + private: const StorageS3Queue * storage; const S3QueueMode mode; @@ -44,12 +68,14 @@ private: const fs::path zookeeper_failed_path; const fs::path zookeeper_cleanup_lock_path; - mutable std::mutex mutex; Poco::Logger * log; std::atomic_bool shutdown = false; BackgroundSchedulePool::TaskHolder task; + FileStatuses file_statuses; + mutable std::mutex file_statuses_mutex; + bool trySetFileAsProcessingForOrderedMode(const std::string & path); bool trySetFileAsProcessingForUnorderedMode(const std::string & path); diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index f2c9cd8ac19..6dbffecbadf 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -20,18 +20,15 @@ class ASTStorage; M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete or keep file in S3 after successful processing", 0) \ M(String, keeper_path, "", "Zookeeper node path", 0) \ M(UInt32, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \ + M(UInt32, s3queue_tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \ + M(UInt32, s3queue_processing_threads_num, 1, "Number of processing threads", 0) \ + M(UInt32, s3queue_enable_logging_to_s3queue_log, 0, "Enable logging to system table system.s3queue_log", 0) \ M(UInt32, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt32, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ - M(UInt32, s3queue_polling_backoff_ms, 0, "Polling backoff", 0) \ - M(UInt32, s3queue_tracked_files_limit, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ - M(UInt32, \ - s3queue_tracked_file_ttl_sec, \ - 0, \ - "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", \ - 0) \ - M(UInt32, s3queue_polling_size, 50, "Maximum files to fetch from S3 with SELECT", 0) \ - M(UInt32, s3queue_cleanup_interval_min_ms, 10000, "Polling backoff min for cleanup for `unordered` mode", 0) \ - M(UInt32, s3queue_cleanup_interval_max_ms, 30000, "Polling backoff max for cleanup for `unordered` mode", 0) \ + M(UInt32, s3queue_polling_backoff_ms, 200, "Polling backoff", 0) \ + M(UInt32, s3queue_tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \ + M(UInt32, s3queue_cleanup_interval_min_ms, 10000, "For unordered mode. Polling backoff min for cleanup", 0) \ + M(UInt32, s3queue_cleanup_interval_max_ms, 30000, "For unordered mode. Polling backoff max for cleanup", 0) \ #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 3a834dae8d0..eec5d34027a 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -11,14 +11,13 @@ namespace CurrentMetrics { -extern const Metric StorageS3Threads; -extern const Metric StorageS3ThreadsActive; + extern const Metric StorageS3Threads; + extern const Metric StorageS3ThreadsActive; } namespace ProfileEvents { -extern const Event S3DeleteObjects; -extern const Event S3ListObjects; + extern const Event S3QueuePullMicroseconds; } namespace DB @@ -50,6 +49,11 @@ StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::FileIterator::next() } } +size_t StorageS3QueueSource::FileIterator::estimatedKeysCount() +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method estimateKeysCount is not implemented"); +} + StorageS3QueueSource::StorageS3QueueSource( String name_, const Block & header_, @@ -58,7 +62,10 @@ StorageS3QueueSource::StorageS3QueueSource( const S3QueueAction & action_, RemoveFileFunc remove_file_func_, const NamesAndTypesList & requested_virtual_columns_, - ContextPtr context_) + ContextPtr context_, + const std::atomic & shutdown_called_, + std::shared_ptr s3_queue_log_, + const StorageID & storage_id_) : ISource(header_) , WithContext(context_) , name(std::move(name_)) @@ -66,12 +73,18 @@ StorageS3QueueSource::StorageS3QueueSource( , files_metadata(files_metadata_) , internal_source(std::move(internal_source_)) , requested_virtual_columns(requested_virtual_columns_) + , shutdown_called(shutdown_called_) + , s3_queue_log(s3_queue_log_) + , storage_id(storage_id_) , remove_file_func(remove_file_func_) , log(&Poco::Logger::get("StorageS3QueueSource")) { reader = std::move(internal_source->reader); if (reader) + { reader_future = std::move(internal_source->reader_future); + file_status = files_metadata->getFileStatus(reader.getFile()); + } } StorageS3QueueSource::~StorageS3QueueSource() @@ -88,19 +101,31 @@ Chunk StorageS3QueueSource::generate() { while (true) { - if (isCancelled() || !reader) + if (!reader) + break; + + if (isCancelled()) { - if (reader) - reader->cancel(); + reader->cancel(); break; } + auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); + SCOPE_EXIT({ CurrentThread::get().attachProfileCountersScope(prev_scope); }); + try { + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueuePullMicroseconds); + Chunk chunk; if (reader->pull(chunk)) { LOG_TEST(log, "Read {} rows from file: {}", chunk.getNumRows(), reader.getPath()); + + file_status->processed_rows += chunk.getNumRows(); + // file_status->profile_counters.increment(ProfileEvents::S3QueuePullMicroseconds, timer.get()); + processed_rows_from_file += chunk.getNumRows(); + VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath()); return chunk; } @@ -109,18 +134,30 @@ Chunk StorageS3QueueSource::generate() { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); files_metadata->setFileFailed(reader.getFile(), e.message()); + appendLogElement(reader.getFile(), processed_rows_from_file, false); throw; } files_metadata->setFileProcessed(reader.getFile()); applyActionAfterProcessing(reader.getFile()); + appendLogElement(reader.getFile(), processed_rows_from_file, true); + processed_rows_from_file = 0; + + if (shutdown_called) + { + LOG_INFO(log, "Shutdown was called, stopping sync"); + break; + } + chassert(reader_future.valid()); reader = reader_future.get(); if (!reader) break; + file_status = files_metadata->getFileStatus(reader.getFile()); + /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. internal_source->create_reader_pool.wait(); @@ -145,6 +182,21 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & path) } } +void StorageS3QueueSource::appendLogElement(const std::string & file_name, size_t processed_rows, bool processed) +{ + if (!s3_queue_log) + return; + + S3QueueLogElement elem + { + .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), + .file_name = file_name, + .rows_processed = processed_rows, + .status = processed ? S3QueueLogElement::S3QueueStatus::Processed : S3QueueLogElement::S3QueueStatus::Failed, + }; + s3_queue_log->add(std::move(elem)); +} + } #endif diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 1ec762d6477..bca10f790c8 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -2,10 +2,11 @@ #include "config.h" #if USE_AWS_S3 +#include #include #include #include -#include +#include namespace Poco { class Logger; } @@ -32,14 +33,14 @@ public: KeyWithInfo next() override; + size_t estimatedKeysCount() override; + private: const std::shared_ptr metadata; const std::unique_ptr glob_iterator; std::mutex mutex; }; - static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); - StorageS3QueueSource( String name_, const Block & header_, @@ -48,10 +49,15 @@ public: const S3QueueAction & action_, RemoveFileFunc remove_file_func_, const NamesAndTypesList & requested_virtual_columns_, - ContextPtr context_); + ContextPtr context_, + const std::atomic & shutdown_called_, + std::shared_ptr s3_queue_log_, + const StorageID & storage_id_); ~StorageS3QueueSource() override; + static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); + String getName() const override; Chunk generate() override; @@ -62,6 +68,9 @@ private: const std::shared_ptr files_metadata; const std::shared_ptr internal_source; const NamesAndTypesList requested_virtual_columns; + const std::atomic & shutdown_called; + const std::shared_ptr s3_queue_log; + const StorageID storage_id; RemoveFileFunc remove_file_func; Poco::Logger * log; @@ -69,8 +78,11 @@ private: using ReaderHolder = StorageS3Source::ReaderHolder; ReaderHolder reader; std::future reader_future; + size_t processed_rows_from_file = 0; + std::shared_ptr file_status; void applyActionAfterProcessing(const String & path); + void appendLogElement(const std::string & file_name, size_t processed_rows, bool processed); }; } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 3e662946b24..f72af72123e 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -163,7 +163,7 @@ Pipe StorageS3Queue::read( ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - size_t /* num_streams */) + size_t num_streams) { if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) { @@ -178,7 +178,8 @@ Pipe StorageS3Queue::read( } Pipes pipes; - pipes.emplace_back(createSource(column_names, storage_snapshot, query_info.query, max_block_size, local_context)); + for (size_t i = 0; i < num_streams; ++i) + pipes.emplace_back(createSource(column_names, storage_snapshot, query_info.query, max_block_size, local_context)); return Pipe::unitePipes(std::move(pipes)); } @@ -219,9 +220,11 @@ std::shared_ptr StorageS3Queue::createSource( LOG_TRACE(log, "Object with path {} was removed from S3", path); } }; + auto s3_queue_log = s3queue_settings->s3queue_enable_logging_to_s3queue_log ? local_context->getS3QueueLog() : nullptr; return std::make_shared( getName(), read_from_format_info.source_header, std::move(internal_source), - files_metadata, after_processing, file_deleter, read_from_format_info.requested_virtual_columns, local_context); + files_metadata, after_processing, file_deleter, read_from_format_info.requested_virtual_columns, + local_context, shutdown_called, s3_queue_log, getStorageID()); } bool StorageS3Queue::hasDependencies(const StorageID & table_id) @@ -280,6 +283,10 @@ void StorageS3Queue::threadFunc() } } } + else + { + LOG_TEST(log, "No attached dependencies"); + } } catch (...) { @@ -317,7 +324,13 @@ void StorageS3Queue::streamToViews() // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true); auto block_io = interpreter.execute(); - auto pipe = Pipe(createSource(block_io.pipeline.getHeader().getNames(), storage_snapshot, nullptr, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context)); + Pipes pipes; + for (size_t i = 0; i < s3queue_settings->s3queue_processing_threads_num; ++i) + { + auto source = createSource(block_io.pipeline.getHeader().getNames(), storage_snapshot, nullptr, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); + pipes.emplace_back(std::move(source)); + } + auto pipe = Pipe::unitePipes(std::move(pipes)); std::atomic_size_t rows = 0; block_io.pipeline.complete(std::move(pipe)); diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index fde3c699142..ff6871ea70f 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -54,6 +54,8 @@ public: zkutil::ZooKeeperPtr getZooKeeper() const; + S3QueueFilesMetadata::FileStatuses getFileStatuses() const { return files_metadata->getFileStateses(); } + private: using FileIterator = StorageS3QueueSource::FileIterator; diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp new file mode 100644 index 00000000000..6d409e29ee4 --- /dev/null +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -0,0 +1,84 @@ +#include "StorageSystemS3Queue.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +NamesAndTypesList StorageSystemS3Queue::getNamesAndTypes() +{ + return { + {"database", std::make_shared()}, + {"table", std::make_shared()}, + {"file_name", std::make_shared()}, + {"rows_processed", std::make_shared()}, + {"status", std::make_shared()}, + {"processing_start_time", std::make_shared(std::make_shared())}, + {"processing_end_time", std::make_shared(std::make_shared())}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, + }; +} + +StorageSystemS3Queue::StorageSystemS3Queue(const StorageID & table_id_) + : IStorageSystemOneBlock(table_id_) +{ +} + +void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +{ + const auto access = context->getAccess(); + const bool show_tables_granted = access->isGranted(AccessType::SHOW_TABLES); + + if (show_tables_granted) + { + auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & db : databases) + { + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + { + StoragePtr storage = iterator->table(); + if (auto * s3queue_table = dynamic_cast(storage.get())) + { + const auto & table_id = s3queue_table->getStorageID(); + auto file_statuses = s3queue_table->getFileStatuses(); + for (const auto & [file_name, file_status] : file_statuses) + { + size_t i = 0; + res_columns[i++]->insert(table_id.database_name); + res_columns[i++]->insert(table_id.table_name); + res_columns[i++]->insert(file_name); + res_columns[i++]->insert(file_status->processed_rows); + res_columns[i++]->insert(magic_enum::enum_name(file_status->state)); + + if (file_status->processing_start_time) + res_columns[i++]->insert(file_status->processing_start_time); + else + res_columns[i++]->insertDefault(); + if (file_status->processing_end_time) + res_columns[i++]->insert(file_status->processing_end_time); + else + res_columns[i++]->insertDefault(); + + ProfileEvents::dumpToMapColumn(file_status->profile_counters.getPartiallyAtomicSnapshot(), res_columns[i++].get(), true); + } + } + } + } + + } +} + +} diff --git a/src/Storages/System/StorageSystemS3Queue.h b/src/Storages/System/StorageSystemS3Queue.h new file mode 100644 index 00000000000..eb6b1533dce --- /dev/null +++ b/src/Storages/System/StorageSystemS3Queue.h @@ -0,0 +1,22 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class StorageSystemS3Queue final : public IStorageSystemOneBlock +{ +public: + explicit StorageSystemS3Queue(const StorageID & table_id_); + + std::string getName() const override { return "SystemS3Queue"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index e4e19ce2e06..ffa225fb929 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -84,6 +84,7 @@ #include #include #include +#include #if USE_RDKAFKA #include @@ -196,6 +197,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "named_collections"); attach(context, system_database, "user_processes"); attach(context, system_database, "jemalloc_bins"); + attach(context, system_database, "s3queue"); if (has_zookeeper) { diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index ffcbdebde16..4c3e9b1753e 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -626,7 +626,6 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" files_to_generate = 300 - poll_size = 30 for i in range(3): table = f"{table_name}_{i + 1}" @@ -638,7 +637,6 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): mode, files_path, additional_settings={ - "s3queue_polling_size": poll_size, "keeper_path": keeper_path, }, ) @@ -700,7 +698,6 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" files_to_generate = 300 - poll_size = 2 for instance in [node, node_2]: create_table( @@ -710,7 +707,6 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): mode, files_path, additional_settings={ - "s3queue_polling_size": poll_size, "keeper_path": keeper_path, }, ) From 251b0165ca33672732e8e3eeec6025c55a7a3a0b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Sep 2023 22:55:41 +0200 Subject: [PATCH 051/634] Better --- .../table-engines/integrations/s3queue.md | 83 ++++++++++++------- src/Interpreters/S3QueueLog.cpp | 15 ++++ src/Interpreters/S3QueueLog.h | 4 + src/Interpreters/SystemLog.cpp | 2 + src/Storages/S3Queue/S3QueueSettings.h | 2 +- src/Storages/S3Queue/S3QueueSource.cpp | 12 ++- src/Storages/S3Queue/S3QueueSource.h | 2 +- src/Storages/S3Queue/StorageS3Queue.cpp | 50 +++++------ src/Storages/S3Queue/StorageS3Queue.h | 2 +- 9 files changed, 106 insertions(+), 66 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 76cc60ad4fd..6ceea8d7dc9 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -200,38 +200,6 @@ Example: SELECT * FROM stats ORDER BY name; ``` -## Introspection {#introspection} - -For introspection use `system.s3queue` stateless table and `system.s3_queue_log` persistent table. - -In order to use `system.s3_queue_log` define its configuration in server config file: - -``` xml - - system - s3_queue_log
-
-``` - -Example: - -``` sql -:) select * from system.s3queue - -SELECT * -FROM system.s3queue - -Query id: bb41964e-c947-4112-be3a-0f01770a1e84 - -┌─database─┬─table───┬─file_name──────────────────────────────────────────┬─rows_processed─┬─status─────┬─processing_start_time─┬─processing_end_time─┬─ProfileEvents──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ default │ s3queue │ hits_compatible/athena_partitioned/hits_60.parquet │ 0 │ Processing │ 2023-09-25 19:56:51 │ ᴺᵁᴸᴸ │ {'S3QueueSetFileProcessingMicroseconds':516} │ -│ default │ s3queue │ hits_compatible/athena_partitioned/hits_54.parquet │ 1000000 │ Processing │ 2023-09-25 19:56:50 │ ᴺᵁᴸᴸ │ {'SelectedRows':1000000,'SelectedBytes':1284181126,'S3ReadMicroseconds':385274,'S3ReadRequestsCount':3,'S3ReadRequestsErrors':1,'S3GetObject':1,'ReadBufferFromS3Microseconds':405970,'ReadBufferFromS3InitMicroseconds':385790,'ReadBufferFromS3Bytes':65536,'ReadBufferFromS3PreservedSessions':1,'S3QueueSetFileProcessingMicroseconds':567,'S3QueuePullMicroseconds':2475045} │ -└──────────┴─────────┴────────────────────────────────────────────────────┴────────────────┴────────────┴───────────────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ - - -SELECT * FROM system.s3_queue_log; -``` - ## Virtual columns {#virtual-columns} - `_path` — Path to the file. @@ -255,3 +223,54 @@ Constructions with `{}` are similar to the [remote](../../../sql-reference/table :::note If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: +## Introspection {#introspection} + +For introspection use `system.s3queue` stateless table and `system.s3queue_log` persistent table. + +The `s3queue` system table has the following structure: + +``` sql +┌─statement──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ CREATE TABLE system.s3queue +( + `database` String, + `table` String, + `file_name` String, + `rows_processed` UInt64, + `status` String, + `processing_start_time` Nullable(DateTime), + `processing_end_time` Nullable(DateTime), + `ProfileEvents` Map(String, UInt64) +) +ENGINE = SystemS3Queue +COMMENT 'SYSTEM TABLE is built on the fly.' │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +In order to use `system.s3queue_log` define its configuration in server config file: + +``` xml + + system + s3queue_log
+
+``` + +Example: + +``` sql +:) select * from system.s3queue + +SELECT * +FROM system.s3queue + +Query id: bb41964e-c947-4112-be3a-0f01770a1e84 + +┌─database─┬─table───┬─file_name──────────────────────────────────────────┬─rows_processed─┬─status─────┬─processing_start_time─┬─processing_end_time─┬─ProfileEvents──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ default │ s3queue │ hits_compatible/athena_partitioned/hits_60.parquet │ 0 │ Processing │ 2023-09-25 19:56:51 │ ᴺᵁᴸᴸ │ {'S3QueueSetFileProcessingMicroseconds':516} │ +│ default │ s3queue │ hits_compatible/athena_partitioned/hits_54.parquet │ 1000000 │ Processing │ 2023-09-25 19:56:50 │ ᴺᵁᴸᴸ │ {'SelectedRows':1000000,'SelectedBytes':1284181126,'S3ReadMicroseconds':385274,'S3ReadRequestsCount':3,'S3ReadRequestsErrors':1,'S3GetObject':1,'ReadBufferFromS3Microseconds':405970,'ReadBufferFromS3InitMicroseconds':385790,'ReadBufferFromS3Bytes':65536,'ReadBufferFromS3PreservedSessions':1,'S3QueueSetFileProcessingMicroseconds':567,'S3QueuePullMicroseconds':2475045} │ +└──────────┴─────────┴────────────────────────────────────────────────────┴────────────────┴────────────┴───────────────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + + +SELECT * FROM system.s3_queue_log; +``` diff --git a/src/Interpreters/S3QueueLog.cpp b/src/Interpreters/S3QueueLog.cpp index 963a4789d35..caccdabf503 100644 --- a/src/Interpreters/S3QueueLog.cpp +++ b/src/Interpreters/S3QueueLog.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -26,6 +27,9 @@ NamesAndTypesList S3QueueLogElement::getNamesAndTypes() {"file_name", std::make_shared()}, {"rows_processed", std::make_shared()}, {"status", status_datatype}, + {"processing_start_time", std::make_shared(std::make_shared())}, + {"processing_end_time", std::make_shared(std::make_shared())}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, }; } @@ -38,6 +42,17 @@ void S3QueueLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(file_name); columns[i++]->insert(rows_processed); columns[i++]->insert(magic_enum::enum_name(status)); + + if (processing_start_time) + columns[i++]->insert(processing_start_time); + else + columns[i++]->insertDefault(); + if (processing_end_time) + columns[i++]->insert(processing_end_time); + else + columns[i++]->insertDefault(); + + ProfileEvents::dumpToMapColumn(counters_snapshot, columns[i++].get(), true); } } diff --git a/src/Interpreters/S3QueueLog.h b/src/Interpreters/S3QueueLog.h index 3c99221026c..b28ef1b82fc 100644 --- a/src/Interpreters/S3QueueLog.h +++ b/src/Interpreters/S3QueueLog.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -20,6 +21,9 @@ struct S3QueueLogElement Failed, }; S3QueueStatus status; + ProfileEvents::Counters::Snapshot counters_snapshot; + time_t processing_start_time; + time_t processing_end_time; static std::string name() { return "S3QueueLog"; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index d2dd4eabaec..9e4129219cb 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -331,6 +331,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf logs.emplace_back(asynchronous_insert_log.get()); if (backup_log) logs.emplace_back(backup_log.get()); + if (s3_queue_log) + logs.emplace_back(s3_queue_log.get()); try { diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 6dbffecbadf..6d7ab4187a5 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -25,7 +25,7 @@ class ASTStorage; M(UInt32, s3queue_enable_logging_to_s3queue_log, 0, "Enable logging to system table system.s3queue_log", 0) \ M(UInt32, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt32, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ - M(UInt32, s3queue_polling_backoff_ms, 200, "Polling backoff", 0) \ + M(UInt32, s3queue_polling_backoff_ms, 1000, "Polling backoff", 0) \ M(UInt32, s3queue_tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \ M(UInt32, s3queue_cleanup_interval_min_ms, 10000, "For unordered mode. Polling backoff min for cleanup", 0) \ M(UInt32, s3queue_cleanup_interval_max_ms, 30000, "For unordered mode. Polling backoff max for cleanup", 0) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index eec5d34027a..f1bce2f2dcd 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -134,14 +134,15 @@ Chunk StorageS3QueueSource::generate() { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); files_metadata->setFileFailed(reader.getFile(), e.message()); - appendLogElement(reader.getFile(), processed_rows_from_file, false); + appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); throw; } files_metadata->setFileProcessed(reader.getFile()); applyActionAfterProcessing(reader.getFile()); - appendLogElement(reader.getFile(), processed_rows_from_file, true); + appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, true); + file_status.reset(); processed_rows_from_file = 0; if (shutdown_called) @@ -182,7 +183,7 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & path) } } -void StorageS3QueueSource::appendLogElement(const std::string & file_name, size_t processed_rows, bool processed) +void StorageS3QueueSource::appendLogElement(const std::string & filename, const S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed) { if (!s3_queue_log) return; @@ -190,9 +191,12 @@ void StorageS3QueueSource::appendLogElement(const std::string & file_name, size_ S3QueueLogElement elem { .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), - .file_name = file_name, + .file_name = filename, .rows_processed = processed_rows, .status = processed ? S3QueueLogElement::S3QueueStatus::Processed : S3QueueLogElement::S3QueueStatus::Failed, + .counters_snapshot = file_status_.profile_counters.getPartiallyAtomicSnapshot(), + .processing_start_time = file_status_.processing_start_time, + .processing_end_time = file_status_.processing_end_time, }; s3_queue_log->add(std::move(elem)); } diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index bca10f790c8..ce8a64022d0 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -82,7 +82,7 @@ private: std::shared_ptr file_status; void applyActionAfterProcessing(const String & path); - void appendLogElement(const std::string & file_name, size_t processed_rows, bool processed); + void appendLogElement(const std::string & filename, const S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); }; } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index f72af72123e..776d3c75ecc 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -34,8 +34,6 @@ namespace ProfileEvents namespace DB { -static const auto MAX_THREAD_WORK_DURATION_MS = 60000; - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -254,34 +252,31 @@ bool StorageS3Queue::hasDependencies(const StorageID & table_id) void StorageS3Queue::threadFunc() { - SCOPE_EXIT({ mv_attached.store(false); }); + if (shutdown_called) + return; + try { - auto table_id = getStorageID(); - size_t dependencies_count = DatabaseCatalog::instance().getDependentViews(table_id).size(); + const size_t dependencies_count = DatabaseCatalog::instance().getDependentViews(getStorageID()).size(); if (dependencies_count) { - auto start_time = std::chrono::steady_clock::now(); - /// Reset reschedule interval. - reschedule_processing_interval_ms = s3queue_settings->s3queue_polling_min_timeout_ms; - /// Disallow parallel selects while streaming to mv. mv_attached.store(true); + SCOPE_EXIT({ mv_attached.store(false); }); - /// Keep streaming as long as there are attached views and streaming is not cancelled - while (!shutdown_called && hasDependencies(table_id)) + LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + + if (streamToViews()) { - LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - - streamToViews(); - - auto now = std::chrono::steady_clock::now(); - auto duration = std::chrono::duration_cast(now - start_time); - if (duration.count() > MAX_THREAD_WORK_DURATION_MS) - { - LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); - break; - } + /// Reset the reschedule interval. + reschedule_processing_interval_ms = s3queue_settings->s3queue_polling_min_timeout_ms; } + else + { + /// Increase the reschedule interval. + reschedule_processing_interval_ms += s3queue_settings->s3queue_polling_backoff_ms; + } + + LOG_DEBUG(log, "Stopped streaming to {} attached views", dependencies_count); } else { @@ -295,15 +290,12 @@ void StorageS3Queue::threadFunc() if (!shutdown_called) { - if (reschedule_processing_interval_ms < s3queue_settings->s3queue_polling_max_timeout_ms) - reschedule_processing_interval_ms += s3queue_settings->s3queue_polling_backoff_ms; - LOG_TRACE(log, "Reschedule S3 Queue processing thread in {} ms", reschedule_processing_interval_ms); task->scheduleAfter(reschedule_processing_interval_ms); } } -void StorageS3Queue::streamToViews() +bool StorageS3Queue::streamToViews() { auto table_id = getStorageID(); auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); @@ -324,10 +316,12 @@ void StorageS3Queue::streamToViews() // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true); auto block_io = interpreter.execute(); + Pipes pipes; for (size_t i = 0; i < s3queue_settings->s3queue_processing_threads_num; ++i) { - auto source = createSource(block_io.pipeline.getHeader().getNames(), storage_snapshot, nullptr, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); + auto source = createSource( + block_io.pipeline.getHeader().getNames(), storage_snapshot, nullptr, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); pipes.emplace_back(std::move(source)); } auto pipe = Pipe::unitePipes(std::move(pipes)); @@ -337,6 +331,8 @@ void StorageS3Queue::streamToViews() block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); + + return rows > 0; } StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context) diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index ff6871ea70f..8ec8bfeeb0a 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -92,7 +92,7 @@ private: ContextPtr local_context); bool hasDependencies(const StorageID & table_id); - void streamToViews(); + bool streamToViews(); void threadFunc(); void createOrCheckMetadata(const StorageInMemoryMetadata & storage_metadata); From 4d78dbacfe4a62089284174b34207a37f662e3cf Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Sep 2023 23:38:16 +0200 Subject: [PATCH 052/634] Fix style check, fix build without s3 --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 1 - src/Storages/S3Queue/S3QueueSource.cpp | 1 + src/Storages/S3Queue/StorageS3Queue.cpp | 1 - src/Storages/System/StorageSystemS3Queue.cpp | 5 +++++ src/Storages/System/StorageSystemS3Queue.h | 5 +++++ 5 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 837d270f8fd..a9be348405f 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -35,7 +35,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int FILE_DOESNT_EXIST; } namespace diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index f1bce2f2dcd..e9a57cbbfd4 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -26,6 +26,7 @@ namespace DB namespace ErrorCodes { extern const int S3_ERROR; + extern const int NOT_IMPLEMENTED; } StorageS3QueueSource::FileIterator::FileIterator( diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 776d3c75ecc..0bc98006463 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -1,6 +1,5 @@ #include "config.h" - #if USE_AWS_S3 #include #include diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index 6d409e29ee4..6f320bbd5f1 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -1,4 +1,7 @@ #include "StorageSystemS3Queue.h" + +#if USE_AWS_S3 + #include #include #include @@ -82,3 +85,5 @@ void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr con } } + +#endif diff --git a/src/Storages/System/StorageSystemS3Queue.h b/src/Storages/System/StorageSystemS3Queue.h index eb6b1533dce..208891aef42 100644 --- a/src/Storages/System/StorageSystemS3Queue.h +++ b/src/Storages/System/StorageSystemS3Queue.h @@ -1,4 +1,7 @@ #pragma once +#include "config.h" + +#if USE_AWS_S3 #include #include @@ -20,3 +23,5 @@ protected: }; } + +#endif From 14b09d3cdcbcfbe62f9780b91eed2611b48958f1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 26 Sep 2023 16:23:24 +0200 Subject: [PATCH 053/634] Add caching --- src/Common/ProfileEvents.cpp | 1 + src/Interpreters/Cache/Metadata.cpp | 1 + src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 283 +++++++++++++----- src/Storages/S3Queue/S3QueueFilesMetadata.h | 37 ++- .../S3Queue/S3QueueMetadataFactory.cpp | 29 ++ src/Storages/S3Queue/S3QueueMetadataFactory.h | 26 ++ src/Storages/S3Queue/StorageS3Queue.cpp | 3 +- src/Storages/S3Queue/StorageS3Queue.h | 2 - src/Storages/System/StorageSystemS3Queue.cpp | 56 ++-- 9 files changed, 315 insertions(+), 123 deletions(-) create mode 100644 src/Storages/S3Queue/S3QueueMetadataFactory.cpp create mode 100644 src/Storages/S3Queue/S3QueueMetadataFactory.h diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index dc6a3108971..a8a3a732784 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -534,6 +534,7 @@ The server successfully detected this situation and will download merged part fr M(S3QueueSetFileFailedMicroseconds, "Time spent to set file as failed")\ M(S3QueueCleanupMaxSetSizeOrTTLMicroseconds, "Time spent to set file as failed")\ M(S3QueuePullMicroseconds, "Time spent to read file data")\ + M(S3QueueLockLocalFileStatusesMicroseconds, "Time spent to lock local file statuses")\ \ M(ServerStartupMilliseconds, "Time elapsed from starting server to listening to sockets in milliseconds")\ M(IOUringSQEsSubmitted, "Total number of io_uring SQEs submitted") \ diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 7a9321e4215..c251cb2dec2 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace fs = std::filesystem; diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index a9be348405f..009f5e3ad75 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -27,6 +27,7 @@ namespace ProfileEvents extern const Event S3QueueSetFileProcessedMicroseconds; extern const Event S3QueueSetFileFailedMicroseconds; extern const Event S3QueueCleanupMaxSetSizeOrTTLMicroseconds; + extern const Event S3QueueLockLocalFileStatusesMicroseconds; }; namespace DB @@ -35,6 +36,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } namespace @@ -53,41 +55,54 @@ namespace } } -S3QueueFilesMetadata::S3QueueFilesMetadata( - const StorageS3Queue * storage_, - const S3QueueSettings & settings_, - ContextPtr context) - : storage(storage_) - , mode(settings_.mode) - , max_set_size(settings_.s3queue_tracked_files_limit.value) - , max_set_age_sec(settings_.s3queue_tracked_file_ttl_sec.value) - , max_loading_retries(settings_.s3queue_loading_retries.value) - , min_cleanup_interval_ms(settings_.s3queue_cleanup_interval_min_ms.value) - , max_cleanup_interval_ms(settings_.s3queue_cleanup_interval_max_ms.value) - , zookeeper_processing_path(storage->getZooKeeperPath() / "processing") - , zookeeper_processed_path(storage->getZooKeeperPath() / "processed") - , zookeeper_failed_path(storage->getZooKeeperPath() / "failed") - , zookeeper_cleanup_lock_path(storage->getZooKeeperPath() / "cleanup_lock") - , log(&Poco::Logger::get("S3QueueFilesMetadata")) +std::unique_lock S3QueueFilesMetadata::LocalFileStatuses::lock() const { - if (mode == S3QueueMode::UNORDERED && (max_set_size || max_set_age_sec)) + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueLockLocalFileStatusesMicroseconds); + return std::unique_lock(mutex); +} + +S3QueueFilesMetadata::FileStatus::State S3QueueFilesMetadata::LocalFileStatuses::state(const std::string & filename) const +{ + auto lk = lock(); + if (auto it = file_statuses.find(filename); it != file_statuses.end()) + return it->second->state; + else + return FileStatus::State::None; +} + +S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::LocalFileStatuses::getAll() const +{ + auto lk = lock(); + return file_statuses; +} + +std::shared_ptr S3QueueFilesMetadata::LocalFileStatuses::get(const std::string & filename, bool create) +{ + auto lk = lock(); + auto it = file_statuses.find(filename); + if (it == file_statuses.end()) { - task = context->getSchedulePool().createTask("S3QueueCleanupFunc", [this] { cleanupThreadFunc(); }); - task->activate(); - task->scheduleAfter(generateRescheduleInterval(min_cleanup_interval_ms, max_cleanup_interval_ms)); + if (create) + it = file_statuses.emplace(filename, std::make_shared()).first; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "File status for {} doesn't exist", filename); } + return it->second; } -S3QueueFilesMetadata::~S3QueueFilesMetadata() +bool S3QueueFilesMetadata::LocalFileStatuses::remove(const std::string & filename, bool if_exists) { - deactivateCleanupTask(); -} - -void S3QueueFilesMetadata::deactivateCleanupTask() -{ - shutdown = true; - if (task) - task->deactivate(); + auto lk = lock(); + auto it = file_statuses.find(filename); + if (it == file_statuses.end()) + { + if (if_exists) + return false; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "File status for {} doesn't exist", filename); + } + file_statuses.erase(it); + return true; } std::string S3QueueFilesMetadata::NodeMetadata::toString() const @@ -117,6 +132,49 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::NodeMetadata::fromStrin return metadata; } +S3QueueFilesMetadata::S3QueueFilesMetadata(const fs::path & zookeeper_path_, const S3QueueSettings & settings_) + : mode(settings_.mode) + , max_set_size(settings_.s3queue_tracked_files_limit.value) + , max_set_age_sec(settings_.s3queue_tracked_file_ttl_sec.value) + , max_loading_retries(settings_.s3queue_loading_retries.value) + , min_cleanup_interval_ms(settings_.s3queue_cleanup_interval_min_ms.value) + , max_cleanup_interval_ms(settings_.s3queue_cleanup_interval_max_ms.value) + , zookeeper_processing_path(zookeeper_path_ / "processing") + , zookeeper_processed_path(zookeeper_path_ / "processed") + , zookeeper_failed_path(zookeeper_path_ / "failed") + , zookeeper_cleanup_lock_path(zookeeper_path_ / "cleanup_lock") + , log(&Poco::Logger::get("S3QueueFilesMetadata")) +{ + if (mode == S3QueueMode::UNORDERED && (max_set_size || max_set_age_sec)) + { + task = Context::getGlobalContextInstance()->getSchedulePool().createTask("S3QueueCleanupFunc", [this] { cleanupThreadFunc(); }); + task->activate(); + task->scheduleAfter(generateRescheduleInterval(min_cleanup_interval_ms, max_cleanup_interval_ms)); + } +} + +S3QueueFilesMetadata::~S3QueueFilesMetadata() +{ + deactivateCleanupTask(); +} + +void S3QueueFilesMetadata::deactivateCleanupTask() +{ + shutdown = true; + if (task) + task->deactivate(); +} + +zkutil::ZooKeeperPtr S3QueueFilesMetadata::getZooKeeper() const +{ + return Context::getGlobalContextInstance()->getZooKeeper(); +} + +std::shared_ptr S3QueueFilesMetadata::getFileStatus(const std::string & path) +{ + return local_file_statuses.get(path, /* create */false); +} + std::string S3QueueFilesMetadata::getNodeName(const std::string & path) { SipHash path_hash; @@ -137,23 +195,39 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( return metadata; } -std::shared_ptr S3QueueFilesMetadata::getFileStatus(const std::string & path) -{ - std::lock_guard lock(file_statuses_mutex); - return file_statuses.at(path); -} - -S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::getFileStateses() const -{ - std::lock_guard lock(file_statuses_mutex); - return file_statuses; -} - bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessingMicroseconds); - bool result; + /// Check locally cached file status. + switch (local_file_statuses.state(path)) + { + case FileStatus::State::Processing: [[fallthrough]]; + case FileStatus::State::Processed: + { + /// File is already processes or processing by current server. + return false; + } + case FileStatus::State::Failed: + { + if (!max_loading_retries) + { + /// File was processes by current server and failed, + /// retries are disabled. + return false; + } + /// TODO save information if file is still retriable. + break; + } + case FileStatus::State::None: + { + /// The file was not processed by current server, + /// check metadata in zookeeper. + break; + } + } + + SetFileProcessingResult result; switch (mode) { case S3QueueMode::ORDERED: @@ -167,21 +241,42 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) break; } } - if (result) + switch (result) { - std::lock_guard lock(file_statuses_mutex); - auto it = file_statuses.emplace(path, std::make_shared()).first; - auto & file_status = it->second; - file_status->state = FileStatus::State::Processing; - file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); - timer.cancel(); - if (!file_status->processing_start_time) - file_status->processing_start_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + case SetFileProcessingResult::Success: + { + auto file_status = local_file_statuses.get(path, /* create */true); + file_status->state = FileStatus::State::Processing; + file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); + timer.cancel(); + if (!file_status->processing_start_time) + file_status->processing_start_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + break; + } + case SetFileProcessingResult::AlreadyProcessed: + { + /// Cache the state. + auto file_status = local_file_statuses.get(path, /* create */true); + file_status->state = FileStatus::State::Processed; + break; + } + case SetFileProcessingResult::AlreadyFailed: + { + /// Cache the state. + auto file_status = local_file_statuses.get(path, /* create */true); + file_status->state = FileStatus::State::Failed; + break; + } + case SetFileProcessingResult::ProcessingByOtherNode: + { + /// We cannot save any local state. + break; + } } - return result; + return result == SetFileProcessingResult::Success; } -bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) +S3QueueFilesMetadata::SetFileProcessingResult S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) { /// Create an ephemenral node in /processing /// if corresponding node does not exist in failed/, processed/ and processing/. @@ -189,7 +284,7 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::str const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); - const auto zk_client = storage->getZooKeeper(); + const auto zk_client = getZooKeeper(); Coordination::Requests requests; zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_processed_path / node_name); @@ -198,10 +293,26 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::str Coordination::Responses responses; auto code = zk_client->tryMulti(requests, responses); - return code == Coordination::Error::ZOK; + + if (code == Coordination::Error::ZOK) + { + return SetFileProcessingResult::Success; + } + else if (responses[0]->error == Coordination::Error::ZOK) + { + if (responses[1]->error == Coordination::Error::ZOK) + { + chassert(responses[2]->error != Coordination::Error::ZOK); + return SetFileProcessingResult::ProcessingByOtherNode; + } + else + return SetFileProcessingResult::AlreadyFailed; + } + else + return SetFileProcessingResult::AlreadyProcessed; } -bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) +S3QueueFilesMetadata::SetFileProcessingResult S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) { /// Create an ephemenral node in /processing /// if corresponding it does not exist in failed/, processing/ and satisfied max processed file check. @@ -209,7 +320,7 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); - const auto zk_client = storage->getZooKeeper(); + const auto zk_client = getZooKeeper(); while (true) { @@ -221,9 +332,16 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin auto code = zk_client->tryMulti(requests, responses); if (code != Coordination::Error::ZOK) { - LOG_TEST(log, "Skipping file `{}`: {}", - path, responses[0]->error != Coordination::Error::ZOK ? "failed" : "processing"); - return false; + if (responses[0]->error == Coordination::Error::ZOK) + { + LOG_TEST(log, "Skipping file `{}`: already processing", path); + return SetFileProcessingResult::ProcessingByOtherNode; + } + else + { + LOG_TEST(log, "Skipping file `{}`: failed", path); + return SetFileProcessingResult::AlreadyFailed; + } } Coordination::Stat processed_node_stat; @@ -234,7 +352,7 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin auto max_processed_file_path = processed_node_metadata.file_path; if (!max_processed_file_path.empty() && path <= max_processed_file_path) - return false; + return SetFileProcessingResult::AlreadyProcessed; requests.clear(); responses.clear(); @@ -244,14 +362,17 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) - return true; + return SetFileProcessingResult::Success; - if (responses[0]->error != Coordination::Error::ZOK - || responses[1]->error != Coordination::Error::ZOK) + if (responses[0]->error != Coordination::Error::ZOK) { - LOG_TEST(log, "Skipping file `{}`: {}", - path, responses[0]->error != Coordination::Error::ZOK ? "failed" : "processing"); - return false; + LOG_TEST(log, "Skipping file `{}`: failed", path); + return SetFileProcessingResult::AlreadyFailed; + } + else if (responses[1]->error != Coordination::Error::ZOK) + { + LOG_TEST(log, "Skipping file `{}`: already processing", path); + return SetFileProcessingResult::ProcessingByOtherNode; } else { @@ -264,8 +385,7 @@ void S3QueueFilesMetadata::setFileProcessed(const String & path) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessedMicroseconds); SCOPE_EXIT({ - std::lock_guard lock(file_statuses_mutex); - auto & file_status = file_statuses.at(path); + auto file_status = local_file_statuses.get(path, /* create */false); file_status->state = FileStatus::State::Processed; file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessedMicroseconds, timer.get()); timer.cancel(); @@ -291,7 +411,7 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(const String & path) const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); - const auto zk_client = storage->getZooKeeper(); + const auto zk_client = getZooKeeper(); Coordination::Requests requests; requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); @@ -316,7 +436,7 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(const String & path) { const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); - const auto zk_client = storage->getZooKeeper(); + const auto zk_client = getZooKeeper(); while (true) { @@ -351,8 +471,7 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileFailedMicroseconds); SCOPE_EXIT_SAFE({ - std::lock_guard lock(file_statuses_mutex); - auto & file_status = file_statuses.at(path); + auto file_status = local_file_statuses.get(path, /* create */false); file_status->state = FileStatus::State::Failed; file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileFailedMicroseconds, timer.get()); timer.cancel(); @@ -361,7 +480,7 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc const auto node_name = getNodeName(path); auto node_metadata = createNodeMetadata(path, exception_message); - const auto zk_client = storage->getZooKeeper(); + const auto zk_client = getZooKeeper(); if (max_loading_retries == 0) { @@ -474,7 +593,7 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() const bool check_nodes_limit = max_set_size > 0; const bool check_nodes_ttl = max_set_age_sec > 0; - const auto zk_client = storage->getZooKeeper(); + const auto zk_client = getZooKeeper(); auto nodes = zk_client->getChildren(zookeeper_processed_path); if (nodes.empty()) { @@ -571,6 +690,8 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TEST(log, "Removing node at path {} ({}) because max files limit is reached", node.metadata.file_path, path.string()); + local_file_statuses.remove(node.metadata.file_path, /* if_exists */true); + code = zk_client->tryRemove(path); if (code == Coordination::Error::ZOK) --nodes_to_remove; @@ -586,6 +707,8 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TEST(log, "Removing node at path {} ({}) because file is reached", node.metadata.file_path, path.string()); + local_file_statuses.remove(node.metadata.file_path, /* if_exists */true); + code = zk_client->tryRemove(path); if (code != Coordination::Error::ZOK) LOG_ERROR(log, "Failed to remove a node `{}` (code: {})", path.string(), code); @@ -608,6 +731,16 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TRACE(log, "Node limits check finished"); } +bool S3QueueFilesMetadata::checkSettings(const S3QueueSettings & settings) const +{ + return mode == settings.mode + && max_set_size == settings.s3queue_tracked_files_limit.value + && max_set_age_sec == settings.s3queue_tracked_file_ttl_sec.value + && max_loading_retries == settings.s3queue_loading_retries.value + && min_cleanup_interval_ms == settings.s3queue_cleanup_interval_min_ms.value + && max_cleanup_interval_ms == settings.s3queue_cleanup_interval_max_ms.value; +} + } #endif diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index 079020514c6..5f3a6448e49 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace fs = std::filesystem; namespace Poco { class Logger; } @@ -18,7 +19,7 @@ class StorageS3Queue; class S3QueueFilesMetadata { public: - S3QueueFilesMetadata(const StorageS3Queue * storage_, const S3QueueSettings & settings_, ContextPtr context); + S3QueueFilesMetadata(const fs::path & zookeeper_path_, const S3QueueSettings & settings_); ~S3QueueFilesMetadata(); @@ -52,10 +53,11 @@ public: std::shared_ptr getFileStatus(const std::string & path); - FileStatuses getFileStateses() const; + FileStatuses getFileStateses() const { return local_file_statuses.getAll(); } + + bool checkSettings(const S3QueueSettings & settings) const; private: - const StorageS3Queue * storage; const S3QueueMode mode; const UInt64 max_set_size; const UInt64 max_set_age_sec; @@ -73,16 +75,22 @@ private: std::atomic_bool shutdown = false; BackgroundSchedulePool::TaskHolder task; - FileStatuses file_statuses; - mutable std::mutex file_statuses_mutex; + std::string getNodeName(const std::string & path); - bool trySetFileAsProcessingForOrderedMode(const std::string & path); - bool trySetFileAsProcessingForUnorderedMode(const std::string & path); + zkutil::ZooKeeperPtr getZooKeeper() const; void setFileProcessedForOrderedMode(const std::string & path); void setFileProcessedForUnorderedMode(const std::string & path); - std::string getNodeName(const std::string & path); + enum class SetFileProcessingResult + { + Success, + ProcessingByOtherNode, + AlreadyProcessed, + AlreadyFailed, + }; + SetFileProcessingResult trySetFileAsProcessingForOrderedMode(const std::string & path); + SetFileProcessingResult trySetFileAsProcessingForUnorderedMode(const std::string & path); struct NodeMetadata { @@ -99,6 +107,19 @@ private: void cleanupThreadFunc(); void cleanupThreadFuncImpl(); + + struct LocalFileStatuses + { + FileStatuses file_statuses; + mutable std::mutex mutex; + + FileStatuses getAll() const; + std::shared_ptr get(const std::string & filename, bool create); + bool remove(const std::string & filename, bool if_exists); + FileStatus::State state(const std::string & filename) const; + std::unique_lock lock() const; + }; + LocalFileStatuses local_file_statuses; }; } diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp new file mode 100644 index 00000000000..b1b0c4aef23 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp @@ -0,0 +1,29 @@ +#include + +namespace DB +{ + +S3QueueMetadataFactory & S3QueueMetadataFactory::instance() +{ + static S3QueueMetadataFactory ret; + return ret; +} + +S3QueueMetadataFactory::MetadataPtr +S3QueueMetadataFactory::getOrCreate(const std::string & zookeeper_path, const S3QueueSettings & settings) +{ + std::lock_guard lock(mutex); + auto it = metadata_by_path.find(zookeeper_path); + if (it == metadata_by_path.end()) + { + it = metadata_by_path.emplace(zookeeper_path, std::make_shared(fs::path(zookeeper_path), settings)).first; + } + else if (!it->second->checkSettings(settings)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Metadata with the same `s3queue_zookeeper_path` " + "was already created but with different settings"); + } + return it->second; +} + +} diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.h b/src/Storages/S3Queue/S3QueueMetadataFactory.h new file mode 100644 index 00000000000..e7a473d863d --- /dev/null +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.h @@ -0,0 +1,26 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class S3QueueMetadataFactory final : private boost::noncopyable +{ +public: + using MetadataPtr = std::shared_ptr; + using MetadataByPath = std::unordered_map; + + static S3QueueMetadataFactory & instance(); + + MetadataPtr getOrCreate(const std::string & zookeeper_path, const S3QueueSettings & settings); + + MetadataByPath getAll() { return metadata_by_path; } + +private: + MetadataByPath metadata_by_path; + std::mutex mutex; +}; + +} diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 0bc98006463..a15cc1bea9b 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -85,7 +86,7 @@ StorageS3Queue::StorageS3Queue( , s3queue_settings(std::move(s3queue_settings_)) , zk_path(chooseZooKeeperPath(table_id_, context_->getSettingsRef(), *s3queue_settings)) , after_processing(s3queue_settings->after_processing) - , files_metadata(std::make_shared(this, *s3queue_settings, context_)) + , files_metadata(S3QueueMetadataFactory::instance().getOrCreate(zk_path, *s3queue_settings)) , configuration{configuration_} , format_settings(format_settings_) , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 8ec8bfeeb0a..2a62078fcca 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -54,8 +54,6 @@ public: zkutil::ZooKeeperPtr getZooKeeper() const; - S3QueueFilesMetadata::FileStatuses getFileStatuses() const { return files_metadata->getFileStateses(); } - private: using FileIterator = StorageS3QueueSource::FileIterator; diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index 6f320bbd5f1..faba5bc671b 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -24,8 +25,7 @@ namespace DB NamesAndTypesList StorageSystemS3Queue::getNamesAndTypes() { return { - {"database", std::make_shared()}, - {"table", std::make_shared()}, + {"zookeeper_path", std::make_shared()}, {"file_name", std::make_shared()}, {"rows_processed", std::make_shared()}, {"status", std::make_shared()}, @@ -40,47 +40,29 @@ StorageSystemS3Queue::StorageSystemS3Queue(const StorageID & table_id_) { } -void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const { - const auto access = context->getAccess(); - const bool show_tables_granted = access->isGranted(AccessType::SHOW_TABLES); - - if (show_tables_granted) + for (const auto & [zookeeper_path, metadata] : S3QueueMetadataFactory::instance().getAll()) { - auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & db : databases) + for (const auto & [file_name, file_status] : metadata->getFileStateses()) { - for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) - { - StoragePtr storage = iterator->table(); - if (auto * s3queue_table = dynamic_cast(storage.get())) - { - const auto & table_id = s3queue_table->getStorageID(); - auto file_statuses = s3queue_table->getFileStatuses(); - for (const auto & [file_name, file_status] : file_statuses) - { - size_t i = 0; - res_columns[i++]->insert(table_id.database_name); - res_columns[i++]->insert(table_id.table_name); - res_columns[i++]->insert(file_name); - res_columns[i++]->insert(file_status->processed_rows); - res_columns[i++]->insert(magic_enum::enum_name(file_status->state)); + size_t i = 0; + res_columns[i++]->insert(zookeeper_path); + res_columns[i++]->insert(file_name); + res_columns[i++]->insert(file_status->processed_rows); + res_columns[i++]->insert(magic_enum::enum_name(file_status->state)); - if (file_status->processing_start_time) - res_columns[i++]->insert(file_status->processing_start_time); - else - res_columns[i++]->insertDefault(); - if (file_status->processing_end_time) - res_columns[i++]->insert(file_status->processing_end_time); - else - res_columns[i++]->insertDefault(); + if (file_status->processing_start_time) + res_columns[i++]->insert(file_status->processing_start_time); + else + res_columns[i++]->insertDefault(); + if (file_status->processing_end_time) + res_columns[i++]->insert(file_status->processing_end_time); + else + res_columns[i++]->insertDefault(); - ProfileEvents::dumpToMapColumn(file_status->profile_counters.getPartiallyAtomicSnapshot(), res_columns[i++].get(), true); - } - } - } + ProfileEvents::dumpToMapColumn(file_status->profile_counters.getPartiallyAtomicSnapshot(), res_columns[i++].get(), true); } - } } From b22b93c60ad1b44461fa29e8fb27b9ffee93dbc8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Sep 2023 18:27:17 +0000 Subject: [PATCH 054/634] More stable test. --- .../02597_projection_materialize_and_replication.sql | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql b/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql index 6e78e04a9e7..39f0f9091da 100644 --- a/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql +++ b/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql @@ -23,9 +23,13 @@ INSERT INTO test SELECT '2', '22', '22' FROM numbers(3); select * from test format Null; select min(c_id) from test group by d format Null; +set mutations_sync=0; +ALTER TABLE test UPDATE d = d || toString(sleepEachRow(0.1)) where 1; -ALTER TABLE test ADD PROJECTION d_order ( SELECT min(c_id) GROUP BY `d`), MATERIALIZE PROJECTION d_order, DROP PROJECTION d_order SETTINGS mutations_sync = 2; +ALTER TABLE test ADD PROJECTION d_order ( SELECT min(c_id) GROUP BY `d`); +ALTER TABLE test MATERIALIZE PROJECTION d_order; +ALTER TABLE test DROP PROJECTION d_order SETTINGS mutations_sync = 2; SELECT * FROM system.mutations WHERE database=currentDatabase() AND table='test' AND NOT is_done; From 5da03e8ba5c8812050e595171188f6383f46b647 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 27 Sep 2023 10:46:39 +0800 Subject: [PATCH 055/634] Add load balancing test_hostname_levenshtein_distance --- .../settings.md | 5 +- docs/en/operations/settings/settings.md | 20 +++++ programs/copier/Internals.cpp | 2 +- programs/server/users.yaml.example | 1 + src/Client/ConnectionPoolWithFailover.cpp | 6 +- src/Common/GetPriorityForLoadBalancing.cpp | 11 ++- src/Common/GetPriorityForLoadBalancing.h | 7 +- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 6 +- src/Common/isLocalAddress.cpp | 10 ++- src/Common/isLocalAddress.h | 6 +- src/Core/SettingsEnums.cpp | 1 + src/Core/SettingsEnums.h | 4 +- tests/config/config.d/zookeeper.xml | 2 +- .../test_distributed_load_balancing/test.py | 8 ++ .../test.py | 90 +++++++++++++++++++ 15 files changed, 162 insertions(+), 17 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 5586dbafd88..bd8e1da2f1e 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2403,7 +2403,8 @@ This section contains the following parameters: - zookeeper_load_balancing - Specifies the algorithm of ZooKeeper node selection. * random - randomly selects one of ZooKeeper nodes. * in_order - selects the first ZooKeeper node, if it's not available then the second, and so on. - * nearest_hostname - selects a ZooKeeper node with a hostname that is most similar to the server’s hostname. + * nearest_hostname - selects a ZooKeeper node with a hostname that is most similar to the server’s hostname, hostname is compared with name prefix. + * hostname_levenshtein_distance - just like nearest_hostname, but it compares hostname in a levenshtein distance manner. * first_or_random - selects the first ZooKeeper node, if it's not available then randomly selects one of remaining ZooKeeper nodes. * round_robin - selects the first ZooKeeper node, if reconnection happens selects the next. @@ -2425,7 +2426,7 @@ This section contains the following parameters: /path/to/zookeeper/node user:password - + random ``` diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index bf705c6e263..4e46d97102b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1413,6 +1413,7 @@ ClickHouse supports the following algorithms of choosing replicas: - [Random](#load_balancing-random) (by default) - [Nearest hostname](#load_balancing-nearest_hostname) +- [Hostname levenshtein distance](#load_balancing-hostname_levenshtein_distance) - [In order](#load_balancing-in_order) - [First or random](#load_balancing-first_or_random) - [Round robin](#load_balancing-round_robin) @@ -1444,6 +1445,25 @@ This method might seem primitive, but it does not require external data about ne Thus, if there are equivalent replicas, the closest one by name is preferred. We can also assume that when sending a query to the same server, in the absence of failures, a distributed query will also go to the same servers. So even if different data is placed on the replicas, the query will return mostly the same results. +### Hostname levenshtein distance {#load_balancing-hostname_levenshtein_distance} + +``` sql +load_balancing = hostname_levenshtein_distance +``` + +Just like `nearest_hostname`, but it compares hostname in a [levenshtein distance](https://en.wikipedia.org/wiki/Levenshtein_distance) manner. For example: + +``` text +example-clickhouse-0-0 ample-clickhouse-0-0 +1 + +example-clickhouse-0-0 example-clickhouse-1-10 +2 + +example-clickhouse-0-0 example-clickhouse-12-0 +3 +``` + ### In Order {#load_balancing-in_order} ``` sql diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index 3fa7090e7d7..0cfff7e3f6c 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -269,7 +269,7 @@ ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std res.hostname_difference = std::numeric_limits::max(); for (const auto & replica : replicas) { - size_t difference = getHostNameDifference(local_hostname, replica.host_name); + size_t difference = getHostNamePrefixDistance(local_hostname, replica.host_name); res.hostname_difference = std::min(difference, res.hostname_difference); } diff --git a/programs/server/users.yaml.example b/programs/server/users.yaml.example index ddd0ca4466a..afae8f2b1ff 100644 --- a/programs/server/users.yaml.example +++ b/programs/server/users.yaml.example @@ -9,6 +9,7 @@ profiles: # random - choose random replica from set of replicas with minimum number of errors # nearest_hostname - from set of replicas with minimum number of errors, choose replica # with minimum number of different symbols between replica's hostname and local hostname (Hamming distance). + # hostname_levenshtein_distance - just the same with nearest_hostname but calculate the difference by Levenshtein distance. # in_order - first live replica is chosen in specified order. # first_or_random - if first replica one has higher number of errors, pick a random one from replicas with minimum number of errors. load_balancing: random diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index bc93d1a44e1..4bbdc300d2b 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -34,11 +34,13 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover( { const std::string & local_hostname = getFQDNOrHostName(); - get_priority_load_balancing.hostname_differences.resize(nested_pools.size()); + get_priority_load_balancing.hostname_prefix_distance.resize(nested_pools.size()); + get_priority_load_balancing.hostname_levenshtein_distance.resize(nested_pools.size()); for (size_t i = 0; i < nested_pools.size(); ++i) { ConnectionPool & connection_pool = dynamic_cast(*nested_pools[i]); - get_priority_load_balancing.hostname_differences[i] = getHostNameDifference(local_hostname, connection_pool.getHost()); + get_priority_load_balancing.hostname_prefix_distance[i] = getHostNamePrefixDistance(local_hostname, connection_pool.getHost()); + get_priority_load_balancing.hostname_levenshtein_distance[i] = getHostNameLevenshteinDistance(local_hostname, connection_pool.getHost()); } } diff --git a/src/Common/GetPriorityForLoadBalancing.cpp b/src/Common/GetPriorityForLoadBalancing.cpp index c4d36acc70c..bc00e047a88 100644 --- a/src/Common/GetPriorityForLoadBalancing.cpp +++ b/src/Common/GetPriorityForLoadBalancing.cpp @@ -15,9 +15,14 @@ std::function GetPriorityForLoadBalancing::getPriorityFu switch (load_balance) { case LoadBalancing::NEAREST_HOSTNAME: - if (hostname_differences.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "It's a bug: hostname_differences is not initialized"); - get_priority = [this](size_t i) { return Priority{static_cast(hostname_differences[i])}; }; + if (hostname_prefix_distance.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "It's a bug: hostname_prefix_distance is not initialized"); + get_priority = [this](size_t i) { return Priority{static_cast(hostname_prefix_distance[i])}; }; + break; + case LoadBalancing::HOSTNAME_LEVENSHTEIN_DISTANCE: + if (hostname_levenshtein_distance.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "It's a bug: hostname_levenshtein_distance is not initialized"); + get_priority = [this](size_t i) { return Priority{static_cast(hostname_levenshtein_distance[i])}; }; break; case LoadBalancing::IN_ORDER: get_priority = [](size_t i) { return Priority{static_cast(i)}; }; diff --git a/src/Common/GetPriorityForLoadBalancing.h b/src/Common/GetPriorityForLoadBalancing.h index 8052185ac13..b4fdbbe9016 100644 --- a/src/Common/GetPriorityForLoadBalancing.h +++ b/src/Common/GetPriorityForLoadBalancing.h @@ -13,7 +13,9 @@ public: bool operator == (const GetPriorityForLoadBalancing & other) const { - return load_balancing == other.load_balancing && hostname_differences == other.hostname_differences; + return load_balancing == other.load_balancing + && hostname_prefix_distance == other.hostname_prefix_distance + && hostname_levenshtein_distance == other.hostname_levenshtein_distance; } bool operator != (const GetPriorityForLoadBalancing & other) const @@ -23,7 +25,8 @@ public: std::function getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const; - std::vector hostname_differences; /// Distances from name of this host to the names of hosts of pools. + std::vector hostname_prefix_distance; /// Prefix distances from name of this host to the names of hosts of pools. + std::vector hostname_levenshtein_distance; /// Levenshtein Distances from name of this host to the names of hosts of pools. LoadBalancing load_balancing = LoadBalancing::RANDOM; diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 5d01294e9b0..55ba2d02e55 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -39,12 +39,14 @@ ZooKeeperArgs::ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, c throw KeeperException::fromMessage(Coordination::Error::ZBADARGUMENTS, "Timeout cannot be negative"); /// init get_priority_load_balancing - get_priority_load_balancing.hostname_differences.resize(hosts.size()); + get_priority_load_balancing.hostname_prefix_distance.resize(hosts.size()); + get_priority_load_balancing.hostname_levenshtein_distance.resize(hosts.size()); const String & local_hostname = getFQDNOrHostName(); for (size_t i = 0; i < hosts.size(); ++i) { const String & node_host = hosts[i].substr(0, hosts[i].find_last_of(':')); - get_priority_load_balancing.hostname_differences[i] = DB::getHostNameDifference(local_hostname, node_host); + get_priority_load_balancing.hostname_prefix_distance[i] = DB::getHostNamePrefixDistance(local_hostname, node_host); + get_priority_load_balancing.hostname_levenshtein_distance[i] = DB::getHostNameLevenshteinDistance(local_hostname, node_host); } } diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index ab830e9bc6b..61483ae18f3 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -122,8 +122,16 @@ bool isLocalAddress(const Poco::Net::SocketAddress & address, UInt16 clickhouse_ return clickhouse_port == address.port() && isLocalAddress(address.host()); } +size_t getHostNamePrefixDistance(const std::string & local_hostname, const std::string & host) +{ + size_t hostname_difference = 0; + for (size_t i = 0; i < std::min(local_hostname.length(), host.length()); ++i) + if (local_hostname[i] != host[i]) + ++hostname_difference; + return hostname_difference; +} -size_t getHostNameDifference(const std::string & local_hostname, const std::string & host) +size_t getHostNameLevenshteinDistance(const std::string & local_hostname, const std::string & host) { return levenshteinDistance(local_hostname, host); } diff --git a/src/Common/isLocalAddress.h b/src/Common/isLocalAddress.h index 42977775b09..c070d7d3213 100644 --- a/src/Common/isLocalAddress.h +++ b/src/Common/isLocalAddress.h @@ -26,6 +26,8 @@ namespace DB bool isLocalAddress(const Poco::Net::SocketAddress & address); bool isLocalAddress(const Poco::Net::IPAddress & address); - /// Returns number of different bytes in hostnames, used for load balancing - size_t getHostNameDifference(const std::string & local_hostname, const std::string & host); + /// Returns host name difference with name prefix, used for load balancing + size_t getHostNamePrefixDistance(const std::string & local_hostname, const std::string & host); + /// Returns host name difference with Levenshtein Distance. + size_t getHostNameLevenshteinDistance(const std::string & local_hostname, const std::string & host); } diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index a30d8040f47..741d5967c35 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes IMPLEMENT_SETTING_ENUM(LoadBalancing, ErrorCodes::UNKNOWN_LOAD_BALANCING, {{"random", LoadBalancing::RANDOM}, {"nearest_hostname", LoadBalancing::NEAREST_HOSTNAME}, + {"hostname_levenshtein_distance", LoadBalancing::HOSTNAME_LEVENSHTEIN_DISTANCE}, {"in_order", LoadBalancing::IN_ORDER}, {"first_or_random", LoadBalancing::FIRST_OR_RANDOM}, {"round_robin", LoadBalancing::ROUND_ROBIN}}) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 034e4c8c887..54da26c3450 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -16,8 +16,10 @@ enum class LoadBalancing /// among replicas with a minimum number of errors selected randomly RANDOM = 0, /// a replica is selected among the replicas with the minimum number of errors - /// with the minimum number of distinguished characters in the replica name and local hostname + /// with the minimum number of distinguished characters in the replica name prefix and local hostname prefix NEAREST_HOSTNAME, + /// just like NEAREST_HOSTNAME, but it count distinguished characters in a levenshtein distance manner + HOSTNAME_LEVENSHTEIN_DISTANCE, // replicas with the same number of errors are accessed in the same order // as they are specified in the configuration. IN_ORDER, diff --git a/tests/config/config.d/zookeeper.xml b/tests/config/config.d/zookeeper.xml index 75b4a00fe67..9f984d4c544 100644 --- a/tests/config/config.d/zookeeper.xml +++ b/tests/config/config.d/zookeeper.xml @@ -1,6 +1,6 @@ - + random localhost diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index e879f09ccc1..729d1a83fc0 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -140,6 +140,14 @@ def test_load_balancing_nearest_hostname(): assert unique_nodes == set(["n1"]) +def test_load_balancing_hostname_levenshtein_distance(): + unique_nodes = set() + for _ in range(0, queries): + unique_nodes.add(get_node(n1, settings={"load_balancing": "hostname_levenshtein_distance"})) + assert len(unique_nodes) == 1, unique_nodes + assert unique_nodes == set(["n1"]) + + def test_load_balancing_in_order(): unique_nodes = set() for _ in range(0, queries): diff --git a/tests/integration/test_zookeeper_config_load_balancing/test.py b/tests/integration/test_zookeeper_config_load_balancing/test.py index 56af7513389..f17e0c3f03f 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test.py @@ -312,6 +312,96 @@ def test_nearest_hostname(started_cluster): change_balancing("nearest_hostname", "random", reload=False) +def test_hostname_levenshtein_distance(started_cluster): + try: + change_balancing("random", "hostname_levenshtein_distance") + print( + str( + node1.exec_in_container( + [ + "bash", + "-c", + "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", + ], + privileged=True, + user="root", + ) + ) + ) + assert ( + "1" + == str( + node1.exec_in_container( + [ + "bash", + "-c", + "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo1_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + ], + privileged=True, + user="root", + ) + ).strip() + ) + + print( + str( + node2.exec_in_container( + [ + "bash", + "-c", + "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", + ], + privileged=True, + user="root", + ) + ) + ) + assert ( + "1" + == str( + node2.exec_in_container( + [ + "bash", + "-c", + "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo2_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + ], + privileged=True, + user="root", + ) + ).strip() + ) + + print( + str( + node3.exec_in_container( + [ + "bash", + "-c", + "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", + ], + privileged=True, + user="root", + ) + ) + ) + assert ( + "1" + == str( + node3.exec_in_container( + [ + "bash", + "-c", + "lsof -a -i4 -i6 -itcp -w | grep 'testzookeeperconfigloadbalancing_zoo3_1.*testzookeeperconfigloadbalancing_default:2181' | grep ESTABLISHED | wc -l", + ], + privileged=True, + user="root", + ) + ).strip() + ) + finally: + change_balancing("hostname_levenshtein_distance", "random", reload=False) + + def test_round_robin(started_cluster): pm = PartitionManager() try: From f3bf1f8da987c58332fdf1b3e425d37debe8c5b8 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 27 Sep 2023 12:56:06 +0800 Subject: [PATCH 056/634] Fix code style --- tests/integration/test_distributed_load_balancing/test.py | 4 +++- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index 729d1a83fc0..d61cca6ce12 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -143,7 +143,9 @@ def test_load_balancing_nearest_hostname(): def test_load_balancing_hostname_levenshtein_distance(): unique_nodes = set() for _ in range(0, queries): - unique_nodes.add(get_node(n1, settings={"load_balancing": "hostname_levenshtein_distance"})) + unique_nodes.add( + get_node(n1, settings={"load_balancing": "hostname_levenshtein_distance"}) + ) assert len(unique_nodes) == 1, unique_nodes assert unique_nodes == set(["n1"]) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index d49e55b53d1..efdb2c48f46 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -432,6 +432,7 @@ Klickhouse Kolmogorov Kubernetes LDAP +Levenshtein LGPL LLDB LLVM's @@ -1708,6 +1709,7 @@ lemmatize lemmatized lengthUTF lessOrEquals +levenshtein lexicographically lgamma libFuzzer From 0d256a381a020b3b9822edc88489a39678744dd4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Wed, 20 Sep 2023 10:49:19 +0800 Subject: [PATCH 057/634] add doc fix tests modify code stype and spelling add tests and modify code style --- .../functions/other-functions.md | 39 +++++++++++++ src/Functions/getHttpHeader.cpp | 18 +++--- src/Interpreters/ClientInfo.h | 1 - src/Interpreters/Context.cpp | 8 +-- src/Interpreters/Context.h | 6 +- src/Server/HTTPHandler.cpp | 10 +--- ...new_functions_must_be_documented.reference | 1 + .../02884_getHttpHeaderFunction.reference | 10 +++- .../02884_getHttpHeaderFunction.sh | 57 ++++++++++++++++--- .../aspell-ignore/en/aspell-dict.txt | 1 + 10 files changed, 110 insertions(+), 41 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index bfbd26551d3..788bb6e92c5 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -68,6 +68,45 @@ WHERE macro = 'test'; └───────┴──────────────┘ ``` +## getHttpHeader +Returns the value of specified http header.If there is no such header or the request method is not http, it will return empty string. + +**Syntax** + +```sql +getHttpHeader(name); +``` + +**Arguments** + +- `name` — Http header name .[String](../../sql-reference/data-types/string.md#string) + +**Returned value** + +Value of the specified header. +Type:[String](../../sql-reference/data-types/string.md#string). + + +When we use `clickhouse-client` to execute this function, we'll always get empty string, because client doesn't use http protocol. +```sql +SELECT getHttpHeader('test') +``` +result: + +```text +┌─getHttpHeader('test')─┐ +│ │ +└───────────────────────┘ +``` +Try to use http request: +```shell +echo "select getHttpHeader('X-Clickhouse-User')" | curl -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' 'http://localhost:8123/' -d @- + +#result +default +``` + + ## FQDN Returns the fully qualified domain name of the ClickHouse server. diff --git a/src/Functions/getHttpHeader.cpp b/src/Functions/getHttpHeader.cpp index ee914dd40b9..70db08f0ef5 100644 --- a/src/Functions/getHttpHeader.cpp +++ b/src/Functions/getHttpHeader.cpp @@ -4,9 +4,8 @@ #include #include #include -#include "Common/CurrentThread.h" -#include -#include "Interpreters/ClientInfo.h" +#include +#include "Disks/DiskType.h" #include "Interpreters/Context_fwd.h" #include #include @@ -18,7 +17,6 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; - extern const int FUNCTION_NOT_ALLOWED; } namespace @@ -28,18 +26,18 @@ namespace * If there no such parameter or the method of request is not * http, the function will return empty string. */ -class FunctionGetHttpHeader : public IFunction +class FunctionGetHttpHeader : public IFunction, WithContext { private: public: - FunctionGetHttpHeader() = default; + explicit FunctionGetHttpHeader(ContextPtr context_): WithContext(context_) {} static constexpr auto name = "getHttpHeader"; - static FunctionPtr create(ContextPtr /*context*/) + static FunctionPtr create(ContextPtr context_) { - return std::make_shared(); + return std::make_shared(context_); } @@ -64,8 +62,8 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { - const auto & query_context = DB::CurrentThread::getQueryContext(); - const auto & method = query_context->getClientInfo().http_method; + const auto & client_info = getContext()->getClientInfo(); + const auto & method = client_info.http_method; const auto & headers = DB::CurrentThread::getQueryContext()->getClientInfo().headers; diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 5878f0b424e..4c3a6c9531e 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -98,7 +98,6 @@ public: /// For mysql and postgresql UInt64 connection_id = 0; Poco::Net::NameValueCollection headers; - void setHttpHeaders(const Poco::Net::NameValueCollection & _headers) { headers = _headers; } /// Comma separated list of forwarded IP addresses (from X-Forwarded-For for HTTP interface). /// It's expected that proxy appends the forwarded address to the end of the list. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 113d862e5d6..86bbf018492 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -86,7 +86,6 @@ #include #include #include -#include "Disks/ObjectStorages/S3/diskSettings.h" #include #include #include @@ -4016,12 +4015,9 @@ void Context::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String client_info.http_user_agent = http_user_agent; client_info.http_referer = http_referer; need_recalculate_access = true; - + if (!http_headers.empty()) - { - for (const auto & http_header : http_headers) - client_info.headers.set(http_header.first, http_header.second); - } + client_info.headers = http_headers; } void Context::setForwardedFor(const String & forwarded_for) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b985f45a091..472b2fd535c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,7 +1,7 @@ #pragma once #include -#include "Core/Types.h" +#include #ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD #include @@ -806,10 +806,6 @@ public: /// Storage of forbidden HTTP headers from config.xml void setHTTPHeaderFilter(const Poco::Util::AbstractConfiguration & config); const HTTPHeaderFilter & getHTTPHeaderFilter() const; - const Poco::Net::NameValueCollection & getHttpHeaders() const - { - return client_info.headers; - } /// The port that the server listens for executing SQL queries. UInt16 getTCPPort() const; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index f787d2abe45..617de29babe 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -501,11 +501,7 @@ bool HTTPHandler::authenticateUser( else if (request.getMethod() == HTTPServerRequest::HTTP_POST) http_method = ClientInfo::HTTPMethod::POST; - NameValueCollection http_headers; - for (const auto & it : request) - http_headers.set(it.first, it.second); - - session->setHttpClientInfo(http_method, request.get("User-Agent", ""), request.get("Referer", ""), http_headers); + session->setHttpClientInfo(http_method, request.get("User-Agent", ""), request.get("Referer", ""), request); session->setForwardedFor(request.get("X-Forwarded-For", "")); session->setQuotaClientKey(quota_key); @@ -585,10 +581,6 @@ void HTTPHandler::processQuery( session->makeSessionContext(); } - NameValueCollection headers; - for (auto it = request.begin(); it != request.end(); ++it) - headers.set(it->first, it->second); - auto context = session->makeQueryContext(); /// This parameter is used to tune the behavior of output formats (such as Native) for compatibility. diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 69f455773b0..3896b67b349 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -319,6 +319,7 @@ geoDistance geohashDecode geohashEncode geohashesInBox +getHttpHeader getMacro getOSKernelVersion getServerPort diff --git a/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference b/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference index 564a057086f..8db2cdd84ce 100644 --- a/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference +++ b/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference @@ -1,2 +1,10 @@ -Code: 81. DB::Exception: Database `02884_getHttpHeaderFunction` does not exist. (UNKNOWN_DATABASE) (version 23.9.1.1) default +default value1 value2 +default value1 value2 +default value1 value2 +1 default row1_value1 row1_value2 row1_value3 row1_value4 row1_value5 row1_value6 row1_value7 +2 default row2_value1 row2_value2 row2_value3 row2_value4 row2_value5 row2_value6 row2_value7 +3 +value_from_query_1 value_from_query_2 value_from_query_3 1 default row1_value1 row1_value2 row1_value3 row1_value4 row1_value5 row1_value6 row1_value7 +value_from_query_1 value_from_query_2 value_from_query_3 2 default row2_value1 row2_value2 row2_value3 row2_value4 row2_value5 row2_value6 row2_value7 +value_from_query_1 value_from_query_2 value_from_query_3 3 diff --git a/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh b/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh index b03b05f7cdb..6c78ca113cb 100755 --- a/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh +++ b/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh @@ -3,19 +3,58 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -db="02884_getHttpHeaderFunction" -$CLICKHOUSE_CLIENT -q "CREATE DATABASE IF NOT EXISTS ${db}" -$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS ${db}.get_http_header (id UInt32, header_value String DEFAULT getHttpHeader('X-Clickhouse-User')) Engine=Memory()" +echo "SELECT getHttpHeader('X-Clickhouse-User')" | curl -s -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' 'http://localhost:8123/' -d @- -#Insert data via tcp client -$CLICKHOUSE_CLIENT -q "INSERT INTO ${db}.get_http_header (id) values (1), (2)" +echo "SELECT getHttpHeader('X-Clickhouse-User'), getHttpHeader('key1'), getHttpHeader('key2')" | curl -s -H 'X-Clickhouse-User: default' \ + -H 'X-ClickHouse-Key: ' -H 'key1: value1' -H 'key2: value2' 'http://localhost:8123/' -d @- + +echo "SELECT getHttpHeader('X-Clickhouse-User'), getHttpHeader('key1'), getHttpHeader('key2')" | curl -s -H 'X-Clickhouse-User: default' \ + -H 'X-ClickHouse-Key: ' -H 'key1: value1' -H 'key2: value2' 'http://localhost:8123/' -d @- + +echo "SELECT getHttpHeader('X-' || 'Clickhouse' || '-User'), getHttpHeader('key1'), getHttpHeader('key2')" | curl -s -H 'X-Clickhouse-User: default' \ + -H 'X-ClickHouse-Key: ' -H 'key1: value1' -H 'key2: value2' 'http://localhost:8123/' -d @- + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02884_get_http_header" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS 02884_get_http_header + (id UInt32, + http_user String DEFAULT getHttpHeader('X-Clickhouse-User'), + http_key1 String DEFAULT getHttpHeader('http_header_key1'), + http_key2 String DEFAULT getHttpHeader('http_header_key2'), + http_key3 String DEFAULT getHttpHeader('http_header_key3'), + http_key4 String DEFAULT getHttpHeader('http_header_key4'), + http_key5 String DEFAULT getHttpHeader('http_header_key5'), + http_key6 String DEFAULT getHttpHeader('http_header_key6'), + http_key7 String DEFAULT getHttpHeader('http_header_key7') + ) + Engine=MergeTree() + ORDER BY id" #Insert data via http request -echo "INSERT INTO ${db}.get_http_header (id) values (3), (4)" | curl -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' 'http://localhost:8123/' -d @- +echo "INSERT INTO test.02884_get_http_header (id) values (1)" | curl -s -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' \ + -H 'http_header_key1: row1_value1'\ + -H 'http_header_key2: row1_value2'\ + -H 'http_header_key3: row1_value3'\ + -H 'http_header_key4: row1_value4'\ + -H 'http_header_key5: row1_value5'\ + -H 'http_header_key6: row1_value6'\ + -H 'http_header_key7: row1_value7' 'http://localhost:8123/' -d @- -$CLICKHOUSE_CLIENT -q "SELECT * FROM ${db}.get_http_header ORDER BY id;" -$CLICKHOUSE_CLIENT -q "DROP DATABASE ${db}" +echo "INSERT INTO test.02884_get_http_header (id) values (2)" | curl -s -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' \ + -H 'http_header_key1: row2_value1'\ + -H 'http_header_key2: row2_value2'\ + -H 'http_header_key3: row2_value3'\ + -H 'http_header_key4: row2_value4'\ + -H 'http_header_key5: row2_value5'\ + -H 'http_header_key6: row2_value6'\ + -H 'http_header_key7: row2_value7' 'http://localhost:8123/' -d @- -echo "SELECT getHttpHeader('X-Clickhouse-User')" | curl -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' 'http://localhost:8123/' -d @- +$CLICKHOUSE_CLIENT -q "SELECT id, http_user, http_key1, http_key2, http_key3, http_key4, http_key5, http_key6, http_key7 FROM test.02884_get_http_header ORDER BY id;" +#Insert data via tcp client +$CLICKHOUSE_CLIENT -q "INSERT INTO 02884_get_http_header (id) values (3)" +$CLICKHOUSE_CLIENT -q "SELECT * FROM 02884_get_http_header where id = 3" +echo "SELECT getHttpHeader('key_from_query_1'), getHttpHeader('key_from_query_2'), getHttpHeader('key_from_query_3'), * FROM test.02884_get_http_header ORDER BY id" | curl -s -H 'X-Clickhouse-User: default' \ + -H 'X-ClickHouse-Key: ' -H 'key_from_query_1: value_from_query_1' -H 'key_from_query_2: value_from_query_2' -H 'key_from_query_3: value_from_query_3' 'http://localhost:8123/' -d @- +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02884_get_http_header" diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index bcb971951e5..ad0ba889fda 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1505,6 +1505,7 @@ geohashesInBox geoip geospatial getMacro +getHttpHeader getOSKernelVersion getServerPort getSetting From 6c778d4b121f560e53253f260cc28a35875767b5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 12:05:13 +0200 Subject: [PATCH 058/634] Fix build without s3 --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 11 ++--------- src/Storages/S3Queue/S3QueueFilesMetadata.h | 3 --- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 009f5e3ad75..f61a200f8f0 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -1,12 +1,6 @@ #include -#include "Common/Exception.h" -#include "Common/ZooKeeper/Types.h" -#include "Common/scope_guard_safe.h" -#include "Interpreters/Context_fwd.h" -#include "Storages/S3Queue/S3QueueSettings.h" #include "config.h" -#if USE_AWS_S3 #include #include #include @@ -14,13 +8,13 @@ #include #include #include -#include #include #include #include #include #include + namespace ProfileEvents { extern const Event S3QueueSetFileProcessingMicroseconds; @@ -226,6 +220,7 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) break; } } + /// TODO lock file token not to go to keeper simultaneously from this server. SetFileProcessingResult result; switch (mode) @@ -742,5 +737,3 @@ bool S3QueueFilesMetadata::checkSettings(const S3QueueSettings & settings) const } } - -#endif diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index 5f3a6448e49..b109f584f5e 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -1,7 +1,6 @@ #pragma once #include "config.h" -#if USE_AWS_S3 #include #include #include @@ -123,5 +122,3 @@ private: }; } - -#endif From d77452c5616ff10f25f6c26ef3e75656c435d612 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 12:06:41 +0200 Subject: [PATCH 059/634] Fix style check --- src/Storages/S3Queue/S3QueueMetadataFactory.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp index b1b0c4aef23..0a2856776e2 100644 --- a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp @@ -2,6 +2,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} S3QueueMetadataFactory & S3QueueMetadataFactory::instance() { From 6b191a1afeb95006c8cf1bc7dcaa05b020f1a823 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 14:54:31 +0200 Subject: [PATCH 060/634] Better --- programs/copier/ClusterCopier.cpp | 2 +- src/Common/ZooKeeper/ZooKeeper.h | 29 +++- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 141 ++++++++---------- src/Storages/S3Queue/S3QueueFilesMetadata.h | 22 ++- src/Storages/S3Queue/S3QueueSource.cpp | 27 +++- src/Storages/S3Queue/S3QueueSource.h | 22 ++- src/Storages/StorageS3.cpp | 74 ++++----- src/Storages/StorageS3.h | 19 ++- src/Storages/StorageS3Cluster.cpp | 2 +- 9 files changed, 197 insertions(+), 141 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 556eca808f6..ccd8caf1c5a 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -391,7 +391,7 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee auto code = zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZOK || code == Coordination::Error::ZNODEEXISTS) - return std::make_shared(current_worker_path, *zookeeper, false, false, description); + return zkutil::EphemeralNodeHolder::existing(current_worker_path, *zookeeper); if (code == Coordination::Error::ZBADVERSION) { diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 4b598147301..d61156b31a9 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -644,11 +644,18 @@ class EphemeralNodeHolder public: using Ptr = std::shared_ptr; - EphemeralNodeHolder(const std::string & path_, ZooKeeper & zookeeper_, bool create, bool sequential, const std::string & data) + EphemeralNodeHolder(const std::string & path_, ZooKeeper & zookeeper_, bool create, bool try_create, bool sequential, const std::string & data) : path(path_), zookeeper(zookeeper_) { if (create) + { path = zookeeper.create(path, data, sequential ? CreateMode::EphemeralSequential : CreateMode::Ephemeral); + need_remove = created = true; + } + else if (try_create) + { + need_remove = created = Coordination::Error::ZOK == zookeeper.tryCreate(path, data, sequential ? CreateMode::EphemeralSequential : CreateMode::Ephemeral); + } } std::string getPath() const @@ -656,19 +663,32 @@ public: return path; } + bool isCreated() const + { + return created; + } + static Ptr create(const std::string & path, ZooKeeper & zookeeper, const std::string & data = "") { - return std::make_shared(path, zookeeper, true, false, data); + return std::make_shared(path, zookeeper, true, false, false, data); + } + + static Ptr tryCreate(const std::string & path, ZooKeeper & zookeeper, const std::string & data = "") + { + auto node = std::make_shared(path, zookeeper, false, true, false, data); + if (node->isCreated()) + return node; + return nullptr; } static Ptr createSequential(const std::string & path, ZooKeeper & zookeeper, const std::string & data = "") { - return std::make_shared(path, zookeeper, true, true, data); + return std::make_shared(path, zookeeper, true, false, true, data); } static Ptr existing(const std::string & path, ZooKeeper & zookeeper) { - return std::make_shared(path, zookeeper, false, false, ""); + return std::make_shared(path, zookeeper, false, false, false, ""); } void setAlreadyRemoved() @@ -702,6 +722,7 @@ private: ZooKeeper & zookeeper; CurrentMetrics::Increment metric_increment{CurrentMetrics::EphemeralNode}; bool need_remove = true; + bool created = false; }; using EphemeralNodeHolderPtr = EphemeralNodeHolder::Ptr; diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index f61a200f8f0..dbcc085575b 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -4,10 +4,13 @@ #include #include #include +#include #include #include #include +#include #include +#include #include #include #include @@ -55,15 +58,6 @@ std::unique_lock S3QueueFilesMetadata::LocalFileStatuses::lock() con return std::unique_lock(mutex); } -S3QueueFilesMetadata::FileStatus::State S3QueueFilesMetadata::LocalFileStatuses::state(const std::string & filename) const -{ - auto lk = lock(); - if (auto it = file_statuses.find(filename); it != file_statuses.end()) - return it->second->state; - else - return FileStatus::State::None; -} - S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::LocalFileStatuses::getAll() const { auto lk = lock(); @@ -106,6 +100,7 @@ std::string S3QueueFilesMetadata::NodeMetadata::toString() const json.set("last_processed_timestamp", getCurrentTime()); json.set("last_exception", last_exception); json.set("retries", retries); + json.set("processing_id", processing_id); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); @@ -123,6 +118,7 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::NodeMetadata::fromStrin metadata.last_processed_timestamp = json->getValue("last_processed_timestamp"); metadata.last_exception = json->getValue("last_exception"); metadata.retries = json->getValue("retries"); + metadata.processing_id = json->getValue("processing_id"); return metadata; } @@ -189,28 +185,28 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( return metadata; } -bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) +S3QueueFilesMetadata::ProcessingHolderPtr S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessingMicroseconds); + auto file_status = local_file_statuses.get(path, /* create */false); /// Check locally cached file status. - switch (local_file_statuses.state(path)) + switch (file_status->state) { case FileStatus::State::Processing: [[fallthrough]]; case FileStatus::State::Processed: { /// File is already processes or processing by current server. - return false; + return nullptr; } case FileStatus::State::Failed: { - if (!max_loading_retries) - { - /// File was processes by current server and failed, - /// retries are disabled. - return false; - } - /// TODO save information if file is still retriable. + /// max_loading_retries == 0 => file is not retriable. + /// file_status->retries is a cached value, so in case file_status->retries >= max_loading retries + /// we can fully rely that it is true, but in other case the value might be outdated, + /// but this is ok, we will recheck with zookeeper. + if (!max_loading_retries || file_status->retries >= max_loading_retries) + return nullptr; break; } case FileStatus::State::None: @@ -220,19 +216,25 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) break; } } - /// TODO lock file token not to go to keeper simultaneously from this server. + std::unique_lock lock(file_status->processing_lock, std::defer_lock); + if (!lock.try_lock()) + { + /// Another thread is already trying to set file as processing. + return nullptr; + } SetFileProcessingResult result; + ProcessingHolderPtr processing_holder; switch (mode) { case S3QueueMode::ORDERED: { - result = trySetFileAsProcessingForOrderedMode(path); + std::tie(result, processing_holder) = trySetFileAsProcessingForOrderedMode(path); break; } case S3QueueMode::UNORDERED: { - result = trySetFileAsProcessingForUnorderedMode(path); + std::tie(result, processing_holder) = trySetFileAsProcessingForUnorderedMode(path); break; } } @@ -240,7 +242,6 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) { case SetFileProcessingResult::Success: { - auto file_status = local_file_statuses.get(path, /* create */true); file_status->state = FileStatus::State::Processing; file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); timer.cancel(); @@ -251,14 +252,12 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) case SetFileProcessingResult::AlreadyProcessed: { /// Cache the state. - auto file_status = local_file_statuses.get(path, /* create */true); file_status->state = FileStatus::State::Processed; break; } case SetFileProcessingResult::AlreadyFailed: { /// Cache the state. - auto file_status = local_file_statuses.get(path, /* create */true); file_status->state = FileStatus::State::Failed; break; } @@ -268,54 +267,60 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) break; } } - return result == SetFileProcessingResult::Success; + + if (result != SetFileProcessingResult::Success) + return nullptr; + + return processing_holder; } -S3QueueFilesMetadata::SetFileProcessingResult S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) +std::pair +S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) { /// Create an ephemenral node in /processing /// if corresponding node does not exist in failed/, processed/ and processing/. /// Return false otherwise. const auto node_name = getNodeName(path); - const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = getZooKeeper(); + auto node_metadata = createNodeMetadata(path); + node_metadata.processing_id = getRandomASCIIString(10); Coordination::Requests requests; zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_processed_path / node_name); zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); - requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata, zkutil::CreateMode::Ephemeral)); + requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata.toString(), zkutil::CreateMode::Ephemeral)); Coordination::Responses responses; auto code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) { - return SetFileProcessingResult::Success; + auto holder = std::make_unique(node_metadata.processing_id, zookeeper_processing_path / node_name, zk_client); + return std::pair{SetFileProcessingResult::Success, std::move(holder)}; } - else if (responses[0]->error == Coordination::Error::ZOK) - { - if (responses[1]->error == Coordination::Error::ZOK) - { - chassert(responses[2]->error != Coordination::Error::ZOK); - return SetFileProcessingResult::ProcessingByOtherNode; - } - else - return SetFileProcessingResult::AlreadyFailed; - } - else - return SetFileProcessingResult::AlreadyProcessed; + + if (responses[0]->error != Coordination::Error::ZOK) + return std::pair{SetFileProcessingResult::AlreadyProcessed, nullptr}; + + if (responses[1]->error != Coordination::Error::ZOK) + return std::pair{SetFileProcessingResult::AlreadyFailed, nullptr}; + + chassert(responses[2]->error != Coordination::Error::ZOK); + return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; } -S3QueueFilesMetadata::SetFileProcessingResult S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) +std::pair +S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) { /// Create an ephemenral node in /processing /// if corresponding it does not exist in failed/, processing/ and satisfied max processed file check. /// Return false otherwise. const auto node_name = getNodeName(path); - const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = getZooKeeper(); + auto node_metadata = createNodeMetadata(path); + node_metadata.processing_id = getRandomASCIIString(10); while (true) { @@ -330,12 +335,12 @@ S3QueueFilesMetadata::SetFileProcessingResult S3QueueFilesMetadata::trySetFileAs if (responses[0]->error == Coordination::Error::ZOK) { LOG_TEST(log, "Skipping file `{}`: already processing", path); - return SetFileProcessingResult::ProcessingByOtherNode; + return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; } else { LOG_TEST(log, "Skipping file `{}`: failed", path); - return SetFileProcessingResult::AlreadyFailed; + return std::pair{SetFileProcessingResult::AlreadyFailed, nullptr}; } } @@ -347,27 +352,30 @@ S3QueueFilesMetadata::SetFileProcessingResult S3QueueFilesMetadata::trySetFileAs auto max_processed_file_path = processed_node_metadata.file_path; if (!max_processed_file_path.empty() && path <= max_processed_file_path) - return SetFileProcessingResult::AlreadyProcessed; + return std::pair{SetFileProcessingResult::AlreadyProcessed, nullptr}; requests.clear(); responses.clear(); zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); - requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata, zkutil::CreateMode::Ephemeral)); + requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata.toString(), zkutil::CreateMode::Ephemeral)); requests.push_back(zkutil::makeCheckRequest(zookeeper_processed_path, processed_node_stat.version)); code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) - return SetFileProcessingResult::Success; + { + auto holder = std::make_unique(node_metadata.processing_id, zookeeper_processing_path / node_name, zk_client); + return std::pair{SetFileProcessingResult::Success, std::move(holder)}; + } if (responses[0]->error != Coordination::Error::ZOK) { LOG_TEST(log, "Skipping file `{}`: failed", path); - return SetFileProcessingResult::AlreadyFailed; + return std::pair{SetFileProcessingResult::AlreadyFailed, nullptr}; } else if (responses[1]->error != Coordination::Error::ZOK) { LOG_TEST(log, "Skipping file `{}`: already processing", path); - return SetFileProcessingResult::ProcessingByOtherNode; + return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; } else { @@ -465,8 +473,8 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileFailedMicroseconds); - SCOPE_EXIT_SAFE({ - auto file_status = local_file_statuses.get(path, /* create */false); + auto file_status = local_file_statuses.get(path, /* create */false); + SCOPE_EXIT({ file_status->state = FileStatus::State::Failed; file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileFailedMicroseconds, timer.get()); timer.cancel(); @@ -505,6 +513,7 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc { auto failed_node_metadata = NodeMetadata::fromString(res); node_metadata.retries = failed_node_metadata.retries + 1; + file_status->retries = node_metadata.retries; } LOG_TEST(log, "File `{}` failed to process, try {}/{} (Error: {})", @@ -605,30 +614,12 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() /// Create a lock so that with distributed processing /// multiple nodes do not execute cleanup in parallel. - Coordination::Error code = zk_client->tryCreate(zookeeper_cleanup_lock_path, - toString(getCurrentTime()), - zkutil::CreateMode::Ephemeral); - if (code == Coordination::Error::ZNODEEXISTS) + auto ephemeral_node = zkutil::EphemeralNodeHolder::create(zookeeper_cleanup_lock_path, *zk_client, toString(getCurrentTime())); + if (!ephemeral_node) { LOG_TEST(log, "Cleanup is already being executed by another node"); return; } - else if (code != Coordination::Error::ZOK) - { - throw Coordination::Exception::fromPath(code, zookeeper_cleanup_lock_path); - } - - SCOPE_EXIT_SAFE({ - try - { - zk_client->remove(zookeeper_cleanup_lock_path); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - chassert(false); - } - }); struct Node { @@ -687,7 +678,7 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() local_file_statuses.remove(node.metadata.file_path, /* if_exists */true); - code = zk_client->tryRemove(path); + auto code = zk_client->tryRemove(path); if (code == Coordination::Error::ZOK) --nodes_to_remove; else @@ -704,7 +695,7 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() local_file_statuses.remove(node.metadata.file_path, /* if_exists */true); - code = zk_client->tryRemove(path); + auto code = zk_client->tryRemove(path); if (code != Coordination::Error::ZOK) LOG_ERROR(log, "Failed to remove a node `{}` (code: {})", path.string(), code); } diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index b109f584f5e..7dec4c11383 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -22,7 +22,17 @@ public: ~S3QueueFilesMetadata(); - bool trySetFileAsProcessing(const std::string & path); + struct ProcessingHolder + { + ProcessingHolder(const std::string & processing_id_, const std::string & zk_node_path_, zkutil::ZooKeeperPtr zk_client_) + : zk_client(zk_client_), zk_node_path(zk_node_path_), processing_id(processing_id_) {} + + zkutil::ZooKeeperPtr zk_client; + std::string zk_node_path; + std::string processing_id; + }; + using ProcessingHolderPtr = std::unique_ptr; + ProcessingHolderPtr trySetFileAsProcessing(const std::string & path); void setFileProcessed(const std::string & path); @@ -47,6 +57,10 @@ public: time_t processing_start_time = 0; time_t processing_end_time = 0; + + size_t retries = 0; + + std::mutex processing_lock; }; using FileStatuses = std::unordered_map>; @@ -88,8 +102,8 @@ private: AlreadyProcessed, AlreadyFailed, }; - SetFileProcessingResult trySetFileAsProcessingForOrderedMode(const std::string & path); - SetFileProcessingResult trySetFileAsProcessingForUnorderedMode(const std::string & path); + std::pair trySetFileAsProcessingForOrderedMode(const std::string & path); + std::pair trySetFileAsProcessingForUnorderedMode(const std::string & path); struct NodeMetadata { @@ -97,6 +111,7 @@ private: UInt64 last_processed_timestamp = 0; std::string last_exception; UInt64 retries = 0; + std::string processing_id; /// For ephemeral processing node. std::string toString() const; static NodeMetadata fromString(const std::string & metadata_str); @@ -115,7 +130,6 @@ private: FileStatuses getAll() const; std::shared_ptr get(const std::string & filename, bool create); bool remove(const std::string & filename, bool if_exists); - FileStatus::State state(const std::string & filename) const; std::unique_lock lock() const; }; LocalFileStatuses local_file_statuses; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index e9a57cbbfd4..ff1a8c86b64 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -29,24 +30,39 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +StorageS3QueueSource::S3QueueKeyWithInfo::S3QueueKeyWithInfo( + const std::string & key_, + std::optional info_, + std::unique_ptr processing_holder_, + std::shared_ptr file_status_) + : StorageS3Source::KeyWithInfo(key_, info_) + , processing_holder(std::move(processing_holder_)) + , file_status(file_status_) +{ +} + StorageS3QueueSource::FileIterator::FileIterator( std::shared_ptr metadata_, std::unique_ptr glob_iterator_) : metadata(metadata_) , glob_iterator(std::move(glob_iterator_)) { } -StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::FileIterator::next() +StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next() { /// List results in s3 are always returned in UTF-8 binary order. /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) while (true) { - KeyWithInfo val = glob_iterator->next(); - if (val.key.empty()) + KeyWithInfoPtr val = glob_iterator->next(); + + if (!val) return {}; - if (metadata->trySetFileAsProcessing(val.key)) - return val; + + if (auto processing_holder = metadata->trySetFileAsProcessing(val->key); processing_holder) + { + return std::make_shared(val->key, val->info, std::move(processing_holder), nullptr); + } } } @@ -77,6 +93,7 @@ StorageS3QueueSource::StorageS3QueueSource( , shutdown_called(shutdown_called_) , s3_queue_log(s3_queue_log_) , storage_id(storage_id_) + , s3_queue_user_id(fmt::format("{}:{}", CurrentThread::getQueryId(), getRandomASCIIString(8))) , remove_file_func(remove_file_func_) , log(&Poco::Logger::get("StorageS3QueueSource")) { diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index ce8a64022d0..634c0803465 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -19,19 +19,30 @@ class StorageS3QueueSource : public ISource, WithContext { public: using IIterator = StorageS3Source::IIterator; + using KeyWithInfoPtr = StorageS3Source::KeyWithInfoPtr; using GlobIterator = StorageS3Source::DisclosedGlobIterator; - using KeyWithInfo = StorageS3Source::KeyWithInfo; using ZooKeeperGetter = std::function; using RemoveFileFunc = std::function; + using Metadata = S3QueueFilesMetadata; + + struct S3QueueKeyWithInfo : public StorageS3Source::KeyWithInfo + { + S3QueueKeyWithInfo( + const std::string & key_, + std::optional info_, + std::unique_ptr processing_holder_, + std::shared_ptr file_status_); + + std::unique_ptr processing_holder; + std::shared_ptr file_status; + }; class FileIterator : public IIterator { public: - FileIterator( - std::shared_ptr metadata_, - std::unique_ptr glob_iterator_); + FileIterator(std::shared_ptr metadata_, std::unique_ptr glob_iterator_); - KeyWithInfo next() override; + KeyWithInfoPtr next() override; size_t estimatedKeysCount() override; @@ -71,6 +82,7 @@ private: const std::atomic & shutdown_called; const std::shared_ptr s3_queue_log; const StorageID storage_id; + const std::string s3_queue_user_id; RemoveFileFunc remove_file_func; Poco::Logger * log; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 3deb22bd32d..3f8919d0f05 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -161,7 +161,7 @@ public: /// We don't have to list bucket, because there is no asterisks. if (key_prefix.size() == globbed_uri.key.size()) { - buffer.emplace_back(globbed_uri.key, std::nullopt); + buffer.emplace_back(std::make_shared(globbed_uri.key, std::nullopt)); buffer_iter = buffer.begin(); is_finished = true; return; @@ -182,7 +182,7 @@ public: fillInternalBufferAssumeLocked(); } - KeyWithInfo next() + KeyWithInfoPtr next() { std::lock_guard lock(mutex); return nextAssumeLocked(); @@ -201,7 +201,7 @@ public: private: using ListObjectsOutcome = Aws::S3::Model::ListObjectsV2Outcome; - KeyWithInfo nextAssumeLocked() + KeyWithInfoPtr nextAssumeLocked() { if (buffer_iter != buffer.end()) { @@ -277,7 +277,7 @@ private: .last_modification_time = row.GetLastModified().Millis() / 1000, }; - temp_buffer.emplace_back(std::move(key), std::move(info)); + temp_buffer.emplace_back(std::make_shared(std::move(key), std::move(info))); } } @@ -289,7 +289,7 @@ private: if (!is_initialized) { - filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, fs::path(globbed_uri.bucket) / temp_buffer.front().key, getContext()); + filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, fs::path(globbed_uri.bucket) / temp_buffer.front()->key, getContext()); is_initialized = true; } @@ -298,7 +298,7 @@ private: std::vector paths; paths.reserve(temp_buffer.size()); for (const auto & key_with_info : temp_buffer) - paths.push_back(fs::path(globbed_uri.bucket) / key_with_info.key); + paths.push_back(fs::path(globbed_uri.bucket) / key_with_info->key); VirtualColumnUtils::filterByPathOrFile(temp_buffer, paths, query, virtual_columns, getContext(), filter_ast); } @@ -307,8 +307,8 @@ private: if (file_progress_callback) { - for (const auto & [_, info] : buffer) - file_progress_callback(FileProgress(0, info->size)); + for (const auto & key_with_info : buffer) + file_progress_callback(FileProgress(0, key_with_info->info->size)); } /// Set iterator only after the whole batch is processed @@ -371,7 +371,7 @@ StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( { } -StorageS3Source::KeyWithInfo StorageS3Source::DisclosedGlobIterator::next() +StorageS3Source::KeyWithInfoPtr StorageS3Source::DisclosedGlobIterator::next() { return pimpl->next(); } @@ -422,11 +422,11 @@ public: if (read_keys_) { for (const auto & key : keys) - read_keys_->push_back({key, {}}); + read_keys_->push_back(std::make_shared(key)); } } - KeyWithInfo next() + KeyWithInfoPtr next() { size_t current_index = index.fetch_add(1, std::memory_order_relaxed); if (current_index >= keys.size()) @@ -439,7 +439,7 @@ public: file_progress_callback(FileProgress(0, info->size)); } - return {key, info}; + return std::make_shared(key, info); } size_t objectsCount() @@ -476,7 +476,7 @@ StorageS3Source::KeysIterator::KeysIterator( { } -StorageS3Source::KeyWithInfo StorageS3Source::KeysIterator::next() +StorageS3Source::KeyWithInfoPtr StorageS3Source::KeysIterator::next() { return pimpl->next(); } @@ -502,14 +502,14 @@ StorageS3Source::ReadTaskIterator::ReadTaskIterator( pool.wait(); buffer.reserve(max_threads_count); for (auto & key_future : keys) - buffer.emplace_back(key_future.get(), std::nullopt); + buffer.emplace_back(std::make_shared(key_future.get(), std::nullopt)); } -StorageS3Source::KeyWithInfo StorageS3Source::ReadTaskIterator::next() +StorageS3Source::KeyWithInfoPtr StorageS3Source::ReadTaskIterator::next() { size_t current_index = index.fetch_add(1, std::memory_order_relaxed); if (current_index >= buffer.size()) - return {callback(), {}}; + return std::make_shared(callback()); return buffer[current_index]; } @@ -566,22 +566,22 @@ StorageS3Source::StorageS3Source( StorageS3Source::ReaderHolder StorageS3Source::createReader() { - KeyWithInfo key_with_info; + KeyWithInfoPtr key_with_info; do { key_with_info = (*file_iterator)(); - if (key_with_info.key.empty()) + if (!key_with_info) return {}; - if (!key_with_info.info) - key_with_info.info = S3::getObjectInfo(*client, bucket, key_with_info.key, version_id, request_settings); + if (!key_with_info->info) + key_with_info->info = S3::getObjectInfo(*client, bucket, key_with_info->key, version_id, request_settings); } - while (getContext()->getSettingsRef().s3_skip_empty_files && key_with_info.info->size == 0); + while (getContext()->getSettingsRef().s3_skip_empty_files && key_with_info->info->size == 0); QueryPipelineBuilder builder; std::shared_ptr source; std::unique_ptr read_buf; - std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files ? tryGetNumRowsFromCache(key_with_info) : std::nullopt; + std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files ? tryGetNumRowsFromCache(*key_with_info) : std::nullopt; if (num_rows_from_cache) { /// We should not return single chunk with all number of rows, @@ -594,8 +594,8 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() } else { - auto compression_method = chooseCompressionMethod(key_with_info.key, compression_hint); - read_buf = createS3ReadBuffer(key_with_info.key, key_with_info.info->size); + auto compression_method = chooseCompressionMethod(key_with_info->key, compression_hint); + read_buf = createS3ReadBuffer(key_with_info->key, key_with_info->info->size); auto input_format = FormatFactory::instance().getInput( format, @@ -639,7 +639,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); - return ReaderHolder{key_with_info, bucket, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{*key_with_info, bucket, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3Source::createReaderAsync() @@ -1494,7 +1494,7 @@ namespace { current_key_with_info = (*file_iterator)(); - if (current_key_with_info.key.empty()) + if (!current_key_with_info) { if (first) throw Exception( @@ -1506,6 +1506,8 @@ namespace return nullptr; } + chassert(!current_key_with_info->key.empty()); + /// S3 file iterator could get new keys after new iteration, check them in schema cache. if (getContext()->getSettingsRef().schema_inference_use_cache_for_s3 && read_keys.size() > prev_read_keys_size) { @@ -1515,15 +1517,15 @@ namespace return nullptr; } - if (getContext()->getSettingsRef().s3_skip_empty_files && current_key_with_info.info && current_key_with_info.info->size == 0) + if (getContext()->getSettingsRef().s3_skip_empty_files && current_key_with_info->info && current_key_with_info->info->size == 0) continue; int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); - auto impl = std::make_unique(configuration.client, configuration.url.bucket, current_key_with_info.key, configuration.url.version_id, configuration.request_settings, getContext()->getReadSettings()); + auto impl = std::make_unique(configuration.client, configuration.url.bucket, current_key_with_info->key, configuration.url.version_id, configuration.request_settings, getContext()->getReadSettings()); if (!getContext()->getSettingsRef().s3_skip_empty_files || !impl->eof()) { first = false; - return wrapReadBufferWithCompressionMethod(std::move(impl), chooseCompressionMethod(current_key_with_info.key, configuration.compression_method), zstd_window_log_max); + return wrapReadBufferWithCompressionMethod(std::move(impl), chooseCompressionMethod(current_key_with_info->key, configuration.compression_method), zstd_window_log_max); } } } @@ -1538,7 +1540,7 @@ namespace if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3) return; - String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket / current_key_with_info.key; + String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket / current_key_with_info->key; auto key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); StorageS3::getSchemaCache(getContext()).addNumRows(key, num_rows); } @@ -1549,7 +1551,7 @@ namespace const StorageS3::Configuration & configuration; const std::optional & format_settings; std::optional columns_from_cache; - StorageS3Source::KeyWithInfo current_key_with_info; + StorageS3Source::KeyWithInfoPtr current_key_with_info; size_t prev_read_keys_size; bool first = true; }; @@ -1689,9 +1691,9 @@ std::optional StorageS3::tryGetColumnsFromCache( auto get_last_mod_time = [&] { time_t last_modification_time = 0; - if (it->info) + if ((*it)->info) { - last_modification_time = it->info->last_modification_time; + last_modification_time = (*it)->info->last_modification_time; } else { @@ -1701,7 +1703,7 @@ std::optional StorageS3::tryGetColumnsFromCache( last_modification_time = S3::getObjectInfo( *configuration.client, configuration.url.bucket, - it->key, + (*it)->key, configuration.url.version_id, configuration.request_settings, /*with_metadata=*/ false, @@ -1712,7 +1714,7 @@ std::optional StorageS3::tryGetColumnsFromCache( return last_modification_time ? std::make_optional(last_modification_time) : std::nullopt; }; - String path = fs::path(configuration.url.bucket) / it->key; + String path = fs::path(configuration.url.bucket) / (*it)->key; String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / path; auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, ctx); auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); @@ -1734,7 +1736,7 @@ void StorageS3::addColumnsToCache( auto host_and_bucket = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket; Strings sources; sources.reserve(keys.size()); - std::transform(keys.begin(), keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket / elem.key; }); + std::transform(keys.begin(), keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket / elem->key; }); auto cache_keys = getKeysForSchemaCache(sources, format_name, format_settings, ctx); auto & schema_cache = getSchemaCache(ctx); schema_cache.addManyColumns(cache_keys, columns); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 088f9000ce8..38cf3a5f65b 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -43,22 +43,21 @@ public: struct KeyWithInfo { KeyWithInfo() = default; - KeyWithInfo(String key_, std::optional info_) - : key(std::move(key_)), info(std::move(info_)) - { - } + explicit KeyWithInfo(String key_, std::optional info_ = std::nullopt) + : key(std::move(key_)), info(std::move(info_)) {} String key; std::optional info; }; + using KeyWithInfoPtr = std::shared_ptr; - using KeysWithInfo = std::vector; + using KeysWithInfo = std::vector; class IIterator { public: virtual ~IIterator() = default; - virtual KeyWithInfo next() = 0; + virtual KeyWithInfoPtr next() = 0; /// Estimates how many streams we need to process all files. /// If keys count >= max_threads_count, the returned number may not represent the actual number of the keys. @@ -66,7 +65,7 @@ public: /// fixme: May underestimate if the glob has a strong filter, so there are few matches among the first 1000 ListObjects results. virtual size_t estimatedKeysCount() = 0; - KeyWithInfo operator ()() { return next(); } + KeyWithInfoPtr operator ()() { return next(); } }; class DisclosedGlobIterator : public IIterator @@ -82,7 +81,7 @@ public: const S3Settings::RequestSettings & request_settings_ = {}, std::function progress_callback_ = {}); - KeyWithInfo next() override; + KeyWithInfoPtr next() override; size_t estimatedKeysCount() override; private: @@ -106,7 +105,7 @@ public: KeysWithInfo * read_keys = nullptr, std::function progress_callback_ = {}); - KeyWithInfo next() override; + KeyWithInfoPtr next() override; size_t estimatedKeysCount() override; private: @@ -120,7 +119,7 @@ public: public: explicit ReadTaskIterator(const ReadTaskCallback & callback_, const size_t max_threads_count); - KeyWithInfo next() override; + KeyWithInfoPtr next() override; size_t estimatedKeysCount() override; private: diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index c8715938c6f..1dd1a410595 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -82,7 +82,7 @@ RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(ASTPtr { auto iterator = std::make_shared( *s3_configuration.client, s3_configuration.url, query, virtual_columns, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback()); - auto callback = std::make_shared>([iterator]() mutable -> String { return iterator->next().key; }); + auto callback = std::make_shared>([iterator]() mutable -> String { return iterator->next()->key; }); return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } From f753b91a3b64d71794b9d15a0fde1dd473e47e49 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 17:17:52 +0200 Subject: [PATCH 061/634] Better maintenance of processing node --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 132 ++++++++++++++---- src/Storages/S3Queue/S3QueueFilesMetadata.h | 52 ++++--- .../S3Queue/S3QueueMetadataFactory.cpp | 43 +++++- src/Storages/S3Queue/S3QueueMetadataFactory.h | 18 ++- src/Storages/S3Queue/S3QueueSource.cpp | 16 ++- src/Storages/S3Queue/S3QueueSource.h | 6 +- src/Storages/S3Queue/StorageS3Queue.cpp | 12 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3.h | 12 +- 9 files changed, 216 insertions(+), 77 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index dbcc085575b..0492a8c2f4d 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -1,4 +1,3 @@ -#include #include "config.h" #include @@ -25,6 +24,7 @@ namespace ProfileEvents extern const Event S3QueueSetFileFailedMicroseconds; extern const Event S3QueueCleanupMaxSetSizeOrTTLMicroseconds; extern const Event S3QueueLockLocalFileStatusesMicroseconds; + extern const Event CannotRemoveEphemeralNode; }; namespace DB @@ -118,7 +118,7 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::NodeMetadata::fromStrin metadata.last_processed_timestamp = json->getValue("last_processed_timestamp"); metadata.last_exception = json->getValue("last_exception"); metadata.retries = json->getValue("retries"); - metadata.processing_id = json->getValue("processing_id"); + metadata.processing_id = json->getValue("processing_id"); return metadata; } @@ -185,10 +185,10 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( return metadata; } -S3QueueFilesMetadata::ProcessingHolderPtr S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) +S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessingMicroseconds); - auto file_status = local_file_statuses.get(path, /* create */false); + auto file_status = local_file_statuses.get(path, /* create */true); /// Check locally cached file status. switch (file_status->state) @@ -224,7 +224,7 @@ S3QueueFilesMetadata::ProcessingHolderPtr S3QueueFilesMetadata::trySetFileAsProc } SetFileProcessingResult result; - ProcessingHolderPtr processing_holder; + ProcessingNodeHolderPtr processing_holder; switch (mode) { case S3QueueMode::ORDERED: @@ -274,7 +274,7 @@ S3QueueFilesMetadata::ProcessingHolderPtr S3QueueFilesMetadata::trySetFileAsProc return processing_holder; } -std::pair +std::pair S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) { /// Create an ephemenral node in /processing @@ -296,7 +296,7 @@ S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & if (code == Coordination::Error::ZOK) { - auto holder = std::make_unique(node_metadata.processing_id, zookeeper_processing_path / node_name, zk_client); + auto holder = std::make_unique(node_metadata.processing_id, path, zookeeper_processing_path / node_name, zk_client); return std::pair{SetFileProcessingResult::Success, std::move(holder)}; } @@ -310,7 +310,7 @@ S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; } -std::pair +std::pair S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) { /// Create an ephemenral node in /processing @@ -356,6 +356,7 @@ S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & p requests.clear(); responses.clear(); + zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata.toString(), zkutil::CreateMode::Ephemeral)); requests.push_back(zkutil::makeCheckRequest(zookeeper_processed_path, processed_node_stat.version)); @@ -363,7 +364,7 @@ S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & p code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) { - auto holder = std::make_unique(node_metadata.processing_id, zookeeper_processing_path / node_name, zk_client); + auto holder = std::make_unique(node_metadata.processing_id, path, zookeeper_processing_path / node_name, zk_client); return std::pair{SetFileProcessingResult::Success, std::move(holder)}; } @@ -384,9 +385,11 @@ S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & p } } -void S3QueueFilesMetadata::setFileProcessed(const String & path) +void S3QueueFilesMetadata::setFileProcessed(ProcessingNodeHolderPtr holder) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessedMicroseconds); + const auto & path = holder->path; + SCOPE_EXIT({ auto file_status = local_file_statuses.get(path, /* create */false); file_status->state = FileStatus::State::Processed; @@ -399,44 +402,44 @@ void S3QueueFilesMetadata::setFileProcessed(const String & path) { case S3QueueMode::ORDERED: { - return setFileProcessedForOrderedMode(path); + return setFileProcessedForOrderedMode(holder); } case S3QueueMode::UNORDERED: { - return setFileProcessedForUnorderedMode(path); + return setFileProcessedForUnorderedMode(holder); } } } -void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(const String & path) +void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolderPtr holder) { /// Create a persistent node in /processed and remove ephemeral node from /processing. + const auto & path = holder->path; const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = getZooKeeper(); Coordination::Requests requests; - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); requests.push_back(zkutil::makeCreateRequest(zookeeper_processed_path / node_name, node_metadata, zkutil::CreateMode::Persistent)); Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) + if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); return; } /// TODO this could be because of the expired session. - if (responses[0]->error != Coordination::Error::ZOK) + if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is not processing"); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is already processed"); } -void S3QueueFilesMetadata::setFileProcessedForOrderedMode(const String & path) +void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPtr holder) { + const auto & path = holder->path; const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = getZooKeeper(); @@ -463,15 +466,22 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(const String & path) } Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) + if (holder->remove(&requests, &responses)) return; + + if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) + continue; + + LOG_ERROR(log, "Cannot set file {} as failed - failed to remove ephemeral processing node", path); + chassert(false); + return; } } -void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exception_message) +void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const String & exception_message) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileFailedMicroseconds); + const auto & path = holder->path; auto file_status = local_file_statuses.get(path, /* create */false); SCOPE_EXIT({ @@ -488,14 +498,12 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc if (max_loading_retries == 0) { Coordination::Requests requests; - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, node_metadata.toString(), zkutil::CreateMode::Persistent)); Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) + if (holder->remove(&requests, &responses)) { LOG_TEST(log, "File `{}` failed to process and will not be retried. " "Error: {}", path, exception_message); @@ -565,6 +573,82 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc } } +S3QueueFilesMetadata::ProcessingNodeHolder::ProcessingNodeHolder( + const std::string & processing_id_, + const std::string & path_, + const std::string & zk_node_path_, + zkutil::ZooKeeperPtr zk_client_) + : zk_client(zk_client_) + , path(path_) + , zk_node_path(zk_node_path_) + , processing_id(processing_id_) + , log(&Poco::Logger::get("ProcessingNodeHolder")) +{ +} + +S3QueueFilesMetadata::ProcessingNodeHolder::~ProcessingNodeHolder() +{ + if (!removed) + remove(); +} + +bool S3QueueFilesMetadata::ProcessingNodeHolder::remove(Coordination::Requests * requests, Coordination::Responses * responses) +{ + if (removed) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Processing file holder is already released"); + + removed = true; + + try + { + if (!zk_client->expired()) + { + /// Is is possible that we created an ephemeral processing node + /// but session expired and someone other created an ephemeral processing node. + /// To avoid deleting this new node, check processing_id. + std::string res; + Coordination::Stat stat; + if (zk_client->tryGet(zk_node_path, res, &stat)) + { + auto node_metadata = NodeMetadata::fromString(res); + if (node_metadata.processing_id == processing_id) + { + if (requests) + { + requests->push_back(zkutil::makeRemoveRequest(zk_node_path, stat.version)); + auto code = zk_client->tryMulti(*requests, *responses); + return code == Coordination::Error::ZOK; + } + else + zk_client->remove(zk_node_path); + return true; + } + else + LOG_WARNING(log, "Cannot remove {} since precessing id changed: {} -> {}", + zk_node_path, processing_id, node_metadata.processing_id); + } + else + LOG_DEBUG(log, "Cannot remove {}, node doesn't exist, " + "probably because of session expiration", zk_node_path); + + /// TODO: this actually would mean that we already processed (or partially processed) + /// the data but another thread will try processing it again and data can be duplicated. + /// This can be solved via persistenly saving last processed offset in the file. + } + else + { + ProfileEvents::increment(ProfileEvents::CannotRemoveEphemeralNode); + LOG_DEBUG(log, "Cannot remove {} since session has been expired", zk_node_path); + } + } + catch (...) + { + ProfileEvents::increment(ProfileEvents::CannotRemoveEphemeralNode); + DB::tryLogCurrentException(__PRETTY_FUNCTION__, "Cannot remove " + zk_node_path); + } + return false; +} + void S3QueueFilesMetadata::cleanupThreadFunc() { /// A background task is responsible for maintaining diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index 7dec4c11383..dd760b29299 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -18,29 +18,18 @@ class StorageS3Queue; class S3QueueFilesMetadata { public: + class ProcessingNodeHolder; + using ProcessingNodeHolderPtr = std::shared_ptr; + S3QueueFilesMetadata(const fs::path & zookeeper_path_, const S3QueueSettings & settings_); ~S3QueueFilesMetadata(); - struct ProcessingHolder - { - ProcessingHolder(const std::string & processing_id_, const std::string & zk_node_path_, zkutil::ZooKeeperPtr zk_client_) - : zk_client(zk_client_), zk_node_path(zk_node_path_), processing_id(processing_id_) {} + ProcessingNodeHolderPtr trySetFileAsProcessing(const std::string & path); - zkutil::ZooKeeperPtr zk_client; - std::string zk_node_path; - std::string processing_id; - }; - using ProcessingHolderPtr = std::unique_ptr; - ProcessingHolderPtr trySetFileAsProcessing(const std::string & path); + void setFileProcessed(ProcessingNodeHolderPtr holder); - void setFileProcessed(const std::string & path); - - void setFileFailed(const std::string & path, const std::string & exception_message); - - using OnProgress = std::function; - - void deactivateCleanupTask(); + void setFileFailed(ProcessingNodeHolderPtr holder, const std::string & exception_message); struct FileStatus { @@ -70,6 +59,8 @@ public: bool checkSettings(const S3QueueSettings & settings) const; + void deactivateCleanupTask(); + private: const S3QueueMode mode; const UInt64 max_set_size; @@ -92,8 +83,8 @@ private: zkutil::ZooKeeperPtr getZooKeeper() const; - void setFileProcessedForOrderedMode(const std::string & path); - void setFileProcessedForUnorderedMode(const std::string & path); + void setFileProcessedForOrderedMode(ProcessingNodeHolderPtr holder); + void setFileProcessedForUnorderedMode(ProcessingNodeHolderPtr holder); enum class SetFileProcessingResult { @@ -102,8 +93,8 @@ private: AlreadyProcessed, AlreadyFailed, }; - std::pair trySetFileAsProcessingForOrderedMode(const std::string & path); - std::pair trySetFileAsProcessingForUnorderedMode(const std::string & path); + std::pair trySetFileAsProcessingForOrderedMode(const std::string & path); + std::pair trySetFileAsProcessingForUnorderedMode(const std::string & path); struct NodeMetadata { @@ -135,4 +126,23 @@ private: LocalFileStatuses local_file_statuses; }; +class S3QueueFilesMetadata::ProcessingNodeHolder +{ + friend class S3QueueFilesMetadata; +public: + ProcessingNodeHolder(const std::string & processing_id_, const std::string & path_, const std::string & zk_node_path_, zkutil::ZooKeeperPtr zk_client_); + + ~ProcessingNodeHolder(); + +private: + bool remove(Coordination::Requests * requests = nullptr, Coordination::Responses * responses = nullptr); + + zkutil::ZooKeeperPtr zk_client; + std::string path; + std::string zk_node_path; + std::string processing_id; + bool removed = false; + Poco::Logger * log; +}; + } diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp index 0a2856776e2..bd01bd52425 100644 --- a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { @@ -13,7 +14,7 @@ S3QueueMetadataFactory & S3QueueMetadataFactory::instance() return ret; } -S3QueueMetadataFactory::MetadataPtr +S3QueueMetadataFactory::FilesMetadataPtr S3QueueMetadataFactory::getOrCreate(const std::string & zookeeper_path, const S3QueueSettings & settings) { std::lock_guard lock(mutex); @@ -22,12 +23,48 @@ S3QueueMetadataFactory::getOrCreate(const std::string & zookeeper_path, const S3 { it = metadata_by_path.emplace(zookeeper_path, std::make_shared(fs::path(zookeeper_path), settings)).first; } - else if (!it->second->checkSettings(settings)) + else if (it->second.metadata->checkSettings(settings)) + { + it->second.ref_count += 1; + } + else { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Metadata with the same `s3queue_zookeeper_path` " "was already created but with different settings"); } - return it->second; + return it->second.metadata; +} + +void S3QueueMetadataFactory::remove(const std::string & zookeeper_path) +{ + std::lock_guard lock(mutex); + auto it = metadata_by_path.find(zookeeper_path); + + if (it == metadata_by_path.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Metadata with zookeeper path {} does not exist", zookeeper_path); + + if (--it->second.ref_count == 0) + { + try + { + auto zk_client = Context::getGlobalContextInstance()->getZooKeeper(); + zk_client->tryRemove(it->first); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + metadata_by_path.erase(it); + } +} + +std::unordered_map S3QueueMetadataFactory::getAll() +{ + std::unordered_map result; + for (const auto & [zk_path, metadata_and_ref_count] : metadata_by_path) + result.emplace(zk_path, metadata_and_ref_count.metadata); + return result; } } diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.h b/src/Storages/S3Queue/S3QueueMetadataFactory.h index e7a473d863d..55c2dfad5dd 100644 --- a/src/Storages/S3Queue/S3QueueMetadataFactory.h +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.h @@ -9,16 +9,26 @@ namespace DB class S3QueueMetadataFactory final : private boost::noncopyable { public: - using MetadataPtr = std::shared_ptr; - using MetadataByPath = std::unordered_map; + using FilesMetadataPtr = std::shared_ptr; static S3QueueMetadataFactory & instance(); - MetadataPtr getOrCreate(const std::string & zookeeper_path, const S3QueueSettings & settings); + FilesMetadataPtr getOrCreate(const std::string & zookeeper_path, const S3QueueSettings & settings); - MetadataByPath getAll() { return metadata_by_path; } + void remove(const std::string & zookeeper_path); + + std::unordered_map getAll(); private: + struct Metadata + { + explicit Metadata(std::shared_ptr metadata_) : metadata(metadata_), ref_count(1) {} + + std::shared_ptr metadata; + size_t ref_count = 0; + }; + using MetadataByPath = std::unordered_map; + MetadataByPath metadata_by_path; std::mutex mutex; }; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index ff1a8c86b64..4171597b619 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -33,11 +33,9 @@ namespace ErrorCodes StorageS3QueueSource::S3QueueKeyWithInfo::S3QueueKeyWithInfo( const std::string & key_, std::optional info_, - std::unique_ptr processing_holder_, - std::shared_ptr file_status_) + Metadata::ProcessingNodeHolderPtr processing_holder_) : StorageS3Source::KeyWithInfo(key_, info_) - , processing_holder(std::move(processing_holder_)) - , file_status(file_status_) + , processing_holder(processing_holder_) { } @@ -61,7 +59,7 @@ StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next() if (auto processing_holder = metadata->trySetFileAsProcessing(val->key); processing_holder) { - return std::make_shared(val->key, val->info, std::move(processing_holder), nullptr); + return std::make_shared(val->key, val->info, processing_holder); } } } @@ -151,12 +149,16 @@ Chunk StorageS3QueueSource::generate() catch (const Exception & e) { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); - files_metadata->setFileFailed(reader.getFile(), e.message()); + + const StorageS3QueueSource::S3QueueKeyWithInfo * key_with_info = assert_cast(&reader.getKeyWithInfo()); + files_metadata->setFileFailed(key_with_info->processing_holder, e.message()); + appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); throw; } - files_metadata->setFileProcessed(reader.getFile()); + const StorageS3QueueSource::S3QueueKeyWithInfo * key_with_info = assert_cast(&reader.getKeyWithInfo()); + files_metadata->setFileProcessed(key_with_info->processing_holder); applyActionAfterProcessing(reader.getFile()); appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, true); diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 634c0803465..304b7d3b661 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -30,11 +30,9 @@ public: S3QueueKeyWithInfo( const std::string & key_, std::optional info_, - std::unique_ptr processing_holder_, - std::shared_ptr file_status_); + Metadata::ProcessingNodeHolderPtr processing_holder_); - std::unique_ptr processing_holder; - std::shared_ptr file_status; + Metadata::ProcessingNodeHolderPtr processing_holder; }; class FileIterator : public IIterator diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index a15cc1bea9b..553211f7772 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -149,6 +149,11 @@ void StorageS3Queue::shutdown() } } +void StorageS3Queue::drop() +{ + S3QueueMetadataFactory::instance().remove(zk_path); +} + bool StorageS3Queue::supportsSubsetOfColumns(const ContextPtr & context_) const { return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context_, format_settings); @@ -422,13 +427,6 @@ std::shared_ptr StorageS3Queue::createFileIterator return std::make_shared(files_metadata, std::move(glob_iterator)); } -void StorageS3Queue::drop() -{ - auto zookeeper = getZooKeeper(); - if (zookeeper->exists(zk_path)) - zookeeper->removeRecursive(zk_path); -} - void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) { factory.registerStorage( diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 3f8919d0f05..1bb2dccbccc 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -639,7 +639,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); - return ReaderHolder{*key_with_info, bucket, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{key_with_info, bucket, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3Source::createReaderAsync() diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 38cf3a5f65b..c053c6f9bd2 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -175,13 +175,13 @@ private: { public: ReaderHolder( - KeyWithInfo key_with_info_, + KeyWithInfoPtr key_with_info_, String bucket_, std::unique_ptr read_buf_, std::shared_ptr source_, std::unique_ptr pipeline_, std::unique_ptr reader_) - : key_with_info(std::move(key_with_info_)) + : key_with_info(key_with_info_) , bucket(std::move(bucket_)) , read_buf(std::move(read_buf_)) , source(std::move(source_)) @@ -215,14 +215,14 @@ private: explicit operator bool() const { return reader != nullptr; } PullingPipelineExecutor * operator->() { return reader.get(); } const PullingPipelineExecutor * operator->() const { return reader.get(); } - String getPath() const { return fs::path(bucket) / key_with_info.key; } - const String & getFile() const { return key_with_info.key; } - const KeyWithInfo & getKeyWithInfo() const { return key_with_info; } + String getPath() const { return fs::path(bucket) / key_with_info->key; } + const String & getFile() const { return key_with_info->key; } + const KeyWithInfo & getKeyWithInfo() const { return *key_with_info; } const IInputFormat * getInputFormat() const { return dynamic_cast(source.get()); } private: - KeyWithInfo key_with_info; + KeyWithInfoPtr key_with_info; String bucket; std::unique_ptr read_buf; std::shared_ptr source; From e0ff76a7f968c8fa0511f55f81907e8b97880984 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 18:44:53 +0200 Subject: [PATCH 062/634] Fix --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 24 +++++++++++-------- src/Storages/S3Queue/S3QueueSource.cpp | 1 - src/Storages/S3Queue/StorageS3Queue.cpp | 9 ++++++- .../integration/test_storage_s3_queue/test.py | 7 +++--- 4 files changed, 26 insertions(+), 15 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 0492a8c2f4d..e82345dd4fb 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -430,11 +430,12 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolder return; } - /// TODO this could be because of the expired session. if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is not processing"); - else throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is already processed"); + + LOG_WARNING(log, "Cannot set file ({}) as processed since processing node " + "does not exist with expected processing id does not exist, " + "this could be a result of expired zookeeper session", path); } void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPtr holder) @@ -472,8 +473,9 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) continue; - LOG_ERROR(log, "Cannot set file {} as failed - failed to remove ephemeral processing node", path); - chassert(false); + LOG_WARNING(log, "Cannot set file ({}) as processed since processing node " + "does not exist with expected processing id does not exist, " + "this could be a result of expired zookeeper session", path); return; } } @@ -531,7 +533,6 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S { /// File is no longer retriable. /// Make a failed/node_name node and remove failed/node_name.retriable node. - /// TODO: always add version for processing node. Coordination::Requests requests; requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); @@ -595,9 +596,9 @@ S3QueueFilesMetadata::ProcessingNodeHolder::~ProcessingNodeHolder() bool S3QueueFilesMetadata::ProcessingNodeHolder::remove(Coordination::Requests * requests, Coordination::Responses * responses) { if (removed) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Processing file holder is already released"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Processing node is already removed"); - removed = true; + LOG_TEST(log, "Removing processing node {} ({})", zk_node_path, path); try { @@ -617,11 +618,14 @@ bool S3QueueFilesMetadata::ProcessingNodeHolder::remove(Coordination::Requests * { requests->push_back(zkutil::makeRemoveRequest(zk_node_path, stat.version)); auto code = zk_client->tryMulti(*requests, *responses); - return code == Coordination::Error::ZOK; + removed = code == Coordination::Error::ZOK; } else + { zk_client->remove(zk_node_path); - return true; + removed = true; + } + return removed; } else LOG_WARNING(log, "Cannot remove {} since precessing id changed: {} -> {}", diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 4171597b619..11415a40302 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -139,7 +139,6 @@ Chunk StorageS3QueueSource::generate() LOG_TEST(log, "Read {} rows from file: {}", chunk.getNumRows(), reader.getPath()); file_status->processed_rows += chunk.getNumRows(); - // file_status->profile_counters.increment(ProfileEvents::S3QueuePullMicroseconds, timer.get()); processed_rows_from_file += chunk.getNumRows(); VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath()); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 553211f7772..6bed73b0315 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -101,6 +101,12 @@ StorageS3Queue::StorageS3Queue( throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs"); } + if (s3queue_settings->mode == S3QueueMode::ORDERED && s3queue_settings->s3queue_processing_threads_num > 1) + { + LOG_WARNING(log, "Parallel processing is not yet supported for Ordered mode"); + s3queue_settings->s3queue_processing_threads_num = 1; + } + configuration.update(context_); FormatFactory::instance().checkFormatName(configuration.format); context_->getRemoteHostFilter().checkURL(configuration.url.uri); @@ -181,7 +187,8 @@ Pipe StorageS3Queue::read( } Pipes pipes; - for (size_t i = 0; i < num_streams; ++i) + const size_t adjusted_num_streams = std::min(num_streams, s3queue_settings->s3queue_processing_threads_num); + for (size_t i = 0; i < adjusted_num_streams; ++i) pipes.emplace_back(createSource(column_names, storage_snapshot, query_info.query, max_block_size, local_context)); return Pipe::unitePipes(std::move(pipes)); } diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 4c3e9b1753e..7d8cfd68dd2 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -558,7 +558,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): }, ) except QueryRuntimeException as e: - assert "Existing table metadata in ZooKeeper differs in engine mode" in str(e) + assert "Metadata with the same `s3queue_zookeeper_path` was already created but with different settings" in str(e) failed = True assert failed is True @@ -836,15 +836,16 @@ def test_max_set_size(started_cluster): "s3queue_tracked_files_limit": 9, "s3queue_cleanup_interval_min_ms": 0, "s3queue_cleanup_interval_max_ms": 0, + "s3queue_processing_threads_num": 1, }, ) total_values = generate_random_files( started_cluster, files_path, files_to_generate, start_ind=0, row_num=1 ) - get_query = f"SELECT * FROM {table_name}" + get_query = f"SELECT * FROM {table_name} ORDER BY column1, column2, column3" res1 = [list(map(int, l.split())) for l in run_query(node, get_query).splitlines()] - assert res1 == total_values + assert res1 == sorted(total_values, key=lambda x: (x[0], x[1], x[2])) print(total_values) time.sleep(10) From 86c70c8f84b4d2a894fbdeba2f6a51d045b34852 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 27 Sep 2023 17:28:08 +0000 Subject: [PATCH 063/634] Add another test without projections. --- ...97_column_update_and_replication.reference | 0 .../02597_column_update_and_replication.sql | 42 +++++++++++++++++++ 2 files changed, 42 insertions(+) create mode 100644 tests/queries/0_stateless/02597_column_update_and_replication.reference create mode 100644 tests/queries/0_stateless/02597_column_update_and_replication.sql diff --git a/tests/queries/0_stateless/02597_column_update_and_replication.reference b/tests/queries/0_stateless/02597_column_update_and_replication.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02597_column_update_and_replication.sql b/tests/queries/0_stateless/02597_column_update_and_replication.sql new file mode 100644 index 00000000000..07b13468ccb --- /dev/null +++ b/tests/queries/0_stateless/02597_column_update_and_replication.sql @@ -0,0 +1,42 @@ +CREATE TABLE test ( + `c_id` String, + `p_id` String, + `d` String +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1') +ORDER BY (c_id, p_id); + + +---CREATE TABLE test_r2 ( +--- `c_id` String, +--- `p_id` String, +--- `d` String +---) +---ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '2') +---ORDER BY (c_id, p_id); + + +INSERT INTO test SELECT '1', '11', '111' FROM numbers(3); + +INSERT INTO test SELECT '2', '22', '22' FROM numbers(3); + +select * from test format Null; +select min(c_id) from test group by d format Null; + +set mutations_sync=0; + +ALTER TABLE test UPDATE d = d || toString(sleepEachRow(0.1)) where 1; + +ALTER TABLE test ADD COLUMN x UInt32 default 0; +ALTER TABLE test UPDATE x = x + 1 where 1; +ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; + +SELECT * FROM system.mutations WHERE database=currentDatabase() AND table='test' AND NOT is_done; + + + +select * from test format Null; + + +DROP TABLE test; +--DROP TABLE test_r2; From 42783894b75661fde201806543dafddc00a387a6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 20:07:19 +0200 Subject: [PATCH 064/634] Add comments --- src/Interpreters/S3QueueLog.cpp | 2 + src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 179 ++++++++++++------ src/Storages/S3Queue/S3QueueFilesMetadata.h | 22 ++- src/Storages/S3Queue/S3QueueSource.cpp | 1 - src/Storages/S3Queue/S3QueueSource.h | 1 - 5 files changed, 141 insertions(+), 64 deletions(-) diff --git a/src/Interpreters/S3QueueLog.cpp b/src/Interpreters/S3QueueLog.cpp index caccdabf503..78c3c8dadf7 100644 --- a/src/Interpreters/S3QueueLog.cpp +++ b/src/Interpreters/S3QueueLog.cpp @@ -33,6 +33,8 @@ NamesAndTypesList S3QueueLogElement::getNamesAndTypes() }; } +/// TODO add last_exception column + void S3QueueLogElement::appendToBlock(MutableColumns & columns) const { size_t i = 0; diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index e82345dd4fb..c2a74e45dab 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -64,7 +64,8 @@ S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::LocalFileStatuses::getA return file_statuses; } -std::shared_ptr S3QueueFilesMetadata::LocalFileStatuses::get(const std::string & filename, bool create) +std::shared_ptr +S3QueueFilesMetadata::LocalFileStatuses::get(const std::string & filename, bool create) { auto lk = lock(); auto it = file_statuses.find(filename); @@ -162,11 +163,16 @@ zkutil::ZooKeeperPtr S3QueueFilesMetadata::getZooKeeper() const std::shared_ptr S3QueueFilesMetadata::getFileStatus(const std::string & path) { + /// Return a locally cached file status. return local_file_statuses.get(path, /* create */false); } std::string S3QueueFilesMetadata::getNodeName(const std::string & path) { + /// Since with are dealing with paths in s3 which can have "/", + /// we cannot create a zookeeper node with the name equal to path. + /// Therefore we use a hash of the path as a node name. + SipHash path_hash; path_hash.update(path); return toString(path_hash.get64()); @@ -177,6 +183,13 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( const std::string & exception, size_t retries) { + /// Create a metadata which will be stored in a node named as getNodeName(path). + + /// Since node name is just a hash we want to know to which file it corresponds, + /// so we keep "file_path" in nodes data. + /// "last_processed_timestamp" is needed for TTL metadata nodes enabled by s3queue_tracked_file_ttl_sec. + /// "last_exception" is kept for introspection, should also be visible in system.s3queue_log if it is enabled. + /// "retries" is kept for retrying the processing enabled by s3queue_loading_retries. NodeMetadata metadata; metadata.file_path = path; metadata.last_processed_timestamp = getCurrentTime(); @@ -191,95 +204,111 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs auto file_status = local_file_statuses.get(path, /* create */true); /// Check locally cached file status. + /// Processed or Failed state is always cached. + /// Processing state is cached only if processing is being done by current clickhouse server + /// (because If another server is doing the processing, + /// we cannot know if state changes without checking with zookeeper so there is no point in cache here). switch (file_status->state) { case FileStatus::State::Processing: [[fallthrough]]; case FileStatus::State::Processed: { - /// File is already processes or processing by current server. return nullptr; } case FileStatus::State::Failed: { - /// max_loading_retries == 0 => file is not retriable. - /// file_status->retries is a cached value, so in case file_status->retries >= max_loading retries - /// we can fully rely that it is true, but in other case the value might be outdated, - /// but this is ok, we will recheck with zookeeper. - if (!max_loading_retries || file_status->retries >= max_loading_retries) + /// If max_loading_retries == 0, file is not retriable. + if (max_loading_retries == 0) return nullptr; + + /// Otherwise file_status->retries is also cached. + /// In case file_status->retries >= max_loading_retries we can fully rely that it is true + /// and will not attempt processing it. + /// But in case file_status->retries < max_loading_retries we cannot be sure + /// (another server could have done a try after we cached retries value), + /// so check with zookeeper here. + if (file_status->retries >= max_loading_retries) + return nullptr; + break; } case FileStatus::State::None: { - /// The file was not processed by current server, + /// The file was not processed by current server and file status was not cached, /// check metadata in zookeeper. break; } } + + /// Another thread could already be trying to set file as processing. + /// So there is no need to attempt the same, better to continue with the next file. std::unique_lock lock(file_status->processing_lock, std::defer_lock); if (!lock.try_lock()) { - /// Another thread is already trying to set file as processing. return nullptr; } + /// Let's go and check metadata in zookeeper and try to create a /processing ephemeral node. + /// If successful, return result with processing node holder. SetFileProcessingResult result; - ProcessingNodeHolderPtr processing_holder; + ProcessingNodeHolderPtr processing_node_holder; + switch (mode) { case S3QueueMode::ORDERED: { - std::tie(result, processing_holder) = trySetFileAsProcessingForOrderedMode(path); + std::tie(result, processing_node_holder) = trySetFileAsProcessingForOrderedMode(path); break; } case S3QueueMode::UNORDERED: { - std::tie(result, processing_holder) = trySetFileAsProcessingForUnorderedMode(path); + std::tie(result, processing_node_holder) = trySetFileAsProcessingForUnorderedMode(path); break; } } + + /// Cache file status, save some statistics. switch (result) { case SetFileProcessingResult::Success: { file_status->state = FileStatus::State::Processing; + file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); timer.cancel(); + if (!file_status->processing_start_time) file_status->processing_start_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + break; } case SetFileProcessingResult::AlreadyProcessed: { - /// Cache the state. file_status->state = FileStatus::State::Processed; break; } case SetFileProcessingResult::AlreadyFailed: { - /// Cache the state. file_status->state = FileStatus::State::Failed; break; } case SetFileProcessingResult::ProcessingByOtherNode: { - /// We cannot save any local state. + /// We cannot save any local state here, see comment above. break; } } - if (result != SetFileProcessingResult::Success) - return nullptr; - - return processing_holder; + return result == SetFileProcessingResult::Success ? processing_node_holder : nullptr; } -std::pair -S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) +std::pair S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) { - /// Create an ephemenral node in /processing - /// if corresponding node does not exist in failed/, processed/ and processing/. - /// Return false otherwise. + /// In one zookeeper transaction do the following: + /// 1. check that corresponding persistent nodes do not exist in processed/ and failed/; + /// 2. create an ephemenral node in /processing if it does not exist; + /// Return corresponding status if any of the step failed. const auto node_name = getNodeName(path); const auto zk_client = getZooKeeper(); @@ -310,12 +339,14 @@ S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; } -std::pair -S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) +std::pair S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) { - /// Create an ephemenral node in /processing - /// if corresponding it does not exist in failed/, processing/ and satisfied max processed file check. - /// Return false otherwise. + /// Same as for Unordered mode. + /// The only difference is the check if the file is already processed. + /// For Ordered mode we do not keep a separate /processed/hash_node for each file + /// but instead we only keep a maximum processed file + /// (since all files are ordered and new files have a lexically bigger name, it makes sense). const auto node_name = getNodeName(path); const auto zk_client = getZooKeeper(); @@ -324,25 +355,13 @@ S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & p while (true) { - Coordination::Requests requests; - zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); - zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_processing_path / node_name); - - Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code != Coordination::Error::ZOK) - { - if (responses[0]->error == Coordination::Error::ZOK) - { - LOG_TEST(log, "Skipping file `{}`: already processing", path); - return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; - } - else - { - LOG_TEST(log, "Skipping file `{}`: failed", path); - return std::pair{SetFileProcessingResult::AlreadyFailed, nullptr}; - } - } + /// Get a /processed node content - max_processed path. + /// Compare our path to it. + /// If file is not yet processed, check corresponding /failed node and try create /processing node + /// and in the same zookeeper transaction als check that /processed node did not change + /// in between, e.g. that stat.version remained the same. + /// If the version did change - retry (since we cannot do Get and Create requests + /// in the same zookeeper transaction, so we use a while loop with tries). Coordination::Stat processed_node_stat; auto data = zk_client->get(zookeeper_processed_path, &processed_node_stat); @@ -354,14 +373,13 @@ S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & p if (!max_processed_file_path.empty() && path <= max_processed_file_path) return std::pair{SetFileProcessingResult::AlreadyProcessed, nullptr}; - requests.clear(); - responses.clear(); - + Coordination::Requests requests; zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata.toString(), zkutil::CreateMode::Ephemeral)); requests.push_back(zkutil::makeCheckRequest(zookeeper_processed_path, processed_node_stat.version)); - code = zk_client->tryMulti(requests, responses); + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) { auto holder = std::make_unique(node_metadata.processing_id, path, zookeeper_processing_path / node_name, zk_client); @@ -431,15 +449,21 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolder } if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is already processed"); + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot create a persistent node in /processed since it already exists"); + } - LOG_WARNING(log, "Cannot set file ({}) as processed since processing node " - "does not exist with expected processing id does not exist, " + LOG_WARNING(log, + "Cannot set file ({}) as processed since ephemeral node in /processing" + "does not exist with expected id, " "this could be a result of expired zookeeper session", path); } void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPtr holder) { + /// Update a presistent node in /processed and remove ephemeral node from /processing. + const auto & path = holder->path; const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); @@ -457,7 +481,20 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt { auto metadata = NodeMetadata::fromString(res); if (metadata.file_path >= path) + { + /// Here we get in the case that maximum processed file is bigger than ours. + /// This is possible to achive in case of parallel processing + /// but for local processing we explicitly disable parallel mode and do everything in a single thread + /// (see constructor of StorageS3Queue where s3queue_processing_threads_num is explicitly set to 1 in case of Ordered mode). + /// Nevertheless, in case of distributed processing we cannot do anything with parallelism. + /// What this means? + /// It means that in scenario "distributed processing + Ordered mode" + /// a setting s3queue_loading_retries will not work. It is possible to fix, it is in TODO. + + /// Return because there is nothing to change, + /// the max processed file is already bigger than ours. return; + } } requests.push_back(zkutil::makeSetRequest(zookeeper_processed_path, node_metadata, stat.version)); } @@ -470,6 +507,7 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt if (holder->remove(&requests, &responses)) return; + /// Failed to update max processed node, retry. if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) continue; @@ -497,13 +535,16 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S auto node_metadata = createNodeMetadata(path, exception_message); const auto zk_client = getZooKeeper(); + /// Is file retriable? if (max_loading_retries == 0) { + /// File is not retriable, + /// just create a node in /failed and remove a node from /processing. + Coordination::Requests requests; requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, node_metadata.toString(), zkutil::CreateMode::Persistent)); - Coordination::Responses responses; if (holder->remove(&requests, &responses)) { @@ -512,13 +553,30 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S return; } - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); + if (responses[0]->error != Coordination::Error::ZOK) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot create a persistent node in /failed since it already exists"); + } + + LOG_WARNING(log, "Cannot set file ({}) as processed since processing node " + "does not exist with expected processing id does not exist, " + "this could be a result of expired zookeeper session", path); + return; } - const auto node_name_with_retriable_suffix = node_name + ".retriable"; + /// So file is retriable. + /// Let's do an optimization here. + /// Instead of creating a persistent /failed/node_hash node + /// we create a persistent /failed/node_hash.retriable node. + /// This allows us to make less zookeeper requests as we avoid checking + /// the number of already done retries in trySetFileAsProcessing. + const auto node_name_with_retriable_suffix = node_name + ".retriable"; Coordination::Stat stat; std::string res; + + /// Extract the number of already done retries from node_hash.retriable node if it exists. if (zk_client->tryGet(zookeeper_failed_path / node_name_with_retriable_suffix, res, &stat)) { auto failed_node_metadata = NodeMetadata::fromString(res); @@ -529,10 +587,11 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S LOG_TEST(log, "File `{}` failed to process, try {}/{} (Error: {})", path, node_metadata.retries, max_loading_retries, exception_message); + /// Check if file can be retried futher or not. if (node_metadata.retries >= max_loading_retries) { /// File is no longer retriable. - /// Make a failed/node_name node and remove failed/node_name.retriable node. + /// Make a persistent node /failed/node_hash, remove /failed/node_hash.retriable node and node in /processing. Coordination::Requests requests; requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); @@ -551,6 +610,8 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S } else { + /// File is still retriable, update retries count and remove node from /processing. + Coordination::Requests requests; requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); if (node_metadata.retries == 0) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index dd760b29299..e9870292348 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -15,6 +15,22 @@ namespace DB struct S3QueueSettings; class StorageS3Queue; +/** + * A class for managing S3Queue metadata in zookeeper, e.g. + * the following folders: + * - /processing + * - /processed + * - /failed + * + * Depending on S3Queue processing mode (ordered or unordered) + * we can differently store metadata in /processed node. + * + * Implements caching of zookeeper metadata for faster responses. + * Cached part is located in LocalFileStatuses. + * + * In case of Unordered mode - if files TTL is enabled or maximum tracked files limit is set + * starts a background cleanup thread which is responsible for maintaining them. + */ class S3QueueFilesMetadata { public: @@ -25,6 +41,7 @@ public: ~S3QueueFilesMetadata(); + /// Set file as processing, if it is not alreaty processed, failed or processing. ProcessingNodeHolderPtr trySetFileAsProcessing(const std::string & path); void setFileProcessed(ProcessingNodeHolderPtr holder); @@ -33,7 +50,6 @@ public: struct FileStatus { - size_t processed_rows = 0; enum class State { Processing, @@ -42,12 +58,12 @@ public: None }; State state = State::None; - ProfileEvents::Counters profile_counters; + size_t processed_rows = 0; time_t processing_start_time = 0; time_t processing_end_time = 0; - size_t retries = 0; + ProfileEvents::Counters profile_counters; std::mutex processing_lock; }; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 11415a40302..a1868238bc3 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -91,7 +91,6 @@ StorageS3QueueSource::StorageS3QueueSource( , shutdown_called(shutdown_called_) , s3_queue_log(s3_queue_log_) , storage_id(storage_id_) - , s3_queue_user_id(fmt::format("{}:{}", CurrentThread::getQueryId(), getRandomASCIIString(8))) , remove_file_func(remove_file_func_) , log(&Poco::Logger::get("StorageS3QueueSource")) { diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 304b7d3b661..fa21b6cdd59 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -80,7 +80,6 @@ private: const std::atomic & shutdown_called; const std::shared_ptr s3_queue_log; const StorageID storage_id; - const std::string s3_queue_user_id; RemoveFileFunc remove_file_func; Poco::Logger * log; From c579f5b32127e5939292c6d8c90dc4d7b38ec765 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 20:10:05 +0200 Subject: [PATCH 065/634] Fix style check --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 2 +- tests/integration/test_storage_s3_queue/test.py | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index c2a74e45dab..aff4a91bf7f 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -546,7 +546,7 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S node_metadata.toString(), zkutil::CreateMode::Persistent)); Coordination::Responses responses; - if (holder->remove(&requests, &responses)) + if (holder->remove(&requests, &responses)) { LOG_TEST(log, "File `{}` failed to process and will not be retried. " "Error: {}", path, exception_message); diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 7d8cfd68dd2..079311422cc 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -558,7 +558,10 @@ def test_multiple_tables_meta_mismatch(started_cluster): }, ) except QueryRuntimeException as e: - assert "Metadata with the same `s3queue_zookeeper_path` was already created but with different settings" in str(e) + assert ( + "Metadata with the same `s3queue_zookeeper_path` was already created but with different settings" + in str(e) + ) failed = True assert failed is True From 57cfb88cb6e79e41476002773bc49c18675e8772 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 22:22:49 +0200 Subject: [PATCH 066/634] Fix typos check, fix build wihtout s3 --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 10 +++++----- src/Storages/System/StorageSystemS3Queue.cpp | 4 ---- src/Storages/System/StorageSystemS3Queue.h | 4 ---- 3 files changed, 5 insertions(+), 13 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index aff4a91bf7f..5d2ac9514bc 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -358,7 +358,7 @@ std::pairpath; const auto node_name = getNodeName(path); @@ -483,7 +483,7 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt if (metadata.file_path >= path) { /// Here we get in the case that maximum processed file is bigger than ours. - /// This is possible to achive in case of parallel processing + /// This is possible to achieve in case of parallel processing /// but for local processing we explicitly disable parallel mode and do everything in a single thread /// (see constructor of StorageS3Queue where s3queue_processing_threads_num is explicitly set to 1 in case of Ordered mode). /// Nevertheless, in case of distributed processing we cannot do anything with parallelism. @@ -587,7 +587,7 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S LOG_TEST(log, "File `{}` failed to process, try {}/{} (Error: {})", path, node_metadata.retries, max_loading_retries, exception_message); - /// Check if file can be retried futher or not. + /// Check if file can be retried further or not. if (node_metadata.retries >= max_loading_retries) { /// File is no longer retriable. @@ -689,7 +689,7 @@ bool S3QueueFilesMetadata::ProcessingNodeHolder::remove(Coordination::Requests * return removed; } else - LOG_WARNING(log, "Cannot remove {} since precessing id changed: {} -> {}", + LOG_WARNING(log, "Cannot remove {} since processing id changed: {} -> {}", zk_node_path, processing_id, node_metadata.processing_id); } else diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index faba5bc671b..ce76469b72a 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -1,7 +1,5 @@ #include "StorageSystemS3Queue.h" -#if USE_AWS_S3 - #include #include #include @@ -67,5 +65,3 @@ void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr, co } } - -#endif diff --git a/src/Storages/System/StorageSystemS3Queue.h b/src/Storages/System/StorageSystemS3Queue.h index 208891aef42..1bb4e3694d2 100644 --- a/src/Storages/System/StorageSystemS3Queue.h +++ b/src/Storages/System/StorageSystemS3Queue.h @@ -1,8 +1,6 @@ #pragma once #include "config.h" -#if USE_AWS_S3 - #include #include @@ -23,5 +21,3 @@ protected: }; } - -#endif From eab7b34e23af312cfb2af2be855fb630664961fc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Thu, 28 Sep 2023 17:10:49 +0800 Subject: [PATCH 067/634] support column query --- src/Functions/getHttpHeader.cpp | 28 +++++++++++++------ .../02884_getHttpHeaderFunction.reference | 6 ++++ .../02884_getHttpHeaderFunction.sh | 16 +++++++++-- 3 files changed, 39 insertions(+), 11 deletions(-) diff --git a/src/Functions/getHttpHeader.cpp b/src/Functions/getHttpHeader.cpp index 70db08f0ef5..1113ccff002 100644 --- a/src/Functions/getHttpHeader.cpp +++ b/src/Functions/getHttpHeader.cpp @@ -40,12 +40,13 @@ public: return std::make_shared(context_); } + bool useDefaultImplementationForConstants() const override { return true; } String getName() const override { return name; } bool isDeterministic() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } size_t getNumberOfArguments() const override @@ -64,22 +65,31 @@ public: { const auto & client_info = getContext()->getClientInfo(); const auto & method = client_info.http_method; - - const auto & headers = DB::CurrentThread::getQueryContext()->getClientInfo().headers; - + const auto & headers = client_info.headers; const IColumn * arg_column = arguments[0].column.get(); - const ColumnString * arg_string = checkAndGetColumnConstData(arg_column); + const ColumnString * arg_string = checkAndGetColumn(arg_column); if (!arg_string) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The argument of function {} must be constant String", getName()); if (method != ClientInfo::HTTPMethod::GET && method != ClientInfo::HTTPMethod::POST) - return result_type->createColumnConst(input_rows_count, ""); + return result_type->createColumnConstWithDefaultValue(input_rows_count); - if (!headers.has(arg_string->getDataAt(0).toString())) - return result_type->createColumnConst(input_rows_count, ""); + auto result_column = ColumnString::create(); - return result_type->createColumnConst(input_rows_count, headers[arg_string->getDataAt(0).toString()]); + const String default_value; + for (size_t row = 0; row < input_rows_count; ++row) + { + auto header_name = arg_string->getDataAt(row).toString(); + + if (!headers.has(header_name)) + result_column->insertData(default_value.data(), default_value.size()); + + const String & value = headers[header_name]; + result_column->insertData(value.data(), value.size()); + } + + return result_column; } }; diff --git a/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference b/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference index 8db2cdd84ce..334f0840fa4 100644 --- a/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference +++ b/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference @@ -8,3 +8,9 @@ default value1 value2 value_from_query_1 value_from_query_2 value_from_query_3 1 default row1_value1 row1_value2 row1_value3 row1_value4 row1_value5 row1_value6 row1_value7 value_from_query_1 value_from_query_2 value_from_query_3 2 default row2_value1 row2_value2 row2_value3 row2_value4 row2_value5 row2_value6 row2_value7 value_from_query_1 value_from_query_2 value_from_query_3 3 +X-Clickhouse-User +http_key1 +http_key2 +default +http_value1 +http_value2 diff --git a/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh b/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh index 6c78ca113cb..fecc54ffd82 100755 --- a/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh +++ b/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh @@ -17,7 +17,8 @@ echo "SELECT getHttpHeader('X-' || 'Clickhouse' || '-User'), getHttpHeader('key1 $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02884_get_http_header" -$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS 02884_get_http_header +$CLICKHOUSE_CLIENT -q " + CREATE TABLE IF NOT EXISTS 02884_get_http_header (id UInt32, http_user String DEFAULT getHttpHeader('X-Clickhouse-User'), http_key1 String DEFAULT getHttpHeader('http_header_key1'), @@ -50,7 +51,7 @@ echo "INSERT INTO test.02884_get_http_header (id) values (2)" | curl -s -H 'X-Cl -H 'http_header_key6: row2_value6'\ -H 'http_header_key7: row2_value7' 'http://localhost:8123/' -d @- -$CLICKHOUSE_CLIENT -q "SELECT id, http_user, http_key1, http_key2, http_key3, http_key4, http_key5, http_key6, http_key7 FROM test.02884_get_http_header ORDER BY id;" +$CLICKHOUSE_CLIENT -q "SELECT id, http_user, http_key1, http_key2, http_key3, http_key4, http_key5, http_key6, http_key7 FROM 02884_get_http_header ORDER BY id;" #Insert data via tcp client $CLICKHOUSE_CLIENT -q "INSERT INTO 02884_get_http_header (id) values (3)" $CLICKHOUSE_CLIENT -q "SELECT * FROM 02884_get_http_header where id = 3" @@ -58,3 +59,14 @@ echo "SELECT getHttpHeader('key_from_query_1'), getHttpHeader('key_from_query_2' -H 'X-ClickHouse-Key: ' -H 'key_from_query_1: value_from_query_1' -H 'key_from_query_2: value_from_query_2' -H 'key_from_query_3: value_from_query_3' 'http://localhost:8123/' -d @- $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02884_get_http_header" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE 02884_header_from_table (header_name String) Engine=Memory" +$CLICKHOUSE_CLIENT -q "INSERT INTO 02884_header_from_table values ('X-Clickhouse-User'), ('http_key1'), ('http_key2')" +$CLICKHOUSE_CLIENT -q "select * from 02884_header_from_table" + +echo "SELECT getHttpHeader(header_name) as value from (select * FROM test.02884_header_from_table) order by value" | curl -s -H 'X-Clickhouse-User: default' \ + -H 'X-ClickHouse-Key: ' -H 'http_key1: http_value1' -H 'http_key2: http_value2' 'http://localhost:8123/' -d @- + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02884_header_from_table" + + From 1749874e7bdbd4436bc2d8d49eccd556745861da Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Sep 2023 13:51:07 +0200 Subject: [PATCH 068/634] Fxi --- src/Storages/StorageS3Cluster.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 1dd1a410595..8649c3f1baa 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -82,7 +82,12 @@ RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(ASTPtr { auto iterator = std::make_shared( *s3_configuration.client, s3_configuration.url, query, virtual_columns, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback()); - auto callback = std::make_shared>([iterator]() mutable -> String { return iterator->next()->key; }); + auto callback = std::make_shared>([iterator]() mutable -> String + { + if (auto next = iterator->next()) + return next->key; + return ""; + }); return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } From 7d91ba109ae1147048bee73b4353292739a7fbe4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Sep 2023 13:58:59 +0200 Subject: [PATCH 069/634] Update doc --- .../table-engines/integrations/s3queue.md | 36 +++++++++++++------ src/Storages/S3Queue/S3QueueSettings.h | 2 +- 2 files changed, 26 insertions(+), 12 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 6ceea8d7dc9..7edc12145e8 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -17,14 +17,17 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) [after_processing = 'keep',] [keeper_path = '',] [s3queue_loading_retries = 0,] + [s3queue_processing_threads_num = 1,] + [s3queue_enable_logging_to_s3queue_log = 0,] [s3queue_polling_min_timeout_ms = 1000,] [s3queue_polling_max_timeout_ms = 10000,] [s3queue_polling_backoff_ms = 0,] - [s3queue_tracked_files_limit = 1000,] [s3queue_tracked_file_ttl_sec = 0,] - [s3queue_polling_size = 50,] + [s3queue_tracked_files_limit = 1000,] + [s3queue_cleanup_interval_min_ms = 10000,] + [s3queue_cleanup_interval_max_ms = 30000,] ``` - + * [ ] **Engine parameters** - `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). @@ -39,7 +42,7 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) CREATE TABLE s3queue_engine_table (name String, value UInt32) ENGINE=S3Queue('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/*', 'CSV', 'gzip') SETTINGS - mode = 'ordered'; + mode = 'unordered'; ``` Using named collections: @@ -102,6 +105,18 @@ Possible values: Default value: `0`. +### s3queue_processing_threads_num {#processing_threads_num} + +Number of threads to perform processing. Applies only for `Unordered` mode. + +Default value: `1`. + +### s3queue_enable_logging_to_s3queue_log {#enable_logging_to_s3queue_log} + +Enable logging to `system.s3queue_log`. + +Default value: `0`. + ### s3queue_polling_min_timeout_ms {#polling_min_timeout_ms} Minimal timeout before next polling (in milliseconds). @@ -154,18 +169,17 @@ Possible values: Default value: `0`. -### s3queue_polling_size {#polling_size} +### s3queue_cleanup_interval_min_ms {#cleanup_interval_min_ms} -Maximum files to fetch from S3 with SELECT or in background task. -Engine takes files for processing from S3 in batches. -We limit the batch size to increase concurrency if multiple table engines with the same `keeper_path` consume files from the same path. +For 'Ordered' mode. Defines a minimum boundary for reschedule interval for a background task, which is responsible for maintaining tracked file TTL and maximum tracked files set. -Possible values: +Default value: `10000`. -- Positive integer. +### s3queue_cleanup_interval_max_ms {#cleanup_interval_max_ms} -Default value: `50`. +For 'Ordered' mode. Defines a maximum boundary for reschedule interval for a background task, which is responsible for maintaining tracked file TTL and maximum tracked files set. +Default value: `30000`. ## S3-related Settings {#s3-settings} diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 6d7ab4187a5..e6a85a834fc 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -20,9 +20,9 @@ class ASTStorage; M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete or keep file in S3 after successful processing", 0) \ M(String, keeper_path, "", "Zookeeper node path", 0) \ M(UInt32, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \ - M(UInt32, s3queue_tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \ M(UInt32, s3queue_processing_threads_num, 1, "Number of processing threads", 0) \ M(UInt32, s3queue_enable_logging_to_s3queue_log, 0, "Enable logging to system table system.s3queue_log", 0) \ + M(UInt32, s3queue_tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \ M(UInt32, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt32, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ M(UInt32, s3queue_polling_backoff_ms, 1000, "Polling backoff", 0) \ From 4a7922507ba2475bb4c3c5162b3952dba0173e93 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Sep 2023 16:18:00 +0200 Subject: [PATCH 070/634] Minor changes --- docs/en/engines/table-engines/integrations/s3queue.md | 2 +- src/Interpreters/Cache/Metadata.cpp | 1 - src/Interpreters/S3QueueLog.cpp | 5 +++-- src/Interpreters/S3QueueLog.h | 1 + src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 8 +++++--- src/Storages/S3Queue/S3QueueFilesMetadata.h | 1 + src/Storages/S3Queue/S3QueueSource.cpp | 5 +++-- src/Storages/StorageS3.h | 3 +++ 8 files changed, 17 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 7edc12145e8..7c8c35d1b5c 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -27,7 +27,7 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) [s3queue_cleanup_interval_min_ms = 10000,] [s3queue_cleanup_interval_max_ms = 30000,] ``` - * [ ] + **Engine parameters** - `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index c251cb2dec2..7a9321e4215 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include namespace fs = std::filesystem; diff --git a/src/Interpreters/S3QueueLog.cpp b/src/Interpreters/S3QueueLog.cpp index 78c3c8dadf7..af40d1dd1e8 100644 --- a/src/Interpreters/S3QueueLog.cpp +++ b/src/Interpreters/S3QueueLog.cpp @@ -30,11 +30,10 @@ NamesAndTypesList S3QueueLogElement::getNamesAndTypes() {"processing_start_time", std::make_shared(std::make_shared())}, {"processing_end_time", std::make_shared(std::make_shared())}, {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, + {"exception", std::make_shared()}, }; } -/// TODO add last_exception column - void S3QueueLogElement::appendToBlock(MutableColumns & columns) const { size_t i = 0; @@ -55,6 +54,8 @@ void S3QueueLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertDefault(); ProfileEvents::dumpToMapColumn(counters_snapshot, columns[i++].get(), true); + + columns[i++]->insert(exception); } } diff --git a/src/Interpreters/S3QueueLog.h b/src/Interpreters/S3QueueLog.h index b28ef1b82fc..76ff5ca0cdc 100644 --- a/src/Interpreters/S3QueueLog.h +++ b/src/Interpreters/S3QueueLog.h @@ -24,6 +24,7 @@ struct S3QueueLogElement ProfileEvents::Counters::Snapshot counters_snapshot; time_t processing_start_time; time_t processing_end_time; + std::string exception; static std::string name() { return "S3QueueLog"; } diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 5d2ac9514bc..dbbf7e2ee1d 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -524,11 +524,13 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S const auto & path = holder->path; auto file_status = local_file_statuses.get(path, /* create */false); + file_status->state = FileStatus::State::Failed; + file_status->last_exception = exception_message; + file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + SCOPE_EXIT({ - file_status->state = FileStatus::State::Failed; file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileFailedMicroseconds, timer.get()); timer.cancel(); - file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); }); const auto node_name = getNodeName(path); @@ -575,11 +577,11 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S const auto node_name_with_retriable_suffix = node_name + ".retriable"; Coordination::Stat stat; std::string res; + auto failed_node_metadata = NodeMetadata::fromString(res); /// Extract the number of already done retries from node_hash.retriable node if it exists. if (zk_client->tryGet(zookeeper_failed_path / node_name_with_retriable_suffix, res, &stat)) { - auto failed_node_metadata = NodeMetadata::fromString(res); node_metadata.retries = failed_node_metadata.retries + 1; file_status->retries = node_metadata.retries; } diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index e9870292348..f6d98d13aaf 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -63,6 +63,7 @@ public: time_t processing_start_time = 0; time_t processing_end_time = 0; size_t retries = 0; + std::string last_exception; ProfileEvents::Counters profile_counters; std::mutex processing_lock; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index a1868238bc3..09713e77f37 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -128,6 +128,8 @@ Chunk StorageS3QueueSource::generate() auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); SCOPE_EXIT({ CurrentThread::get().attachProfileCountersScope(prev_scope); }); + const auto * key_with_info = dynamic_cast(&reader.getKeyWithInfo()); + try { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueuePullMicroseconds); @@ -148,14 +150,12 @@ Chunk StorageS3QueueSource::generate() { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); - const StorageS3QueueSource::S3QueueKeyWithInfo * key_with_info = assert_cast(&reader.getKeyWithInfo()); files_metadata->setFileFailed(key_with_info->processing_holder, e.message()); appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); throw; } - const StorageS3QueueSource::S3QueueKeyWithInfo * key_with_info = assert_cast(&reader.getKeyWithInfo()); files_metadata->setFileProcessed(key_with_info->processing_holder); applyActionAfterProcessing(reader.getFile()); @@ -215,6 +215,7 @@ void StorageS3QueueSource::appendLogElement(const std::string & filename, const .counters_snapshot = file_status_.profile_counters.getPartiallyAtomicSnapshot(), .processing_start_time = file_status_.processing_start_time, .processing_end_time = file_status_.processing_end_time, + .exception = file_status_.last_exception, }; s3_queue_log->add(std::move(elem)); } diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index c053c6f9bd2..adcfa6ebdb9 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -43,9 +43,12 @@ public: struct KeyWithInfo { KeyWithInfo() = default; + explicit KeyWithInfo(String key_, std::optional info_ = std::nullopt) : key(std::move(key_)), info(std::move(info_)) {} + virtual ~KeyWithInfo() = default; + String key; std::optional info; }; From d6449921921e0b432dd8f246f376df155d47c55d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Sep 2023 16:25:04 +0200 Subject: [PATCH 071/634] Fxi --- src/Storages/StorageS3.cpp | 6 ++---- src/Storages/StorageS3Cluster.cpp | 1 + 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 1bb2dccbccc..8d391a02d30 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -570,7 +570,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() do { key_with_info = (*file_iterator)(); - if (!key_with_info) + if (!key_with_info || key_with_info->key.empty()) return {}; if (!key_with_info->info) @@ -1494,7 +1494,7 @@ namespace { current_key_with_info = (*file_iterator)(); - if (!current_key_with_info) + if (!current_key_with_info || current_key_with_info->key.empty()) { if (first) throw Exception( @@ -1506,8 +1506,6 @@ namespace return nullptr; } - chassert(!current_key_with_info->key.empty()); - /// S3 file iterator could get new keys after new iteration, check them in schema cache. if (getContext()->getSettingsRef().schema_inference_use_cache_for_s3 && read_keys.size() > prev_read_keys_size) { diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 8649c3f1baa..824dae6bc3e 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -82,6 +82,7 @@ RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(ASTPtr { auto iterator = std::make_shared( *s3_configuration.client, s3_configuration.url, query, virtual_columns, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback()); + auto callback = std::make_shared>([iterator]() mutable -> String { if (auto next = iterator->next()) From a7f389a7601021137ed9cf32290935ff5ecfedba Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 29 Sep 2023 15:14:25 +0200 Subject: [PATCH 072/634] Fix --- .../DataLakes/IcebergMetadataParser.cpp | 110 ++++++++++++------ 1 file changed, 74 insertions(+), 36 deletions(-) diff --git a/src/Storages/DataLakes/IcebergMetadataParser.cpp b/src/Storages/DataLakes/IcebergMetadataParser.cpp index 3820bb08247..666e046c169 100644 --- a/src/Storages/DataLakes/IcebergMetadataParser.cpp +++ b/src/Storages/DataLakes/IcebergMetadataParser.cpp @@ -154,13 +154,12 @@ struct IcebergMetadataParser::Impl */ void processManifestList(Metadata & metadata, const Configuration & configuration, ContextPtr context) { - static constexpr auto manifest_path = "manifest_path"; - auto buf = MetadataReadHelper::createReadBuffer(metadata.manifest_list, context, configuration); auto file_reader = std::make_unique(std::make_unique(*buf)); auto data_type = AvroSchemaReader::avroNodeToDataType(file_reader->dataSchema().root()->leafAt(0)); - auto columns = parseAvro(*file_reader, data_type, manifest_path, getFormatSettings(context)); + Block header{{data_type->createColumn(), data_type, "manifest_path"}}; + auto columns = parseAvro(*file_reader, header, getFormatSettings(context)); auto & col = columns.at(0); if (col->getDataType() != TypeIndex::String) @@ -207,9 +206,7 @@ struct IcebergMetadataParser::Impl */ Strings getFilesForRead(const Metadata & metadata, const Configuration & configuration, ContextPtr context) { - static constexpr auto manifest_path = "data_file"; - - Strings keys; + NameSet keys; for (const auto & manifest_file : metadata.manifest_files) { auto buffer = MetadataReadHelper::createReadBuffer(manifest_file, context, configuration); @@ -226,59 +223,100 @@ struct IcebergMetadataParser::Impl root_node->leaves(), expected_min_num); } + avro::NodePtr status_node = root_node->leafAt(0); + if (status_node->type() != avro::Type::AVRO_INT) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `status` field should be Int type, got {}", + magic_enum::enum_name(status_node->type())); + } + avro::NodePtr data_file_node = root_node->leafAt(static_cast(leaves_num) - 1); if (data_file_node->type() != avro::Type::AVRO_RECORD) { throw Exception( ErrorCodes::ILLEGAL_COLUMN, "The parsed column from Avro file of `data_file` field should be Tuple type, got {}", - data_file_node->type()); - } - auto data_type = AvroSchemaReader::avroNodeToDataType(data_file_node); - const auto columns = parseAvro(*file_reader, data_type, manifest_path, getFormatSettings(context)); - const auto col_tuple = typeid_cast(columns.at(0).get()); - - ColumnPtr col_str; - if (metadata.format_version == 1) - col_str = col_tuple->getColumnPtr(0); - else - col_str = col_tuple->getColumnPtr(1); - - if (col_str->getDataType() != TypeIndex::String) - { - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `file_path` field should be String type, got {}", - col_str->getFamilyName()); + magic_enum::enum_name(data_file_node->type())); } - const auto * str_col = assert_cast(col_str.get()); - for (size_t i = 0; i < str_col->size(); ++i) + auto status_col_data_type = AvroSchemaReader::avroNodeToDataType(status_node); + auto data_col_data_type = AvroSchemaReader::avroNodeToDataType(data_file_node); + Block header{ + {status_col_data_type->createColumn(), status_col_data_type, "status"}, + {data_col_data_type->createColumn(), data_col_data_type, "data_file"}}; + + const auto columns = parseAvro(*file_reader, header, getFormatSettings(context)); + if (columns.size() != 2) { - const auto data_path = std::string(str_col->getDataAt(i).toView()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Unexpected number of columns. Expected 2, got {}", columns.size()); + } + + if (columns.at(0)->getDataType() != TypeIndex::Int32) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `status` field should be Int32 type, got {}", + columns.at(0)->getFamilyName()); + } + if (columns.at(1)->getDataType() != TypeIndex::Tuple) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `file_path` field should be Tuple type, got {}", + columns.at(1)->getFamilyName()); + } + + const auto status_int_column = assert_cast(columns.at(0).get()); + const auto data_file_tuple_column = assert_cast(columns.at(1).get()); + + if (status_int_column->size() != data_file_tuple_column->size()) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `file_path` and `status` have different rows number: {} and {}", + status_int_column->size(), data_file_tuple_column->size()); + } + + const auto * data_file_name_column = metadata.format_version == 1 + ? data_file_tuple_column->getColumnPtr(0).get() + : data_file_tuple_column->getColumnPtr(1).get(); + + if (data_file_name_column->getDataType() != TypeIndex::String) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `file_path` field should be String type, got {}", + data_file_name_column->getFamilyName()); + } + auto file_name_str_column = assert_cast(data_file_name_column); + + for (size_t i = 0; i < status_int_column->size(); ++i) + { + const auto status = status_int_column->getInt(i); + const auto data_path = std::string(file_name_str_column->getDataAt(i).toView()); const auto pos = data_path.find(configuration.url.key); + const auto file_path = data_path.substr(pos); if (pos == std::string::npos) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected to find {} in data path: {}", configuration.url.key, data_path); - keys.emplace_back(data_path.substr(pos)); + + if (status == 2) + keys.erase(file_path); + else + keys.insert(file_path); } } - return keys; + return std::vector(keys.begin(), keys.end()); } MutableColumns parseAvro( avro::DataFileReaderBase & file_reader, - const DataTypePtr & data_type, - const String & field_name, + const Block & header, const FormatSettings & settings) { - auto deserializer = std::make_unique( - Block{{data_type->createColumn(), data_type, field_name}}, file_reader.dataSchema(), true, true, settings); + auto deserializer = std::make_unique(header, file_reader.dataSchema(), true, true, settings); + MutableColumns columns = header.cloneEmptyColumns(); file_reader.init(); - MutableColumns columns; - columns.emplace_back(data_type->createColumn()); - RowReadExtension ext; while (file_reader.hasMore()) { From 1ef21bab3d7f64c0594a61bb73b640c2651baf64 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 29 Sep 2023 16:02:51 +0200 Subject: [PATCH 073/634] Fix data race --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 81 +++++++++++-------- src/Storages/S3Queue/S3QueueFilesMetadata.h | 3 +- src/Storages/S3Queue/S3QueueSource.cpp | 26 +++--- src/Storages/S3Queue/S3QueueSource.h | 2 +- src/Storages/System/StorageSystemS3Queue.cpp | 5 +- 5 files changed, 71 insertions(+), 46 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index dbbf7e2ee1d..1856490c92e 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -208,42 +208,46 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs /// Processing state is cached only if processing is being done by current clickhouse server /// (because If another server is doing the processing, /// we cannot know if state changes without checking with zookeeper so there is no point in cache here). - switch (file_status->state) + { - case FileStatus::State::Processing: [[fallthrough]]; - case FileStatus::State::Processed: + std::lock_guard lock(file_status->metadata_lock); + switch (file_status->state) { - return nullptr; - } - case FileStatus::State::Failed: - { - /// If max_loading_retries == 0, file is not retriable. - if (max_loading_retries == 0) + case FileStatus::State::Processing: [[fallthrough]]; + case FileStatus::State::Processed: + { return nullptr; + } + case FileStatus::State::Failed: + { + /// If max_loading_retries == 0, file is not retriable. + if (max_loading_retries == 0) + return nullptr; - /// Otherwise file_status->retries is also cached. - /// In case file_status->retries >= max_loading_retries we can fully rely that it is true - /// and will not attempt processing it. - /// But in case file_status->retries < max_loading_retries we cannot be sure - /// (another server could have done a try after we cached retries value), - /// so check with zookeeper here. - if (file_status->retries >= max_loading_retries) - return nullptr; + /// Otherwise file_status->retries is also cached. + /// In case file_status->retries >= max_loading_retries we can fully rely that it is true + /// and will not attempt processing it. + /// But in case file_status->retries < max_loading_retries we cannot be sure + /// (another server could have done a try after we cached retries value), + /// so check with zookeeper here. + if (file_status->retries >= max_loading_retries) + return nullptr; - break; - } - case FileStatus::State::None: - { - /// The file was not processed by current server and file status was not cached, - /// check metadata in zookeeper. - break; + break; + } + case FileStatus::State::None: + { + /// The file was not processed by current server and file status was not cached, + /// check metadata in zookeeper. + break; + } } } /// Another thread could already be trying to set file as processing. /// So there is no need to attempt the same, better to continue with the next file. - std::unique_lock lock(file_status->processing_lock, std::defer_lock); - if (!lock.try_lock()) + std::unique_lock processing_lock(file_status->processing_lock, std::defer_lock); + if (!processing_lock.try_lock()) { return nullptr; } @@ -272,6 +276,7 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs { case SetFileProcessingResult::Success: { + std::lock_guard lock(file_status->metadata_lock); file_status->state = FileStatus::State::Processing; file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); @@ -284,11 +289,13 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs } case SetFileProcessingResult::AlreadyProcessed: { + std::lock_guard lock(file_status->metadata_lock); file_status->state = FileStatus::State::Processed; break; } case SetFileProcessingResult::AlreadyFailed: { + std::lock_guard lock(file_status->metadata_lock); file_status->state = FileStatus::State::Failed; break; } @@ -408,12 +415,16 @@ void S3QueueFilesMetadata::setFileProcessed(ProcessingNodeHolderPtr holder) auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessedMicroseconds); const auto & path = holder->path; - SCOPE_EXIT({ - auto file_status = local_file_statuses.get(path, /* create */false); + auto file_status = local_file_statuses.get(path, /* create */false); + { + std::lock_guard lock(file_status->metadata_lock); file_status->state = FileStatus::State::Processed; + file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + } + + SCOPE_EXIT({ file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessedMicroseconds, timer.get()); timer.cancel(); - file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); }); switch (mode) @@ -524,9 +535,12 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S const auto & path = holder->path; auto file_status = local_file_statuses.get(path, /* create */false); - file_status->state = FileStatus::State::Failed; - file_status->last_exception = exception_message; - file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + { + std::lock_guard lock(file_status->metadata_lock); + file_status->state = FileStatus::State::Failed; + file_status->last_exception = exception_message; + file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + } SCOPE_EXIT({ file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileFailedMicroseconds, timer.get()); @@ -583,6 +597,8 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S if (zk_client->tryGet(zookeeper_failed_path / node_name_with_retriable_suffix, res, &stat)) { node_metadata.retries = failed_node_metadata.retries + 1; + + std::lock_guard lock(file_status->metadata_lock); file_status->retries = node_metadata.retries; } @@ -771,6 +787,7 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TEST(log, "Cleanup is already being executed by another node"); return; } + /// TODO because of this lock we might not update local file statuses on time on one of the nodes. struct Node { diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index f6d98d13aaf..7eb5c40eb26 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -59,7 +59,7 @@ public: }; State state = State::None; - size_t processed_rows = 0; + std::atomic processed_rows = 0; time_t processing_start_time = 0; time_t processing_end_time = 0; size_t retries = 0; @@ -67,6 +67,7 @@ public: ProfileEvents::Counters profile_counters; std::mutex processing_lock; + std::mutex metadata_lock; }; using FileStatuses = std::unordered_map>; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 09713e77f37..d86fc6fe1ce 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -201,22 +201,26 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & path) } } -void StorageS3QueueSource::appendLogElement(const std::string & filename, const S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed) +void StorageS3QueueSource::appendLogElement(const std::string & filename, S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed) { if (!s3_queue_log) return; - S3QueueLogElement elem + S3QueueLogElement elem{}; { - .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), - .file_name = filename, - .rows_processed = processed_rows, - .status = processed ? S3QueueLogElement::S3QueueStatus::Processed : S3QueueLogElement::S3QueueStatus::Failed, - .counters_snapshot = file_status_.profile_counters.getPartiallyAtomicSnapshot(), - .processing_start_time = file_status_.processing_start_time, - .processing_end_time = file_status_.processing_end_time, - .exception = file_status_.last_exception, - }; + std::lock_guard lock(file_status_.metadata_lock); + elem = S3QueueLogElement + { + .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), + .file_name = filename, + .rows_processed = processed_rows, + .status = processed ? S3QueueLogElement::S3QueueStatus::Processed : S3QueueLogElement::S3QueueStatus::Failed, + .counters_snapshot = file_status_.profile_counters.getPartiallyAtomicSnapshot(), + .processing_start_time = file_status_.processing_start_time, + .processing_end_time = file_status_.processing_end_time, + .exception = file_status_.last_exception, + }; + } s3_queue_log->add(std::move(elem)); } diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index fa21b6cdd59..9bd2cad9c72 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -91,7 +91,7 @@ private: std::shared_ptr file_status; void applyActionAfterProcessing(const String & path); - void appendLogElement(const std::string & filename, const S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); + void appendLogElement(const std::string & filename, S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); }; } diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index ce76469b72a..235f4ff6f8f 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -47,7 +47,10 @@ void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr, co size_t i = 0; res_columns[i++]->insert(zookeeper_path); res_columns[i++]->insert(file_name); - res_columns[i++]->insert(file_status->processed_rows); + + std::lock_guard lock(file_status->metadata_lock); + + res_columns[i++]->insert(file_status->processed_rows.load()); res_columns[i++]->insert(magic_enum::enum_name(file_status->state)); if (file_status->processing_start_time) From d1f688811933af46b709815a860162e607898f0e Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 29 Sep 2023 16:17:02 +0200 Subject: [PATCH 074/634] Allow unique consumer identifier --- .../database-engines/materialized-postgresql.md | 5 +++++ .../PostgreSQL/DatabaseMaterializedPostgreSQL.cpp | 12 +++++++++++- .../PostgreSQL/MaterializedPostgreSQLSettings.h | 1 + .../PostgreSQL/StorageMaterializedPostgreSQL.cpp | 11 ++++++++++- 4 files changed, 27 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 33d75dc9582..4e978947e36 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -197,6 +197,11 @@ Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.htm ALTER DATABASE postgres_database MODIFY SETTING materialized_postgresql_max_block_size = ; ``` +### `materialized_postgresql_use_unique_replication_consumer_identifier` {#materialized_postgresql_use_unique_replication_consumer_identifier} + +Use a unique replication consumer identifier for replication. Default: `0`. +If set to `1`, allows to setup several `MaterializedPostgreSQL` tables pointing to the same `PostgreSQL` table. + ## Notes {#notes} ### Failover of the logical replication slot {#logical-replication-slot-failover} diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index f2b970a39af..71a508e818b 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -63,8 +63,18 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() if (shutdown_called) return; + String replication_identifier; + if (settings->materialized_postgresql_use_unique_replication_consumer_identifier) + { + replication_identifier = fmt::format("{}_{}", getUUID(), TSA_SUPPRESS_WARNING_FOR_READ(database_name)); + } + else + { + replication_identifier = TSA_SUPPRESS_WARNING_FOR_READ(database_name); + } + replication_handler = std::make_unique( - /* replication_identifier */ TSA_SUPPRESS_WARNING_FOR_READ(database_name), /// FIXME + replication_identifier, remote_database_name, TSA_SUPPRESS_WARNING_FOR_READ(database_name), /// FIXME connection_info, diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h index d3d2faba497..18473e1bb26 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h @@ -24,6 +24,7 @@ namespace DB M(UInt64, materialized_postgresql_backoff_min_ms, 200, "Poll backoff start point", 0) \ M(UInt64, materialized_postgresql_backoff_max_ms, 10000, "Poll backoff max point", 0) \ M(UInt64, materialized_postgresql_backoff_factor, 2, "Poll backoff factor", 0) \ + M(Bool, materialized_postgresql_use_unique_replication_consumer_identifier, false, "Should a unique consumer be registered for table replication", 0) \ DECLARE_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index e6389da32fd..d871b846ece 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -74,7 +74,16 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( setInMemoryMetadata(storage_metadata); - String replication_identifier = remote_database_name + "_" + remote_table_name_; + String replication_identifier; + if (replication_settings->materialized_postgresql_use_unique_replication_consumer_identifier) + { + replication_identifier = fmt::format("{}_{}_{}", table_id_.uuid, remote_database_name, remote_table_name_); + } + else + { + replication_identifier = fmt::format("{}_{}", remote_database_name, remote_table_name_); + } + replication_settings->materialized_postgresql_tables_list = remote_table_name_; replication_handler = std::make_unique( From c64e5e282c3e8d9fc91acb87d63955677046875b Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 29 Sep 2023 17:44:32 +0000 Subject: [PATCH 075/634] Fix file cache temporary file segment range in FileSegment::reserve --- src/Interpreters/Cache/FileSegment.cpp | 3 +- .../tests/gtest_lru_file_cache.cpp | 49 ++++++++++++++++--- 2 files changed, 44 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index bb3216cb20e..60c843a3df5 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -506,7 +506,8 @@ bool FileSegment::reserve(size_t size_to_reserve, FileCacheReserveStat * reserve /// This (resizable file segments) is allowed only for single threaded use of file segment. /// Currently it is used only for temporary files through cache. if (is_unbound && is_file_segment_size_exceeded) - segment_range.right = range().left + expected_downloaded_size + size_to_reserve; + /// Note: segment_range.right is inclusive. + segment_range.right = range().left + expected_downloaded_size + size_to_reserve - 1; /// if reserve_stat is not passed then use dummy stat and discard the result. FileCacheReserveStat dummy_stat; diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 2f749cd23fa..3ed04f367ad 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -1,6 +1,12 @@ #include #include #include + +#include +#include +#include + +#include #include #include #include @@ -180,6 +186,12 @@ public: else setupLogs(TEST_LOG_LEVEL); + UInt64 seed = randomSeed(); + if (const char * random_seed = std::getenv("TEST_RANDOM_SEED")) // NOLINT(concurrency-mt-unsafe) + seed = std::stoull(random_seed); + std::cout << "TEST_RANDOM_SEED=" << seed << std::endl; + rng = pcg64(seed); + if (fs::exists(cache_base_path)) fs::remove_all(cache_base_path); fs::create_directories(cache_base_path); @@ -191,6 +203,7 @@ public: fs::remove_all(cache_base_path); } + pcg64 rng; }; TEST_F(FileCacheTest, get) @@ -672,7 +685,7 @@ TEST_F(FileCacheTest, writeBuffer) FileCache cache("6", settings); cache.initialize(); - auto write_to_cache = [&cache](const String & key, const Strings & data, bool flush) + auto write_to_cache = [&cache, this](const String & key, const Strings & data, bool flush, ReadBufferPtr * out_read_buffer = nullptr) { CreateFileSegmentSettings segment_settings; segment_settings.kind = FileSegmentKind::Temporary; @@ -687,24 +700,32 @@ TEST_F(FileCacheTest, writeBuffer) WriteBufferToFileSegment out(&segment); std::list threads; std::mutex mu; - for (const auto & s : data) + + /// get random permutation of indexes + std::vector indexes(data.size()); + std::iota(indexes.begin(), indexes.end(), 0); + std::shuffle(indexes.begin(), indexes.end(), rng); + + for (auto i : indexes) { /// Write from diffetent threads to check /// that no assertions inside cache related to downloaderId are triggered + const auto & s = data[i]; threads.emplace_back([&] { std::unique_lock lock(mu); out.write(s.data(), s.size()); /// test different buffering scenarios if (flush) - { out.next(); - } }); } for (auto & t : threads) t.join(); + out.finalize(); + if (out_read_buffer) + *out_read_buffer = out.tryGetReadBuffer(); return holder; }; @@ -714,17 +735,31 @@ TEST_F(FileCacheTest, writeBuffer) file_segment_paths.emplace_back(holder->front().getPathInLocalCache()); ASSERT_EQ(fs::file_size(file_segment_paths.back()), 7); - ASSERT_TRUE(holder->front().range() == FileSegment::Range(0, 7)); + EXPECT_EQ(holder->front().range().size(), 7); + EXPECT_EQ(holder->front().range().left, 0); ASSERT_EQ(cache.getUsedCacheSize(), 7); { - auto holder2 = write_to_cache("key2", {"1", "22", "333", "4444", "55555"}, true); + ReadBufferPtr reader = nullptr; + + auto holder2 = write_to_cache("key2", {"22", "333", "4444", "55555", "1"}, true, &reader); file_segment_paths.emplace_back(holder2->front().getPathInLocalCache()); std::cerr << "\nFile segments: " << holder2->toString() << "\n"; ASSERT_EQ(fs::file_size(file_segment_paths.back()), 15); - ASSERT_EQ(holder2->front().range(), FileSegment::Range(0, 15)); + EXPECT_TRUE(reader); + if (reader) + { + String result; + readStringUntilEOF(result, *reader); + /// sort result to make it independent of the order of writes + std::sort(result.begin(), result.end()); + EXPECT_EQ(result, "122333444455555"); + } + + EXPECT_EQ(holder2->front().range().size(), 15); + EXPECT_EQ(holder2->front().range().left, 0); ASSERT_EQ(cache.getUsedCacheSize(), 22); } ASSERT_FALSE(fs::exists(file_segment_paths.back())); From bff832c7f0e47a2bd84da7b424a33d4b5a5ee3cf Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Thu, 17 Aug 2023 08:19:20 +0200 Subject: [PATCH 076/634] Disable logic max_threads=max_distributed_connections when async_socket_for_remote=1 See #53287 --- src/Interpreters/InterpreterSelectQuery.cpp | 27 +- src/Planner/PlannerJoinTree.cpp | 27 +- tests/analyzer_tech_debt.txt | 1 + ...ads_count_in_distributed_queries.reference | 15 ++ ...hreads_count_in_distributed_queries.sql.j2 | 238 ++++++++++++++++++ 5 files changed, 286 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02845_threads_count_in_distributed_queries.reference create mode 100644 tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e1faa8c8958..6037e205eca 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2328,17 +2328,22 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc /// General limit for the number of threads. size_t max_threads_execute_query = settings.max_threads; - /** With distributed query processing, almost no computations are done in the threads, - * but wait and receive data from remote servers. - * If we have 20 remote servers, and max_threads = 8, then it would not be very good - * connect and ask only 8 servers at a time. - * To simultaneously query more remote servers, - * instead of max_threads, max_distributed_connections is used. + /** + * To simultaneously query more remote servers when async_socket_for_remote is off + * instead of max_threads, max_distributed_connections is used: + * since threads there mostly spend time waiting for data from remote servers, + * we can increase the degree of parallelism to avoid sequential querying of remote servers. + * + * DANGER: that can lead to insane number of threads working if there are a lot of stream and prefer_localhost_replica is used. + * + * That is not needed when async_socket_for_remote is on, because in that case + * threads are not blocked waiting for data from remote servers. + * */ - bool is_remote = false; - if (storage && storage->isRemote()) + bool is_sync_remote = false; + if (storage && storage->isRemote() && !settings.async_socket_for_remote) { - is_remote = true; + is_sync_remote = true; max_threads_execute_query = max_streams = settings.max_distributed_connections; } @@ -2415,7 +2420,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc max_streams = 1; /// If necessary, we request more sources than the number of threads - to distribute the work evenly over the threads. - if (max_streams > 1 && !is_remote) + if (max_streams > 1 && !is_sync_remote) max_streams = static_cast(max_streams * settings.max_streams_to_max_threads_ratio); auto & prewhere_info = analysis_result.prewhere_info; @@ -2513,7 +2518,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc /// The inner local query (that is done in the same process, without /// network interaction), it will setMaxThreads earlier and distributed /// query will not update it. - if (!query_plan.getMaxThreads() || is_remote) + if (!query_plan.getMaxThreads() || is_sync_remote) query_plan.setMaxThreads(max_threads_execute_query); query_plan.setConcurrencyControl(settings.use_concurrency_control); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index c95671da6be..d70f901164f 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -557,15 +557,20 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres size_t max_streams = settings.max_threads; size_t max_threads_execute_query = settings.max_threads; - /** With distributed query processing, almost no computations are done in the threads, - * but wait and receive data from remote servers. - * If we have 20 remote servers, and max_threads = 8, then it would not be efficient to - * connect and ask only 8 servers at a time. - * To simultaneously query more remote servers, - * instead of max_threads, max_distributed_connections is used. - */ - bool is_remote = table_expression_data.isRemote(); - if (is_remote) + /** + * To simultaneously query more remote servers when async_socket_for_remote is off + * instead of max_threads, max_distributed_connections is used: + * since threads there mostly spend time waiting for data from remote servers, + * we can increase the degree of parallelism to avoid sequential querying of remote servers. + * + * DANGER: that can lead to insane number of threads working if there are a lot of stream and prefer_localhost_replica is used. + * + * That is not needed when async_socket_for_remote is on, because in that case + * threads are not blocked waiting for data from remote servers. + * + */ + bool is_sync_remote = table_expression_data.isRemote() && !settings.async_socket_for_remote; + if (is_sync_remote) { max_streams = settings.max_distributed_connections; max_threads_execute_query = settings.max_distributed_connections; @@ -632,7 +637,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres max_streams = 1; /// If necessary, we request more sources than the number of threads - to distribute the work evenly over the threads - if (max_streams > 1 && !is_remote) + if (max_streams > 1 && !is_sync_remote) max_streams = static_cast(max_streams * settings.max_streams_to_max_threads_ratio); if (table_node) @@ -792,7 +797,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres * network interaction), it will setMaxThreads earlier and distributed * query will not update it. */ - if (!query_plan.getMaxThreads() || is_remote) + if (!query_plan.getMaxThreads() || is_sync_remote) query_plan.setMaxThreads(max_threads_execute_query); query_plan.setConcurrencyControl(settings.use_concurrency_control); diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 652ab0b99de..352bb0a0c94 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -128,4 +128,5 @@ 02790_optimize_skip_unused_shards_join 01940_custom_tld_sharding_key 02815_range_dict_no_direct_join +02845_threads_count_in_distributed_queries 02861_join_on_nullsafe_compare diff --git a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.reference b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.reference new file mode 100644 index 00000000000..0e3f6336468 --- /dev/null +++ b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.reference @@ -0,0 +1,15 @@ +prefer_localhost_replica=1, remote query with a lot of union all +379 +ok +prefer_localhost_replica=0, remote query with a lot of union all +379 +ok +prefer_localhost_replica=1, async_socket_for_remote=0, remote query with a lot of union all (lot of threads) +379 +ok +prepare test schema +500 +prefer_localhost_replica=1, remote query with read in order +ok +prefer_localhost_replica=1 + async_socket_for_remote=0, remote query with read in order (lot of threads) +ok diff --git a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 new file mode 100644 index 00000000000..6039e12071d --- /dev/null +++ b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 @@ -0,0 +1,238 @@ +-- enforce some defaults to be sure that the env settings will not affect the test +SET max_threads=5, async_socket_for_remote=1, async_socket_for_remote=1, optimize_read_in_order=1; + + +-- we use query_thread_log to check peak thread usage +-- after https://github.com/ClickHouse/ClickHouse/issues/53417 there is a simpler way to check it +-- but that will not allow to backport the test to older versions +SET log_query_threads=1; + + +-------------------- +SELECT 'prefer_localhost_replica=1, remote query with a lot of union all' AS testname; + +-- query with lot of dummy union all will create a lot of streams +-- let's check how many threads clickhouse will start for that + +select count() from remote('127.0.0.1:9000', view( +{% for n in range(379) -%} +SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} +{% endfor -%} + )) SETTINGS log_comment='check_concurrency_in_remote_queries1'; + +SYSTEM FLUSH LOGS; + +WITH + maxIntersections( + toUnixTimestamp64Micro(query_start_time_microseconds), + toUnixTimestamp64Micro(event_time_microseconds) + ) as peak_threads +SELECT + if(peak_threads BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads) ) AS result +FROM system.query_thread_log +WHERE + event_time > now() - 60 + AND query_id = ( + SELECT query_id + FROM system.query_log + WHERE + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries1' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1 + ); + +-------------------- +SELECT 'prefer_localhost_replica=0, remote query with a lot of union all' AS testname; + +select count() from remote('127.0.0.1:9000', view( +{% for n in range(379) -%} +SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} +{% endfor -%} + )) SETTINGS log_comment='check_concurrency_in_remote_queries2', prefer_localhost_replica=0; + +SYSTEM FLUSH LOGS; + +WITH + maxIntersections( + toUnixTimestamp64Micro(query_start_time_microseconds), + toUnixTimestamp64Micro(event_time_microseconds) + ) as peak_threads +SELECT + if(peak_threads BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads) ) AS result +FROM system.query_thread_log +WHERE + event_time > now() - 60 + AND query_id = ( + SELECT query_id + FROM system.query_log + WHERE + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries2' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1 + ); + +-------------------- +SELECT 'prefer_localhost_replica=1, async_socket_for_remote=0, remote query with a lot of union all (lot of threads)' AS testname; + +-- that is actually a bad behaviour, but it used to work like that for a long time. +-- now is happens only for async_socket_for_remote=0 (while it is 1 by default) +-- see https://github.com/ClickHouse/ClickHouse/issues/53287 + +select count() from remote('127.0.0.1:9000', view( +{% for n in range(379) -%} +SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} +{% endfor -%} + )) SETTINGS log_comment='check_concurrency_in_remote_queries3', async_socket_for_remote=0, prefer_localhost_replica=1; + +SYSTEM FLUSH LOGS; + +WITH + maxIntersections( + toUnixTimestamp64Micro(query_start_time_microseconds), + toUnixTimestamp64Micro(event_time_microseconds) + ) as peak_threads +SELECT + if(peak_threads >= 379, 'ok', 'too few threads: ' || toString(peak_threads) ) AS result +FROM system.query_thread_log +WHERE + event_time > now() - 60 + AND query_id = ( + SELECT query_id + FROM system.query_log + WHERE + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries3' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1 + ); + +-- less synthetic testcase from the issue https://github.com/ClickHouse/ClickHouse/issues/53287 +-- it creates lot of streams because of many parts (one per part) + optimize_read_in_order=1 feature + +SELECT 'prepare test schema' AS stage; + +DROP TABLE IF EXISTS test_lot_of_parts_distributed; +DROP TABLE IF EXISTS test_lot_of_parts; + +CREATE TABLE test_lot_of_parts +( + `a` String, + `b` LowCardinality(String), + `c` DateTime64(3), + `val` String, +) +ENGINE = MergeTree +ORDER BY (a, b, c) +SETTINGS parts_to_delay_insert=0; + +CREATE TABLE test_lot_of_parts_distributed +( + `a` String, + `b` LowCardinality(String), + `c` DateTime64(3), + `val` String, +) +ENGINE = Distributed(test_shard_localhost, currentDatabase(), 'test_lot_of_parts', rand()); + +-- we need a lot of parts to make sure that we will have a lot of streams +SYSTEM STOP MERGES test_lot_of_parts; +INSERT INTO test_lot_of_parts (a, b, c, val) + SELECT + 'foo' as a, + 'bar' as b, + _CAST('1683504000', 'DateTime64') as c, + 'baz' as val + FROM numbers_mt(500) + SETTINGS max_block_size = 1, min_insert_block_size_bytes=1, min_insert_block_size_rows=1; --every row will be in separate part + +select count() from system.parts where table = 'test_lot_of_parts' and active and database = currentDatabase(); + +SELECT 'prefer_localhost_replica=1, remote query with read in order' AS testname; + +--- it seem like the number of threads here reported here is still higher than expected +--- max_threads=1 => peak_threads=7 +--- max_threads=2 => peak_threads=14 +--- max_threads=5 => peak_threads=25 +--- may be it's a matter of the way how we count threads in thread_log, or there is another problem here +--- anyway it still much better than before (see the next test in that file - it uses 500 threads). + +-- query which uses optimize_read_in_order=1 +SELECT DISTINCT + 'val' AS fieldType, + val AS value +FROM test_lot_of_parts_distributed +WHERE a = 'foo' AND value != '' AND positionCaseInsensitiveUTF8(value, 'baz') > 0 AND b = 'bar' AND c >= _CAST('1683504000', 'DateTime64') +ORDER BY c DESC +LIMIT 5 +SETTINGS log_comment='check_concurrency_in_remote_queries4' FORMAT Null; + +SYSTEM FLUSH LOGS; + +WITH + maxIntersections( + toUnixTimestamp64Micro(query_start_time_microseconds), + toUnixTimestamp64Micro(event_time_microseconds) + ) as peak_threads +SELECT + -- it should be max_threads+2 not max_threads*8 + if(peak_threads BETWEEN 1 AND toUInt64(getSetting('max_threads')) * 8, 'ok', 'too many threads: ' || toString(peak_threads) ) AS result +FROM system.query_thread_log +WHERE + event_time > now() - 60 + AND query_id = ( + SELECT query_id + FROM system.query_log + WHERE + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries4' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1 + ); + + +SELECT 'prefer_localhost_replica=1 + async_socket_for_remote=0, remote query with read in order (lot of threads)' AS testname; + +-- that is actually a bad behaviour, but it used to work like that for a long time. +-- now is happens only for async_socket_for_remote=0 (while it is 1 by default) + +SELECT DISTINCT + 'val' AS fieldType, + val AS value +FROM test_lot_of_parts_distributed +WHERE a = 'foo' AND value != '' AND positionCaseInsensitiveUTF8(value, 'baz') > 0 AND b = 'bar' AND c >= _CAST('1683504000', 'DateTime64') +ORDER BY c DESC +LIMIT 5 +SETTINGS log_comment='check_concurrency_in_remote_queries5', async_socket_for_remote=0 FORMAT Null; + +SYSTEM FLUSH LOGS; + +WITH + maxIntersections( + toUnixTimestamp64Micro(query_start_time_microseconds), + toUnixTimestamp64Micro(event_time_microseconds) + ) as peak_threads +SELECT + if(peak_threads >= 500, 'ok', 'too few threads: ' || toString(peak_threads) ) AS result + -- we have 500 parts +FROM system.query_thread_log +WHERE + event_time > now() - 60 + AND query_id = ( + SELECT query_id + FROM system.query_log + WHERE + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries5' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1 + ); + +DROP TABLE IF EXISTS test_lot_of_parts_distributed; +DROP TABLE IF EXISTS test_lot_of_parts; From bf1aa653d1711920329ac787a4e69ea6d3f950a0 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 27 Sep 2023 14:49:23 +0200 Subject: [PATCH 077/634] put limit on number of threads back, measure them via peak_threads_usage --- ...hreads_count_in_distributed_queries.sql.j2 | 137 +++++------------- 1 file changed, 36 insertions(+), 101 deletions(-) diff --git a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 index 6039e12071d..5be37bbacf6 100644 --- a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 +++ b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 @@ -1,5 +1,5 @@ -- enforce some defaults to be sure that the env settings will not affect the test -SET max_threads=5, async_socket_for_remote=1, async_socket_for_remote=1, optimize_read_in_order=1; +SET max_threads=5, async_socket_for_remote=1, async_socket_for_remote=1, optimize_read_in_order=1, prefer_localhost_replica=1; -- we use query_thread_log to check peak thread usage @@ -22,26 +22,15 @@ SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} SYSTEM FLUSH LOGS; -WITH - maxIntersections( - toUnixTimestamp64Micro(query_start_time_microseconds), - toUnixTimestamp64Micro(event_time_microseconds) - ) as peak_threads SELECT - if(peak_threads BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads) ) AS result -FROM system.query_thread_log + if(peak_threads_usage BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads_usage) ) AS result +FROM system.query_log WHERE - event_time > now() - 60 - AND query_id = ( - SELECT query_id - FROM system.query_log - WHERE - type = 'QueryFinish' - AND event_time > now() - 60 - AND log_comment = 'check_concurrency_in_remote_queries1' - AND current_database = currentDatabase() - ORDER BY event_time DESC LIMIT 1 - ); + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries1' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1; -------------------- SELECT 'prefer_localhost_replica=0, remote query with a lot of union all' AS testname; @@ -54,26 +43,15 @@ SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} SYSTEM FLUSH LOGS; -WITH - maxIntersections( - toUnixTimestamp64Micro(query_start_time_microseconds), - toUnixTimestamp64Micro(event_time_microseconds) - ) as peak_threads SELECT - if(peak_threads BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads) ) AS result -FROM system.query_thread_log + if(peak_threads_usage BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads_usage) ) AS result +FROM system.query_log WHERE - event_time > now() - 60 - AND query_id = ( - SELECT query_id - FROM system.query_log - WHERE - type = 'QueryFinish' - AND event_time > now() - 60 - AND log_comment = 'check_concurrency_in_remote_queries2' - AND current_database = currentDatabase() - ORDER BY event_time DESC LIMIT 1 - ); + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries2' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1; -------------------- SELECT 'prefer_localhost_replica=1, async_socket_for_remote=0, remote query with a lot of union all (lot of threads)' AS testname; @@ -90,26 +68,15 @@ SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} SYSTEM FLUSH LOGS; -WITH - maxIntersections( - toUnixTimestamp64Micro(query_start_time_microseconds), - toUnixTimestamp64Micro(event_time_microseconds) - ) as peak_threads SELECT - if(peak_threads >= 379, 'ok', 'too few threads: ' || toString(peak_threads) ) AS result -FROM system.query_thread_log + if(peak_threads_usage >= 379, 'ok', 'too few threads: ' || toString(peak_threads_usage) ) AS result +FROM system.query_log WHERE - event_time > now() - 60 - AND query_id = ( - SELECT query_id - FROM system.query_log - WHERE - type = 'QueryFinish' - AND event_time > now() - 60 - AND log_comment = 'check_concurrency_in_remote_queries3' - AND current_database = currentDatabase() - ORDER BY event_time DESC LIMIT 1 - ); + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries3' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1; -- less synthetic testcase from the issue https://github.com/ClickHouse/ClickHouse/issues/53287 -- it creates lot of streams because of many parts (one per part) + optimize_read_in_order=1 feature @@ -154,13 +121,6 @@ select count() from system.parts where table = 'test_lot_of_parts' and active an SELECT 'prefer_localhost_replica=1, remote query with read in order' AS testname; ---- it seem like the number of threads here reported here is still higher than expected ---- max_threads=1 => peak_threads=7 ---- max_threads=2 => peak_threads=14 ---- max_threads=5 => peak_threads=25 ---- may be it's a matter of the way how we count threads in thread_log, or there is another problem here ---- anyway it still much better than before (see the next test in that file - it uses 500 threads). - -- query which uses optimize_read_in_order=1 SELECT DISTINCT 'val' AS fieldType, @@ -173,28 +133,15 @@ SETTINGS log_comment='check_concurrency_in_remote_queries4' FORMAT Null; SYSTEM FLUSH LOGS; -WITH - maxIntersections( - toUnixTimestamp64Micro(query_start_time_microseconds), - toUnixTimestamp64Micro(event_time_microseconds) - ) as peak_threads SELECT - -- it should be max_threads+2 not max_threads*8 - if(peak_threads BETWEEN 1 AND toUInt64(getSetting('max_threads')) * 8, 'ok', 'too many threads: ' || toString(peak_threads) ) AS result -FROM system.query_thread_log + if(peak_threads_usage BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads_usage) ) AS result +FROM system.query_log WHERE - event_time > now() - 60 - AND query_id = ( - SELECT query_id - FROM system.query_log - WHERE - type = 'QueryFinish' - AND event_time > now() - 60 - AND log_comment = 'check_concurrency_in_remote_queries4' - AND current_database = currentDatabase() - ORDER BY event_time DESC LIMIT 1 - ); - + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries4' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1; SELECT 'prefer_localhost_replica=1 + async_socket_for_remote=0, remote query with read in order (lot of threads)' AS testname; @@ -212,27 +159,15 @@ SETTINGS log_comment='check_concurrency_in_remote_queries5', async_socket_for_re SYSTEM FLUSH LOGS; -WITH - maxIntersections( - toUnixTimestamp64Micro(query_start_time_microseconds), - toUnixTimestamp64Micro(event_time_microseconds) - ) as peak_threads SELECT - if(peak_threads >= 500, 'ok', 'too few threads: ' || toString(peak_threads) ) AS result - -- we have 500 parts -FROM system.query_thread_log + if(peak_threads_usage >= 500, 'ok', 'too few threads: ' || toString(peak_threads_usage) ) AS result +FROM system.query_log WHERE - event_time > now() - 60 - AND query_id = ( - SELECT query_id - FROM system.query_log - WHERE - type = 'QueryFinish' - AND event_time > now() - 60 - AND log_comment = 'check_concurrency_in_remote_queries5' - AND current_database = currentDatabase() - ORDER BY event_time DESC LIMIT 1 - ); + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries5' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1; DROP TABLE IF EXISTS test_lot_of_parts_distributed; DROP TABLE IF EXISTS test_lot_of_parts; From e699b4e93772f7a1cac50d32502726db7095a690 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 29 Sep 2023 11:07:25 +0200 Subject: [PATCH 078/634] Revert "put limit on number of threads back, measure them via peak_threads_usage" This reverts commit 4da2d7ca879749bb2930c0936cd33156c9c62ee0. --- ...hreads_count_in_distributed_queries.sql.j2 | 137 +++++++++++++----- 1 file changed, 101 insertions(+), 36 deletions(-) diff --git a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 index 5be37bbacf6..6039e12071d 100644 --- a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 +++ b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 @@ -1,5 +1,5 @@ -- enforce some defaults to be sure that the env settings will not affect the test -SET max_threads=5, async_socket_for_remote=1, async_socket_for_remote=1, optimize_read_in_order=1, prefer_localhost_replica=1; +SET max_threads=5, async_socket_for_remote=1, async_socket_for_remote=1, optimize_read_in_order=1; -- we use query_thread_log to check peak thread usage @@ -22,15 +22,26 @@ SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} SYSTEM FLUSH LOGS; +WITH + maxIntersections( + toUnixTimestamp64Micro(query_start_time_microseconds), + toUnixTimestamp64Micro(event_time_microseconds) + ) as peak_threads SELECT - if(peak_threads_usage BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads_usage) ) AS result -FROM system.query_log + if(peak_threads BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads) ) AS result +FROM system.query_thread_log WHERE - type = 'QueryFinish' - AND event_time > now() - 60 - AND log_comment = 'check_concurrency_in_remote_queries1' - AND current_database = currentDatabase() - ORDER BY event_time DESC LIMIT 1; + event_time > now() - 60 + AND query_id = ( + SELECT query_id + FROM system.query_log + WHERE + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries1' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1 + ); -------------------- SELECT 'prefer_localhost_replica=0, remote query with a lot of union all' AS testname; @@ -43,15 +54,26 @@ SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} SYSTEM FLUSH LOGS; +WITH + maxIntersections( + toUnixTimestamp64Micro(query_start_time_microseconds), + toUnixTimestamp64Micro(event_time_microseconds) + ) as peak_threads SELECT - if(peak_threads_usage BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads_usage) ) AS result -FROM system.query_log + if(peak_threads BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads) ) AS result +FROM system.query_thread_log WHERE - type = 'QueryFinish' - AND event_time > now() - 60 - AND log_comment = 'check_concurrency_in_remote_queries2' - AND current_database = currentDatabase() - ORDER BY event_time DESC LIMIT 1; + event_time > now() - 60 + AND query_id = ( + SELECT query_id + FROM system.query_log + WHERE + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries2' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1 + ); -------------------- SELECT 'prefer_localhost_replica=1, async_socket_for_remote=0, remote query with a lot of union all (lot of threads)' AS testname; @@ -68,15 +90,26 @@ SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} SYSTEM FLUSH LOGS; +WITH + maxIntersections( + toUnixTimestamp64Micro(query_start_time_microseconds), + toUnixTimestamp64Micro(event_time_microseconds) + ) as peak_threads SELECT - if(peak_threads_usage >= 379, 'ok', 'too few threads: ' || toString(peak_threads_usage) ) AS result -FROM system.query_log + if(peak_threads >= 379, 'ok', 'too few threads: ' || toString(peak_threads) ) AS result +FROM system.query_thread_log WHERE - type = 'QueryFinish' - AND event_time > now() - 60 - AND log_comment = 'check_concurrency_in_remote_queries3' - AND current_database = currentDatabase() - ORDER BY event_time DESC LIMIT 1; + event_time > now() - 60 + AND query_id = ( + SELECT query_id + FROM system.query_log + WHERE + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries3' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1 + ); -- less synthetic testcase from the issue https://github.com/ClickHouse/ClickHouse/issues/53287 -- it creates lot of streams because of many parts (one per part) + optimize_read_in_order=1 feature @@ -121,6 +154,13 @@ select count() from system.parts where table = 'test_lot_of_parts' and active an SELECT 'prefer_localhost_replica=1, remote query with read in order' AS testname; +--- it seem like the number of threads here reported here is still higher than expected +--- max_threads=1 => peak_threads=7 +--- max_threads=2 => peak_threads=14 +--- max_threads=5 => peak_threads=25 +--- may be it's a matter of the way how we count threads in thread_log, or there is another problem here +--- anyway it still much better than before (see the next test in that file - it uses 500 threads). + -- query which uses optimize_read_in_order=1 SELECT DISTINCT 'val' AS fieldType, @@ -133,15 +173,28 @@ SETTINGS log_comment='check_concurrency_in_remote_queries4' FORMAT Null; SYSTEM FLUSH LOGS; +WITH + maxIntersections( + toUnixTimestamp64Micro(query_start_time_microseconds), + toUnixTimestamp64Micro(event_time_microseconds) + ) as peak_threads SELECT - if(peak_threads_usage BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads_usage) ) AS result -FROM system.query_log + -- it should be max_threads+2 not max_threads*8 + if(peak_threads BETWEEN 1 AND toUInt64(getSetting('max_threads')) * 8, 'ok', 'too many threads: ' || toString(peak_threads) ) AS result +FROM system.query_thread_log WHERE - type = 'QueryFinish' - AND event_time > now() - 60 - AND log_comment = 'check_concurrency_in_remote_queries4' - AND current_database = currentDatabase() - ORDER BY event_time DESC LIMIT 1; + event_time > now() - 60 + AND query_id = ( + SELECT query_id + FROM system.query_log + WHERE + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries4' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1 + ); + SELECT 'prefer_localhost_replica=1 + async_socket_for_remote=0, remote query with read in order (lot of threads)' AS testname; @@ -159,15 +212,27 @@ SETTINGS log_comment='check_concurrency_in_remote_queries5', async_socket_for_re SYSTEM FLUSH LOGS; +WITH + maxIntersections( + toUnixTimestamp64Micro(query_start_time_microseconds), + toUnixTimestamp64Micro(event_time_microseconds) + ) as peak_threads SELECT - if(peak_threads_usage >= 500, 'ok', 'too few threads: ' || toString(peak_threads_usage) ) AS result -FROM system.query_log + if(peak_threads >= 500, 'ok', 'too few threads: ' || toString(peak_threads) ) AS result + -- we have 500 parts +FROM system.query_thread_log WHERE - type = 'QueryFinish' - AND event_time > now() - 60 - AND log_comment = 'check_concurrency_in_remote_queries5' - AND current_database = currentDatabase() - ORDER BY event_time DESC LIMIT 1; + event_time > now() - 60 + AND query_id = ( + SELECT query_id + FROM system.query_log + WHERE + type = 'QueryFinish' + AND event_time > now() - 60 + AND log_comment = 'check_concurrency_in_remote_queries5' + AND current_database = currentDatabase() + ORDER BY event_time DESC LIMIT 1 + ); DROP TABLE IF EXISTS test_lot_of_parts_distributed; DROP TABLE IF EXISTS test_lot_of_parts; From d8cddeaf907f669221972c44707bc49871c3dd8a Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 29 Sep 2023 17:31:35 +0200 Subject: [PATCH 079/634] Trying proposal by Azat --- ...02845_threads_count_in_distributed_queries.sql.j2 | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 index 6039e12071d..fe5744eccf1 100644 --- a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 +++ b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 @@ -1,5 +1,5 @@ -- enforce some defaults to be sure that the env settings will not affect the test -SET max_threads=5, async_socket_for_remote=1, async_socket_for_remote=1, optimize_read_in_order=1; +SET max_threads=5, async_socket_for_remote=1, optimize_read_in_order=1, load_marks_asynchronously=0, local_filesystem_read_method='pread'; -- we use query_thread_log to check peak thread usage @@ -154,13 +154,6 @@ select count() from system.parts where table = 'test_lot_of_parts' and active an SELECT 'prefer_localhost_replica=1, remote query with read in order' AS testname; ---- it seem like the number of threads here reported here is still higher than expected ---- max_threads=1 => peak_threads=7 ---- max_threads=2 => peak_threads=14 ---- max_threads=5 => peak_threads=25 ---- may be it's a matter of the way how we count threads in thread_log, or there is another problem here ---- anyway it still much better than before (see the next test in that file - it uses 500 threads). - -- query which uses optimize_read_in_order=1 SELECT DISTINCT 'val' AS fieldType, @@ -179,8 +172,7 @@ WITH toUnixTimestamp64Micro(event_time_microseconds) ) as peak_threads SELECT - -- it should be max_threads+2 not max_threads*8 - if(peak_threads BETWEEN 1 AND toUInt64(getSetting('max_threads')) * 8, 'ok', 'too many threads: ' || toString(peak_threads) ) AS result + if(peak_threads BETWEEN 1 AND toUInt64(getSetting('max_threads')) + 2, 'ok', 'too many threads: ' || toString(peak_threads) ) AS result FROM system.query_thread_log WHERE event_time > now() - 60 From bbba89977803200b83d7f35637304be886311b4d Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Mon, 2 Oct 2023 12:44:54 +0200 Subject: [PATCH 080/634] missed setting --- .../02845_threads_count_in_distributed_queries.sql.j2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 index fe5744eccf1..cd714fdc091 100644 --- a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 +++ b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 @@ -1,5 +1,5 @@ -- enforce some defaults to be sure that the env settings will not affect the test -SET max_threads=5, async_socket_for_remote=1, optimize_read_in_order=1, load_marks_asynchronously=0, local_filesystem_read_method='pread'; +SET max_threads=5, async_socket_for_remote=1, prefer_localhost_replica=1, optimize_read_in_order=1, load_marks_asynchronously=0, local_filesystem_read_method='pread'; -- we use query_thread_log to check peak thread usage From 46fbe00c9a32c828078b304f0f4a8e23c3d6fc3f Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Mon, 2 Oct 2023 18:50:56 +0100 Subject: [PATCH 081/634] Implement byteswap for following dtypes. UInt[8|16|32|64] TODOs: - Improve NOT_IMPLEMENTED error message. - Add implementation for FixedStrings (reverse the bytes). - See whether this needs to be implemented for UInt[128|256] and signed integers as well. --- src/Functions/byteSwap.cpp | 45 ++++++++++++++++---------------------- 1 file changed, 19 insertions(+), 26 deletions(-) diff --git a/src/Functions/byteSwap.cpp b/src/Functions/byteSwap.cpp index 071e50bb786..b1c2d571fb8 100644 --- a/src/Functions/byteSwap.cpp +++ b/src/Functions/byteSwap.cpp @@ -1,7 +1,5 @@ -#include #include #include -#include namespace DB { @@ -12,53 +10,46 @@ extern const int NOT_IMPLEMENTED; namespace { - template -requires std::is_integral_v && (sizeof(T) <= sizeof(UInt32)) -inline T roundDownToPowerOfTwo(T x) +requires std::is_same_v +inline T byteSwap(T x) { - return x <= 0 ? 0 : (T(1) << (31 - __builtin_clz(x))); + return x; } template -requires std::is_integral_v && (sizeof(T) == sizeof(UInt64)) -inline T roundDownToPowerOfTwo(T x) +requires std::is_same_v +inline T byteSwap(T x) { - return x <= 0 ? 0 : (T(1) << (63 - __builtin_clzll(x))); + return __builtin_bswap16(x); } template -requires std::is_same_v -inline T roundDownToPowerOfTwo(T x) +requires std::is_same_v +inline T byteSwap(T x) { - return bit_cast(bit_cast(x) & ~((1ULL << 23) - 1)); + return __builtin_bswap32(x); } template -requires std::is_same_v -inline T roundDownToPowerOfTwo(T x) +requires std::is_same_v +inline T byteSwap(T x) { - return bit_cast(bit_cast(x) & ~((1ULL << 52) - 1)); + return __builtin_bswap64(x); } template -requires is_big_int_v -inline T roundDownToPowerOfTwo(T) +inline T byteSwap(T) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "roundToExp2() for big integers is not implemented"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "byteSwap() is not implemented for {} datatype", demangle(typeid(T).name())); } - template struct ByteSwapImpl { using ResultType = T; - - static inline T apply(T x) - { - // return roundDownToPowerOfTwo(x); - return x; - } + static constexpr const bool allow_string_or_fixed_string = false; + static inline T apply(T x) { return byteSwap(x); } #if USE_EMBEDDED_COMPILER static constexpr bool compilable = false; @@ -74,8 +65,10 @@ using FunctionByteSwap = FunctionUnaryArithmetic -struct FunctionUnaryArithmeticMonotonicity : PositiveMonotonicity +struct FunctionUnaryArithmeticMonotonicity { + static bool has() { return false; } + static IFunction::Monotonicity get(const Field &, const Field &) { return {}; } }; REGISTER_FUNCTION(ByteSwap) From 1abe0013c0b2042c79b0dcd75c18ef2364f8aecf Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Mon, 2 Oct 2023 22:49:23 +0200 Subject: [PATCH 082/634] Update 02845_threads_count_in_distributed_queries.sql.j2 --- .../02845_threads_count_in_distributed_queries.sql.j2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 index cd714fdc091..eb3fa6fadd7 100644 --- a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 +++ b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 @@ -1,5 +1,5 @@ -- enforce some defaults to be sure that the env settings will not affect the test -SET max_threads=5, async_socket_for_remote=1, prefer_localhost_replica=1, optimize_read_in_order=1, load_marks_asynchronously=0, local_filesystem_read_method='pread'; +SET max_threads=5, async_socket_for_remote=1, prefer_localhost_replica=1, optimize_read_in_order=1, load_marks_asynchronously=0, local_filesystem_read_method='pread', remote_filesystem_read_method='read'; -- we use query_thread_log to check peak thread usage From 540d966033515e3365837675e46600473ded285b Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Mon, 2 Oct 2023 22:51:58 +0100 Subject: [PATCH 083/634] Docs and comment --- .../sql-reference/functions/math-functions.md | 25 +++++++++++++------ src/Functions/byteSwap.cpp | 1 + 2 files changed, 19 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 81603cba869..a187fe413ce 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -714,7 +714,7 @@ Result: ## byteSwap -Accepts an integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. +Accepts an unsigned integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. **Syntax** @@ -736,10 +736,21 @@ Result: └──────────────────────┘ ``` -The above example can be understood in the following manner: -1. First, the integer operand (base 10) is converted to bytes (base 2) in little-endian i.e. 3351772109 -> CD FB C7 C7 -2. Then, the bytes are reversed i.e CD FB C7 C7 -> C7 C7 FB CD -3. Finally, the bytes are interpreted back to an integer assuming little-endian i.e. C7 C7 FB CD -> 3455829959 +The above example can be worked out in the following manner: +1. First, convert the integer operand (base 10) to its equivalent hexadecimal interpretation (base 16) in big-endian format i.e. 3351772109 -> C7 C7 FB CD (4 bytes) +2. Then, reverse the bytes i.e. C7 C7 FB CD -> CD FB C7 C7 +3. Finally, the convert the hexadecimal number back to an integer assuming big-endian i.e. CD FB C7 C7 -> 3455829959 -Note that, in step#1, we can also choose to convert the operand to bytes in big-endian as long as we also assume big-endian when -converting back to integer in step#3. +Note that, in step#1, one can also choose to convert the operand to bytes in little-endian as long as one also assumes little-endian when converting back to integer in step#3. + +This can be particularly useful when one wants to reverse values of data-types which are stored as unsigned integers under the hood and allow conversions from unsigned integers to themselves (such as IPV4). For example: + +```result +┌─toIPv4(3351772109)─┐ +│ 199.199.251.205 │ +└────────────────────┘ + +┌─toIPv4(byteSwap(3351772109))─┐ +│ 205.251.199.199 │ +└──────────────────────────────┘ +``` \ No newline at end of file diff --git a/src/Functions/byteSwap.cpp b/src/Functions/byteSwap.cpp index b1c2d571fb8..f84d6ec5a2c 100644 --- a/src/Functions/byteSwap.cpp +++ b/src/Functions/byteSwap.cpp @@ -48,6 +48,7 @@ template struct ByteSwapImpl { using ResultType = T; + // byteSwap on a String/FixedString is equivalent to `reverse` which is already implemented. static constexpr const bool allow_string_or_fixed_string = false; static inline T apply(T x) { return byteSwap(x); } From aa2bf5ac02ff079a8367959398bead42d411037f Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 3 Oct 2023 10:03:17 +0200 Subject: [PATCH 084/634] make test faster --- ...threads_count_in_distributed_queries.reference | 8 ++++---- ...45_threads_count_in_distributed_queries.sql.j2 | 15 +++++++-------- 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.reference b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.reference index 0e3f6336468..c8338ebaf7c 100644 --- a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.reference +++ b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.reference @@ -1,14 +1,14 @@ prefer_localhost_replica=1, remote query with a lot of union all -379 +77 ok prefer_localhost_replica=0, remote query with a lot of union all -379 +77 ok prefer_localhost_replica=1, async_socket_for_remote=0, remote query with a lot of union all (lot of threads) -379 +77 ok prepare test schema -500 +95 prefer_localhost_replica=1, remote query with read in order ok prefer_localhost_replica=1 + async_socket_for_remote=0, remote query with read in order (lot of threads) diff --git a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 index eb3fa6fadd7..ffdd4e3400e 100644 --- a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 +++ b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 @@ -1,7 +1,6 @@ -- enforce some defaults to be sure that the env settings will not affect the test SET max_threads=5, async_socket_for_remote=1, prefer_localhost_replica=1, optimize_read_in_order=1, load_marks_asynchronously=0, local_filesystem_read_method='pread', remote_filesystem_read_method='read'; - -- we use query_thread_log to check peak thread usage -- after https://github.com/ClickHouse/ClickHouse/issues/53417 there is a simpler way to check it -- but that will not allow to backport the test to older versions @@ -15,7 +14,7 @@ SELECT 'prefer_localhost_replica=1, remote query with a lot of union all' AS tes -- let's check how many threads clickhouse will start for that select count() from remote('127.0.0.1:9000', view( -{% for n in range(379) -%} +{% for n in range(77) -%} SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} {% endfor -%} )) SETTINGS log_comment='check_concurrency_in_remote_queries1'; @@ -47,7 +46,7 @@ WHERE SELECT 'prefer_localhost_replica=0, remote query with a lot of union all' AS testname; select count() from remote('127.0.0.1:9000', view( -{% for n in range(379) -%} +{% for n in range(77) -%} SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} {% endfor -%} )) SETTINGS log_comment='check_concurrency_in_remote_queries2', prefer_localhost_replica=0; @@ -83,7 +82,7 @@ SELECT 'prefer_localhost_replica=1, async_socket_for_remote=0, remote query with -- see https://github.com/ClickHouse/ClickHouse/issues/53287 select count() from remote('127.0.0.1:9000', view( -{% for n in range(379) -%} +{% for n in range(77) -%} SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} {% endfor -%} )) SETTINGS log_comment='check_concurrency_in_remote_queries3', async_socket_for_remote=0, prefer_localhost_replica=1; @@ -96,7 +95,7 @@ WITH toUnixTimestamp64Micro(event_time_microseconds) ) as peak_threads SELECT - if(peak_threads >= 379, 'ok', 'too few threads: ' || toString(peak_threads) ) AS result + if(peak_threads >= 77, 'ok', 'too few threads: ' || toString(peak_threads) ) AS result FROM system.query_thread_log WHERE event_time > now() - 60 @@ -147,7 +146,7 @@ INSERT INTO test_lot_of_parts (a, b, c, val) 'bar' as b, _CAST('1683504000', 'DateTime64') as c, 'baz' as val - FROM numbers_mt(500) + FROM numbers_mt(95) SETTINGS max_block_size = 1, min_insert_block_size_bytes=1, min_insert_block_size_rows=1; --every row will be in separate part select count() from system.parts where table = 'test_lot_of_parts' and active and database = currentDatabase(); @@ -210,8 +209,8 @@ WITH toUnixTimestamp64Micro(event_time_microseconds) ) as peak_threads SELECT - if(peak_threads >= 500, 'ok', 'too few threads: ' || toString(peak_threads) ) AS result - -- we have 500 parts + if(peak_threads >= 95, 'ok', 'too few threads: ' || toString(peak_threads) ) AS result + -- we have 95 parts FROM system.query_thread_log WHERE event_time > now() - 60 From d54186df30428752e217a6cfe12f0b945dcfd15d Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 4 Oct 2023 17:21:27 -0300 Subject: [PATCH 085/634] tmp --- .../settings.md | 111 +++++++++++++ .../EnvironmentProxyConfigurationResolver.cpp | 21 +-- src/Common/ProxyConfiguration.h | 18 ++- .../ProxyConfigurationResolverProvider.cpp | 152 ++++++++---------- .../ProxyConfigurationResolverProvider.h | 2 + .../gtest_proxy_environment_configuration.cpp | 37 ----- src/Disks/ObjectStorages/S3/diskSettings.cpp | 6 +- .../configs/config.d/proxy_list.xml | 2 +- .../proxy-resolver/resolver.py | 14 -- 9 files changed, 199 insertions(+), 164 deletions(-) delete mode 100644 tests/integration/test_s3_storage_conf_new_proxy/proxy-resolver/resolver.py diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 5586dbafd88..e2d288b6e07 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2630,3 +2630,114 @@ Possible values: - 1 — Enabled. Default value: 0. + +## proxy {#proxy} + +Define proxy servers for HTTP and HTTPS requests, currently supported by S3 storage, S3 table functions, and URL functions. + +There are three ways to define proxy servers: environment variables, proxy lists, and remote proxy resolvers. + +### Environment variables + +The `http_proxy` and `https_proxy` environment variables allow you to specify a +proxy server for a given protocol. If you have it set on your system, it should work seamlessly. + +This is the simplest approach if a given protocol has +only one proxy server and that proxy server doesn't change. + +### Proxy lists + +This approach allows you to specify one or more +proxy servers for a protocol. If more than one proxy server is defined, +ClickHouse uses the different proxies on a round-robin basis, balancing the +load across the servers. This is the simplest approach if there is more than +one proxy server for a protocol and the list of proxy servers doesn't change. + +### Configuration template + +``` xml + + + http://proxy1 + http://proxy2:3128 + + + http://proxy1:3128 + + +``` + +`` fields + +* `` - A list of one or more HTTP proxies +* `` - A list of one or more HTTPS proxies + +`` and `` fields + +* `` - The URI of the proxy + +### Remote proxy resolvers + +It's possible that the proxy servers change dynamically. In that +case, you can define the endpoint of a resolver. ClickHouse sends +an empty GET request to that endpoint, the remote resolver should return the proxy host. +ClickHouse will use it to form the proxy URI using the following template: `{proxy_scheme}://{proxy_host}:{proxy_port}` + +### Configuration template + +``` xml + + + + http://resolver:8080/hostname + http + 80 + 10 + + + + + + http://resolver:8080/hostname + http + 3128 + 10 + + + + +``` + +`` fields + +* `` - A list of one or more resolvers* +* `` - A list of one or more resolvers* + +`` and `` fields + +* `` - The endpoint and other details for a resolver. + You can have multiple `` elements, but only the first + `` for a given protocol is used. Any other `` + elements for that protocol are ignored. That means load balancing + (if needed) should be implemented by the remote resolver. + +`` fields + +* `` - The URI of the proxy resolver +* `` - The protocol of the final proxy URI. This can be either `http` or `https`. +* `` - The port number of the proxy resolver +* `` - The time in seconds that values from the resolver + should be cached by ClickHouse. Setting this value to `0` causes ClickHouse + to contact the resolver for every HTTP or HTTPS request. + +### Precedence + +Proxy settings are determined in the following order: + +1. Remote proxy resolvers +2. Proxy lists +3. Environment variables + +ClickHouse will check the highest priority resolver type for the request protocol. If it is not defined, +it will check the next highest priority resolver type, until it reaches the environment resolver. +This also allows a mix of resolver types can be used. diff --git a/src/Common/EnvironmentProxyConfigurationResolver.cpp b/src/Common/EnvironmentProxyConfigurationResolver.cpp index 184e2ac5ca5..79b50964cf5 100644 --- a/src/Common/EnvironmentProxyConfigurationResolver.cpp +++ b/src/Common/EnvironmentProxyConfigurationResolver.cpp @@ -25,25 +25,12 @@ namespace * getenv is safe to use here because ClickHouse code does not make any call to `setenv` or `putenv` * aside from tests and a very early call during startup: https://github.com/ClickHouse/ClickHouse/blob/master/src/Daemon/BaseDaemon.cpp#L791 * */ - - if (protocol == DB::ProxyConfiguration::Protocol::HTTP) + switch (protocol) { - return std::getenv(PROXY_HTTP_ENVIRONMENT_VARIABLE); // NOLINT(concurrency-mt-unsafe) - } - else if (protocol == DB::ProxyConfiguration::Protocol::HTTPS) - { - return std::getenv(PROXY_HTTPS_ENVIRONMENT_VARIABLE); // NOLINT(concurrency-mt-unsafe) - } - else - { - if (const char * http_proxy_host = std::getenv(PROXY_HTTP_ENVIRONMENT_VARIABLE)) // NOLINT(concurrency-mt-unsafe) - { - return http_proxy_host; - } - else - { + case ProxyConfiguration::Protocol::HTTP: + return std::getenv(PROXY_HTTP_ENVIRONMENT_VARIABLE); // NOLINT(concurrency-mt-unsafe) + case ProxyConfiguration::Protocol::HTTPS: return std::getenv(PROXY_HTTPS_ENVIRONMENT_VARIABLE); // NOLINT(concurrency-mt-unsafe) - } } } } diff --git a/src/Common/ProxyConfiguration.h b/src/Common/ProxyConfiguration.h index cc951c004bc..421c9fb10d9 100644 --- a/src/Common/ProxyConfiguration.h +++ b/src/Common/ProxyConfiguration.h @@ -1,17 +1,23 @@ #pragma once #include +#include namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + + struct ProxyConfiguration { enum class Protocol { HTTP, - HTTPS, - ANY + HTTPS }; static auto protocolFromString(const std::string & str) @@ -24,10 +30,8 @@ struct ProxyConfiguration { return Protocol::HTTPS; } - else - { - return Protocol::ANY; - } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown proxy protocol: {}", str); } static auto protocolToString(Protocol protocol) @@ -38,8 +42,6 @@ struct ProxyConfiguration return "http"; case Protocol::HTTPS: return "https"; - case Protocol::ANY: - return "any"; } } diff --git a/src/Common/ProxyConfigurationResolverProvider.cpp b/src/Common/ProxyConfigurationResolverProvider.cpp index 30164c776e5..c7f50b70479 100644 --- a/src/Common/ProxyConfigurationResolverProvider.cpp +++ b/src/Common/ProxyConfigurationResolverProvider.cpp @@ -20,12 +20,13 @@ namespace std::shared_ptr getRemoteResolver( const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration) { - auto endpoint = Poco::URI(configuration.getString(config_prefix + ".endpoint")); - auto proxy_scheme = configuration.getString(config_prefix + ".proxy_scheme"); + auto resolver_prefix = config_prefix + ".resolver"; + auto endpoint = Poco::URI(configuration.getString(resolver_prefix + ".endpoint")); + auto proxy_scheme = configuration.getString(resolver_prefix + ".proxy_scheme"); if (proxy_scheme != "http" && proxy_scheme != "https") throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only HTTP/HTTPS schemas allowed in proxy resolver config: {}", proxy_scheme); - auto proxy_port = configuration.getUInt(config_prefix + ".proxy_port"); - auto cache_ttl = configuration.getUInt(config_prefix + ".proxy_cache_time", 10); + auto proxy_port = configuration.getUInt(resolver_prefix + ".proxy_port"); + auto cache_ttl = configuration.getUInt(resolver_prefix + ".proxy_cache_time", 10); LOG_DEBUG(&Poco::Logger::get("ProxyConfigurationResolverProvider"), "Configured remote proxy resolver: {}, Scheme: {}, Port: {}", endpoint.toString(), proxy_scheme, proxy_port); @@ -33,31 +34,6 @@ namespace return std::make_shared(endpoint, proxy_scheme, proxy_port, cache_ttl); } - std::shared_ptr getRemoteResolver( - ProxyConfiguration::Protocol protocol, const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration) - { - std::vector keys; - configuration.keys(config_prefix, keys); - - std::vector uris; - for (const auto & key : keys) - { - if (startsWith(key, "resolver")) - { - auto prefix_with_key = config_prefix + "." + key; - auto proxy_scheme_config_string = prefix_with_key + ".proxy_scheme"; - auto config_protocol = configuration.getString(proxy_scheme_config_string); - - if (ProxyConfiguration::Protocol::ANY == protocol || config_protocol == ProxyConfiguration::protocolToString(protocol)) - { - return getRemoteResolver(prefix_with_key, configuration); - } - } - } - - return nullptr; - } - auto extractURIList(const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration) { std::vector keys; @@ -84,34 +60,7 @@ namespace return uris; } - std::shared_ptr getListResolverNewSyntax( - ProxyConfiguration::Protocol protocol, - const String & config_prefix, - const Poco::Util::AbstractConfiguration & configuration - ) - { - std::vector uris; - - bool include_http_uris = ProxyConfiguration::Protocol::ANY == protocol || ProxyConfiguration::Protocol::HTTP == protocol; - - if (include_http_uris && configuration.has(config_prefix + ".http")) - { - auto http_uris = extractURIList(config_prefix + ".http", configuration); - uris.insert(uris.end(), http_uris.begin(), http_uris.end()); - } - - bool include_https_uris = ProxyConfiguration::Protocol::ANY == protocol || ProxyConfiguration::Protocol::HTTPS == protocol; - - if (include_https_uris && configuration.has(config_prefix + ".https")) - { - auto https_uris = extractURIList(config_prefix + ".https", configuration); - uris.insert(uris.end(), https_uris.begin(), https_uris.end()); - } - - return uris.empty() ? nullptr : std::make_shared(uris); - } - - std::shared_ptr getListResolverOldSyntax( + std::shared_ptr getListResolver( const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration ) @@ -121,29 +70,58 @@ namespace return uris.empty() ? nullptr : std::make_shared(uris); } - std::shared_ptr getListResolver( - ProxyConfiguration::Protocol protocol, const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration + bool hasRemoteResolver(const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration) + { + return configuration.has(config_prefix + ".resolver"); + } + + bool hasListResolver(const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration) + { + return configuration.has(config_prefix + ".uri"); + } + + /* + * New syntax requires protocol prefix " or " + * */ + std::optional getProtocolPrefix( + ProxyConfiguration::Protocol request_protocol, + const String & config_prefix, + const Poco::Util::AbstractConfiguration & configuration ) { - std::vector keys; - configuration.keys(config_prefix, keys); + auto protocol_prefix = config_prefix + "." + ProxyConfiguration::protocolToString(request_protocol); + if (!configuration.has(protocol_prefix)) + { + return std::nullopt; + } - bool new_setting_syntax = std::find_if( - keys.begin(), - keys.end(), - [](const String & key) - { - return startsWith(key, "http") || startsWith(key, "https"); - }) != keys.end(); + return protocol_prefix; + } - return new_setting_syntax ? getListResolverNewSyntax(protocol, config_prefix, configuration) - : getListResolverOldSyntax(config_prefix, configuration); + template + std::optional calculatePrefixBasedOnSettingsSyntax( + ProxyConfiguration::Protocol request_protocol, + const String & config_prefix, + const Poco::Util::AbstractConfiguration & configuration + ) + { + if (!configuration.has(config_prefix)) + { + return std::nullopt; + } + + if constexpr (new_syntax) + { + return getProtocolPrefix(request_protocol, config_prefix, configuration); + } + + return config_prefix; } } std::shared_ptr ProxyConfigurationResolverProvider::get(Protocol protocol, const Poco::Util::AbstractConfiguration & configuration) { - if (auto resolver = getFromSettings(protocol, "", configuration)) + if (auto resolver = getFromSettings(protocol, "proxy", configuration)) { return resolver; } @@ -151,34 +129,36 @@ std::shared_ptr ProxyConfigurationResolverProvider:: return std::make_shared(protocol); } +template std::shared_ptr ProxyConfigurationResolverProvider::getFromSettings( - Protocol protocol, + Protocol request_protocol, const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration ) { - auto proxy_prefix = config_prefix.empty() ? "proxy" : config_prefix + ".proxy"; + auto prefix_opt = calculatePrefixBasedOnSettingsSyntax(request_protocol, config_prefix, configuration); - if (configuration.has(proxy_prefix)) + if (!prefix_opt) { - std::vector config_keys; - configuration.keys(proxy_prefix, config_keys); + return nullptr; + } - if (auto remote_resolver = getRemoteResolver(protocol, proxy_prefix, configuration)) - { - return remote_resolver; - } + auto prefix = *prefix_opt; - if (auto list_resolver = getListResolver(protocol, proxy_prefix, configuration)) - { - return list_resolver; - } + if (hasRemoteResolver(prefix, configuration)) + { + return getRemoteResolver(prefix, configuration); + } + else if (hasListResolver(prefix, configuration)) + { + return getListResolver(prefix, configuration); } return nullptr; } std::shared_ptr ProxyConfigurationResolverProvider::getFromOldSettingsFormat( + Protocol request_protocol, const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration ) @@ -187,7 +167,7 @@ std::shared_ptr ProxyConfigurationResolverProvider:: * First try to get it from settings only using the combination of config_prefix and configuration. * This logic exists for backward compatibility with old S3 storage specific proxy configuration. * */ - if (auto resolver = ProxyConfigurationResolverProvider::getFromSettings(Protocol::ANY, config_prefix, configuration)) + if (auto resolver = ProxyConfigurationResolverProvider::getFromSettings(request_protocol, config_prefix + ".proxy", configuration)) { return resolver; } @@ -196,7 +176,7 @@ std::shared_ptr ProxyConfigurationResolverProvider:: * In case the combination of config_prefix and configuration does not provide a resolver, try to get it from general / new settings. * Falls back to Environment resolver if no configuration is found. * */ - return ProxyConfigurationResolverProvider::get(Protocol::ANY, configuration); + return ProxyConfigurationResolverProvider::get(request_protocol, configuration); } } diff --git a/src/Common/ProxyConfigurationResolverProvider.h b/src/Common/ProxyConfigurationResolverProvider.h index 26b2261b218..ebf22f7e92a 100644 --- a/src/Common/ProxyConfigurationResolverProvider.h +++ b/src/Common/ProxyConfigurationResolverProvider.h @@ -27,11 +27,13 @@ public: * If no configuration is found, returns nullptr. * */ static std::shared_ptr getFromOldSettingsFormat( + Protocol request_protocol, const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration ); private: + template static std::shared_ptr getFromSettings( Protocol protocol, const String & config_prefix, diff --git a/src/Common/tests/gtest_proxy_environment_configuration.cpp b/src/Common/tests/gtest_proxy_environment_configuration.cpp index b14307d879a..6b189030b1a 100644 --- a/src/Common/tests/gtest_proxy_environment_configuration.cpp +++ b/src/Common/tests/gtest_proxy_environment_configuration.cpp @@ -57,40 +57,3 @@ TEST(EnvironmentProxyConfigurationResolver, TestHTTPsNoEnv) ASSERT_EQ(configuration.protocol, DB::ProxyConfiguration::Protocol::HTTP); ASSERT_EQ(configuration.port, 0u); } - -TEST(EnvironmentProxyConfigurationResolver, TestANYHTTP) -{ - EnvironmentProxySetter setter(http_proxy_server, {}); - - DB::EnvironmentProxyConfigurationResolver resolver(DB::ProxyConfiguration::Protocol::ANY); - - auto configuration = resolver.resolve(); - - ASSERT_EQ(configuration.host, http_proxy_server.getHost()); - ASSERT_EQ(configuration.port, http_proxy_server.getPort()); - ASSERT_EQ(configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_proxy_server.getScheme())); -} - -TEST(EnvironmentProxyConfigurationResolver, TestANYHTTPS) -{ - EnvironmentProxySetter setter({}, https_proxy_server); - - DB::EnvironmentProxyConfigurationResolver resolver(DB::ProxyConfiguration::Protocol::ANY); - - auto configuration = resolver.resolve(); - - ASSERT_EQ(configuration.host, https_proxy_server.getHost()); - ASSERT_EQ(configuration.port, https_proxy_server.getPort()); - ASSERT_EQ(configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_proxy_server.getScheme())); -} - -TEST(EnvironmentProxyConfigurationResolver, TestANYNoEnv) -{ - DB::EnvironmentProxyConfigurationResolver resolver(DB::ProxyConfiguration::Protocol::ANY); - - auto configuration = resolver.resolve(); - - ASSERT_EQ(configuration.host, ""); - ASSERT_EQ(configuration.protocol, DB::ProxyConfiguration::Protocol::HTTP); - ASSERT_EQ(configuration.port, 0u); -} diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 3e38d0a7912..de88c876922 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -71,7 +71,11 @@ std::unique_ptr getClient( /* * Override proxy configuration for backwards compatibility with old configuration format. * */ - auto proxy_config = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat(config_prefix, config); + auto proxy_config = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( + ProxyConfiguration::protocolFromString(uri.uri.getScheme()), + config_prefix, + config + ); if (proxy_config) { client_configuration.per_request_configuration diff --git a/tests/integration/test_s3_storage_conf_new_proxy/configs/config.d/proxy_list.xml b/tests/integration/test_s3_storage_conf_new_proxy/configs/config.d/proxy_list.xml index af5687d88ac..24c1eb29fbc 100644 --- a/tests/integration/test_s3_storage_conf_new_proxy/configs/config.d/proxy_list.xml +++ b/tests/integration/test_s3_storage_conf_new_proxy/configs/config.d/proxy_list.xml @@ -5,4 +5,4 @@ http://proxy2 - \ No newline at end of file + diff --git a/tests/integration/test_s3_storage_conf_new_proxy/proxy-resolver/resolver.py b/tests/integration/test_s3_storage_conf_new_proxy/proxy-resolver/resolver.py deleted file mode 100644 index eaea4c1dab2..00000000000 --- a/tests/integration/test_s3_storage_conf_new_proxy/proxy-resolver/resolver.py +++ /dev/null @@ -1,14 +0,0 @@ -import random - -import bottle - - -@bottle.route("/hostname") -def index(): - if random.randrange(2) == 0: - return "proxy1" - else: - return "proxy2" - - -bottle.run(host="0.0.0.0", port=8080) From 406dfe277c18b9c7b18dc1ef0011e0d627266699 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 5 Oct 2023 11:58:00 +0200 Subject: [PATCH 086/634] fix style --- src/Interpreters/tests/gtest_lru_file_cache.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index f99a41729c3..ab2a128de34 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -1,8 +1,9 @@ +#include + #include #include #include -#include #include #include @@ -22,7 +23,6 @@ #include #include #include -#include #include #include #include From c8216d2b00deb73918d03dfb0a7b7b8dd3dfd008 Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Fri, 6 Oct 2023 19:24:49 +0000 Subject: [PATCH 087/634] Add tests --- .../0_stateless/02887_byteswap.reference | 15 +++++++++++++++ tests/queries/0_stateless/02887_byteswap.sql | 18 ++++++++++++++++++ 2 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/02887_byteswap.reference create mode 100644 tests/queries/0_stateless/02887_byteswap.sql diff --git a/tests/queries/0_stateless/02887_byteswap.reference b/tests/queries/0_stateless/02887_byteswap.reference new file mode 100644 index 00000000000..9538075290a --- /dev/null +++ b/tests/queries/0_stateless/02887_byteswap.reference @@ -0,0 +1,15 @@ +0 +1 +255 +1 +10000 +4135 +65535 +256 +3455829959 +3351772109 +4294967295 +16777216 +18439412204227788800 +123294967295 +18446744073709551615 diff --git a/tests/queries/0_stateless/02887_byteswap.sql b/tests/queries/0_stateless/02887_byteswap.sql new file mode 100644 index 00000000000..570135f1490 --- /dev/null +++ b/tests/queries/0_stateless/02887_byteswap.sql @@ -0,0 +1,18 @@ +SELECT byteSwap(0); +SELECT byteSwap(1); +SELECT byteSwap(255); + +SELECT byteSwap(256); +SELECT byteSwap(4135); +SELECT byteSwap(10000); +SELECT byteSwap(65535); + +SELECT byteSwap(65536); +SELECT byteSwap(3351772109); +SELECT byteSwap(3455829959); +SELECT byteSwap(4294967295); + +SELECT byteSwap(4294967296); +SELECT byteSwap(123294967295); +SELECT byteSwap(18439412204227788800); +SELECT byteSwap(18446744073709551615); \ No newline at end of file From ed2c85bbce36360fe1a4cf4590937281aa6aaae5 Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Fri, 6 Oct 2023 19:35:11 +0000 Subject: [PATCH 088/634] Remove comment --- src/Functions/byteSwap.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/byteSwap.cpp b/src/Functions/byteSwap.cpp index f84d6ec5a2c..b1c2d571fb8 100644 --- a/src/Functions/byteSwap.cpp +++ b/src/Functions/byteSwap.cpp @@ -48,7 +48,6 @@ template struct ByteSwapImpl { using ResultType = T; - // byteSwap on a String/FixedString is equivalent to `reverse` which is already implemented. static constexpr const bool allow_string_or_fixed_string = false; static inline T apply(T x) { return byteSwap(x); } From 09c7607d07726c6f831bf24b7f72ef3a51249e43 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Sat, 7 Oct 2023 11:11:44 +0800 Subject: [PATCH 089/634] fix tests --- .../02884_getHttpHeaderFunction.reference | 3 --- .../0_stateless/02884_getHttpHeaderFunction.sh | 18 +++++++++--------- 2 files changed, 9 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference b/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference index 334f0840fa4..de5112f13ed 100644 --- a/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference +++ b/tests/queries/0_stateless/02884_getHttpHeaderFunction.reference @@ -8,9 +8,6 @@ default value1 value2 value_from_query_1 value_from_query_2 value_from_query_3 1 default row1_value1 row1_value2 row1_value3 row1_value4 row1_value5 row1_value6 row1_value7 value_from_query_1 value_from_query_2 value_from_query_3 2 default row2_value1 row2_value2 row2_value3 row2_value4 row2_value5 row2_value6 row2_value7 value_from_query_1 value_from_query_2 value_from_query_3 3 -X-Clickhouse-User -http_key1 -http_key2 default http_value1 http_value2 diff --git a/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh b/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh index fecc54ffd82..64b68ed5dd0 100755 --- a/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh +++ b/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh @@ -15,10 +15,11 @@ echo "SELECT getHttpHeader('X-Clickhouse-User'), getHttpHeader('key1'), getHttpH echo "SELECT getHttpHeader('X-' || 'Clickhouse' || '-User'), getHttpHeader('key1'), getHttpHeader('key2')" | curl -s -H 'X-Clickhouse-User: default' \ -H 'X-ClickHouse-Key: ' -H 'key1: value1' -H 'key2: value2' 'http://localhost:8123/' -d @- +$CLICKHOUSE_CLIENT -q "CREATE DATABASE IF NOT EXISTS test" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02884_get_http_header" $CLICKHOUSE_CLIENT -q " - CREATE TABLE IF NOT EXISTS 02884_get_http_header + CREATE TABLE IF NOT EXISTS test.02884_get_http_header (id UInt32, http_user String DEFAULT getHttpHeader('X-Clickhouse-User'), http_key1 String DEFAULT getHttpHeader('http_header_key1'), @@ -51,22 +52,21 @@ echo "INSERT INTO test.02884_get_http_header (id) values (2)" | curl -s -H 'X-Cl -H 'http_header_key6: row2_value6'\ -H 'http_header_key7: row2_value7' 'http://localhost:8123/' -d @- -$CLICKHOUSE_CLIENT -q "SELECT id, http_user, http_key1, http_key2, http_key3, http_key4, http_key5, http_key6, http_key7 FROM 02884_get_http_header ORDER BY id;" +$CLICKHOUSE_CLIENT -q "SELECT id, http_user, http_key1, http_key2, http_key3, http_key4, http_key5, http_key6, http_key7 FROM test.02884_get_http_header ORDER BY id;" #Insert data via tcp client -$CLICKHOUSE_CLIENT -q "INSERT INTO 02884_get_http_header (id) values (3)" -$CLICKHOUSE_CLIENT -q "SELECT * FROM 02884_get_http_header where id = 3" +$CLICKHOUSE_CLIENT -q "INSERT INTO test.02884_get_http_header (id) values (3)" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test.02884_get_http_header where id = 3" echo "SELECT getHttpHeader('key_from_query_1'), getHttpHeader('key_from_query_2'), getHttpHeader('key_from_query_3'), * FROM test.02884_get_http_header ORDER BY id" | curl -s -H 'X-Clickhouse-User: default' \ -H 'X-ClickHouse-Key: ' -H 'key_from_query_1: value_from_query_1' -H 'key_from_query_2: value_from_query_2' -H 'key_from_query_3: value_from_query_3' 'http://localhost:8123/' -d @- -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02884_get_http_header" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.02884_get_http_header" -$CLICKHOUSE_CLIENT -q "CREATE TABLE 02884_header_from_table (header_name String) Engine=Memory" -$CLICKHOUSE_CLIENT -q "INSERT INTO 02884_header_from_table values ('X-Clickhouse-User'), ('http_key1'), ('http_key2')" -$CLICKHOUSE_CLIENT -q "select * from 02884_header_from_table" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test.02884_header_from_table (header_name String) Engine=Memory" +$CLICKHOUSE_CLIENT -q "INSERT INTO test.02884_header_from_table values ('X-Clickhouse-User'), ('http_key1'), ('http_key2')" echo "SELECT getHttpHeader(header_name) as value from (select * FROM test.02884_header_from_table) order by value" | curl -s -H 'X-Clickhouse-User: default' \ -H 'X-ClickHouse-Key: ' -H 'http_key1: http_value1' -H 'http_key2: http_value2' 'http://localhost:8123/' -d @- -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02884_header_from_table" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.02884_header_from_table" From 719bae2ed0bf02e5b15ea191be4e7ccb876ccaeb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Sat, 7 Oct 2023 14:48:31 +0800 Subject: [PATCH 090/634] fix tests --- tests/queries/0_stateless/02884_getHttpHeaderFunction.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh b/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh index 64b68ed5dd0..54e3f8715d7 100755 --- a/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh +++ b/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh @@ -61,7 +61,7 @@ echo "SELECT getHttpHeader('key_from_query_1'), getHttpHeader('key_from_query_2' $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.02884_get_http_header" -$CLICKHOUSE_CLIENT -q "CREATE TABLE test.02884_header_from_table (header_name String) Engine=Memory" +$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS test.02884_header_from_table (header_name String) Engine=Memory" $CLICKHOUSE_CLIENT -q "INSERT INTO test.02884_header_from_table values ('X-Clickhouse-User'), ('http_key1'), ('http_key2')" echo "SELECT getHttpHeader(header_name) as value from (select * FROM test.02884_header_from_table) order by value" | curl -s -H 'X-Clickhouse-User: default' \ From 302291e17b39aa443882975b3851e321b0879415 Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Sat, 7 Oct 2023 23:05:07 +0000 Subject: [PATCH 091/634] Address a few review comments. - Consider byteswap injective. - Make function case-insensitive. - Add in-code documentation and copy-paste it to the markdown docs. --- .../sql-reference/functions/math-functions.md | 9 +--- src/Functions/byteSwap.cpp | 50 ++++++++++++++++++- 2 files changed, 50 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index a187fe413ce..2be1b06385c 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -716,11 +716,7 @@ Result: Accepts an unsigned integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. -**Syntax** - -```sql -byteSwap(operand) -``` +Currently, this is implemented for UInt8, UInt16, UInt32 and UInt64. **Example** @@ -743,8 +739,7 @@ The above example can be worked out in the following manner: Note that, in step#1, one can also choose to convert the operand to bytes in little-endian as long as one also assumes little-endian when converting back to integer in step#3. -This can be particularly useful when one wants to reverse values of data-types which are stored as unsigned integers under the hood and allow conversions from unsigned integers to themselves (such as IPV4). For example: - +One use-case of this function is reversing IPv4s: ```result ┌─toIPv4(3351772109)─┐ │ 199.199.251.205 │ diff --git a/src/Functions/byteSwap.cpp b/src/Functions/byteSwap.cpp index b1c2d571fb8..9374ad4fa59 100644 --- a/src/Functions/byteSwap.cpp +++ b/src/Functions/byteSwap.cpp @@ -60,7 +60,7 @@ struct NameByteSwap { static constexpr auto name = "byteSwap"; }; -using FunctionByteSwap = FunctionUnaryArithmetic; +using FunctionByteSwap = FunctionUnaryArithmetic; } @@ -73,7 +73,53 @@ struct FunctionUnaryArithmeticMonotonicity REGISTER_FUNCTION(ByteSwap) { - factory.registerFunction(); + factory.registerFunction( + FunctionDocumentation{ + .description = R"( +Accepts an unsigned integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. + +Currently, this is implemented for UInt8, UInt16, UInt32 and UInt64. + +**Example** + +```sql +byteSwap(3351772109) +``` + +Result: + +```result +┌─byteSwap(3351772109)─┐ +│ 3455829959 │ +└──────────────────────┘ +``` + +The above example can be worked out in the following manner: +1. First, convert the integer operand (base 10) to its equivalent hexadecimal interpretation (base 16) in big-endian format i.e. 3351772109 -> C7 C7 FB CD (4 bytes) +2. Then, reverse the bytes i.e. C7 C7 FB CD -> CD FB C7 C7 +3. Finally, the convert the hexadecimal number back to an integer assuming big-endian i.e. CD FB C7 C7 -> 3455829959 + +Note that, in step#1, one can also choose to convert the operand to bytes in little-endian as long as one also assumes little-endian when converting back to integer in step#3. + +One use-case of this function is reversing IPv4s: +```result +┌─toIPv4(3351772109)─┐ +│ 199.199.251.205 │ +└────────────────────┘ + +┌─toIPv4(byteSwap(3351772109))─┐ +│ 205.251.199.199 │ +└──────────────────────────────┘ +``` +)", + .examples{ + {"8-bit", "SELECT byteSwap(54)", "54"}, + {"16-bit", "SELECT byteSwap(4135)", "10000"}, + {"32-bit", "SELECT byteSwap(3351772109)", "3455829959"}, + {"64-bit", "SELECT byteSwap(123294967295)", "18439412204227788800"}, + }, + .categories{"Mathematical"}}, + FunctionFactory::CaseInsensitive); } } From 2c04370e5f2c7cf1e76146c4b969fd6ddb79f6bb Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Sat, 7 Oct 2023 23:59:24 +0000 Subject: [PATCH 092/634] Use std::byteswap instead of custom implementations. - Also allow signed ints now because std::byteswap accepts them. - Fix for style check. --- .../sql-reference/functions/math-functions.md | 2 +- src/Functions/byteSwap.cpp | 29 +++---------------- .../0_stateless/02887_byteswap.reference | 12 ++++++++ tests/queries/0_stateless/02887_byteswap.sql | 18 +++++++++++- .../aspell-ignore/en/aspell-dict.txt | 1 + 5 files changed, 35 insertions(+), 27 deletions(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 2be1b06385c..90413b109dc 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -716,7 +716,7 @@ Result: Accepts an unsigned integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. -Currently, this is implemented for UInt8, UInt16, UInt32 and UInt64. +Currently, this is supported for up to 64-bit (signed and unsigned) integers. **Example** diff --git a/src/Functions/byteSwap.cpp b/src/Functions/byteSwap.cpp index 9374ad4fa59..e87671c40c5 100644 --- a/src/Functions/byteSwap.cpp +++ b/src/Functions/byteSwap.cpp @@ -11,31 +11,10 @@ extern const int NOT_IMPLEMENTED; namespace { template -requires std::is_same_v +requires std::is_integral_v inline T byteSwap(T x) { - return x; -} - -template -requires std::is_same_v -inline T byteSwap(T x) -{ - return __builtin_bswap16(x); -} - -template -requires std::is_same_v -inline T byteSwap(T x) -{ - return __builtin_bswap32(x); -} - -template -requires std::is_same_v -inline T byteSwap(T x) -{ - return __builtin_bswap64(x); + return std::byteswap(x); } template @@ -76,9 +55,9 @@ REGISTER_FUNCTION(ByteSwap) factory.registerFunction( FunctionDocumentation{ .description = R"( -Accepts an unsigned integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. +Accepts an integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. -Currently, this is implemented for UInt8, UInt16, UInt32 and UInt64. +Currently, this is supported for up to 64-bit (signed and unsigned) integers. **Example** diff --git a/tests/queries/0_stateless/02887_byteswap.reference b/tests/queries/0_stateless/02887_byteswap.reference index 9538075290a..47e5a37f872 100644 --- a/tests/queries/0_stateless/02887_byteswap.reference +++ b/tests/queries/0_stateless/02887_byteswap.reference @@ -13,3 +13,15 @@ 18439412204227788800 123294967295 18446744073709551615 +0 +-1 +-128 +32767 +-9745 +128 +-8388609 +3676125013305458687 +128 +-549755813889 +4039370097989451775 +128 diff --git a/tests/queries/0_stateless/02887_byteswap.sql b/tests/queries/0_stateless/02887_byteswap.sql index 570135f1490..70f3b3ce4f6 100644 --- a/tests/queries/0_stateless/02887_byteswap.sql +++ b/tests/queries/0_stateless/02887_byteswap.sql @@ -15,4 +15,20 @@ SELECT byteSwap(4294967295); SELECT byteSwap(4294967296); SELECT byteSwap(123294967295); SELECT byteSwap(18439412204227788800); -SELECT byteSwap(18446744073709551615); \ No newline at end of file +SELECT byteSwap(18446744073709551615); + +SELECT byteSwap(-0); +SELECT byteSwap(-1); +SELECT byteSwap(-128); + +SELECT byteSwap(-129); +SELECT byteSwap(-4135); +SELECT byteSwap(-32768); + +SELECT byteSwap(-32769); +SELECT byteSwap(-3351772109); +SELECT byteSwap(-2147483648); + +SELECT byteSwap(-2147483649); +SELECT byteSwap(-1242525266376); +SELECT byteswap(-9223372036854775808); \ No newline at end of file diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a418676ad6d..d4a307c15da 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1415,6 +1415,7 @@ encodeXMLComponent encodings encryptions endian +endianness endsWith endsWithUTF enum From 28f17bec96a521213360f41f29f1915b9e4393fb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Oct 2023 16:06:00 +0200 Subject: [PATCH 093/634] Avoid possible memory leaks in case of missing buffer finalization Sometimes it is still possible, the pattern could looks like this: try { writer->finalize(); writer->flush(); write_buf->finalize(); } catch (...) { /// Stop ParallelFormattingOutputFormat correctly. release(); throw; } Here, write_buf will not be finalized, in case of exception during writer finalize()/flush(). Signed-off-by: Azat Khuzhin --- src/IO/LZMADeflatingWriteBuffer.cpp | 6 +++++- src/IO/Lz4DeflatingWriteBuffer.cpp | 7 ++++++- src/IO/ZlibDeflatingWriteBuffer.cpp | 6 +++++- src/IO/ZstdDeflatingWriteBuffer.cpp | 7 ++++++- 4 files changed, 22 insertions(+), 4 deletions(-) diff --git a/src/IO/LZMADeflatingWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp index c70ec1507cb..a77b2bb7b39 100644 --- a/src/IO/LZMADeflatingWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -44,7 +44,11 @@ LZMADeflatingWriteBuffer::LZMADeflatingWriteBuffer( LZMA_VERSION_STRING); } -LZMADeflatingWriteBuffer::~LZMADeflatingWriteBuffer() = default; +LZMADeflatingWriteBuffer::~LZMADeflatingWriteBuffer() +{ + /// It is OK to call deflateEnd() twice (one from the finalizeAfter()) + lzma_end(&lstr); +} void LZMADeflatingWriteBuffer::nextImpl() { diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index 27c945f92cf..aab8dacef38 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -40,7 +40,11 @@ Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer( LZ4F_VERSION); } -Lz4DeflatingWriteBuffer::~Lz4DeflatingWriteBuffer() = default; +Lz4DeflatingWriteBuffer::~Lz4DeflatingWriteBuffer() +{ + if (ctx) + LZ4F_freeCompressionContext(ctx); +} void Lz4DeflatingWriteBuffer::nextImpl() { @@ -156,6 +160,7 @@ void Lz4DeflatingWriteBuffer::finalizeBefore() void Lz4DeflatingWriteBuffer::finalizeAfter() { LZ4F_freeCompressionContext(ctx); + ctx = nullptr; } } diff --git a/src/IO/ZlibDeflatingWriteBuffer.cpp b/src/IO/ZlibDeflatingWriteBuffer.cpp index 5455adcb7c9..6e4ab742413 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -72,7 +72,11 @@ void ZlibDeflatingWriteBuffer::nextImpl() } } -ZlibDeflatingWriteBuffer::~ZlibDeflatingWriteBuffer() = default; +ZlibDeflatingWriteBuffer::~ZlibDeflatingWriteBuffer() +{ + /// It is OK to call deflateEnd() twice (one from the finalizeAfter() that does the proper error checking) + deflateEnd(&zstr); +} void ZlibDeflatingWriteBuffer::finalizeBefore() { diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index 83d8487e3e7..949d65926b3 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -30,7 +30,11 @@ ZstdDeflatingWriteBuffer::ZstdDeflatingWriteBuffer( output = {nullptr, 0, 0}; } -ZstdDeflatingWriteBuffer::~ZstdDeflatingWriteBuffer() = default; +ZstdDeflatingWriteBuffer::~ZstdDeflatingWriteBuffer() +{ + if (cctx) + ZSTD_freeCCtx(cctx); +} void ZstdDeflatingWriteBuffer::flush(ZSTD_EndDirective mode) { @@ -88,6 +92,7 @@ void ZstdDeflatingWriteBuffer::finalizeAfter() try { size_t err = ZSTD_freeCCtx(cctx); + cctx = nullptr; /// This is just in case, since it is impossible to get an error by using this wrapper. if (unlikely(err)) throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, "ZSTD_freeCCtx failed: error: '{}'; zstd version: {}", From 63b0df9f1d79f77109c622f5566f88eda5665b43 Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Sun, 8 Oct 2023 13:25:09 +0000 Subject: [PATCH 094/634] byteSwap is now documented; remove it from the list of undocumented functions --- .../02415_all_new_functions_must_be_documented.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 16cab29649b..69f455773b0 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -192,7 +192,6 @@ blockSerializedSize blockSize buildId byteSize -byteSwap caseWithExpr caseWithExpression caseWithoutExpr From b6d4ae51c60767431a9c74a070252326c982e1ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 9 Oct 2023 02:40:08 +0200 Subject: [PATCH 095/634] Better recursion depth check --- src/Interpreters/ActionsVisitor.cpp | 3 +++ src/Parsers/parseQuery.cpp | 4 ++++ tests/queries/0_stateless/02894_ast_depth_check.reference | 1 + tests/queries/0_stateless/02894_ast_depth_check.sh | 7 +++++++ 4 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/02894_ast_depth_check.reference create mode 100755 tests/queries/0_stateless/02894_ast_depth_check.sh diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index bbbbe454781..3c4ed435f91 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -691,6 +692,8 @@ bool ActionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr & child) void ActionsMatcher::visit(const ASTPtr & ast, Data & data) { + checkStackSize(); + if (const auto * identifier = ast->as()) visit(*identifier, ast, data); else if (const auto * table = ast->as()) diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index dd9a6023b0b..aa6357ec12f 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -263,6 +263,10 @@ ASTPtr tryParseQuery( const auto last_token = token_iterator.max(); _out_query_end = last_token.end; + /// Also check on the AST level, because the generated AST depth can be greater than the recursion depth of the parser. + if (max_parser_depth) + res->checkDepth(max_parser_depth); + ASTInsertQuery * insert = nullptr; if (parse_res) { diff --git a/tests/queries/0_stateless/02894_ast_depth_check.reference b/tests/queries/0_stateless/02894_ast_depth_check.reference new file mode 100644 index 00000000000..880a8212188 --- /dev/null +++ b/tests/queries/0_stateless/02894_ast_depth_check.reference @@ -0,0 +1 @@ +TOO_DEEP_AST diff --git a/tests/queries/0_stateless/02894_ast_depth_check.sh b/tests/queries/0_stateless/02894_ast_depth_check.sh new file mode 100755 index 00000000000..381853b7f3e --- /dev/null +++ b/tests/queries/0_stateless/02894_ast_depth_check.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL --query "SELECT * FROM format('TSV', \$\$ a UInt8, x ALIAS ''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN''IN'' \$\$, '1')" 2>&1 | grep -oF 'TOO_DEEP_AST' From 8f82d64129b3aaa52a0ff17287b5714c7ca84c7f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Mon, 9 Oct 2023 17:06:31 +0800 Subject: [PATCH 096/634] support rename table without keyword TABLE --- src/Parsers/ParserRenameQuery.cpp | 27 ++++++++++--------- ...891_rename_table_without_keyword.reference | 3 +++ .../02891_rename_table_without_keyword.sql | 14 ++++++++++ 3 files changed, 32 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02891_rename_table_without_keyword.reference create mode 100644 tests/queries/0_stateless/02891_rename_table_without_keyword.sql diff --git a/src/Parsers/ParserRenameQuery.cpp b/src/Parsers/ParserRenameQuery.cpp index cb595846633..56845d781b3 100644 --- a/src/Parsers/ParserRenameQuery.cpp +++ b/src/Parsers/ParserRenameQuery.cpp @@ -11,6 +11,7 @@ namespace DB bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { + ParserKeyword s_rename("RENAME"); ParserKeyword s_rename_table("RENAME TABLE"); ParserKeyword s_exchange_tables("EXCHANGE TABLES"); ParserKeyword s_rename_dictionary("RENAME DICTIONARY"); @@ -24,18 +25,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool exchange = false; bool dictionary = false; - if (s_rename_table.ignore(pos, expected)) - ; - else if (s_exchange_tables.ignore(pos, expected)) - exchange = true; - else if (s_rename_dictionary.ignore(pos, expected)) - dictionary = true; - else if (s_exchange_dictionaries.ignore(pos, expected)) - { - exchange = true; - dictionary = true; - } - else if (s_rename_database.ignore(pos, expected)) + if (s_rename_database.ignore(pos, expected)) { ASTPtr from_db; ASTPtr to_db; @@ -67,6 +57,19 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) node = query; return true; } + else if(s_rename_table.ignore(pos, expected)) + ; + else if (s_exchange_tables.ignore(pos, expected)) + exchange = true; + else if (s_rename_dictionary.ignore(pos, expected)) + dictionary = true; + else if (s_exchange_dictionaries.ignore(pos, expected)) + { + exchange = true; + dictionary = true; + } + else if (s_rename.ignore(pos, expected)) + ; else return false; diff --git a/tests/queries/0_stateless/02891_rename_table_without_keyword.reference b/tests/queries/0_stateless/02891_rename_table_without_keyword.reference new file mode 100644 index 00000000000..74c047b102a --- /dev/null +++ b/tests/queries/0_stateless/02891_rename_table_without_keyword.reference @@ -0,0 +1,3 @@ +r1 +r2 +r3 diff --git a/tests/queries/0_stateless/02891_rename_table_without_keyword.sql b/tests/queries/0_stateless/02891_rename_table_without_keyword.sql new file mode 100644 index 00000000000..d92b3f69e5d --- /dev/null +++ b/tests/queries/0_stateless/02891_rename_table_without_keyword.sql @@ -0,0 +1,14 @@ +DROP DATABASE IF EXISTS rename_db; +CREATE DATABASE rename_db; + +CREATE TABLE rename_db.r1 (name String) Engine=Memory(); +SHOW TABLES FROM rename_db; + +RENAME TABLE rename_db.r1 TO rename_db.r2; +SHOW TABLES FROM rename_db; + +RENAME rename_db.r2 TO rename_db.r3; +SHOW TABLES FROM rename_db; + +DROP DATABASE rename_db; + From 2e4709fd10277dc6936d619732350b2ac83ed16e Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Mon, 9 Oct 2023 21:16:46 +0000 Subject: [PATCH 097/634] Remove "unsigned" from docs now that we support all ints. --- docs/en/sql-reference/functions/math-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 90413b109dc..39b9f1d29ee 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -714,7 +714,7 @@ Result: ## byteSwap -Accepts an unsigned integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. +Accepts an integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. Currently, this is supported for up to 64-bit (signed and unsigned) integers. From 9677dedd870b05e8a6040191faddfd2c7e35c349 Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Mon, 9 Oct 2023 21:50:23 +0000 Subject: [PATCH 098/634] Add negative tests. Also: - Add comments in tests. - Add an example in docs where an IPv4 is casted to an int, byteswapped and then casted back to an IPv4. --- .../sql-reference/functions/math-functions.md | 12 +++---- src/Functions/byteSwap.cpp | 10 ++---- .../0_stateless/02887_byteswap.reference | 2 ++ tests/queries/0_stateless/02887_byteswap.sql | 34 ++++++++++++++++++- 4 files changed, 42 insertions(+), 16 deletions(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 39b9f1d29ee..2671a4b9743 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -741,11 +741,7 @@ Note that, in step#1, one can also choose to convert the operand to bytes in lit One use-case of this function is reversing IPv4s: ```result -┌─toIPv4(3351772109)─┐ -│ 199.199.251.205 │ -└────────────────────┘ - -┌─toIPv4(byteSwap(3351772109))─┐ -│ 205.251.199.199 │ -└──────────────────────────────┘ -``` \ No newline at end of file +┌─toIPv4(byteSwap(toUInt32(toIPv4('205.251.199.199'))))─┐ +│ 199.199.251.205 │ +└───────────────────────────────────────────────────────┘ +``` diff --git a/src/Functions/byteSwap.cpp b/src/Functions/byteSwap.cpp index e87671c40c5..ccbae8ab268 100644 --- a/src/Functions/byteSwap.cpp +++ b/src/Functions/byteSwap.cpp @@ -82,13 +82,9 @@ Note that, in step#1, one can also choose to convert the operand to bytes in lit One use-case of this function is reversing IPv4s: ```result -┌─toIPv4(3351772109)─┐ -│ 199.199.251.205 │ -└────────────────────┘ - -┌─toIPv4(byteSwap(3351772109))─┐ -│ 205.251.199.199 │ -└──────────────────────────────┘ +┌─toIPv4(byteSwap(toUInt32(toIPv4('205.251.199.199'))))─┐ +│ 199.199.251.205 │ +└───────────────────────────────────────────────────────┘ ``` )", .examples{ diff --git a/tests/queries/0_stateless/02887_byteswap.reference b/tests/queries/0_stateless/02887_byteswap.reference index 47e5a37f872..d4269a00aca 100644 --- a/tests/queries/0_stateless/02887_byteswap.reference +++ b/tests/queries/0_stateless/02887_byteswap.reference @@ -25,3 +25,5 @@ -549755813889 4039370097989451775 128 +0 +1 diff --git a/tests/queries/0_stateless/02887_byteswap.sql b/tests/queries/0_stateless/02887_byteswap.sql index 70f3b3ce4f6..2da0bebbaed 100644 --- a/tests/queries/0_stateless/02887_byteswap.sql +++ b/tests/queries/0_stateless/02887_byteswap.sql @@ -1,34 +1,66 @@ +/* UInt8 */ SELECT byteSwap(0); SELECT byteSwap(1); SELECT byteSwap(255); +/* UInt16 */ SELECT byteSwap(256); SELECT byteSwap(4135); SELECT byteSwap(10000); SELECT byteSwap(65535); +/* UInt32 */ SELECT byteSwap(65536); SELECT byteSwap(3351772109); SELECT byteSwap(3455829959); SELECT byteSwap(4294967295); +/* UInt64 */ SELECT byteSwap(4294967296); SELECT byteSwap(123294967295); SELECT byteSwap(18439412204227788800); SELECT byteSwap(18446744073709551615); +/* Int8 */ SELECT byteSwap(-0); SELECT byteSwap(-1); SELECT byteSwap(-128); +/* Int16 */ SELECT byteSwap(-129); SELECT byteSwap(-4135); SELECT byteSwap(-32768); +/* Int32 */ SELECT byteSwap(-32769); SELECT byteSwap(-3351772109); SELECT byteSwap(-2147483648); +/* Int64 */ SELECT byteSwap(-2147483649); SELECT byteSwap(-1242525266376); -SELECT byteswap(-9223372036854775808); \ No newline at end of file +SELECT byteSwap(-9223372036854775808); + +/* Booleans are interpreted as UInt8 */ +SELECT byteSwap(false); +SELECT byteSwap(true); + +/* Integer overflows */ +SELECT byteSwap(18446744073709551616); -- { serverError 48 } +SELECT byteSwap(-9223372036854775809); -- { serverError 48 } + +/* Number of arguments should equal 1 */ +SELECT byteSwap(); -- { serverError 42 } +SELECT byteSwap(128, 129); -- { serverError 42 } + +/* Input should be "integral" */ +SELECT byteSwap('abc'); -- { serverError 43 } +SELECT byteSwap(reinterpretAsFixedString(3351772109)); -- { serverError 43 } +SELECT byteSwap(toDate('2019-01-01')); -- { serverError 43 } +SELECT byteSwap(toDate32('2019-01-01')); -- { serverError 43 } +SELECT byteSwap(toDateTime32(1546300800)); -- { serverError 43 } +SELECT byteSwap(toDateTime64(1546300800, 3)); -- { serverError 43 } +SELECT byteSwap(generateUUIDv4()); -- { serverError 43 } +SELECT byteSwap(toDecimal32(2, 4)); -- { serverError 43 } +SELECT byteSwap(toFloat32(123.456)); -- { serverError 48 } +SELECT byteSwap(toFloat64(123.456)); -- { serverError 48 } From e29a828d6a0fa78a8e58a9c3714441c720bdd3f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Tue, 10 Oct 2023 10:58:13 +0800 Subject: [PATCH 099/634] rebase master From 06a7b679f65bc4503f9083fa874daa866c3b69a3 Mon Sep 17 00:00:00 2001 From: Joey Wang Date: Tue, 10 Oct 2023 14:51:51 +0800 Subject: [PATCH 100/634] support merging of nested paths --- .../sql-reference/functions/json-functions.md | 6 +-- .../{jsonMerge.cpp => jsonMergePatch.cpp} | 46 ++++++++++++------- .../02874_json_merge_function_test.sql | 11 ----- ..._json_merge_patch_function_test.reference} | 3 ++ .../02874_json_merge_patch_function_test.sql | 14 ++++++ .../aspell-ignore/en/aspell-dict.txt | 2 +- 6 files changed, 51 insertions(+), 31 deletions(-) rename src/Functions/{jsonMerge.cpp => jsonMergePatch.cpp} (73%) delete mode 100644 tests/queries/0_stateless/02874_json_merge_function_test.sql rename tests/queries/0_stateless/{02874_json_merge_function_test.reference => 02874_json_merge_patch_function_test.reference} (65%) create mode 100644 tests/queries/0_stateless/02874_json_merge_patch_function_test.sql diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index e5ff13763ad..83900532de8 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -511,14 +511,14 @@ SELECT ``` -## jsonMerge +## jsonMergePatch Return the merged JSON object string which is formed by merging multiple JSON objects. **Syntax** ``` sql -jsonMerge(json1, json2, ...) +jsonMergePatch(json1, json2, ...) ``` **Arguments** @@ -534,7 +534,7 @@ Type: [String](../../sql-reference/data-types/string.md). **Example** ``` sql -SELECT jsonMerge('{"a":1}', '{"name": "joey"}', '{"name": "tom"}', '{"name": "zoey"}') AS res +SELECT jsonMergePatch('{"a":1}', '{"name": "joey"}', '{"name": "tom"}', '{"name": "zoey"}') AS res ┌─res───────────────────┐ │ {"a":1,"name":"zoey"} │ diff --git a/src/Functions/jsonMerge.cpp b/src/Functions/jsonMergePatch.cpp similarity index 73% rename from src/Functions/jsonMerge.cpp rename to src/Functions/jsonMergePatch.cpp index e57b56d73a2..7fd366528ee 100644 --- a/src/Functions/jsonMerge.cpp +++ b/src/Functions/jsonMergePatch.cpp @@ -29,17 +29,17 @@ namespace ErrorCodes namespace { - // select jsonMerge('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); + // select jsonMergePatch('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); // || // \/ // ┌───────────────────────┐ // │ {"a":1,"name":"zoey"} │ // └───────────────────────┘ - class FunctionjsonMerge : public IFunction + class FunctionjsonMergePatch : public IFunction { public: - static constexpr auto name = "jsonMerge"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + static constexpr auto name = "jsonMergePatch"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -63,6 +63,29 @@ namespace merged_json.SetObject(); rapidjson::Document::AllocatorType& allocator = merged_json.GetAllocator(); + std::function mergeObjects; + mergeObjects = [&mergeObjects, &allocator](rapidjson::Value& dest, const rapidjson::Value& src) -> void + { + if (!src.IsObject()) + return; + for (auto it = src.MemberBegin(); it != src.MemberEnd(); ++it) + { + rapidjson::Value key(it->name, allocator); + rapidjson::Value value(it->value, allocator); + if (dest.HasMember(key)) + { + if (dest[key].IsObject() && value.IsObject()) + mergeObjects(dest[key], value); + else + dest[key] = value; + } + else + { + dest.AddMember(key, value, allocator); + } + } + }; + for (const auto & arg : arguments) { const ColumnPtr column = arg.column; @@ -78,16 +101,7 @@ namespace document.Parse(json); if (!document.IsObject()) throw Exception(ErrorCodes::ILLEGAL_JSON_OBJECT_FORMAT, "Wrong input Json object format"); - - for (auto it = document.MemberBegin(); it != document.MemberEnd(); ++it) - { - rapidjson::Value key(it->name, allocator); - rapidjson::Value value(it->value, allocator); - if (merged_json.HasMember(key)) - merged_json[key] = value; - else - merged_json.AddMember(key, value, allocator); - } + mergeObjects(merged_json, document); } } @@ -105,9 +119,9 @@ namespace } -REGISTER_FUNCTION(jsonMerge) +REGISTER_FUNCTION(jsonMergePatch) { - factory.registerFunction(FunctionDocumentation{ + factory.registerFunction(FunctionDocumentation{ .description="Return the merged JSON object string, which is formed by merging multiple JSON objects."}); } diff --git a/tests/queries/0_stateless/02874_json_merge_function_test.sql b/tests/queries/0_stateless/02874_json_merge_function_test.sql deleted file mode 100644 index b6287d13778..00000000000 --- a/tests/queries/0_stateless/02874_json_merge_function_test.sql +++ /dev/null @@ -1,11 +0,0 @@ --- Tags: no-fasttest -select jsonMerge(null); -select jsonMerge('{"a":1}'); -select jsonMerge('{"a":1}', '{"b":1}'); -select jsonMerge('{"a":1}', '{"b":1}', '{"c":[1,2]}'); -select jsonMerge('{"a":1}', '{"b":1}', '{"c":[{"d":1},2]}'); -select jsonMerge('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); -select jsonMerge('{"a": "1","b": 2,"c": [true,{"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); - -select jsonMerge('[1]'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } -select jsonMerge('{"a": "1","b": 2,"c": [true,"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } diff --git a/tests/queries/0_stateless/02874_json_merge_function_test.reference b/tests/queries/0_stateless/02874_json_merge_patch_function_test.reference similarity index 65% rename from tests/queries/0_stateless/02874_json_merge_function_test.reference rename to tests/queries/0_stateless/02874_json_merge_patch_function_test.reference index 11bc968e6c6..85caab63611 100644 --- a/tests/queries/0_stateless/02874_json_merge_function_test.reference +++ b/tests/queries/0_stateless/02874_json_merge_patch_function_test.reference @@ -5,3 +5,6 @@ {"a":1,"b":1,"c":[{"d":1},2]} {"a":1,"name":"zoey"} {"a":"1","b":2,"c":"1"} +{"a":{"b":[3,4],"c":2}} +{"a":5,"b":2,"c":4,"d":6} +{"a":1,"b":null} diff --git a/tests/queries/0_stateless/02874_json_merge_patch_function_test.sql b/tests/queries/0_stateless/02874_json_merge_patch_function_test.sql new file mode 100644 index 00000000000..51e520bd0e8 --- /dev/null +++ b/tests/queries/0_stateless/02874_json_merge_patch_function_test.sql @@ -0,0 +1,14 @@ +-- Tags: no-fasttest +select jsonMergePatch(null); +select jsonMergePatch('{"a":1}'); +select jsonMergePatch('{"a":1}', '{"b":1}'); +select jsonMergePatch('{"a":1}', '{"b":1}', '{"c":[1,2]}'); +select jsonMergePatch('{"a":1}', '{"b":1}', '{"c":[{"d":1},2]}'); +select jsonMergePatch('{"a":1}','{"name": "joey"}','{"name": "tom"}','{"name": "zoey"}'); +select jsonMergePatch('{"a": "1","b": 2,"c": [true,{"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); +select jsonMergePatch('{"a": {"b": 1, "c": 2}}', '{"a": {"b": [3, 4]}}'); +select jsonMergePatch('{ "a": 1, "b":2 }','{ "a": 3, "c":4 }','{ "a": 5, "d":6 }'); +select jsonMergePatch('{"a":1, "b":2}', '{"b":null}'); + +select jsonMergePatch('[1]'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } +select jsonMergePatch('{"a": "1","b": 2,"c": [true,"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 26ac93d56b7..c7f2a567fcb 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1660,7 +1660,7 @@ jsoncompactstringseachrowwithnames jsoncompactstringseachrowwithnamesandtypes jsoneachrow jsoneachrowwithprogress -jsonMerge +jsonMergePatch jsonobjecteachrow jsonstrings jsonstringseachrow From 31ae84604f2ea987523c24fc526418a5a543378b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 10 Oct 2023 09:54:00 +0000 Subject: [PATCH 101/634] Some fixups --- .../functions/arithmetic-functions.md | 37 +++++++ .../sql-reference/functions/math-functions.md | 34 ------- src/Functions/byteSwap.cpp | 19 ++-- .../0_stateless/02887_byteswap.reference | 2 +- tests/queries/0_stateless/02887_byteswap.sql | 97 +++++++++---------- 5 files changed, 90 insertions(+), 99 deletions(-) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 69f1816b7df..1aa9c982f33 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -441,3 +441,40 @@ DB::Exception: Decimal result's scale is less than argument's one: While process │ -12 │ 2.1 │ -5.7 │ -5.71428 │ └─────┴─────┴────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────┘ ``` + +## byteSwap + +Reverses the bytes of an integer, i.e. changes its [endianness](https://en.wikipedia.org/wiki/Endianness). Currently, integers of up to 64 bit are supported. + +**Syntax** + +```sql +byteSwap(a) +``` + +**Example** + +```sql +byteSwap(3351772109) +``` + +Result: + +```result +┌─byteSwap(3351772109)─┐ +│ 3455829959 │ +└──────────────────────┘ +``` + +The above example can be worked out in the following manner: +1. Convert the base-10 integer to its equivalent hexadecimal format in big-endian format, i.e. 3351772109 -> C7 C7 FB CD (4 bytes) +2. Reverse the bytes, i.e. C7 C7 FB CD -> CD FB C7 C7 +3. Convert the result back to an integer assuming big-endian, i.e. CD FB C7 C7 -> 3455829959 + +One use case of this function is reversing IPv4s: + +```result +┌─toIPv4(byteSwap(toUInt32(toIPv4('205.251.199.199'))))─┐ +│ 199.199.251.205 │ +└───────────────────────────────────────────────────────┘ +``` diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 2671a4b9743..9eab2274210 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -711,37 +711,3 @@ Result: │ 11 │ └──────────────────────────────────┘ ``` - -## byteSwap - -Accepts an integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. - -Currently, this is supported for up to 64-bit (signed and unsigned) integers. - -**Example** - -```sql -byteSwap(3351772109) -``` - -Result: - -```result -┌─byteSwap(3351772109)─┐ -│ 3455829959 │ -└──────────────────────┘ -``` - -The above example can be worked out in the following manner: -1. First, convert the integer operand (base 10) to its equivalent hexadecimal interpretation (base 16) in big-endian format i.e. 3351772109 -> C7 C7 FB CD (4 bytes) -2. Then, reverse the bytes i.e. C7 C7 FB CD -> CD FB C7 C7 -3. Finally, the convert the hexadecimal number back to an integer assuming big-endian i.e. CD FB C7 C7 -> 3455829959 - -Note that, in step#1, one can also choose to convert the operand to bytes in little-endian as long as one also assumes little-endian when converting back to integer in step#3. - -One use-case of this function is reversing IPv4s: -```result -┌─toIPv4(byteSwap(toUInt32(toIPv4('205.251.199.199'))))─┐ -│ 199.199.251.205 │ -└───────────────────────────────────────────────────────┘ -``` diff --git a/src/Functions/byteSwap.cpp b/src/Functions/byteSwap.cpp index ccbae8ab268..2de6df4a4b3 100644 --- a/src/Functions/byteSwap.cpp +++ b/src/Functions/byteSwap.cpp @@ -12,13 +12,13 @@ namespace { template requires std::is_integral_v -inline T byteSwap(T x) +T byteSwap(T x) { return std::byteswap(x); } template -inline T byteSwap(T) +T byteSwap(T) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "byteSwap() is not implemented for {} datatype", demangle(typeid(T).name())); } @@ -28,7 +28,7 @@ struct ByteSwapImpl { using ResultType = T; static constexpr const bool allow_string_or_fixed_string = false; - static inline T apply(T x) { return byteSwap(x); } + static T apply(T x) { return byteSwap(x); } #if USE_EMBEDDED_COMPILER static constexpr bool compilable = false; @@ -55,9 +55,7 @@ REGISTER_FUNCTION(ByteSwap) factory.registerFunction( FunctionDocumentation{ .description = R"( -Accepts an integer `operand` and returns the integer which is obtained by swapping the **endianness** of `operand` i.e. reversing the bytes of the `operand`. - -Currently, this is supported for up to 64-bit (signed and unsigned) integers. +Reverses the bytes of an integer, i.e. changes its [endianness](https://en.wikipedia.org/wiki/Endianness). Currently, integers of up to 64 bit are supported. **Example** @@ -74,13 +72,12 @@ Result: ``` The above example can be worked out in the following manner: -1. First, convert the integer operand (base 10) to its equivalent hexadecimal interpretation (base 16) in big-endian format i.e. 3351772109 -> C7 C7 FB CD (4 bytes) -2. Then, reverse the bytes i.e. C7 C7 FB CD -> CD FB C7 C7 -3. Finally, the convert the hexadecimal number back to an integer assuming big-endian i.e. CD FB C7 C7 -> 3455829959 +1. Convert the base-10 integer to its equivalent hexadecimal format in big-endian format, i.e. 3351772109 -> C7 C7 FB CD (4 bytes) +2. Reverse the bytes, i.e. C7 C7 FB CD -> CD FB C7 C7 +3. Convert the result back to an integer assuming big-endian, i.e. CD FB C7 C7 -> 3455829959 -Note that, in step#1, one can also choose to convert the operand to bytes in little-endian as long as one also assumes little-endian when converting back to integer in step#3. +One use-case of this function is reversing IPv5s: -One use-case of this function is reversing IPv4s: ```result ┌─toIPv4(byteSwap(toUInt32(toIPv4('205.251.199.199'))))─┐ │ 199.199.251.205 │ diff --git a/tests/queries/0_stateless/02887_byteswap.reference b/tests/queries/0_stateless/02887_byteswap.reference index d4269a00aca..b55b6f9cd6b 100644 --- a/tests/queries/0_stateless/02887_byteswap.reference +++ b/tests/queries/0_stateless/02887_byteswap.reference @@ -20,7 +20,7 @@ -9745 128 -8388609 -3676125013305458687 +855914552 128 -549755813889 4039370097989451775 diff --git a/tests/queries/0_stateless/02887_byteswap.sql b/tests/queries/0_stateless/02887_byteswap.sql index 2da0bebbaed..297132c7e7d 100644 --- a/tests/queries/0_stateless/02887_byteswap.sql +++ b/tests/queries/0_stateless/02887_byteswap.sql @@ -1,66 +1,57 @@ -/* UInt8 */ -SELECT byteSwap(0); -SELECT byteSwap(1); -SELECT byteSwap(255); +SELECT byteSwap(0::UInt8); +SELECT byteSwap(1::UInt8); +SELECT byteSwap(255::UInt8); -/* UInt16 */ -SELECT byteSwap(256); -SELECT byteSwap(4135); -SELECT byteSwap(10000); -SELECT byteSwap(65535); +SELECT byteSwap(256::UInt16); +SELECT byteSwap(4135::UInt16); +SELECT byteSwap(10000::UInt16); +SELECT byteSwap(65535::UInt16); -/* UInt32 */ -SELECT byteSwap(65536); -SELECT byteSwap(3351772109); -SELECT byteSwap(3455829959); -SELECT byteSwap(4294967295); +SELECT byteSwap(65536::UInt32); +SELECT byteSwap(3351772109::UInt32); +SELECT byteSwap(3455829959::UInt32); +SELECT byteSwap(4294967295::UInt32); -/* UInt64 */ -SELECT byteSwap(4294967296); -SELECT byteSwap(123294967295); -SELECT byteSwap(18439412204227788800); -SELECT byteSwap(18446744073709551615); +SELECT byteSwap(4294967296::UInt64); +SELECT byteSwap(123294967295::UInt64); +SELECT byteSwap(18439412204227788800::UInt64); +SELECT byteSwap(18446744073709551615::UInt64); -/* Int8 */ -SELECT byteSwap(-0); -SELECT byteSwap(-1); -SELECT byteSwap(-128); +SELECT byteSwap(-0::Int8); +SELECT byteSwap(-1::Int8); +SELECT byteSwap(-128::Int8); -/* Int16 */ -SELECT byteSwap(-129); -SELECT byteSwap(-4135); -SELECT byteSwap(-32768); +SELECT byteSwap(-129::Int16); +SELECT byteSwap(-4135::Int16); +SELECT byteSwap(-32768::Int16); -/* Int32 */ -SELECT byteSwap(-32769); -SELECT byteSwap(-3351772109); -SELECT byteSwap(-2147483648); +SELECT byteSwap(-32769::Int32); +SELECT byteSwap(-3351772109::Int32); +SELECT byteSwap(-2147483648::Int32); -/* Int64 */ -SELECT byteSwap(-2147483649); -SELECT byteSwap(-1242525266376); -SELECT byteSwap(-9223372036854775808); +SELECT byteSwap(-2147483649::Int64); +SELECT byteSwap(-1242525266376::Int64); +SELECT byteSwap(-9223372036854775808::Int64); -/* Booleans are interpreted as UInt8 */ +-- Booleans are interpreted as UInt8 SELECT byteSwap(false); SELECT byteSwap(true); -/* Integer overflows */ -SELECT byteSwap(18446744073709551616); -- { serverError 48 } -SELECT byteSwap(-9223372036854775809); -- { serverError 48 } +-- Number of arguments should equal 1 +SELECT byteSwap(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT byteSwap(128, 129); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -/* Number of arguments should equal 1 */ -SELECT byteSwap(); -- { serverError 42 } -SELECT byteSwap(128, 129); -- { serverError 42 } +-- Input should be integral +SELECT byteSwap('abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT byteSwap(toFixedString('abc', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT byteSwap(toDate('2019-01-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT byteSwap(toDate32('2019-01-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT byteSwap(toDateTime32(1546300800)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT byteSwap(toDateTime64(1546300800, 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT byteSwap(generateUUIDv4()); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT byteSwap(toDecimal32(2, 4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT byteSwap(toFloat32(123.456)); -- { serverError NOT_IMPLEMENTED } +SELECT byteSwap(toFloat64(123.456)); -- { serverError NOT_IMPLEMENTED } +SELECT byteSwap(18446744073709551616::UInt128); -- { serverError NOT_IMPLEMENTED } +SELECT byteSwap(-9223372036854775809::Int128); -- { serverError NOT_IMPLEMENTED } -/* Input should be "integral" */ -SELECT byteSwap('abc'); -- { serverError 43 } -SELECT byteSwap(reinterpretAsFixedString(3351772109)); -- { serverError 43 } -SELECT byteSwap(toDate('2019-01-01')); -- { serverError 43 } -SELECT byteSwap(toDate32('2019-01-01')); -- { serverError 43 } -SELECT byteSwap(toDateTime32(1546300800)); -- { serverError 43 } -SELECT byteSwap(toDateTime64(1546300800, 3)); -- { serverError 43 } -SELECT byteSwap(generateUUIDv4()); -- { serverError 43 } -SELECT byteSwap(toDecimal32(2, 4)); -- { serverError 43 } -SELECT byteSwap(toFloat32(123.456)); -- { serverError 48 } -SELECT byteSwap(toFloat64(123.456)); -- { serverError 48 } From 8b7d1021cd207a9edef6b2b0077f498c4c6d98e8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 10 Oct 2023 13:32:20 +0000 Subject: [PATCH 102/634] Docs: Update field descriptions of system.part_log --- docs/en/operations/system-tables/part_log.md | 26 ++++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/en/operations/system-tables/part_log.md b/docs/en/operations/system-tables/part_log.md index c9e34962c79..b9185434e01 100644 --- a/docs/en/operations/system-tables/part_log.md +++ b/docs/en/operations/system-tables/part_log.md @@ -11,21 +11,21 @@ The `system.part_log` table contains the following columns: - `query_id` ([String](../../sql-reference/data-types/string.md)) — Identifier of the `INSERT` query that created this data part. - `event_type` ([Enum8](../../sql-reference/data-types/enum.md)) — Type of the event that occurred with the data part. Can have one of the following values: - - `NEW_PART` — Inserting of a new data part. - - `MERGE_PARTS` — Merging of data parts. - - `DOWNLOAD_PART` — Downloading a data part. - - `REMOVE_PART` — Removing or detaching a data part using [DETACH PARTITION](../../sql-reference/statements/alter/partition.md#alter_detach-partition). - - `MUTATE_PART` — Mutating of a data part. - - `MOVE_PART` — Moving the data part from the one disk to another one. + - `NewPart` — Inserting of a new data part. + - `MergeParts` — Merging of data parts. + - `DownloadParts` — Downloading a data part. + - `RemovePart` — Removing or detaching a data part using [DETACH PARTITION](../../sql-reference/statements/alter/partition.md#alter_detach-partition). + - `MutatePart` — Mutating of a data part. + - `MovePart` — Moving the data part from the one disk to another one. - `merge_reason` ([Enum8](../../sql-reference/data-types/enum.md)) — The reason for the event with type `MERGE_PARTS`. Can have one of the following values: - - `NOT_A_MERGE` — The current event has the type other than `MERGE_PARTS`. - - `REGULAR_MERGE` — Some regular merge. - - `TTL_DELETE_MERGE` — Cleaning up expired data. - - `TTL_RECOMPRESS_MERGE` — Recompressing data part with the. + - `NotAMerge` — The current event has the type other than `MERGE_PARTS`. + - `RegularMerge` — Some regular merge. + - `TTLDeleteMerge` — Cleaning up expired data. + - `TTLRecompressMerge` — Recompressing data part with the. - `merge_algorithm` ([Enum8](../../sql-reference/data-types/enum.md)) — Merge algorithm for the event with type `MERGE_PARTS`. Can have one of the following values: - - `UNDECIDED` - - `HORIZONTAL` - - `VERTICAL` + - `Undecided` + - `Horizontal` + - `Vertical` - `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time. - `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Event time with microseconds precision. From dda3463944d9be2b97c2186ee25bab863f664c87 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 10 Oct 2023 11:44:06 -0300 Subject: [PATCH 103/634] add some tests --- ..._proxy_configuration_resolver_provider.cpp | 74 +++++++++++++++++++ 1 file changed, 74 insertions(+) diff --git a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp index 884040fcf52..bd6e08522c4 100644 --- a/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp +++ b/src/Common/tests/gtest_proxy_configuration_resolver_provider.cpp @@ -120,4 +120,78 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListBoth) ASSERT_EQ(https_proxy_configuration.port, https_list_proxy_server.getPort()); } +TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolConfigurationHTTP) +{ + /* + * Since there is no way to call `ProxyConfigurationResolver::resolve` on remote resolver, + * it is hard to verify the remote resolver was actually picked. One hackish way to assert + * the remote resolver was OR was not picked based on the configuration, is to use the + * environment resolver. Since the environment resolver is always returned as a fallback, + * we can assert the remote resolver was not picked if `ProxyConfigurationResolver::resolve` + * succeeds and returns an environment proxy configuration. + * */ + EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server); + + ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); + + config->setString("proxy", ""); + config->setString("proxy.https", ""); + config->setString("proxy.https.resolver", ""); + config->setString("proxy.https.resolver.endpoint", "http://resolver:8080/hostname"); + + // even tho proxy protocol / scheme is http, it should not be picked (prior to this PR, it would be picked) + config->setString("proxy.https.resolver.proxy_scheme", "http"); + config->setString("proxy.https.resolver.proxy_port", "80"); + config->setString("proxy.https.resolver.proxy_cache_time", "10"); + + context->setConfig(config); + + auto http_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config)->resolve(); + + /* + * Asserts env proxy is used and not the remote resolver. If the remote resolver is picked, it is an error because + * there is no `http` specification for remote resolver + * */ + ASSERT_EQ(http_proxy_configuration.host, http_env_proxy_server.getHost()); + ASSERT_EQ(http_proxy_configuration.port, http_env_proxy_server.getPort()); + ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_env_proxy_server.getScheme())); +} + +TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolConfigurationHTTPS) +{ + /* + * Since there is no way to call `ProxyConfigurationResolver::resolve` on remote resolver, + * it is hard to verify the remote resolver was actually picked. One hackish way to assert + * the remote resolver was OR was not picked based on the configuration, is to use the + * environment resolver. Since the environment resolver is always returned as a fallback, + * we can assert the remote resolver was not picked if `ProxyConfigurationResolver::resolve` + * succeeds and returns an environment proxy configuration. + * */ + EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server); + + ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); + + config->setString("proxy", ""); + config->setString("proxy.http", ""); + config->setString("proxy.http.resolver", ""); + config->setString("proxy.http.resolver.endpoint", "http://resolver:8080/hostname"); + + // even tho proxy protocol / scheme is https, it should not be picked (prior to this PR, it would be picked) + config->setString("proxy.http.resolver.proxy_scheme", "https"); + config->setString("proxy.http.resolver.proxy_port", "80"); + config->setString("proxy.http.resolver.proxy_cache_time", "10"); + + context->setConfig(config); + + auto http_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config)->resolve(); + + /* + * Asserts env proxy is used and not the remote resolver. If the remote resolver is picked, it is an error because + * there is no `http` specification for remote resolver + * */ + ASSERT_EQ(http_proxy_configuration.host, https_env_proxy_server.getHost()); + ASSERT_EQ(http_proxy_configuration.port, https_env_proxy_server.getPort()); + ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_env_proxy_server.getScheme())); +} + // remote resolver is tricky to be tested in unit tests From 50e7f02dc55f3f6acc17875868ed478ccbd758c6 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 10 Oct 2023 12:18:59 -0300 Subject: [PATCH 104/634] trigger ci From eca411ec922f090c7e852b5dd429aa94e1c4b9f7 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 10 Oct 2023 13:14:41 -0300 Subject: [PATCH 105/634] that it? --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index b43f9cab0f1..d2d68fcc984 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2136,6 +2136,7 @@ repo representable requestor requireTLSv +resolvers resharding reshards resultset From ab09aee75433aa013bc70f7b3cdc713c5c35087b Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 10 Oct 2023 19:59:48 +0200 Subject: [PATCH 106/634] Initial implementation --- src/Storages/AlterCommands.cpp | 5 ++ src/Storages/AlterCommands.h | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 53 +++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 6 +++ .../MergeTree/ReplicatedMergeTreeQueue.cpp | 16 ++++++ .../MergeTree/ReplicatedMergeTreeQueue.h | 8 +-- src/Storages/StorageMergeTree.cpp | 27 ++++++++++ src/Storages/StorageMergeTree.h | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 5 ++ src/Storages/StorageReplicatedMergeTree.h | 2 + 10 files changed, 123 insertions(+), 3 deletions(-) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 3ade4474b6b..fd77e33e97e 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -845,6 +845,11 @@ bool AlterCommand::isRemovingProperty() const return to_remove != RemoveProperty::NO_PROPERTY; } +bool AlterCommand::isDropSomething() const +{ + return type == Type::DROP_COLUMN || type == Type::DROP_INDEX || type == Type::DROP_CONSTRAINT; +} + std::optional AlterCommand::tryConvertToMutationCommand(StorageInMemoryMetadata & metadata, ContextPtr context) const { if (!isRequireMutationStage(metadata)) diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index c06872f9757..74a11d2beaa 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -167,6 +167,8 @@ struct AlterCommand /// Command removing some property from column or table bool isRemovingProperty() const; + bool isDropSomething() const; + /// If possible, convert alter command to mutation command. In other case /// return empty optional. Some storages may execute mutations after /// metadata changes. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6c014307c3c..1e7c35418ab 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2961,9 +2961,11 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context NamesAndTypesList columns_to_check_conversion; + auto unfinished_mutations = getUnfinishedMutationCommands(); std::optional name_deps{}; for (const AlterCommand & command : commands) { + checkDropCommandDoesntAffectInProgressMutations(command, unfinished_mutations, local_context); /// Just validate partition expression if (command.partition) { @@ -8082,6 +8084,57 @@ bool MergeTreeData::canUsePolymorphicParts() const return canUsePolymorphicParts(*getSettings(), unused); } + +void MergeTreeData::checkDropCommandDoesntAffectInProgressMutations(const AlterCommand & command, const std::map & unfinished_mutations, ContextPtr /*local_context*/) const +{ + if (!command.isDropSomething()) + return; + + for (const auto & [mutation_name, commands] : unfinished_mutations) + { + for (const MutationCommand & mutation_command : commands) + { + if (command.type == AlterCommand::DROP_INDEX && mutation_command.index_name == command.index_name) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot drop index {} because it's affected by mutation with ID '{}' which is not finished yet. " + "Wait this mutation, or KILL it with command " + "\"KILL MUTATION WHERE mutation_id = '{}'\"", + command.index_name, + mutation_name, + mutation_name); + } + else if (command.type == AlterCommand::DROP_PROJECTION + && mutation_command.projection_name == command.projection_name) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot drop projection {} because it's affected by mutation with ID '{}' which is not finished yet. " + "Wait this mutation, or KILL it with command " + "\"KILL MUTATION WHERE mutation_id = '{}'\"", + command.index_name, + mutation_name, + mutation_name); + } + else if (command.type == AlterCommand::DROP_COLUMN) + { + if (mutation_command.column_name == command.column_name) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot drop column {} because it's affected by mutation with ID '{}' which is not finished yet. " + "Wait this mutation, or KILL it with command " + "\"KILL MUTATION WHERE mutation_id = '{}'\"", + command.index_name, + mutation_name, + mutation_name); + } + } + } + } +} + bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, String & out_reason) const { if (!canUseAdaptiveGranularity()) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 55b4b23e351..346bfee076a 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -47,6 +47,7 @@ namespace DB /// Number of streams is not number parts, but number or parts*files, hence 1000. const size_t DEFAULT_DELAYED_STREAMS_FOR_PARALLEL_WRITE = 1000; +struct AlterCommand; class AlterCommands; class MergeTreePartsMover; class MergeTreeDataMergerMutator; @@ -721,6 +722,11 @@ public: /// If something is wrong, throws an exception. void checkAlterIsPossible(const AlterCommands & commands, ContextPtr context) const override; + void checkDropCommandDoesntAffectInProgressMutations( + const AlterCommand & command, const std::map & unfinished_mutations, ContextPtr context) const; + + virtual std::map getUnfinishedMutationCommands() const = 0; + /// Checks if the Mutation can be performed. /// (currently no additional checks: always ok) void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 3e61a02c29a..f993515573a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -8,6 +8,7 @@ #include #include #include +#include "Storages/MutationCommands.h" #include #include @@ -1777,6 +1778,21 @@ size_t ReplicatedMergeTreeQueue::countUnfinishedMutations() const } +std::map ReplicatedMergeTreeQueue::getUnfinishedMutations() const +{ + std::map result; + std::lock_guard lock(state_mutex); + + for (const auto & [name, status] : mutations_by_znode | std::views::reverse) + { + if (status.is_done) + break; + result.emplace(name, status.entry->commands); + } + + return result; +} + ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zkutil::ZooKeeperPtr & zookeeper, std::optional && partition_ids_hint) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index d5d85e58cb5..452dee22ea1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -391,9 +391,11 @@ public: /// Count the total number of active mutations that are not finished (is_done = false). size_t countUnfinishedMutations() const; - /// Returns functor which used by MergeTreeMergerMutator to select parts for merge - ReplicatedMergeTreeMergePredicate getMergePredicate(zkutil::ZooKeeperPtr & zookeeper, - std::optional && partition_ids_hint); + std::map getUnfinishedMutations() const; + + /// Returns functor which used by MergeTreeMergerMutator to select parts for merge + ReplicatedMergeTreeMergePredicate + getMergePredicate(zkutil::ZooKeeperPtr & zookeeper, std::optional && partition_ids_hint); MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, Strings & mutation_ids) const; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 470e30b7947..2b1eb6b25d0 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -714,6 +714,33 @@ std::optional StorageMergeTree::getIncompleteMutationsS return result; } +std::map StorageMergeTree::getUnfinishedMutationCommands() const +{ + std::lock_guard lock(currently_processing_in_background_mutex); + std::vector part_versions_with_names; + auto data_parts = getDataPartsVectorForInternalUsage(); + part_versions_with_names.reserve(data_parts.size()); + for (const auto & part : data_parts) + part_versions_with_names.emplace_back(PartVersionWithName{part->info.getDataVersion(), part->name}); + std::sort(part_versions_with_names.begin(), part_versions_with_names.end(), comparator); + + std::map result; + + for (const auto & kv : current_mutations_by_version) + { + Int64 mutation_version = kv.first; + const MergeTreeMutationEntry & entry = kv.second; + const PartVersionWithName needle{mutation_version, ""}; + auto versions_it = std::lower_bound( + part_versions_with_names.begin(), part_versions_with_names.end(), needle, comparator); + + size_t parts_to_do = versions_it - part_versions_with_names.begin(); + if (parts_to_do > 0) + result.emplace(entry.file_name, entry.commands); + } + return result; +} + std::vector StorageMergeTree::getMutationsStatus() const { std::lock_guard lock(currently_processing_in_background_mutex); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 89da9ab839e..a6d2f3a3272 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -114,6 +114,8 @@ public: size_t getNumberOfUnfinishedMutations() const override; + std::map getUnfinishedMutationCommands() const override; + MergeTreeDeduplicationLog * getDeduplicationLog() { return deduplication_log.get(); } private: diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 63f785a93e5..421d04087ee 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8715,6 +8715,11 @@ size_t StorageReplicatedMergeTree::getNumberOfUnfinishedMutations() const return queue.countUnfinishedMutations(); } +std::map StorageReplicatedMergeTree::getUnfinishedMutationCommands() const +{ + return queue.getUnfinishedMutations(); +} + void StorageReplicatedMergeTree::createTableSharedID() const { LOG_DEBUG(log, "Creating shared ID for table {}", getStorageID().getNameForLogs()); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 794991d8e06..b027735fda2 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -347,6 +347,8 @@ public: // Return table id, common for different replicas String getTableSharedID() const override; + std::map getUnfinishedMutationCommands() const override; + size_t getNumberOfUnfinishedMutations() const override; /// Returns the same as getTableSharedID(), but extracts it from a create query. From b107712e0cf8e2671da7c1641b268df57e54bd7a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 9 Oct 2023 16:08:22 +0200 Subject: [PATCH 107/634] Fix filtering by virtual columns with OR filter in query The problem with the initial implementation #52653 was: - OR can have multiple arguments - It simply not correct to assume that if there are two arguments this is OK. Consider the following example: "WHERE (column_not_from_partition_by = 1) OR false OR false" Will be converted to: "WHERE false OR false" And it will simply read nothing. Yes, we could apply some optimization for bool, but this will not always work, since to optimize things like "0 = 1" we need to execute it. And the only way to make handle this correctly (with ability to ignore some commands during filtering) is to make is_constant() function return has it use something from the input block, so that we can be sure, that we have some sensible, and not just "false". Plus we cannot simply ignore the difference of the input and output arguments of handling OR, we need to add always-true (1/true) if the size is different, since otherwise it could break invariants (see comment in the code). This includes (but not limited to): - _part* filtering for MergeTree - _path/_file for various File/HDFS/... engines - _table for Merge - ... P.S. analyzer does not have this bug, since it execute expression as whole, and this is what filterBlockWithQuery() should do actually instead, but this will be a more complex patch. Signed-off-by: Azat Khuzhin --- src/Storages/VirtualColumnUtils.cpp | 55 +++++++++++++++---- .../02840_merge__table_or_filter.sql.j2 | 5 ++ .../0_stateless/02896_multiple_OR.reference | 14 +++++ .../queries/0_stateless/02896_multiple_OR.sql | 28 ++++++++++ 4 files changed, 92 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02896_multiple_OR.reference create mode 100644 tests/queries/0_stateless/02896_multiple_OR.sql diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index d0d6233728e..219043f25c6 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -81,14 +82,33 @@ bool extractFunctions(const ASTPtr & expression, const std::functionname == "or") { - bool ret = true; + bool ret = false; ASTs or_args; for (const auto & child : function->arguments->children) - ret &= extractFunctions(child, is_constant, or_args); - /// We can keep condition only if it still OR condition (i.e. we - /// have dependent conditions for columns at both sides) - if (or_args.size() == 2) + ret |= extractFunctions(child, is_constant, or_args); + + if (!or_args.empty()) + { + /// In case of there are less number of arguments for which + /// is_constant() == true, we need to add always-true + /// implicitly to avoid breaking AND invariant. + /// + /// Consider the following: + /// + /// ((value = 10) OR (_table = 'v2')) AND ((_table = 'v1') OR (value = 20)) + /// + /// Without implicit always-true: + /// + /// (_table = 'v2') AND (_table = 'v1') + /// + /// With: + /// + /// (_table = 'v2' OR 1) AND (_table = 'v1' OR 1) -> (_table = 'v2') OR (_table = 'v1') + /// + if (or_args.size() != function->arguments->children.size()) + or_args.push_back(std::make_shared(Field(1))); result.push_back(makeASTForLogicalOr(std::move(or_args))); + } return ret; } } @@ -165,8 +185,10 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block if (!select.where() && !select.prewhere()) return unmodified; - // Provide input columns as constant columns to check if an expression is constant. - std::function is_constant = [&block, &context](const ASTPtr & node) + // Provide input columns as constant columns to check if an expression is + // constant and depends on the columns from provided block (the last is + // required to allow skipping some conditions for handling OR). + std::function is_constant = [&block, &context](const ASTPtr & expr) { auto actions = std::make_shared(block.getColumnsWithTypeAndName()); PreparedSetsPtr prepared_sets = std::make_shared(); @@ -178,13 +200,26 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block context, SizeLimits{}, 1, source_columns, std::move(actions), prepared_sets, true, true, true, { aggregation_keys, grouping_set_keys, GroupByKind::NONE }); - ActionsVisitor(visitor_data).visit(node); + ActionsVisitor(visitor_data).visit(expr); actions = visitor_data.getActions(); + auto expr_column_name = expr->getColumnName(); + + const auto * expr_const_node = actions->tryFindInOutputs(expr_column_name); + if (!expr_const_node) + return false; + auto filter_actions = ActionsDAG::buildFilterActionsDAG({expr_const_node}, {}, context); + const auto & nodes = filter_actions->getNodes(); + bool has_dependent_columns = std::any_of(nodes.begin(), nodes.end(), [&](const auto & node) + { + return block.has(node.result_name); + }); + if (!has_dependent_columns) + return false; + auto expression_actions = std::make_shared(actions); auto block_with_constants = block; expression_actions->execute(block_with_constants); - auto column_name = node->getColumnName(); - return block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column); + return block_with_constants.has(expr_column_name) && isColumnConst(*block_with_constants.getByName(expr_column_name).column); }; /// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. diff --git a/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 b/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 index a87ef7302c6..286e4545ef7 100644 --- a/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 +++ b/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 @@ -18,6 +18,11 @@ create view v2 as select * from d2; create table m as v1 engine=Merge(currentDatabase(), '^(v1|v2)$'); +{# -- FIXME: +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') or 0 or 0 settings {{ settings }}; +select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') or 0 or 0 settings {{ settings }}; +#} + -- avoid reorder set max_threads=1; -- { echoOn } diff --git a/tests/queries/0_stateless/02896_multiple_OR.reference b/tests/queries/0_stateless/02896_multiple_OR.reference new file mode 100644 index 00000000000..96480a75d11 --- /dev/null +++ b/tests/queries/0_stateless/02896_multiple_OR.reference @@ -0,0 +1,14 @@ +-- { echoOn } +SELECT * FROM or_bug WHERE (key = 1) OR false OR false; +1 +SELECT * FROM or_bug WHERE (key = 1) OR false; +1 +SELECT * FROM or_bug WHERE (key = 1); +1 +-- { echoOn } +select * from forms where text_field like '%this%' or 0 = 1 or 0 = 1; +5840ead423829c1eab29fa97 this is a test +select * from forms where text_field like '%this%' or 0 = 1; +5840ead423829c1eab29fa97 this is a test +select * from forms where text_field like '%this%'; +5840ead423829c1eab29fa97 this is a test diff --git a/tests/queries/0_stateless/02896_multiple_OR.sql b/tests/queries/0_stateless/02896_multiple_OR.sql new file mode 100644 index 00000000000..653ddebca1f --- /dev/null +++ b/tests/queries/0_stateless/02896_multiple_OR.sql @@ -0,0 +1,28 @@ +-- https://github.com/ClickHouse/ClickHouse/pull/52653 +DROP TABLE IF EXISTS or_bug; +CREATE TABLE or_bug (key UInt8) ENGINE=MergeTree ORDER BY key; +INSERT INTO TABLE or_bug VALUES (0), (1); + +-- { echoOn } +SELECT * FROM or_bug WHERE (key = 1) OR false OR false; +SELECT * FROM or_bug WHERE (key = 1) OR false; +SELECT * FROM or_bug WHERE (key = 1); +-- { echoOff } + +-- https://github.com/ClickHouse/ClickHouse/issues/55288 +DROP TABLE IF EXISTS forms; +CREATE TABLE forms +( + `form_id` FixedString(24), + `text_field` String +) +ENGINE = MergeTree +PRIMARY KEY form_id +ORDER BY form_id; +insert into forms values ('5840ead423829c1eab29fa97','this is a test'); + +-- { echoOn } +select * from forms where text_field like '%this%' or 0 = 1 or 0 = 1; +select * from forms where text_field like '%this%' or 0 = 1; +select * from forms where text_field like '%this%'; +-- { echoOff } From 06f3b82cc90787d18bbbc170b05bafa3d127052c Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Tue, 10 Oct 2023 19:56:22 +0000 Subject: [PATCH 108/634] remove moving/ dir if allow_remove_stale_ is off --- src/Storages/MergeTree/MergeTreePartsMover.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index f4dc6c8d042..e781dfa641f 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -229,10 +229,16 @@ MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const Me String relative_path = part->getDataPartStorage().getPartDirectory(); if (disk->exists(path_to_clone + relative_path)) { - throw Exception(ErrorCodes::DIRECTORY_ALREADY_EXISTS, - "Cannot clone part {} from '{}' to '{}': path '{}' already exists", - part->name, part->getDataPartStorage().getDiskName(), disk->getName(), + // If setting is on, we should've already cleaned moving/ dir on startup + if (data->allowRemoveStaleMovingParts()) + throw Exception(ErrorCodes::DIRECTORY_ALREADY_EXISTS, + "Cannot clone part {} from '{}' to '{}': path '{}' already exists", + part->name, part->getDataPartStorage().getDiskName(), disk->getName(), + fullPath(disk, path_to_clone + relative_path)); + + LOG_DEBUG(log, "Path {} already exists. Will remove it and clone again.", fullPath(disk, path_to_clone + relative_path)); + disk->removeRecursive(fs::path(path_to_clone) / relative_path / ""); } disk->createDirectories(path_to_clone); From 96e3a4f9b441628f1ee6b9a226d9b15c4f49ef8a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 10 Oct 2023 23:45:23 +0200 Subject: [PATCH 109/634] Update src/Functions/byteSwap.cpp Co-authored-by: Priyansh Agrawal --- src/Functions/byteSwap.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/byteSwap.cpp b/src/Functions/byteSwap.cpp index 2de6df4a4b3..f16fef2e54b 100644 --- a/src/Functions/byteSwap.cpp +++ b/src/Functions/byteSwap.cpp @@ -76,7 +76,7 @@ The above example can be worked out in the following manner: 2. Reverse the bytes, i.e. C7 C7 FB CD -> CD FB C7 C7 3. Convert the result back to an integer assuming big-endian, i.e. CD FB C7 C7 -> 3455829959 -One use-case of this function is reversing IPv5s: +One use-case of this function is reversing IPv4s: ```result ┌─toIPv4(byteSwap(toUInt32(toIPv4('205.251.199.199'))))─┐ From b7936cb953b4a8fe2cdbe0a69d9d3a603885ade6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 10 Oct 2023 23:45:57 +0200 Subject: [PATCH 110/634] Update src/Functions/byteSwap.cpp Co-authored-by: Priyansh Agrawal --- src/Functions/byteSwap.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/byteSwap.cpp b/src/Functions/byteSwap.cpp index f16fef2e54b..bdc6eb6c386 100644 --- a/src/Functions/byteSwap.cpp +++ b/src/Functions/byteSwap.cpp @@ -90,7 +90,7 @@ One use-case of this function is reversing IPv4s: {"32-bit", "SELECT byteSwap(3351772109)", "3455829959"}, {"64-bit", "SELECT byteSwap(123294967295)", "18439412204227788800"}, }, - .categories{"Mathematical"}}, + .categories{"Mathematical", "Arithmetic"}}, FunctionFactory::CaseInsensitive); } From bd43b84bf8a82c5a1bcdb3c7aa1f6e3fd47d5375 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 10 Oct 2023 21:44:08 +0000 Subject: [PATCH 111/634] Make use_mysql_types_in_show_columns affect only SHOW COLUMNS --- src/DataTypes/DataTypeAggregateFunction.h | 1 - src/DataTypes/DataTypeArray.h | 4 -- src/DataTypes/DataTypeDate.h | 1 - src/DataTypes/DataTypeDate32.h | 1 - src/DataTypes/DataTypeDateTime.h | 1 - src/DataTypes/DataTypeDateTime64.h | 1 - src/DataTypes/DataTypeEnum.cpp | 24 -------- src/DataTypes/DataTypeEnum.h | 2 - src/DataTypes/DataTypeFixedString.h | 2 - src/DataTypes/DataTypeFunction.h | 1 - src/DataTypes/DataTypeIPv4andIPv6.h | 2 - src/DataTypes/DataTypeInterval.h | 1 - src/DataTypes/DataTypeLowCardinality.h | 1 - src/DataTypes/DataTypeMap.h | 1 - src/DataTypes/DataTypeNothing.h | 1 - src/DataTypes/DataTypeNullable.h | 1 - src/DataTypes/DataTypeNumberBase.cpp | 28 --------- src/DataTypes/DataTypeNumberBase.h | 1 - src/DataTypes/DataTypeObject.h | 1 - src/DataTypes/DataTypeSet.h | 1 - src/DataTypes/DataTypeString.h | 2 - src/DataTypes/DataTypeTuple.h | 1 - src/DataTypes/DataTypeUUID.h | 1 - src/DataTypes/DataTypesDecimal.cpp | 12 ---- src/DataTypes/DataTypesDecimal.h | 1 - src/DataTypes/IDataType.h | 2 - src/Formats/SchemaInferenceUtils.cpp | 5 -- .../InterpreterShowColumnsQuery.cpp | 58 +++++++++++++++++-- src/Storages/System/StorageSystemColumns.cpp | 4 +- 29 files changed, 54 insertions(+), 108 deletions(-) diff --git a/src/DataTypes/DataTypeAggregateFunction.h b/src/DataTypes/DataTypeAggregateFunction.h index 6331c23222f..7d1bb355ccf 100644 --- a/src/DataTypes/DataTypeAggregateFunction.h +++ b/src/DataTypes/DataTypeAggregateFunction.h @@ -45,7 +45,6 @@ public: String doGetName() const override; String getNameWithoutVersion() const; const char * getFamilyName() const override { return "AggregateFunction"; } - String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; } Array getParameters() const { return parameters; } diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index 3a83babbc23..6a09b3b530d 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -35,10 +35,6 @@ public: { return "Array"; } - String getSQLCompatibleName() const override - { - return "TEXT"; - } bool canBeInsideNullable() const override { diff --git a/src/DataTypes/DataTypeDate.h b/src/DataTypes/DataTypeDate.h index 0d557cad5f0..2f17207cc07 100644 --- a/src/DataTypes/DataTypeDate.h +++ b/src/DataTypes/DataTypeDate.h @@ -13,7 +13,6 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Date; } const char * getFamilyName() const override { return family_name; } - String getSQLCompatibleName() const override { return "DATE"; } bool canBeUsedAsVersion() const override { return true; } bool canBeInsideNullable() const override { return true; } diff --git a/src/DataTypes/DataTypeDate32.h b/src/DataTypes/DataTypeDate32.h index 0879a404179..9160b62dc15 100644 --- a/src/DataTypes/DataTypeDate32.h +++ b/src/DataTypes/DataTypeDate32.h @@ -13,7 +13,6 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Date32; } const char * getFamilyName() const override { return family_name; } - String getSQLCompatibleName() const override { return "DATE"; } Field getDefault() const override { diff --git a/src/DataTypes/DataTypeDateTime.h b/src/DataTypes/DataTypeDateTime.h index a473aae1faf..a4a05917ba5 100644 --- a/src/DataTypes/DataTypeDateTime.h +++ b/src/DataTypes/DataTypeDateTime.h @@ -38,7 +38,6 @@ public: static constexpr auto family_name = "DateTime"; const char * getFamilyName() const override { return family_name; } - String getSQLCompatibleName() const override { return "DATETIME"; } String doGetName() const override; TypeIndex getTypeId() const override { return TypeIndex::DateTime; } diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index 7663518807f..64cedd798d1 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -28,7 +28,6 @@ public: DataTypeDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_info); const char * getFamilyName() const override { return family_name; } - String getSQLCompatibleName() const override { return "DATETIME"; } std::string doGetName() const override; TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index 1750ae785bf..e5efb73cfca 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -36,30 +36,6 @@ const char * DataTypeEnum::getFamilyName() const return EnumName::value; } -template -std::string DataTypeEnum::generateMySQLName(const Values & values) -{ - WriteBufferFromOwnString out; - - writeString("ENUM", out); - writeChar('(', out); - - auto first = true; - for (const auto & name_and_value : values) - { - if (!first) - writeString(", ", out); - - first = false; - - writeQuotedString(name_and_value.first, out); - } - - writeChar(')', out); - - return out.str(); -} - template std::string DataTypeEnum::generateName(const Values & values) { diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index d148f753c82..2f607fc2aa6 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -46,14 +46,12 @@ public: private: std::string type_name; static std::string generateName(const Values & values); - static std::string generateMySQLName(const Values & values); public: explicit DataTypeEnum(const Values & values_); std::string doGetName() const override { return type_name; } const char * getFamilyName() const override; - String getSQLCompatibleName() const override { return generateMySQLName(this->getValues()); } TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeFixedString.h b/src/DataTypes/DataTypeFixedString.h index 22ec793208d..8d114121c1a 100644 --- a/src/DataTypes/DataTypeFixedString.h +++ b/src/DataTypes/DataTypeFixedString.h @@ -42,8 +42,6 @@ public: TypeIndex getTypeId() const override { return type_id; } const char * getFamilyName() const override { return "FixedString"; } - /// Use TEXT for compatibility with MySQL to allow arbitrary bytes. - String getSQLCompatibleName() const override { return "TEXT"; } size_t getN() const { diff --git a/src/DataTypes/DataTypeFunction.h b/src/DataTypes/DataTypeFunction.h index 9acec676ce0..e8ce6871d8d 100644 --- a/src/DataTypes/DataTypeFunction.h +++ b/src/DataTypes/DataTypeFunction.h @@ -24,7 +24,6 @@ public: std::string doGetName() const override; const char * getFamilyName() const override { return "Function"; } - String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::Function; } const DataTypes & getArgumentTypes() const diff --git a/src/DataTypes/DataTypeIPv4andIPv6.h b/src/DataTypes/DataTypeIPv4andIPv6.h index 487ce04f67c..5aea55751a7 100644 --- a/src/DataTypes/DataTypeIPv4andIPv6.h +++ b/src/DataTypes/DataTypeIPv4andIPv6.h @@ -19,7 +19,6 @@ public: static constexpr auto type_id = TypeToTypeIndex; const char * getFamilyName() const override { return TypeName.data(); } - String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return type_id; } @@ -61,7 +60,6 @@ public: static constexpr auto type_id = TypeToTypeIndex; const char * getFamilyName() const override { return TypeName.data(); } - String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypeInterval.h b/src/DataTypes/DataTypeInterval.h index c398a54268e..b0e747555e3 100644 --- a/src/DataTypes/DataTypeInterval.h +++ b/src/DataTypes/DataTypeInterval.h @@ -27,7 +27,6 @@ public: SerializationPtr doGetDefaultSerialization() const override; std::string doGetName() const override { return fmt::format("Interval{}", kind.toString()); } const char * getFamilyName() const override { return "Interval"; } - String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::Interval; } bool equals(const IDataType & rhs) const override; diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index d2a414cb073..527f0475720 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -23,7 +23,6 @@ public: return "LowCardinality(" + dictionary_type->getName() + ")"; } const char * getFamilyName() const override { return "LowCardinality"; } - String getSQLCompatibleName() const override { return dictionary_type->getSQLCompatibleName(); } TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; } diff --git a/src/DataTypes/DataTypeMap.h b/src/DataTypes/DataTypeMap.h index 619815fdf20..257888a8e44 100644 --- a/src/DataTypes/DataTypeMap.h +++ b/src/DataTypes/DataTypeMap.h @@ -31,7 +31,6 @@ public: std::string doGetName() const override; std::string doGetPrettyName(size_t indent) const override; const char * getFamilyName() const override { return "Map"; } - String getSQLCompatibleName() const override { return "JSON"; } bool canBeInsideNullable() const override { return false; } diff --git a/src/DataTypes/DataTypeNothing.h b/src/DataTypes/DataTypeNothing.h index c3a7e2d09f0..3cafaecc7cc 100644 --- a/src/DataTypes/DataTypeNothing.h +++ b/src/DataTypes/DataTypeNothing.h @@ -16,7 +16,6 @@ public: static constexpr bool is_parametric = false; const char * getFamilyName() const override { return "Nothing"; } - String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::Nothing; } diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index e3165414c07..06d46fb15ed 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -16,7 +16,6 @@ public: explicit DataTypeNullable(const DataTypePtr & nested_data_type_); std::string doGetName() const override { return "Nullable(" + nested_data_type->getName() + ")"; } const char * getFamilyName() const override { return "Nullable"; } - String getSQLCompatibleName() const override { return nested_data_type->getSQLCompatibleName(); } TypeIndex getTypeId() const override { return TypeIndex::Nullable; } MutableColumnPtr createColumn() const override; diff --git a/src/DataTypes/DataTypeNumberBase.cpp b/src/DataTypes/DataTypeNumberBase.cpp index 4cefc4945c6..be448fe1491 100644 --- a/src/DataTypes/DataTypeNumberBase.cpp +++ b/src/DataTypes/DataTypeNumberBase.cpp @@ -11,34 +11,6 @@ Field DataTypeNumberBase::getDefault() const { return NearestFieldType(); } -template -String DataTypeNumberBase::getSQLCompatibleName() const -{ - if constexpr (std::is_same_v) - return "TINYINT"; - else if constexpr (std::is_same_v) - return "SMALLINT"; - else if constexpr (std::is_same_v) - return "INTEGER"; - else if constexpr (std::is_same_v) - return "BIGINT"; - else if constexpr (std::is_same_v) - return "TINYINT UNSIGNED"; - else if constexpr (std::is_same_v) - return "SMALLINT UNSIGNED"; - else if constexpr (std::is_same_v) - return "INTEGER UNSIGNED"; - else if constexpr (std::is_same_v) - return "BIGINT UNSIGNED"; - else if constexpr (std::is_same_v) - return "FLOAT"; - else if constexpr (std::is_same_v) - return "DOUBLE"; - /// Unsupported types are converted to TEXT - else - return "TEXT"; -} - template MutableColumnPtr DataTypeNumberBase::createColumn() const { diff --git a/src/DataTypes/DataTypeNumberBase.h b/src/DataTypes/DataTypeNumberBase.h index d902c62505e..3a5b11c5124 100644 --- a/src/DataTypes/DataTypeNumberBase.h +++ b/src/DataTypes/DataTypeNumberBase.h @@ -25,7 +25,6 @@ public: using ColumnType = ColumnVector; const char * getFamilyName() const override { return TypeName.data(); } - String getSQLCompatibleName() const override; TypeIndex getTypeId() const override { return TypeToTypeIndex; } Field getDefault() const override; diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 2e1e5398f7e..937a9091371 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -23,7 +23,6 @@ public: DataTypeObject(const String & schema_format_, bool is_nullable_); const char * getFamilyName() const override { return "Object"; } - String getSQLCompatibleName() const override { return "JSON"; } String doGetName() const override; TypeIndex getTypeId() const override { return TypeIndex::Object; } diff --git a/src/DataTypes/DataTypeSet.h b/src/DataTypes/DataTypeSet.h index e71a345a195..1c898de9c6d 100644 --- a/src/DataTypes/DataTypeSet.h +++ b/src/DataTypes/DataTypeSet.h @@ -15,7 +15,6 @@ class DataTypeSet final : public IDataTypeDummy public: static constexpr bool is_parametric = true; const char * getFamilyName() const override { return "Set"; } - String getSQLCompatibleName() const override { return "TEXT"; } TypeIndex getTypeId() const override { return TypeIndex::Set; } bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); } diff --git a/src/DataTypes/DataTypeString.h b/src/DataTypes/DataTypeString.h index c39fa90f6e7..5f3bde43a13 100644 --- a/src/DataTypes/DataTypeString.h +++ b/src/DataTypes/DataTypeString.h @@ -21,8 +21,6 @@ public: return "String"; } - String getSQLCompatibleName() const override { return "BLOB"; } - TypeIndex getTypeId() const override { return type_id; } MutableColumnPtr createColumn() const override; diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 0a1e78e23a0..db49b7f22d1 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -34,7 +34,6 @@ public: std::string doGetName() const override; std::string doGetPrettyName(size_t indent) const override; const char * getFamilyName() const override { return "Tuple"; } - String getSQLCompatibleName() const override { return "JSON"; } bool canBeInsideNullable() const override { return false; } bool supportsSparseSerialization() const override { return true; } diff --git a/src/DataTypes/DataTypeUUID.h b/src/DataTypes/DataTypeUUID.h index 8664c3bcfd1..90cdd90d68d 100644 --- a/src/DataTypes/DataTypeUUID.h +++ b/src/DataTypes/DataTypeUUID.h @@ -18,7 +18,6 @@ public: static constexpr auto type_id = TypeIndex::UUID; const char * getFamilyName() const override { return "UUID"; } - String getSQLCompatibleName() const override { return "CHAR"; } TypeIndex getTypeId() const override { return type_id; } diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index 49214f4aa45..7ad9f0b6fd8 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -28,18 +28,6 @@ std::string DataTypeDecimal::doGetName() const return fmt::format("Decimal({}, {})", this->precision, this->scale); } -template -std::string DataTypeDecimal::getSQLCompatibleName() const -{ - /// See https://dev.mysql.com/doc/refman/8.0/en/precision-math-decimal-characteristics.html - /// DECIMAL(M,D) - /// M is the maximum number of digits (the precision). It has a range of 1 to 65. - /// D is the number of digits to the right of the decimal point (the scale). It has a range of 0 to 30 and must be no larger than M. - if (this->precision > 65 || this->scale > 30) - return "TEXT"; - return fmt::format("DECIMAL({}, {})", this->precision, this->scale); -} - template bool DataTypeDecimal::equals(const IDataType & rhs) const { diff --git a/src/DataTypes/DataTypesDecimal.h b/src/DataTypes/DataTypesDecimal.h index 5e4cfab7928..e2b433cbe2f 100644 --- a/src/DataTypes/DataTypesDecimal.h +++ b/src/DataTypes/DataTypesDecimal.h @@ -39,7 +39,6 @@ public: static constexpr auto family_name = "Decimal"; const char * getFamilyName() const override { return family_name; } - String getSQLCompatibleName() const override; std::string doGetName() const override; TypeIndex getTypeId() const override { return TypeToTypeIndex; } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index c30b009c931..fb0a015821d 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -83,8 +83,6 @@ public: /// Name of data type family (example: FixedString, Array). virtual const char * getFamilyName() const = 0; - /// Name of corresponding data type in MySQL (exampe: Bigint, Blob, etc) - virtual String getSQLCompatibleName() const = 0; /// Data type id. It's used for runtime type checks. virtual TypeIndex getTypeId() const = 0; diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index d4d66ec76da..94166aa9002 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -83,11 +83,6 @@ namespace String doGetName() const override { return finalize()->getName(); } TypeIndex getTypeId() const override { return TypeIndex::JSONPaths; } - String getSQLCompatibleName() const override - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getSQLCompatibleName is not implemented for JSONObjectForInference type"); - } - bool isParametric() const override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method isParametric is not implemented for JSONObjectForInference type"); diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index e1f736ba4fb..0c78907ffee 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -24,6 +24,8 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() { const auto & query = query_ptr->as(); + [[maybe_unused]] const bool use_mysql_types = getContext()->getSettingsRef().use_mysql_types_in_show_columns; + WriteBufferFromOwnString buf_database; String resolved_database = getContext()->resolveDatabase(query.database); writeEscapedString(resolved_database, buf_database); @@ -33,18 +35,64 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() writeEscapedString(query.table, buf_table); String table = buf_table.str(); - String rewritten_query = R"( + String rewritten_query; + if (use_mysql_types) + /// Cheapskate mapping from native to MySQL types, see https://dev.mysql.com/doc/refman/8.0/en/data-types.html + /// Known issues: + /// - Enums are translated to TEXT + rewritten_query += R"( +WITH map( + 'Int8', 'TINYINT', + 'Int16', 'SMALLINT', + 'Int32', 'INTEGER', + 'Int64', 'BIGINT', + 'UInt8', 'TINYINT UNSIGNED', + 'UInt16', 'SMALLINT UNSIGNED', + 'UInt32', 'INTEGER UNSIGNED', + 'UInt64', 'BIGINT UNSIGNED', + 'Float32', 'FLOAT', + 'Float64', 'DOUBLE', + 'String', 'BLOB', + 'UUID', 'CHAR', + 'Bool', 'TINYINT', + 'Date', 'DATE', + 'Date32', 'DATE', + 'DateTime', 'DATETIME', + 'DateTime64', 'DATETIME', + 'Map', 'JSON', + 'Tuple', 'JSON', + 'Object', 'JSON') AS native_to_mysql_mapping, + splitByRegexp('\(|\)', type) AS split, + multiIf(startsWith(type, 'LowCardinality(Nullable'), split[3], + startsWith(type, 'LowCardinality'), split[2], + startsWith(type, 'Nullable'), split[2], + split[1]) AS inner_type, + if(length(split) > 1, splitByString(', ', split[2]), []) AS decimal_scale_and_precision, + multiIf(inner_type = 'Decimal' AND toInt8(decimal_scale_and_precision[1]) <= 65 AND toInt8(decimal_scale_and_precision[2]) <= 30, concat('DECIMAL(', decimal_scale_and_precision[1], ', ', decimal_scale_and_precision[2], ')'), + mapContains(native_to_mysql_mapping, inner_type) = true, native_to_mysql_mapping[inner_type], + 'TEXT') AS mysql_type + )"; + + rewritten_query += R"( SELECT name AS field, + )"; + + if (use_mysql_types) + rewritten_query += R"( + mysql_type AS type, + )"; + else + rewritten_query += R"( type AS type, - if (startsWith(type, 'Nullable'), 'YES', 'NO') AS `null`, + )"; + + rewritten_query += R"( + multiIf(startsWith(type, 'Nullable('), 'YES', startsWith(type, 'LowCardinality(Nullable('), 'YES', 'NO') AS `null`, trim(concatWithSeparator(' ', if (is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, if (default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, '' AS extra )"; - // Known issue: Field 'null' is wrong for types like 'LowCardinality(Nullable(String))'. Can't simply replace 'startsWith' by - // `hasSubsequence` as that would return `true` for non-nullable types such as `Tuple(Nullable(String), String)`... - // TODO Interpret query.extended. It is supposed to show internal/virtual columns. Need to fetch virtual column names, see // IStorage::getVirtuals(). We can't easily do that via SQL. diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index bf81ef1917d..ad65f6f5476 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -76,7 +76,6 @@ public: , columns_mask(std::move(columns_mask_)), max_block_size(max_block_size_) , databases(std::move(databases_)), tables(std::move(tables_)), storages(std::move(storages_)) , client_info_interface(context->getClientInfo().interface) - , use_mysql_types(context->getSettingsRef().use_mysql_types_in_show_columns) , total_tables(tables->size()), access(context->getAccess()) , query_id(context->getCurrentQueryId()), lock_acquire_timeout(context->getSettingsRef().lock_acquire_timeout) { @@ -149,7 +148,7 @@ protected: if (columns_mask[src_index++]) res_columns[res_index++]->insert(column.name); if (columns_mask[src_index++]) - res_columns[res_index++]->insert(use_mysql_types ? (column.type->getSQLCompatibleName()) : (column.type->getName())); + res_columns[res_index++]->insert(column.type->getName()); if (columns_mask[src_index++]) res_columns[res_index++]->insert(position); @@ -285,7 +284,6 @@ private: ColumnPtr tables; Storages storages; ClientInfo::Interface client_info_interface; - bool use_mysql_types; size_t db_table_num = 0; size_t total_tables; std::shared_ptr access; From 3df0e77ea3d5f03823f6f4e4b2a42296eb4afcb5 Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Wed, 11 Oct 2023 10:28:28 +0800 Subject: [PATCH 112/634] keeper_client add get_all_children_number command --- programs/keeper-client/Commands.cpp | 41 +++++++++++++++++++++++++ programs/keeper-client/Commands.h | 14 +++++++++ programs/keeper-client/KeeperClient.cpp | 2 ++ 3 files changed, 57 insertions(+) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 42057cf202c..68313ae0e54 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -475,4 +475,45 @@ void FourLetterWordCommand::execute(const ASTKeeperQuery * query, KeeperClient * std::cout << client->executeFourLetterCommand(query->args[0].safeGet()) << "\n"; } +bool GetAllChildrenNumberCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String path; + if (!parseKeeperPath(pos, expected, path)) + path = "."; + + node->args.push_back(std::move(path)); + + return true; +} + +void GetAllChildrenNumberCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + auto path = client->getAbsolutePath(query->args[0].safeGet()); + + std::queue queue; + queue.push(path); + Coordination::Stat stat; + client->zookeeper->get(path, &stat); + + int totalNumChildren = stat.numChildren; + while (!queue.empty()) + { + auto next_path = queue.front(); + queue.pop(); + + auto children = client->zookeeper->getChildren(next_path); + std::transform(children.cbegin(), children.cend(), children.begin(), [&](const String & child) { return next_path / child; }); + + auto response = client->zookeeper->get(children); + + for (size_t i = 0; i < response.size(); ++i) + { + totalNumChildren += response[i].stat.numChildren; + queue.push(children[i]); + } + } + + std::cout << totalNumChildren << "\n"; +} + } diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index a58a63e8a0a..d543dce6634 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -238,4 +238,18 @@ class FourLetterWordCommand : public IKeeperClientCommand String getHelpMessage() const override { return "{} -- Executes four-letter-word command"; } }; +class GetAllChildrenNumberCommand : public IKeeperClientCommand +{ + String getName() const override { return "get_all_children_number"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override + { + return "{} [path] -- Returns the total number of znode nodes for all children"; + } +}; + } diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 1f22a3cd644..f96975f5ab1 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -2,6 +2,7 @@ #include "Commands.h" #include #include +#include "Common/VersionNumber.h" #include #include #include @@ -206,6 +207,7 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) std::make_shared(), std::make_shared(), std::make_shared(), + std::make_shared(), }); String home_path; From ed89451efcf1e55070453e54aeafd59367fe4536 Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Wed, 11 Oct 2023 10:41:43 +0800 Subject: [PATCH 113/634] add get_all_children_number command docs in clickhouse-keeper-client.md --- docs/en/operations/utilities/clickhouse-keeper-client.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md index 2e84cd26071..d55163b375c 100644 --- a/docs/en/operations/utilities/clickhouse-keeper-client.md +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -55,6 +55,7 @@ keeper foo bar - `rmr ` -- Recursively deletes path. Confirmation required - `flwc ` -- Executes four-letter-word command - `help` -- Prints this message +- `get_all_children_number [path]` -- Returns the total number of znode nodes for all children - `get_stat [path]` -- Returns the node's stat (default `.`) - `find_super_nodes [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`) - `delete_stale_backups` -- Deletes ClickHouse nodes used for backups that are now inactive From 7d003fe5be1a5c43eee1828e6f03ac9c41838c5e Mon Sep 17 00:00:00 2001 From: Ryadh DAHIMENE Date: Wed, 11 Oct 2023 06:43:50 +0200 Subject: [PATCH 114/634] Update mysql.md Remove the Private Preview Note --- docs/en/interfaces/mysql.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/en/interfaces/mysql.md b/docs/en/interfaces/mysql.md index 32c612dfa5f..ce5ab24ecb0 100644 --- a/docs/en/interfaces/mysql.md +++ b/docs/en/interfaces/mysql.md @@ -10,10 +10,6 @@ ClickHouse supports the MySQL wire protocol. This allow tools that are MySQL-com ## Enabling the MySQL Interface On ClickHouse Cloud -:::note -The MySQL interface for ClickHouse Cloud is currently in private preview. Please contact support@clickhouse.com to enable this feature for your ClickHouse Cloud service. -::: - 1. After creating your ClickHouse Cloud Service, on the credentials screen, select the MySQL tab ![Credentials screen - Prompt](./images/mysql1.png) From 3bcd0b559d8eb1bd3e1d9aadbbbb5e886992adb0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Wed, 11 Oct 2023 16:32:03 +0800 Subject: [PATCH 115/634] fix tests --- .../02884_getHttpHeaderFunction.sh | 32 +++++++++---------- 1 file changed, 15 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh b/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh index 54e3f8715d7..cb1c22ea4b0 100755 --- a/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh +++ b/tests/queries/0_stateless/02884_getHttpHeaderFunction.sh @@ -15,11 +15,11 @@ echo "SELECT getHttpHeader('X-Clickhouse-User'), getHttpHeader('key1'), getHttpH echo "SELECT getHttpHeader('X-' || 'Clickhouse' || '-User'), getHttpHeader('key1'), getHttpHeader('key2')" | curl -s -H 'X-Clickhouse-User: default' \ -H 'X-ClickHouse-Key: ' -H 'key1: value1' -H 'key2: value2' 'http://localhost:8123/' -d @- -$CLICKHOUSE_CLIENT -q "CREATE DATABASE IF NOT EXISTS test" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02884_get_http_header" +db_name=db_$(echo $RANDOM |md5sum |cut -c 1-4) -$CLICKHOUSE_CLIENT -q " - CREATE TABLE IF NOT EXISTS test.02884_get_http_header +$CLICKHOUSE_CLIENT -q "CREATE DATABASE IF NOT EXISTS ${db_name};" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE ${db_name}.02884_get_http_header (id UInt32, http_user String DEFAULT getHttpHeader('X-Clickhouse-User'), http_key1 String DEFAULT getHttpHeader('http_header_key1'), @@ -34,7 +34,7 @@ $CLICKHOUSE_CLIENT -q " ORDER BY id" #Insert data via http request -echo "INSERT INTO test.02884_get_http_header (id) values (1)" | curl -s -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' \ +echo "INSERT INTO ${db_name}.02884_get_http_header (id) values (1)" | curl -s -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' \ -H 'http_header_key1: row1_value1'\ -H 'http_header_key2: row1_value2'\ -H 'http_header_key3: row1_value3'\ @@ -43,7 +43,7 @@ echo "INSERT INTO test.02884_get_http_header (id) values (1)" | curl -s -H 'X-Cl -H 'http_header_key6: row1_value6'\ -H 'http_header_key7: row1_value7' 'http://localhost:8123/' -d @- -echo "INSERT INTO test.02884_get_http_header (id) values (2)" | curl -s -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' \ +echo "INSERT INTO ${db_name}.02884_get_http_header (id) values (2)" | curl -s -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' \ -H 'http_header_key1: row2_value1'\ -H 'http_header_key2: row2_value2'\ -H 'http_header_key3: row2_value3'\ @@ -52,21 +52,19 @@ echo "INSERT INTO test.02884_get_http_header (id) values (2)" | curl -s -H 'X-Cl -H 'http_header_key6: row2_value6'\ -H 'http_header_key7: row2_value7' 'http://localhost:8123/' -d @- -$CLICKHOUSE_CLIENT -q "SELECT id, http_user, http_key1, http_key2, http_key3, http_key4, http_key5, http_key6, http_key7 FROM test.02884_get_http_header ORDER BY id;" +$CLICKHOUSE_CLIENT -q "SELECT id, http_user, http_key1, http_key2, http_key3, http_key4, http_key5, http_key6, http_key7 FROM ${db_name}.02884_get_http_header ORDER BY id;" #Insert data via tcp client -$CLICKHOUSE_CLIENT -q "INSERT INTO test.02884_get_http_header (id) values (3)" -$CLICKHOUSE_CLIENT -q "SELECT * FROM test.02884_get_http_header where id = 3" -echo "SELECT getHttpHeader('key_from_query_1'), getHttpHeader('key_from_query_2'), getHttpHeader('key_from_query_3'), * FROM test.02884_get_http_header ORDER BY id" | curl -s -H 'X-Clickhouse-User: default' \ +$CLICKHOUSE_CLIENT --param_db="$db_name" -q "INSERT INTO ${db_name}.02884_get_http_header (id) values (3)" +$CLICKHOUSE_CLIENT --param_db="$db_name" -q "SELECT * FROM ${db_name}.02884_get_http_header where id = 3" +echo "SELECT getHttpHeader('key_from_query_1'), getHttpHeader('key_from_query_2'), getHttpHeader('key_from_query_3'), * FROM ${db_name}.02884_get_http_header ORDER BY id" | curl -s -H 'X-Clickhouse-User: default' \ -H 'X-ClickHouse-Key: ' -H 'key_from_query_1: value_from_query_1' -H 'key_from_query_2: value_from_query_2' -H 'key_from_query_3: value_from_query_3' 'http://localhost:8123/' -d @- -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.02884_get_http_header" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ${db_name}.02884_get_http_header" -$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS test.02884_header_from_table (header_name String) Engine=Memory" -$CLICKHOUSE_CLIENT -q "INSERT INTO test.02884_header_from_table values ('X-Clickhouse-User'), ('http_key1'), ('http_key2')" +$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS ${db_name}.02884_header_from_table (header_name String) Engine=Memory" +$CLICKHOUSE_CLIENT -q "INSERT INTO ${db_name}.02884_header_from_table values ('X-Clickhouse-User'), ('http_key1'), ('http_key2')" -echo "SELECT getHttpHeader(header_name) as value from (select * FROM test.02884_header_from_table) order by value" | curl -s -H 'X-Clickhouse-User: default' \ +echo "SELECT getHttpHeader(header_name) as value from (select * FROM ${db_name}.02884_header_from_table) order by value" | curl -s -H 'X-Clickhouse-User: default' \ -H 'X-ClickHouse-Key: ' -H 'http_key1: http_value1' -H 'http_key2: http_value2' 'http://localhost:8123/' -d @- -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.02884_header_from_table" - - +$CLICKHOUSE_CLIENT -q "DROP DATABASE ${db_name}" From 18a6e624c9f05934e928141840995d7955b88508 Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Wed, 11 Oct 2023 17:24:08 +0800 Subject: [PATCH 116/634] fix description --- docs/en/operations/utilities/clickhouse-keeper-client.md | 2 +- programs/keeper-client/Commands.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md index d55163b375c..67aa4babb3f 100644 --- a/docs/en/operations/utilities/clickhouse-keeper-client.md +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -55,7 +55,7 @@ keeper foo bar - `rmr ` -- Recursively deletes path. Confirmation required - `flwc ` -- Executes four-letter-word command - `help` -- Prints this message -- `get_all_children_number [path]` -- Returns the total number of znode nodes for all children +- `get_all_children_number [path]` -- Get all numbers of children nodes under a specific path - `get_stat [path]` -- Returns the node's stat (default `.`) - `find_super_nodes [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`) - `delete_stale_backups` -- Deletes ClickHouse nodes used for backups that are now inactive diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index d543dce6634..45a7c85266d 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -248,7 +248,7 @@ class GetAllChildrenNumberCommand : public IKeeperClientCommand String getHelpMessage() const override { - return "{} [path] -- Returns the total number of znode nodes for all children"; + return "{} [path] -- Get all numbers of children nodes under a specific path"; } }; From 31ff669edc6120ea4ebb1b8853b78617afe57c9a Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Wed, 11 Oct 2023 17:47:38 +0800 Subject: [PATCH 117/634] change the code to for loop --- programs/keeper-client/Commands.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 68313ae0e54..c7043baa33c 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -324,8 +324,9 @@ void FindBigFamily::execute(const ASTKeeperQuery * query, KeeperClient * client) queue.pop(); auto children = client->zookeeper->getChildren(next_path); - std::transform(children.cbegin(), children.cend(), children.begin(), [&](const String & child) { return next_path / child; }); - + for (auto & child : children) + child = next_path / child; + auto response = client->zookeeper->get(children); for (size_t i = 0; i < response.size(); ++i) @@ -502,7 +503,8 @@ void GetAllChildrenNumberCommand::execute(const ASTKeeperQuery * query, KeeperCl queue.pop(); auto children = client->zookeeper->getChildren(next_path); - std::transform(children.cbegin(), children.cend(), children.begin(), [&](const String & child) { return next_path / child; }); + for (auto & child : children) + child = next_path / child; auto response = client->zookeeper->get(children); From 57dfe29f158cb8ad959ecfa4b3b8cbb86f8a92fe Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 5 May 2023 10:33:50 +0000 Subject: [PATCH 118/634] Fix 02267_join_dup_columns_issue36199 --- tests/analyzer_tech_debt.txt | 1 - .../02267_join_dup_columns_issue36199.reference | 7 +++++++ .../02267_join_dup_columns_issue36199.sql | 15 ++++++++++++++- 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 652ab0b99de..6581e93e3c2 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -77,7 +77,6 @@ 02139_MV_with_scalar_subquery 02174_cte_scalar_cache_mv 02242_join_rocksdb -02267_join_dup_columns_issue36199 02302_s3_file_pruning 02341_global_join_cte 02345_implicit_transaction diff --git a/tests/queries/0_stateless/02267_join_dup_columns_issue36199.reference b/tests/queries/0_stateless/02267_join_dup_columns_issue36199.reference index c075b08e533..3807180b735 100644 --- a/tests/queries/0_stateless/02267_join_dup_columns_issue36199.reference +++ b/tests/queries/0_stateless/02267_join_dup_columns_issue36199.reference @@ -12,3 +12,10 @@ y 0 0 1 3 0 0 1 0 y 0 0 1 3 \N 0 \N 10000000000 \N +2 1 1 2 +0 1 1 0 +0 1 1 0 +0 1 1 0 +0 0 0 0 1 1 0 +y 0 0 0 1 1 3 +\N 0 \N \N 10000000000 \N diff --git a/tests/queries/0_stateless/02267_join_dup_columns_issue36199.sql b/tests/queries/0_stateless/02267_join_dup_columns_issue36199.sql index b51b3cc22e2..fbcc374ba10 100644 --- a/tests/queries/0_stateless/02267_join_dup_columns_issue36199.sql +++ b/tests/queries/0_stateless/02267_join_dup_columns_issue36199.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 0; + SET join_algorithm = 'hash'; SELECT * FROM ( SELECT 2 AS x ) AS t1 RIGHT JOIN ( SELECT count('x'), count('y'), 2 AS x ) AS t2 ON t1.x = t2.x; @@ -15,7 +17,18 @@ SELECT * FROM ( SELECT 2 AS x ) AS t1 RIGHT JOIN ( SELECT count('x'), count('y') SELECT * FROM ( SELECT 2 AS x ) as t1 RIGHT JOIN ( SELECT count('x'), count('y'), 0 AS x ) AS t2 ON t1.x = t2.x; SELECT * FROM ( SELECT 2 AS x ) as t1 RIGHT JOIN ( SELECT count('x') :: Nullable(Int32), count('y'), 0 AS x ) AS t2 ON t1.x = t2.x; SELECT * FROM ( SELECT 2 AS x ) as t1 RIGHT JOIN ( SELECT count('x') :: Nullable(Int32), count('y') :: Nullable(Int32), 0 AS x ) AS t2 ON t1.x = t2.x; - +SELECT * FROM ( SELECT count('a'), count('b'), count('c'), 2 AS x ) as t1 RIGHT JOIN ( SELECT count('x'), count('y'), 0 AS x ) AS t2 ON t1.x = t2.x; + +SELECT 'y', * FROM (SELECT count('y'), count('y'), 2 AS x) AS t1 RIGHT JOIN (SELECT count('x'), count('y'), 3 AS x) AS t2 ON t1.x = t2.x; +SELECT * FROM (SELECT arrayJoin([NULL]), 9223372036854775806, arrayJoin([NULL]), NULL AS x) AS t1 RIGHT JOIN (SELECT arrayJoin([arrayJoin([10000000000.])]), NULL AS x) AS t2 ON t1.x = t2.x; + +SET allow_experimental_analyzer = 1; +SET join_algorithm = 'hash'; + +SELECT * FROM ( SELECT 2 AS x ) AS t1 RIGHT JOIN ( SELECT count('x'), count('y'), 2 AS x ) AS t2 ON t1.x = t2.x; +SELECT * FROM ( SELECT 2 AS x ) as t1 RIGHT JOIN ( SELECT count('x'), count('y'), 0 AS x ) AS t2 ON t1.x = t2.x; +SELECT * FROM ( SELECT 2 AS x ) as t1 RIGHT JOIN ( SELECT count('x') :: Nullable(Int32), count('y'), 0 AS x ) AS t2 ON t1.x = t2.x; +SELECT * FROM ( SELECT 2 AS x ) as t1 RIGHT JOIN ( SELECT count('x') :: Nullable(Int32), count('y') :: Nullable(Int32), 0 AS x ) AS t2 ON t1.x = t2.x; SELECT * FROM ( SELECT count('a'), count('b'), count('c'), 2 AS x ) as t1 RIGHT JOIN ( SELECT count('x'), count('y'), 0 AS x ) AS t2 ON t1.x = t2.x; SELECT 'y', * FROM (SELECT count('y'), count('y'), 2 AS x) AS t1 RIGHT JOIN (SELECT count('x'), count('y'), 3 AS x) AS t2 ON t1.x = t2.x; From 7a9c3ca98ab612b0939c6952a87a65ff5259a9cc Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 5 May 2023 10:52:35 +0000 Subject: [PATCH 119/634] Fix 01353_low_cardinality_join_types --- tests/analyzer_tech_debt.txt | 2 +- ...01353_low_cardinality_join_types.reference | 18 +++++++++ .../01353_low_cardinality_join_types.sql | 40 +++++++++++++++++++ 3 files changed, 59 insertions(+), 1 deletion(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 6581e93e3c2..2b25411ad2a 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -29,7 +29,7 @@ 01268_shard_avgweighted 01270_optimize_skip_unused_shards_low_cardinality 01319_optimize_skip_unused_shards_nesting -01353_low_cardinality_join_types +01428_nullable_asof_join 01455_shard_leaf_max_rows_bytes_to_read 01495_subqueries_in_with_statement 01504_rocksdb diff --git a/tests/queries/0_stateless/01353_low_cardinality_join_types.reference b/tests/queries/0_stateless/01353_low_cardinality_join_types.reference index 85d3f3d598b..6ef0887d978 100644 --- a/tests/queries/0_stateless/01353_low_cardinality_join_types.reference +++ b/tests/queries/0_stateless/01353_low_cardinality_join_types.reference @@ -34,3 +34,21 @@ UInt64 LowCardinality(UInt64) LowCardinality(String) String LowCardinality(UInt64) LowCardinality(UInt64) LowCardinality(String) LowCardinality(String) LowCardinality(UInt64) LowCardinality(UInt64) LowCardinality(String) LowCardinality(String) LowCardinality(UInt64) LowCardinality(UInt64) LowCardinality(String) LowCardinality(String) +- +UInt64 UInt64 String LowCardinality(String) +- +UInt64 UInt64 LowCardinality(String) String +- +LowCardinality(UInt64) LowCardinality(UInt64) LowCardinality(String) LowCardinality(String) +- +UInt64 UInt64 String LowCardinality(String) +UInt64 UInt64 String LowCardinality(String) +UInt64 UInt64 String LowCardinality(String) +- +UInt64 UInt64 LowCardinality(String) String +UInt64 UInt64 LowCardinality(String) String +UInt64 UInt64 LowCardinality(String) String +- +LowCardinality(UInt64) LowCardinality(UInt64) LowCardinality(String) LowCardinality(String) +LowCardinality(UInt64) LowCardinality(UInt64) LowCardinality(String) LowCardinality(String) +LowCardinality(UInt64) LowCardinality(UInt64) LowCardinality(String) LowCardinality(String) diff --git a/tests/queries/0_stateless/01353_low_cardinality_join_types.sql b/tests/queries/0_stateless/01353_low_cardinality_join_types.sql index 545b84d923d..93953f1d74a 100644 --- a/tests/queries/0_stateless/01353_low_cardinality_join_types.sql +++ b/tests/queries/0_stateless/01353_low_cardinality_join_types.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer = 0; set join_algorithm = 'hash'; select '-'; @@ -73,3 +74,42 @@ select toTypeName(materialize(js1.k)), toTypeName(materialize(js2.k)), toTypeNam from (select toLowCardinality(number) k, toLowCardinality(toString(number)) s from numbers(2)) as js1 full join (select toLowCardinality(number+1) k, toLowCardinality(toString(number+1)) s from numbers(2)) as js2 using k order by js1.k, js2.k; + +SET allow_experimental_analyzer = 1; +set join_algorithm = 'hash'; + +select '-'; +select toTypeName(materialize(js1.k)), toTypeName(materialize(js2.k)), toTypeName(materialize(js1.s)), toTypeName(materialize(js2.s)) +from (select toLowCardinality(number) k, toString(number) s from numbers(2)) as js1 +join (select number+1 k, toLowCardinality(toString(number+1)) s from numbers(2)) as js2 +using k order by js1.k, js2.k; + +select '-'; +select toTypeName(materialize(js1.k)), toTypeName(materialize(js2.k)), toTypeName(materialize(js1.s)), toTypeName(materialize(js2.s)) +from (select number k, toLowCardinality(toString(number)) s from numbers(2)) as js1 +join (select toLowCardinality(number+1) k, toString(number+1) s from numbers(2)) as js2 +using k order by js1.k, js2.k; + +select '-'; +select toTypeName(materialize(js1.k)), toTypeName(materialize(js2.k)), toTypeName(materialize(js1.s)), toTypeName(materialize(js2.s)) +from (select toLowCardinality(number) k, toLowCardinality(toString(number)) s from numbers(2)) as js1 +join (select toLowCardinality(number+1) k, toLowCardinality(toString(number+1)) s from numbers(2)) as js2 +using k order by js1.k, js2.k; + +select '-'; +select toTypeName(materialize(js1.k)), toTypeName(materialize(js2.k)), toTypeName(materialize(js1.s)), toTypeName(materialize(js2.s)) +from (select toLowCardinality(number) k, toString(number) s from numbers(2)) as js1 +full join (select number+1 k, toLowCardinality(toString(number+1)) s from numbers(2)) as js2 +using k order by js1.k, js2.k; + +select '-'; +select toTypeName(materialize(js1.k)), toTypeName(materialize(js2.k)), toTypeName(materialize(js1.s)), toTypeName(materialize(js2.s)) +from (select number k, toLowCardinality(toString(number)) s from numbers(2)) as js1 +full join (select toLowCardinality(number+1) k, toString(number+1) s from numbers(2)) as js2 +using k order by js1.k, js2.k; + +select '-'; +select toTypeName(materialize(js1.k)), toTypeName(materialize(js2.k)), toTypeName(materialize(js1.s)), toTypeName(materialize(js2.s)) +from (select toLowCardinality(number) k, toLowCardinality(toString(number)) s from numbers(2)) as js1 +full join (select toLowCardinality(number+1) k, toLowCardinality(toString(number+1)) s from numbers(2)) as js2 +using k order by js1.k, js2.k; From 1c70e0430137dd3ddf515f7a3d4d89a41510ca99 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 5 May 2023 15:05:57 +0000 Subject: [PATCH 120/634] Fix 02242_join_rocksdb --- src/Planner/PlannerJoins.cpp | 18 +++++++++++------- tests/analyzer_tech_debt.txt | 1 - .../queries/0_stateless/02242_join_rocksdb.sql | 2 +- 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index e87930a4d6b..2f7c08b25ba 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -584,13 +584,17 @@ std::shared_ptr tryDirectJoin(const std::shared_ptrgetTableExpressionDataOrThrow(right_table_expression); - const auto * table_column_name = right_table_expression_data.getColumnNameOrNull(key_name); - if (!table_column_name) - return {}; - const auto & storage_primary_key = storage->getPrimaryKey(); - if (storage_primary_key.size() != 1 || storage_primary_key[0] != *table_column_name) + if (const auto * table_column_name = right_table_expression_data.getColumnNameOrNull(key_name)) + { + const auto & storage_primary_key = storage->getPrimaryKey(); + if (storage_primary_key.size() != 1 || storage_primary_key[0] != *table_column_name) + return {}; + } + else + { return {}; + } /** For right table expression during execution columns have unique name. * Direct key value join implementation during storage querying must use storage column names. @@ -608,8 +612,8 @@ std::shared_ptr tryDirectJoin(const std::shared_ptr Date: Mon, 8 May 2023 14:43:46 +0000 Subject: [PATCH 121/634] [wip] Fixing 01049_join_low_card_bug_long --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 310 ++++----- .../01049_join_low_card_bug_long.reference | 615 ------------------ .../01049_join_low_card_bug_long.reference.j2 | 320 +++++++++ .../01049_join_low_card_bug_long.sql.j2 | 12 +- 4 files changed, 485 insertions(+), 772 deletions(-) delete mode 100644 tests/queries/0_stateless/01049_join_low_card_bug_long.reference create mode 100644 tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 7a405a2ef78..d0dd59d99d2 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1313,6 +1313,13 @@ private: IdentifierResolveScope & scope, IdentifierResolveSettings identifier_resolve_settings = {}); + QueryTreeNodePtr resolveIdentifierFromStorageOrThrow( + const Identifier & identifier, + const QueryTreeNodePtr & table_expression_node, + const TableExpressionData & table_expression_data, + IdentifierResolveScope & scope, + size_t identifier_column_qualifier_parts); + /// Resolve query tree nodes functions void qualifyColumnNodesWithProjectionNames(const QueryTreeNodes & column_nodes, @@ -2796,6 +2803,156 @@ bool QueryAnalyzer::tryBindIdentifierToTableExpressions(const IdentifierLookup & return can_bind_identifier_to_table_expression; } +QueryTreeNodePtr QueryAnalyzer::resolveIdentifierFromStorageOrThrow( + const Identifier & identifier, + const QueryTreeNodePtr & table_expression_node, + const TableExpressionData & table_expression_data, + IdentifierResolveScope & scope, + size_t identifier_column_qualifier_parts) +{ + auto identifier_without_column_qualifier = identifier; + identifier_without_column_qualifier.popFirst(identifier_column_qualifier_parts); + + /** Compound identifier cannot be resolved directly from storage if storage is not table. + * + * Example: SELECT test_table.id.value1.value2 FROM test_table; + * In table storage column test_table.id.value1.value2 will exists. + * + * Example: SELECT test_subquery.compound_expression.value FROM (SELECT compound_expression AS value) AS test_subquery; + * Here there is no column with name test_subquery.compound_expression.value, and additional wrap in tuple element is required. + */ + + QueryTreeNodePtr result_expression; + bool match_full_identifier = false; + + auto it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.getFullName()); + if (it != table_expression_data.column_name_to_column_node.end()) + { + match_full_identifier = true; + result_expression = it->second; + } + else + { + it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.at(0)); + if (it != table_expression_data.column_name_to_column_node.end()) + result_expression = it->second; + } + + bool clone_is_needed = true; + + String table_expression_source = table_expression_data.table_expression_description; + if (!table_expression_data.table_expression_name.empty()) + table_expression_source += " with name " + table_expression_data.table_expression_name; + + if (result_expression && !match_full_identifier && identifier_without_column_qualifier.isCompound()) + { + size_t identifier_bind_size = identifier_column_qualifier_parts + 1; + result_expression = tryResolveIdentifierFromCompoundExpression(identifier, + identifier_bind_size, + result_expression, + table_expression_source, + scope); + clone_is_needed = false; + } + + if (!result_expression) + { + QueryTreeNodes nested_column_nodes; + DataTypes nested_types; + Array nested_names_array; + + for (const auto & [column_name, _] : table_expression_data.column_names_and_types) + { + Identifier column_name_identifier_without_last_part(column_name); + auto column_name_identifier_last_part = column_name_identifier_without_last_part.getParts().back(); + column_name_identifier_without_last_part.popLast(); + + if (identifier_without_column_qualifier.getFullName() != column_name_identifier_without_last_part.getFullName()) + continue; + + auto column_node_it = table_expression_data.column_name_to_column_node.find(column_name); + if (column_node_it == table_expression_data.column_name_to_column_node.end()) + continue; + + const auto & column_node = column_node_it->second; + const auto & column_type = column_node->getColumnType(); + const auto * column_type_array = typeid_cast(column_type.get()); + if (!column_type_array) + continue; + + nested_column_nodes.push_back(column_node); + nested_types.push_back(column_type_array->getNestedType()); + nested_names_array.push_back(Field(std::move(column_name_identifier_last_part))); + } + + if (!nested_types.empty()) + { + auto nested_function_node = std::make_shared("nested"); + auto & nested_function_node_arguments = nested_function_node->getArguments().getNodes(); + + auto nested_function_names_array_type = std::make_shared(std::make_shared()); + auto nested_function_names_constant_node = std::make_shared(std::move(nested_names_array), + std::move(nested_function_names_array_type)); + nested_function_node_arguments.push_back(std::move(nested_function_names_constant_node)); + nested_function_node_arguments.insert(nested_function_node_arguments.end(), + nested_column_nodes.begin(), + nested_column_nodes.end()); + + auto nested_function = FunctionFactory::instance().get(nested_function_node->getFunctionName(), scope.context); + nested_function_node->resolveAsFunction(nested_function->build(nested_function_node->getArgumentColumns())); + + clone_is_needed = false; + result_expression = std::move(nested_function_node); + } + } + + if (!result_expression) + { + std::unordered_set valid_identifiers; + collectTableExpressionValidIdentifiersForTypoCorrection(identifier, + table_expression_node, + table_expression_data, + valid_identifiers); + + auto hints = collectIdentifierTypoHints(identifier, valid_identifiers); + + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Identifier '{}' cannot be resolved from {}. In scope {}{}", + identifier.getFullName(), + table_expression_source, + scope.scope_node->formatASTForErrorMessage(), + getHintsErrorMessageSuffix(hints)); + } + + if (clone_is_needed) + result_expression = result_expression->clone(); + + auto qualified_identifier = identifier; + + for (size_t i = 0; i < identifier_column_qualifier_parts; ++i) + { + auto qualified_identifier_with_removed_part = qualified_identifier; + qualified_identifier_with_removed_part.popFirst(); + + if (qualified_identifier_with_removed_part.empty()) + break; + + IdentifierLookup column_identifier_lookup = {qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION}; + if (tryBindIdentifierToAliases(column_identifier_lookup, scope)) + break; + + if (table_expression_data.should_qualify_columns && + tryBindIdentifierToTableExpressions(column_identifier_lookup, table_expression_node, scope)) + break; + + qualified_identifier = std::move(qualified_identifier_with_removed_part); + } + + auto qualified_identifier_full_name = qualified_identifier.getFullName(); + node_to_projection_name.emplace(result_expression, std::move(qualified_identifier_full_name)); + + return result_expression; +} + QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) @@ -2836,151 +2993,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id return {}; } - auto resolve_identifier_from_storage_or_throw = [&](size_t identifier_column_qualifier_parts) -> QueryTreeNodePtr - { - auto identifier_without_column_qualifier = identifier; - identifier_without_column_qualifier.popFirst(identifier_column_qualifier_parts); - - /** Compound identifier cannot be resolved directly from storage if storage is not table. - * - * Example: SELECT test_table.id.value1.value2 FROM test_table; - * In table storage column test_table.id.value1.value2 will exists. - * - * Example: SELECT test_subquery.compound_expression.value FROM (SELECT compound_expression AS value) AS test_subquery; - * Here there is no column with name test_subquery.compound_expression.value, and additional wrap in tuple element is required. - */ - - QueryTreeNodePtr result_expression; - bool match_full_identifier = false; - - auto it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.getFullName()); - if (it != table_expression_data.column_name_to_column_node.end()) - { - match_full_identifier = true; - result_expression = it->second; - } - else - { - it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.at(0)); - if (it != table_expression_data.column_name_to_column_node.end()) - result_expression = it->second; - } - - bool clone_is_needed = true; - - String table_expression_source = table_expression_data.table_expression_description; - if (!table_expression_data.table_expression_name.empty()) - table_expression_source += " with name " + table_expression_data.table_expression_name; - - if (result_expression && !match_full_identifier && identifier_without_column_qualifier.isCompound()) - { - size_t identifier_bind_size = identifier_column_qualifier_parts + 1; - result_expression = tryResolveIdentifierFromCompoundExpression(identifier_lookup.identifier, - identifier_bind_size, - result_expression, - table_expression_source, - scope); - clone_is_needed = false; - } - - if (!result_expression) - { - QueryTreeNodes nested_column_nodes; - DataTypes nested_types; - Array nested_names_array; - - for (auto & [column_name, _] : table_expression_data.column_names_and_types) - { - Identifier column_name_identifier_without_last_part(column_name); - auto column_name_identifier_last_part = column_name_identifier_without_last_part.getParts().back(); - column_name_identifier_without_last_part.popLast(); - - if (identifier_without_column_qualifier.getFullName() != column_name_identifier_without_last_part.getFullName()) - continue; - - auto column_node_it = table_expression_data.column_name_to_column_node.find(column_name); - if (column_node_it == table_expression_data.column_name_to_column_node.end()) - continue; - - const auto & column_node = column_node_it->second; - const auto & column_type = column_node->getColumnType(); - const auto * column_type_array = typeid_cast(column_type.get()); - if (!column_type_array) - continue; - - nested_column_nodes.push_back(column_node); - nested_types.push_back(column_type_array->getNestedType()); - nested_names_array.push_back(Field(std::move(column_name_identifier_last_part))); - } - - if (!nested_types.empty()) - { - auto nested_function_node = std::make_shared("nested"); - auto & nested_function_node_arguments = nested_function_node->getArguments().getNodes(); - - auto nested_function_names_array_type = std::make_shared(std::make_shared()); - auto nested_function_names_constant_node = std::make_shared(std::move(nested_names_array), - std::move(nested_function_names_array_type)); - nested_function_node_arguments.push_back(std::move(nested_function_names_constant_node)); - nested_function_node_arguments.insert(nested_function_node_arguments.end(), - nested_column_nodes.begin(), - nested_column_nodes.end()); - - auto nested_function = FunctionFactory::instance().get(nested_function_node->getFunctionName(), scope.context); - nested_function_node->resolveAsFunction(nested_function->build(nested_function_node->getArgumentColumns())); - - clone_is_needed = false; - result_expression = std::move(nested_function_node); - } - } - - if (!result_expression) - { - std::unordered_set valid_identifiers; - collectTableExpressionValidIdentifiersForTypoCorrection(identifier, - table_expression_node, - table_expression_data, - valid_identifiers); - - auto hints = collectIdentifierTypoHints(identifier, valid_identifiers); - - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Identifier '{}' cannot be resolved from {}. In scope {}{}", - identifier.getFullName(), - table_expression_source, - scope.scope_node->formatASTForErrorMessage(), - getHintsErrorMessageSuffix(hints)); - } - - if (clone_is_needed) - result_expression = result_expression->clone(); - - auto qualified_identifier = identifier; - - for (size_t i = 0; i < identifier_column_qualifier_parts; ++i) - { - auto qualified_identifier_with_removed_part = qualified_identifier; - qualified_identifier_with_removed_part.popFirst(); - - if (qualified_identifier_with_removed_part.empty()) - break; - - IdentifierLookup column_identifier_lookup = {qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION}; - if (tryBindIdentifierToAliases(column_identifier_lookup, scope)) - break; - - if (table_expression_data.should_qualify_columns && - tryBindIdentifierToTableExpressions(column_identifier_lookup, table_expression_node, scope)) - break; - - qualified_identifier = std::move(qualified_identifier_with_removed_part); - } - - auto qualified_identifier_full_name = qualified_identifier.getFullName(); - node_to_projection_name.emplace(result_expression, std::move(qualified_identifier_full_name)); - - return result_expression; - }; - /** If identifier first part binds to some column start or table has full identifier name. Then we can try to find whole identifier in table. * 1. Try to bind identifier first part to column in table, if true get full identifier from table or throw exception. * 2. Try to bind identifier first part to table name or storage alias, if true remove first part and try to get full identifier from table or throw exception. @@ -2988,24 +3000,24 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id * 3. Try to bind identifier first parts to database name and table name, if true remove first two parts and try to get full identifier from table or throw exception. */ if (table_expression_data.hasFullIdentifierName(IdentifierView(identifier))) - return resolve_identifier_from_storage_or_throw(0 /*identifier_column_qualifier_parts*/); + return resolveIdentifierFromStorageOrThrow(identifier, table_expression_node, table_expression_data, scope, 0 /*identifier_column_qualifier_parts*/); if (table_expression_data.canBindIdentifier(IdentifierView(identifier))) - return resolve_identifier_from_storage_or_throw(0 /*identifier_column_qualifier_parts*/); + return resolveIdentifierFromStorageOrThrow(identifier, table_expression_node, table_expression_data, scope, 0 /*identifier_column_qualifier_parts*/); if (identifier.getPartsSize() == 1) return {}; const auto & table_name = table_expression_data.table_name; if ((!table_name.empty() && path_start == table_name) || (table_expression_node->hasAlias() && path_start == table_expression_node->getAlias())) - return resolve_identifier_from_storage_or_throw(1 /*identifier_column_qualifier_parts*/); + return resolveIdentifierFromStorageOrThrow(identifier, table_expression_node, table_expression_data, scope, 1 /*identifier_column_qualifier_parts*/); if (identifier.getPartsSize() == 2) return {}; const auto & database_name = table_expression_data.database_name; if (!database_name.empty() && path_start == database_name && identifier[1] == table_name) - return resolve_identifier_from_storage_or_throw(2 /*identifier_column_qualifier_parts*/); + return resolveIdentifierFromStorageOrThrow(identifier, table_expression_node, table_expression_data, scope, 2 /*identifier_column_qualifier_parts*/); return {}; } diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference deleted file mode 100644 index 6587fab28d2..00000000000 --- a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference +++ /dev/null @@ -1,615 +0,0 @@ --- join_algorithm = default, join_use_nulls = 0 -- -str LowCardinality(String) - LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str LowCardinality(String) - LowCardinality(String) -str_l LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str_l LowCardinality(String) -LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) -LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) -LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) -LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l -LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) -LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l -str LowCardinality(String) - LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str LowCardinality(String) - LowCardinality(String) -str_l LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str_l LowCardinality(String) -String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) -String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) -String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) -String String LowCardinality(String) LowCardinality(String) str_l str_l -String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) -String String LowCardinality(String) LowCardinality(String) str_l str_l -str String - String -str String -str_r String -str String - String -str_l String -str String -str_r String -str_l String -LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String -LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String -LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String -LowCardinality(String) LowCardinality(String) String String str_l str_l -LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String -LowCardinality(String) LowCardinality(String) String String str_l str_l -str LowCardinality(String) - LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str LowCardinality(String) - LowCardinality(String) -str_l LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str_l LowCardinality(String) -Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) -Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) -Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) -Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l -Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) -Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l -str Nullable(String) -\N Nullable(String) -str Nullable(String) -str_r Nullable(String) -str Nullable(String) -\N Nullable(String) -str_l Nullable(String) -str Nullable(String) -str_l Nullable(String) -str_r Nullable(String) -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) str_l str_l -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) str_l str_l -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N --- join_algorithm = default, join_use_nulls = 1 -- -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -LowCardinality(String) LowCardinality(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -LowCardinality(String) LowCardinality(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -String String str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -String String str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -String String str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -String String str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -str Nullable(String) -\N Nullable(String) -str Nullable(String) -str_r Nullable(String) -str Nullable(String) -\N Nullable(String) -str_l Nullable(String) -str Nullable(String) -str_l Nullable(String) -str_r Nullable(String) -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -str Nullable(String) -\N Nullable(String) -str Nullable(String) -str_r Nullable(String) -str Nullable(String) -\N Nullable(String) -str_l Nullable(String) -str Nullable(String) -str_l Nullable(String) -str_r Nullable(String) -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N --- join_algorithm = partial_merge, join_use_nulls = 0 -- -str LowCardinality(String) - LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str LowCardinality(String) - LowCardinality(String) -str_l LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str_l LowCardinality(String) -LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) -LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) -LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) -LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l -LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) -LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l -str LowCardinality(String) - LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str LowCardinality(String) - LowCardinality(String) -str_l LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str_l LowCardinality(String) -String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) -String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) -String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) -String String LowCardinality(String) LowCardinality(String) str_l str_l -String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) -String String LowCardinality(String) LowCardinality(String) str_l str_l -str String - String -str String -str_r String -str String - String -str_l String -str String -str_r String -str_l String -LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String -LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String -LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String -LowCardinality(String) LowCardinality(String) String String str_l str_l -LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String -LowCardinality(String) LowCardinality(String) String String str_l str_l -str LowCardinality(String) - LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str LowCardinality(String) - LowCardinality(String) -str_l LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str_l LowCardinality(String) -Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) -Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) -Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) -Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l -Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) -Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l -str Nullable(String) -\N Nullable(String) -str Nullable(String) -str_r Nullable(String) -str Nullable(String) -\N Nullable(String) -str_l Nullable(String) -str Nullable(String) -str_l Nullable(String) -str_r Nullable(String) -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) str_l str_l -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) str_l str_l -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N --- join_algorithm = partial_merge, join_use_nulls = 1 -- -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -LowCardinality(String) LowCardinality(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -LowCardinality(String) LowCardinality(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -String String str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -String String str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -String String str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -String String str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -str Nullable(String) -\N Nullable(String) -str Nullable(String) -str_r Nullable(String) -str Nullable(String) -\N Nullable(String) -str_l Nullable(String) -str Nullable(String) -str_l Nullable(String) -str_r Nullable(String) -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -str Nullable(String) -\N Nullable(String) -str Nullable(String) -str_r Nullable(String) -str Nullable(String) -\N Nullable(String) -str_l Nullable(String) -str Nullable(String) -str_l Nullable(String) -str_r Nullable(String) -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N --- join_algorithm = parallel_hash, join_use_nulls = 0 -- -str LowCardinality(String) - LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str LowCardinality(String) - LowCardinality(String) -str_l LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str_l LowCardinality(String) -LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) -LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) -LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) -LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l -LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) -LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l -str LowCardinality(String) - LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str LowCardinality(String) - LowCardinality(String) -str_l LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str_l LowCardinality(String) -String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) -String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) -String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) -String String LowCardinality(String) LowCardinality(String) str_l str_l -String String str str LowCardinality(String) LowCardinality(String) str str -String String str_r str_r LowCardinality(String) LowCardinality(String) -String String LowCardinality(String) LowCardinality(String) str_l str_l -str String - String -str String -str_r String -str String - String -str_l String -str String -str_r String -str_l String -LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String -LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String -LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String -LowCardinality(String) LowCardinality(String) String String str_l str_l -LowCardinality(String) LowCardinality(String) str str String String str str -LowCardinality(String) LowCardinality(String) str_r str_r String String -LowCardinality(String) LowCardinality(String) String String str_l str_l -str LowCardinality(String) - LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str LowCardinality(String) - LowCardinality(String) -str_l LowCardinality(String) -str LowCardinality(String) -str_r LowCardinality(String) -str_l LowCardinality(String) -Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) -Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) -Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) -Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l -Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) -Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l -str Nullable(String) -\N Nullable(String) -str Nullable(String) -str_r Nullable(String) -str Nullable(String) -\N Nullable(String) -str_l Nullable(String) -str Nullable(String) -str_l Nullable(String) -str_r Nullable(String) -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) str_l str_l -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) str_l str_l -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N --- join_algorithm = parallel_hash, join_use_nulls = 1 -- -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -LowCardinality(String) LowCardinality(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -LowCardinality(String) LowCardinality(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -String String str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -String String str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -String String str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -String String str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -str Nullable(String) -\N Nullable(String) -str Nullable(String) -str_r Nullable(String) -str Nullable(String) -\N Nullable(String) -str_l Nullable(String) -str Nullable(String) -str_l Nullable(String) -str_r Nullable(String) -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -\N LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str LowCardinality(Nullable(String)) -str_l LowCardinality(Nullable(String)) -str_r LowCardinality(Nullable(String)) -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str -Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l -Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N -str Nullable(String) -\N Nullable(String) -str Nullable(String) -str_r Nullable(String) -str Nullable(String) -\N Nullable(String) -str_l Nullable(String) -str Nullable(String) -str_l Nullable(String) -str_r Nullable(String) -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str -LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l -LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N --- - LowCardinality(String) -str LowCardinality(String) - - -str -str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) -str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) -\N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 new file mode 100644 index 00000000000..879811f6764 --- /dev/null +++ b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 @@ -0,0 +1,320 @@ +-- { echoOn } +{% for join_algorithm in ['default', 'partial_merge', 'parallel_hash'] -%} +SET join_algorithm = '{{ join_algorithm }}'; +SET join_use_nulls = 0; +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, lc; +str LowCardinality(String) + LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, lc; +str LowCardinality(String) +str_r LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, lc; +str LowCardinality(String) + LowCardinality(String) +str_l LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, lc; +str LowCardinality(String) +str_r LowCardinality(String) +str_l LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l +-- + +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x, lc; +str LowCardinality(String) + LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x, lc; +str LowCardinality(String) +str_r LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x, lc; +str LowCardinality(String) + LowCardinality(String) +str_l LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x, lc; +str LowCardinality(String) +str_r LowCardinality(String) +str_l LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x, r.lc, l.lc; +String String str str LowCardinality(String) LowCardinality(String) str str +String String str_r str_r LowCardinality(String) LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x, r.lc, l.lc; +String String str str LowCardinality(String) LowCardinality(String) str str +String String str_r str_r LowCardinality(String) LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x, r.lc, l.lc; +String String str str LowCardinality(String) LowCardinality(String) str str +String String str_r str_r LowCardinality(String) LowCardinality(String) +String String LowCardinality(String) LowCardinality(String) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x, r.lc, l.lc; +String String str str LowCardinality(String) LowCardinality(String) str str +String String str_r str_r LowCardinality(String) LowCardinality(String) +String String LowCardinality(String) LowCardinality(String) str_l str_l +-- + +SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (x) ORDER BY x, lc; +str String + String +SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (lc) ORDER BY x, lc; +str String +str_r String +SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (x) ORDER BY x, lc; +str String + String +str_l String +SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (lc) ORDER BY x, lc; +str String +str_r String +str_l String +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str String String str str +LowCardinality(String) LowCardinality(String) str_r str_r String String +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str String String str str +LowCardinality(String) LowCardinality(String) str_r str_r String String +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str String String str str +LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) String String str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str String String str str +LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) String String str_l str_l +-- + +SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (x) ORDER BY x, lc; +str LowCardinality(String) + LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (lc) ORDER BY x, lc; +str LowCardinality(String) +str_r LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (x) ORDER BY x, lc; +str LowCardinality(String) + LowCardinality(String) +str_l LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (lc) ORDER BY x, lc; +str LowCardinality(String) +str_r LowCardinality(String) +str_l LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l +-- + +SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (x) ORDER BY x, lc; +str Nullable(String) +\N Nullable(String) +SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (lc) ORDER BY x, lc; +str Nullable(String) +str_r Nullable(String) +SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (x) ORDER BY x, lc; +str Nullable(String) +\N Nullable(String) +str_l Nullable(String) +SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (lc) ORDER BY x, lc; +str Nullable(String) +str_l Nullable(String) +str_r Nullable(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N +LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str +LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) str_l str_l +LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N +SET join_use_nulls = 1; +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +\N LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +str_r LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +\N LowCardinality(Nullable(String)) +str_l LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +str_l LowCardinality(Nullable(String)) +str_r LowCardinality(Nullable(String)) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +-- + +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +\N LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +str_r LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +\N LowCardinality(Nullable(String)) +str_l LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +str_l LowCardinality(Nullable(String)) +str_r LowCardinality(Nullable(String)) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x, r.lc, l.lc; +String String str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +String String str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x, r.lc, l.lc; +String String str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +String String str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l +Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +-- + +SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (x) ORDER BY x, lc; +str Nullable(String) +\N Nullable(String) +SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (lc) ORDER BY x, lc; +str Nullable(String) +str_r Nullable(String) +SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (x) ORDER BY x, lc; +str Nullable(String) +\N Nullable(String) +str_l Nullable(String) +SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (lc) ORDER BY x, lc; +str Nullable(String) +str_l Nullable(String) +str_r Nullable(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N +-- + +SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (x) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +\N LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (lc) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +str_r LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (x) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +\N LowCardinality(Nullable(String)) +str_l LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (lc) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +str_l LowCardinality(Nullable(String)) +str_r LowCardinality(Nullable(String)) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l +Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +-- + +SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (x) ORDER BY x, lc; +str Nullable(String) +\N Nullable(String) +SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (lc) ORDER BY x, lc; +str Nullable(String) +str_r Nullable(String) +SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (x) ORDER BY x, lc; +str Nullable(String) +\N Nullable(String) +str_l Nullable(String) +SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (lc) ORDER BY x, lc; +str Nullable(String) +str_l Nullable(String) +str_r Nullable(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N +{% endfor -%} +SELECT '--'; +-- +SET join_use_nulls = 0; +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY l.lc; + LowCardinality(String) +str LowCardinality(String) +SET join_algorithm = 'partial_merge'; +SET join_use_nulls = 1; +SELECT r.lc, materialize(r.lc), toTypeName(r.lc), toTypeName(materialize(r.lc)) FROM l_lc AS l FULL OUTER JOIN r_lc AS r USING (x) ORDER BY r.lc; +str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) +str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) +\N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) +DROP TABLE l; +DROP TABLE r; +DROP TABLE nl; +DROP TABLE nr; +DROP TABLE l_lc; +DROP TABLE r_lc; diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 b/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 index 9dd8f810b40..aa666079e5a 100644 --- a/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 +++ b/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 @@ -22,15 +22,13 @@ INSERT INTO l VALUES (0, 'str'), (2, 'str_l'); INSERT INTO nl VALUES (0, 'str'), (2, 'str_l'); INSERT INTO l_lc VALUES (0, 'str'), (2, 'str_l'); -{% for join_algorithm in [None, 'partial_merge', 'parallel_hash'] -%} +-- { echoOn } +{% for join_algorithm in ['default', 'partial_merge', 'parallel_hash'] -%} +SET join_algorithm = '{{ join_algorithm }}'; + {% for join_use_nulls in [0, 1] -%} - -SELECT '-- join_algorithm = {{ join_algorithm or 'default' }}, join_use_nulls = {{ join_use_nulls }} --'; - -{% if join_algorithm %}SET join_algorithm = '{{ join_algorithm }}';{% endif -%} SET join_use_nulls = {{ join_use_nulls }}; - SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, lc; SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, lc; SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, lc; @@ -98,8 +96,6 @@ SET join_use_nulls = 0; SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY l.lc; -SELECT lowCardinalityKeys(lc.lc) FROM r FULL JOIN l_lc as lc USING (lc) ORDER BY lowCardinalityKeys(lc.lc); - SET join_algorithm = 'partial_merge'; SET join_use_nulls = 1; From 3b36982bd8f1115a6261f803b3edcad7d9582e3f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 11 Oct 2023 13:10:55 +0200 Subject: [PATCH 122/634] Improve tests to avoid flakiness --- ...llel_replicas_automatic_decision.reference | 48 ++++++++++++------- ...84_parallel_replicas_automatic_decision.sh | 18 ++++--- ...replicas_automatic_decision_join.reference | 18 +++++-- ...rallel_replicas_automatic_decision_join.sh | 43 +++++++++++------ 4 files changed, 85 insertions(+), 42 deletions(-) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference index 10c63626c90..f78389c8d62 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.reference @@ -1,16 +1,32 @@ -02784_automatic_parallel_replicas-default_whole_table_10M_pure 0 -02784_automatic_parallel_replicas-default_whole_table_6M_pure 0 -02784_automatic_parallel_replicas-default_whole_table_5M_pure 2 -02784_automatic_parallel_replicas-default_whole_table_1M_pure 3 -02784_automatic_parallel_replicas-default_pruning_10M_pure 0 -02784_automatic_parallel_replicas-default_pruning_1M_pure 2 -02784_automatic_parallel_replicas-default_pruning_500k_pure 3 -02784_automatic_parallel_replicas-default_index_1M_pure 0 -02784_automatic_parallel_replicas-default_index_200k_pure 2 -02784_automatic_parallel_replicas-default_index_100k_pure 3 -02784_automatic_parallel_replicas-default_limit_10M_pure 0 -02784_automatic_parallel_replicas-default_limit_1M_pure 0 -02784_automatic_parallel_replicas-default_limit_500k_pure 2 -02784_automatic_parallel_replicas-default_useless_limit_500k_pure 3 -02784_automatic_parallel_replicas-default_helpless_filter_10M_pure 0 -02784_automatic_parallel_replicas-default_helpless_filter_5M_pure 2 +02784_automatic_parallel_replicas-default_whole_table_10M_pure 1 estimated parallel replicas +02784_automatic_parallel_replicas-default_whole_table_6M_pure 1 estimated parallel replicas +02784_automatic_parallel_replicas-default_whole_table_5M_pure 2 estimated parallel replicas +02784_automatic_parallel_replicas-default_whole_table_1M_pure 10 estimated parallel replicas +02784_automatic_parallel_replicas-default_pruning_10M_pure 0 estimated parallel replicas +02784_automatic_parallel_replicas-default_pruning_1M_pure 2 estimated parallel replicas +02784_automatic_parallel_replicas-default_pruning_500k_pure 4 estimated parallel replicas +02784_automatic_parallel_replicas-default_index_1M_pure 0 estimated parallel replicas +02784_automatic_parallel_replicas-default_index_200k_pure 2 estimated parallel replicas +02784_automatic_parallel_replicas-default_index_100k_pure 5 estimated parallel replicas +02784_automatic_parallel_replicas-default_limit_10M_pure 0 estimated parallel replicas +02784_automatic_parallel_replicas-default_limit_1M_pure 1 estimated parallel replicas +02784_automatic_parallel_replicas-default_limit_500k_pure 2 estimated parallel replicas +02784_automatic_parallel_replicas-default_useless_limit_500k_pure 20 estimated parallel replicas +02784_automatic_parallel_replicas-default_helpless_filter_10M_pure 1 estimated parallel replicas +02784_automatic_parallel_replicas-default_helpless_filter_5M_pure 2 estimated parallel replicas +02784_automatic_parallel_replicas-default_whole_table_10M_pure Used parallel replicas: false +02784_automatic_parallel_replicas-default_whole_table_6M_pure Used parallel replicas: false +02784_automatic_parallel_replicas-default_whole_table_5M_pure Used parallel replicas: true +02784_automatic_parallel_replicas-default_whole_table_1M_pure Used parallel replicas: true +02784_automatic_parallel_replicas-default_pruning_10M_pure Used parallel replicas: false +02784_automatic_parallel_replicas-default_pruning_1M_pure Used parallel replicas: true +02784_automatic_parallel_replicas-default_pruning_500k_pure Used parallel replicas: true +02784_automatic_parallel_replicas-default_index_1M_pure Used parallel replicas: false +02784_automatic_parallel_replicas-default_index_200k_pure Used parallel replicas: true +02784_automatic_parallel_replicas-default_index_100k_pure Used parallel replicas: true +02784_automatic_parallel_replicas-default_limit_10M_pure Used parallel replicas: false +02784_automatic_parallel_replicas-default_limit_1M_pure Used parallel replicas: false +02784_automatic_parallel_replicas-default_limit_500k_pure Used parallel replicas: true +02784_automatic_parallel_replicas-default_useless_limit_500k_pure Used parallel replicas: true +02784_automatic_parallel_replicas-default_helpless_filter_10M_pure Used parallel replicas: false +02784_automatic_parallel_replicas-default_helpless_filter_5M_pure Used parallel replicas: true diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 57613d3cc8b..5a0ef074112 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -4,12 +4,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -function involved_parallel_replicas () { +function were_parallel_replicas_used () { # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it $CLICKHOUSE_CLIENT --query " SELECT initial_query_id, - (count() - 2) / 2 as number_of_parallel_replicas + concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used FROM system.query_log WHERE event_date >= yesterday() AND initial_query_id LIKE '$1%' @@ -34,20 +34,26 @@ $CLICKHOUSE_CLIENT --query " SELECT number, 3 AS p FROM numbers(10_000_000, 8_000_000) " -function run_query_with_pure_parallel_replicas () { # $1 -> query_id # $2 -> min rows per replica # $3 -> query +function run_query_with_pure_parallel_replicas () { + # Note that we look into the logs to know how many parallel replicas were estimated because, although the coordinator + # might decide to use N replicas, one of them might be fast and do all the work before others start up. This means + # that those replicas wouldn't log into the system.query_log and the test would be flaky + $CLICKHOUSE_CLIENT \ --query "$3" \ --query_id "${1}_pure" \ --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ --use_hedged_requests 0 \ - --cluster_for_parallel_replicas 'parallel_replicas' \ + --cluster_for_parallel_replicas "parallel_replicas" \ --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ - --parallel_replicas_min_number_of_rows_per_replica "$2" -- + --parallel_replicas_min_number_of_rows_per_replica "$2" \ + --send_logs_level "trace" \ + |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' } query_id_base="02784_automatic_parallel_replicas-$CLICKHOUSE_DATABASE" @@ -87,6 +93,6 @@ run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_10M" 100 run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_5M" 5000000 "$helpless_filter_query" $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" -involved_parallel_replicas "${query_id_base}" +were_parallel_replicas_used "${query_id_base}" $CLICKHOUSE_CLIENT --query "DROP TABLE test_parallel_replicas_automatic_count" diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference index a4b89a975d1..802d55f8ae3 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference @@ -1,5 +1,13 @@ -02784_automatic_parallel_replicas_join-default_simple_join_0_pure 6 -02784_automatic_parallel_replicas_join-default_simple_join_10M_pure 0 -02784_automatic_parallel_replicas_join-default_simple_join_5M_pure 2 -02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 3 -02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 5 +02784_automatic_parallel_replicas_join-default_simple_join_10M_pure 0 estimated parallel replicas +02784_automatic_parallel_replicas_join-default_simple_join_10M_pure 1 estimated parallel replicas +02784_automatic_parallel_replicas_join-default_simple_join_5M_pure 0 estimated parallel replicas +02784_automatic_parallel_replicas_join-default_simple_join_5M_pure 2 estimated parallel replicas +02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 1 estimated parallel replicas +02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 10 estimated parallel replicas +02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 3 estimated parallel replicas +02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 33 estimated parallel replicas +02784_automatic_parallel_replicas_join-default_simple_join_0_pure Distinct parallel subqueries: 2 Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_10M_pure Distinct parallel subqueries: 0 Used parallel replicas: false +02784_automatic_parallel_replicas_join-default_simple_join_5M_pure Distinct parallel subqueries: 1 Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_1M_pure Distinct parallel subqueries: 1 Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_300k_pure Distinct parallel subqueries: 2 Used parallel replicas: true diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh index f5a534892c8..a2ed41bebe3 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh @@ -1,15 +1,18 @@ #!/usr/bin/env bash +## Note: The analyzer doesn't support JOIN with parallel replicas yet + CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -function involved_parallel_replicas () { +function were_parallel_replicas_used () { # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it $CLICKHOUSE_CLIENT --query " SELECT initial_query_id, - (count() - 2) / 2 as number_of_parallel_replicas + concat('Distinct parallel subqueries: ' , countDistinctIf(query, initial_query_id != query_id)::String) as subqueries_parallelized, + concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used FROM system.query_log WHERE event_date >= yesterday() AND initial_query_id LIKE '$1%' @@ -47,39 +50,49 @@ $CLICKHOUSE_CLIENT --query " SELECT number, number % 2 AS v FROM numbers(1_000_000) " -function run_query_with_pure_parallel_replicas () { # $1 -> query_id # $2 -> min rows per replica # $3 -> query +function run_query_with_pure_parallel_replicas () { + # Note that we look into the logs to know how many parallel replicas were estimated because, although the coordinator + # might decide to use N replicas, one of them might be fast and do all the work before others start up. This means + # that those replicas wouldn't log into the system.query_log and the test would be flaky + $CLICKHOUSE_CLIENT \ --query "$3" \ --query_id "${1}_pure" \ --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ --use_hedged_requests 0 \ - --cluster_for_parallel_replicas 'parallel_replicas' \ + --cluster_for_parallel_replicas "parallel_replicas" \ --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ - --parallel_replicas_min_number_of_rows_per_replica "$2" + --parallel_replicas_min_number_of_rows_per_replica "$2" \ + --send_logs_level "trace" \ + |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' } query_id_base="02784_automatic_parallel_replicas_join-$CLICKHOUSE_DATABASE" #### JOIN (left side 10M, right side 1M) -#### As the right side of the JOIN is a table, ideally it shouldn't be executed with parallel replicas and instead passed as is to the replicas -#### so each of them executes the join with the assigned granules of the left table, but that's not implemented yet -#### https://github.com/ClickHouse/ClickHouse/issues/49301#issuecomment-1619897920 -#### Note that this currently fails with the analyzer since it doesn't support JOIN with parallel replicas +#### As the right side of the JOIN is a table and not a subquery, ideally the right side should be left untouched and +#### pushed down into each replica. This isn't implemented yet and the right side of the join is being transformed into +#### a subquery, which then is executed in parallel (https://github.com/ClickHouse/ClickHouse/issues/49301#issuecomment-1619897920) +#### This is why when we print estimation it happens twice, once for each side of the join simple_join_query="SELECT sum(value) FROM test_parallel_replicas_automatic_left_side INNER JOIN test_parallel_replicas_automatic_count_right_side USING number format Null" -run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_0" 0 "$simple_join_query" # 3 replicas for the right side first, 3 replicas for the left -run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_10M" 10000000 "$simple_join_query" # Right: 0. Left: 0 + +# With 0 rows we won't have any estimation (no logs either). Both queries will be executed in parallel +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_0" 0 "$simple_join_query" + +# Once a limit is set we get estimation. One message for each part of the join (see message above) +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_10M" 10000000 "$simple_join_query" # Right: 0. Left: 1->0 run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_5M" 5000000 "$simple_join_query" # Right: 0. Left: 2 run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_1M" 1000000 "$simple_join_query" # Right: 1->0. Left: 10->3 -run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_300k" 400000 "$simple_join_query" # Right: 2. Left: 3 +run_query_with_pure_parallel_replicas "${query_id_base}_simple_join_300k" 300000 "$simple_join_query" # Right: 2. Left: 33->3 $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" -involved_parallel_replicas "${query_id_base}" +were_parallel_replicas_used "${query_id_base}" -$CLICKHOUSE_CLIENT --query "DROP TABLE test_parallel_replicas_automatic_left_side" -$CLICKHOUSE_CLIENT --query "DROP TABLE test_parallel_replicas_automatic_count_right_side" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_parallel_replicas_automatic_left_side" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_parallel_replicas_automatic_count_right_side" From b63471130322ddace340be22c67dacd184f38186 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 11 Oct 2023 13:14:01 +0200 Subject: [PATCH 123/634] Update src/Storages/MergeTree/MergeTreePartsMover.cpp --- src/Storages/MergeTree/MergeTreePartsMover.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index e781dfa641f..c4defc28ff6 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -236,7 +236,7 @@ MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const Me part->name, part->getDataPartStorage().getDiskName(), disk->getName(), fullPath(disk, path_to_clone + relative_path)); - LOG_DEBUG(log, "Path {} already exists. Will remove it and clone again.", + LOG_WARNING(log, "Path {} already exists. Will remove it and clone again", fullPath(disk, path_to_clone + relative_path)); disk->removeRecursive(fs::path(path_to_clone) / relative_path / ""); } From d3f9145351fd443d166c0ea20561fac3849e4952 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 11 Oct 2023 13:17:54 +0200 Subject: [PATCH 124/634] Update src/Storages/MergeTree/MergeTreePartsMover.cpp --- src/Storages/MergeTree/MergeTreePartsMover.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index c4defc28ff6..0a3416a2700 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -236,7 +236,7 @@ MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const Me part->name, part->getDataPartStorage().getDiskName(), disk->getName(), fullPath(disk, path_to_clone + relative_path)); - LOG_WARNING(log, "Path {} already exists. Will remove it and clone again", + LOG_DEBUG(log, "Path {} already exists. Will remove it and clone again", fullPath(disk, path_to_clone + relative_path)); disk->removeRecursive(fs::path(path_to_clone) / relative_path / ""); } From 0859fc8de9e553be37cbfcd53def1ccad79e0c0e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 11 Oct 2023 14:02:35 +0300 Subject: [PATCH 125/634] AggregatingTransform initGenerate race condition fix --- src/Processors/Transforms/AggregatingTransform.cpp | 9 +++++++-- src/Processors/Transforms/AggregatingTransform.h | 2 +- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index b4d2785bed2..9546d396523 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -623,7 +623,9 @@ IProcessor::Status AggregatingTransform::prepare() void AggregatingTransform::work() { if (is_consume_finished) + { initGenerate(); + } else { consume(std::move(current_chunk)); @@ -676,11 +678,14 @@ void AggregatingTransform::consume(Chunk chunk) void AggregatingTransform::initGenerate() { - if (is_generate_initialized) + if (is_generate_initialized.load(std::memory_order_acquire)) return; std::lock_guard lock(snapshot_mutex); - is_generate_initialized = true; + if (is_generate_initialized.load(std::memory_order_relaxed)) + return; + + is_generate_initialized.store(true, std::memory_order_release); /// If there was no data, and we aggregate without keys, and we must return single row with the result of empty aggregation. /// To do this, we pass a block with zero rows to aggregate. diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 791cd12326f..7b13b1a34f6 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -217,7 +217,7 @@ private: UInt64 src_rows = 0; UInt64 src_bytes = 0; - bool is_generate_initialized = false; + std::atomic is_generate_initialized = false; bool is_consume_finished = false; bool is_pipeline_created = false; From 380ba7c9e510d9ac3ceb06a81e41c994cb5f5df5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 11 Oct 2023 12:11:44 +0000 Subject: [PATCH 126/634] Rewrite to avoid CYCLIC_ALIAS errors --- .../InterpreterShowColumnsQuery.cpp | 43 +++++++++++-------- 1 file changed, 25 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index 0c78907ffee..4c4b21c1a67 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -37,7 +37,7 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() String rewritten_query; if (use_mysql_types) - /// Cheapskate mapping from native to MySQL types, see https://dev.mysql.com/doc/refman/8.0/en/data-types.html + /// Cheapskate SQL-based mapping from native types to MySQL types, see https://dev.mysql.com/doc/refman/8.0/en/data-types.html /// Known issues: /// - Enums are translated to TEXT rewritten_query += R"( @@ -62,10 +62,10 @@ WITH map( 'Map', 'JSON', 'Tuple', 'JSON', 'Object', 'JSON') AS native_to_mysql_mapping, - splitByRegexp('\(|\)', type) AS split, - multiIf(startsWith(type, 'LowCardinality(Nullable'), split[3], - startsWith(type, 'LowCardinality'), split[2], - startsWith(type, 'Nullable'), split[2], + splitByRegexp('\(|\)', type_) AS split, + multiIf(startsWith(type_, 'LowCardinality(Nullable'), split[3], + startsWith(type_, 'LowCardinality'), split[2], + startsWith(type_, 'Nullable'), split[2], split[1]) AS inner_type, if(length(split) > 1, splitByString(', ', split[2]), []) AS decimal_scale_and_precision, multiIf(inner_type = 'Decimal' AND toInt8(decimal_scale_and_precision[1]) <= 65 AND toInt8(decimal_scale_and_precision[2]) <= 30, concat('DECIMAL(', decimal_scale_and_precision[1], ', ', decimal_scale_and_precision[2], ')'), @@ -75,7 +75,7 @@ WITH map( rewritten_query += R"( SELECT - name AS field, + name_ AS field, )"; if (use_mysql_types) @@ -84,13 +84,13 @@ SELECT )"; else rewritten_query += R"( - type AS type, + type_ AS type, )"; rewritten_query += R"( - multiIf(startsWith(type, 'Nullable('), 'YES', startsWith(type, 'LowCardinality(Nullable('), 'YES', 'NO') AS `null`, - trim(concatWithSeparator(' ', if (is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, - if (default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, + multiIf(startsWith(type_, 'Nullable('), 'YES', startsWith(type_, 'LowCardinality(Nullable('), 'YES', 'NO') AS `null`, + trim(concatWithSeparator(' ', if (is_in_primary_key_, 'PRI', ''), if (is_in_sorting_key_, 'SOR', ''))) AS key, + if (default_kind_ IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression_, NULL) AS default, '' AS extra )"; // TODO Interpret query.extended. It is supposed to show internal/virtual columns. Need to fetch virtual column names, see @@ -104,19 +104,29 @@ SELECT /// - privileges: rewritten_query += R"(, NULL AS collation, - comment, + comment_ AS comment, '' AS privileges )"; } rewritten_query += fmt::format(R"( -FROM system.columns +-- need to rename columns of the base table to avoid "CYCLIC_ALIASES" errors +FROM (SELECT name AS name_, + database AS database_, + table AS table_, + type AS type_, + is_in_primary_key AS is_in_primary_key_, + is_in_sorting_key AS is_in_sorting_key_, + default_kind AS default_kind_, + default_expression AS default_expression_, + comment AS comment_ + FROM system.columns) WHERE - database = '{}' - AND table = '{}' )", database, table); + database_ = '{}' + AND table_ = '{}' )", database, table); if (!query.like.empty()) { - rewritten_query += " AND name "; + rewritten_query += " AND field "; if (query.not_like) rewritten_query += "NOT "; if (query.case_insensitive_like) @@ -128,9 +138,6 @@ WHERE else if (query.where_expression) rewritten_query += fmt::format(" AND ({})", query.where_expression); - /// Sorting is strictly speaking not necessary but 1. it is convenient for users, 2. SQL currently does not allow to - /// sort the output of SHOW COLUMNS otherwise (SELECT * FROM (SHOW COLUMNS ...) ORDER BY ...) is rejected) and 3. some - /// SQL tests can take advantage of this. rewritten_query += " ORDER BY field, type, null, key, default, extra"; if (query.limit_length) From e6f5fb2ffe3968faa23060ccec1c873541858c8f Mon Sep 17 00:00:00 2001 From: Daniel Byta Date: Wed, 11 Oct 2023 15:44:00 +0300 Subject: [PATCH 127/634] Update zh client-libraries.md --- docs/zh/interfaces/third-party/client-libraries.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/zh/interfaces/third-party/client-libraries.md b/docs/zh/interfaces/third-party/client-libraries.md index 4ee74f1d2c3..ed7825486be 100644 --- a/docs/zh/interfaces/third-party/client-libraries.md +++ b/docs/zh/interfaces/third-party/client-libraries.md @@ -39,6 +39,8 @@ Yandex**没有**维护下面列出的库,也没有做过任何广泛的测试 - [nestjs-clickhouse](https://github.com/depyronick/nestjs-clickhouse) - [clickhouse-client](https://github.com/depyronick/clickhouse-client) - [node-clickhouse-orm](https://github.com/zimv/node-clickhouse-orm) + - [clickhouse-ts](https://github.com/bytadaniel/clickhouse-ts) + - [clickcache](https://github.com/bytadaniel/clickcache) - Perl - [perl-DBD-ClickHouse](https://github.com/elcamlost/perl-DBD-ClickHouse) - [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse) From f7e64ff20098955da9edc22168a3861bfc6a1e90 Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Wed, 11 Oct 2023 20:45:38 +0800 Subject: [PATCH 128/634] add test_keeper_client --- tests/integration/helpers/keeper_utils.py | 3 +++ tests/integration/test_keeper_client/test.py | 21 ++++++++++++++++++++ 2 files changed, 24 insertions(+) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 3a88688d110..9135ab72c84 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -145,6 +145,9 @@ class KeeperClient(object): def delete_stale_backups(self, timeout: float = 60.0) -> str: return self.execute_query("delete_stale_backups", timeout) + + def get_all_children_number(self, path: str, timeout: float = 60.0) -> str: + return self.execute_query(f"get_all_children_number {path}", timeout) def reconfig( self, diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 77217dd2dcf..08d0825c118 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -216,3 +216,24 @@ def test_quoted_argument_parsing(client: KeeperClient): client.execute_query(f"set '{node_path}' \"value4 with some whitespace\" 3") assert client.get(node_path) == "value4 with some whitespace" + +def test_get_all_children_number(client: KeeperClient): + client.touch("/test_get_all_children_number") + client.touch("/test_get_all_children_number/1") + client.touch("/test_get_all_children_number/1/1") + client.touch("/test_get_all_children_number/1/2") + client.touch("/test_get_all_children_number/1/3") + client.touch("/test_get_all_children_number/1/4") + client.touch("/test_get_all_children_number/1/5") + client.touch("/test_get_all_children_number/2") + client.touch("/test_get_all_children_number/2/1") + client.touch("/test_get_all_children_number/2/2") + client.touch("/test_get_all_children_number/2/3") + client.touch("/test_get_all_children_number/2/4") + + response = client.get_all_children_number("/test_get_all_children_number") + assert response == TSV( + [ + ["11"] + ] + ) From 5e86362abeb7c988479b6a8e59d306b8c5397fcc Mon Sep 17 00:00:00 2001 From: Daniel Byta Date: Wed, 11 Oct 2023 15:46:42 +0300 Subject: [PATCH 129/634] Update client-libraries.md --- docs/en/interfaces/third-party/client-libraries.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/interfaces/third-party/client-libraries.md b/docs/en/interfaces/third-party/client-libraries.md index e229198bdfd..7b5c4f27a2a 100644 --- a/docs/en/interfaces/third-party/client-libraries.md +++ b/docs/en/interfaces/third-party/client-libraries.md @@ -44,6 +44,8 @@ ClickHouse Inc does **not** maintain the libraries listed below and hasn’t don - [nestjs-clickhouse](https://github.com/depyronick/nestjs-clickhouse) - [clickhouse-client](https://github.com/depyronick/clickhouse-client) - [node-clickhouse-orm](https://github.com/zimv/node-clickhouse-orm) + - [clickhouse-ts](https://github.com/bytadaniel/clickhouse-ts) + - [clickcache](https://github.com/bytadaniel/clickcache) ### Perl - [perl-DBD-ClickHouse](https://github.com/elcamlost/perl-DBD-ClickHouse) - [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse) From cd0c15759bce8d79f397ed3d72d76705827d1906 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 11 Oct 2023 10:10:19 -0300 Subject: [PATCH 130/634] fix tests --- .../configs/config.d/proxy_remote.xml | 14 ++++++---- .../configs/config.d/proxy_remote.xml | 28 +++++++++++-------- .../test.py | 1 + 3 files changed, 25 insertions(+), 18 deletions(-) diff --git a/tests/integration/test_s3_table_function_with_http_proxy/configs/config.d/proxy_remote.xml b/tests/integration/test_s3_table_function_with_http_proxy/configs/config.d/proxy_remote.xml index 30d99605458..72546d49111 100644 --- a/tests/integration/test_s3_table_function_with_http_proxy/configs/config.d/proxy_remote.xml +++ b/tests/integration/test_s3_table_function_with_http_proxy/configs/config.d/proxy_remote.xml @@ -5,11 +5,13 @@ Proxy host is returned as string in response body. Then S3 client uses proxy URL formed as proxy_scheme://proxy_host:proxy_port to make request. --> - - http://resolver:8080/hostname - http - 80 - 10 - + + + http://resolver:8080/hostname + http + 80 + 10 + + diff --git a/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote.xml b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote.xml index c0f5975224d..f8fc4294b0c 100644 --- a/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote.xml +++ b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote.xml @@ -5,18 +5,22 @@ Proxy host is returned as string in response body. Then S3 client uses proxy URL formed as proxy_scheme://proxy_host:proxy_port to make request. --> - - http://resolver:8080/hostname - http - 80 - 10 - + + + http://resolver:8080/hostname + http + 80 + 10 + + - - http://resolver:8080/hostname - https - 443 - 10 - + + + http://resolver:8080/hostname + https + 443 + 10 + + diff --git a/tests/integration/test_s3_table_function_with_https_proxy/test.py b/tests/integration/test_s3_table_function_with_https_proxy/test.py index a498410a4d4..83af407093c 100644 --- a/tests/integration/test_s3_table_function_with_https_proxy/test.py +++ b/tests/integration/test_s3_table_function_with_https_proxy/test.py @@ -41,6 +41,7 @@ def cluster(): env_variables={ "https_proxy": "https://proxy1", }, + instance_env_variables=True, ) logging.info("Starting cluster...") From 0baee0c612a5c0d692240be377b64bd3d29287fc Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 11 Oct 2023 10:52:31 -0300 Subject: [PATCH 131/634] remove useless http resolver --- .../configs/config.d/proxy_remote.xml | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote.xml b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote.xml index f8fc4294b0c..a310ea999d1 100644 --- a/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote.xml +++ b/tests/integration/test_s3_table_function_with_https_proxy/configs/config.d/proxy_remote.xml @@ -1,19 +1,5 @@ - - - - http://resolver:8080/hostname - http - 80 - 10 - - - http://resolver:8080/hostname From 03d139764c7a1f7507227a40cafc68928570e810 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 11 Oct 2023 15:54:42 +0200 Subject: [PATCH 132/634] fix lots of 'Structure does not match' warnings in ci --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 0d6c0e368db..30a6dd2e9fa 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -15,7 +15,7 @@ CLICKHOUSE_CI_LOGS_USER=${CLICKHOUSE_CI_LOGS_USER:-ci} # Pre-configured destination cluster, where to export the data CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export} -EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime, check_name LowCardinality(String), instance_type LowCardinality(String), instance_id String, "} +EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime, check_name String, instance_type String, instance_id String, "} EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"0 AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type, '' AS instance_id"} EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "} From 9741e7b17584b7af6ec1970186ea86a5ffdaa4c6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 11 Oct 2023 14:32:42 +0000 Subject: [PATCH 133/634] Fix style --- src/Interpreters/InterpreterShowColumnsQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index 4c4b21c1a67..c5fbaf55e4a 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -67,7 +67,7 @@ WITH map( startsWith(type_, 'LowCardinality'), split[2], startsWith(type_, 'Nullable'), split[2], split[1]) AS inner_type, - if(length(split) > 1, splitByString(', ', split[2]), []) AS decimal_scale_and_precision, + if (length(split) > 1, splitByString(', ', split[2]), []) AS decimal_scale_and_precision, multiIf(inner_type = 'Decimal' AND toInt8(decimal_scale_and_precision[1]) <= 65 AND toInt8(decimal_scale_and_precision[2]) <= 30, concat('DECIMAL(', decimal_scale_and_precision[1], ', ', decimal_scale_and_precision[2], ')'), mapContains(native_to_mysql_mapping, inner_type) = true, native_to_mysql_mapping[inner_type], 'TEXT') AS mysql_type From bb637fa9af8e7166107f3d6c7e789b065283fe57 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 11 Oct 2023 14:34:28 +0000 Subject: [PATCH 134/634] Update docs --- docs/en/operations/settings/settings.md | 2 +- docs/en/operations/system-tables/columns.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f703429cb70..2cb85a61be5 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3281,7 +3281,7 @@ Default value: `0`. ## use_mysql_types_in_show_columns {#use_mysql_types_in_show_columns} -Show the names of MySQL data types corresponding to ClickHouse data types in [SHOW COLUMNS](../../sql-reference/statements/show.md#show_columns) and SELECTs on [system.columns](../system-tables/columns.md). +Show the names of MySQL data types corresponding to ClickHouse data types in [SHOW COLUMNS](../../sql-reference/statements/show.md#show_columns). Possible values: diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index 3a82f56a58d..2915b053458 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -14,7 +14,7 @@ The `system.columns` table contains the following columns (the column type is sh - `database` ([String](../../sql-reference/data-types/string.md)) — Database name. - `table` ([String](../../sql-reference/data-types/string.md)) — Table name. - `name` ([String](../../sql-reference/data-types/string.md)) — Column name. -- `type` ([String](../../sql-reference/data-types/string.md)) — Column type. If setting `[use_mysql_types_in_show_columns](../../operations/settings/settings.md#use_mysql_types_in_show_columns) = 1` (default: 0), then the equivalent type name in MySQL is shown. +- `type` ([String](../../sql-reference/data-types/string.md)) — Column type. - `position` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Ordinal position of a column in a table starting with 1. - `default_kind` ([String](../../sql-reference/data-types/string.md)) — Expression type (`DEFAULT`, `MATERIALIZED`, `ALIAS`) for the default value, or an empty string if it is not defined. - `default_expression` ([String](../../sql-reference/data-types/string.md)) — Expression for the default value, or an empty string if it is not defined. From 2fb6a7deeff8bcb597dded98d1625c061d1e12ba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 11 Oct 2023 17:00:29 +0200 Subject: [PATCH 135/634] Clean up diff with master --- .../02784_parallel_replicas_automatic_decision.sh | 3 ++- .../0_stateless/02811_parallel_replicas_prewhere_count.sql | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 5a0ef074112..6620495222e 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -18,6 +18,7 @@ function were_parallel_replicas_used () { FORMAT TSV" } +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_parallel_replicas_automatic_count" $CLICKHOUSE_CLIENT --query " CREATE TABLE IF NOT EXISTS test_parallel_replicas_automatic_count ( @@ -95,4 +96,4 @@ run_query_with_pure_parallel_replicas "${query_id_base}_helpless_filter_5M" 5000 $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" were_parallel_replicas_used "${query_id_base}" -$CLICKHOUSE_CLIENT --query "DROP TABLE test_parallel_replicas_automatic_count" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_parallel_replicas_automatic_count" diff --git a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql index 18c1e0a67f8..374d73d7d03 100644 --- a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql +++ b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql @@ -1,14 +1,13 @@ +DROP TABLE IF EXISTS users; CREATE TABLE users (uid Int16, name String, age Int16) ENGINE=MergeTree() ORDER BY uid; INSERT INTO users VALUES (111, 'JFK', 33); INSERT INTO users VALUES (6666, 'KLM', 48); INSERT INTO users VALUES (88888, 'AMS', 50); - SELECT '-- count() ------------------------------'; SELECT count() FROM users PREWHERE uid > 2000; - -- enable parallel replicas but with high rows threshold SET skip_unavailable_shards=1, @@ -19,6 +18,7 @@ cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1, parallel_replicas_min_number_of_rows_per_replica=1000; - SELECT '-- count() with parallel replicas -------'; SELECT count() FROM users PREWHERE uid > 2000; + +DROP TABLE IF EXISTS users; From 9d95f4e1b6eb8d840b29a3321aeeeec6381c180e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 11 Oct 2023 15:04:59 +0000 Subject: [PATCH 136/634] Cleanup: parallel replica coordinator usage --- src/Interpreters/ClusterProxy/SelectStreamFactory.h | 1 - src/Interpreters/ClusterProxy/executeQuery.cpp | 1 + src/Interpreters/Context.h | 5 ----- src/Processors/QueryPlan/DistributedCreateLocalPlan.h | 1 - src/Processors/QueryPlan/ReadFromRemote.h | 6 ++++-- src/QueryPipeline/RemoteQueryExecutor.cpp | 1 + src/QueryPipeline/RemoteQueryExecutor.h | 3 ++- 7 files changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index a821730657d..938f683f209 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -6,7 +6,6 @@ #include #include #include -#include #include namespace DB diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 84ae382487d..41235d107cd 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 24441ff9ab8..4fc29f0a7d2 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -194,9 +194,6 @@ using MergeTreeReadTaskCallback = std::function; -class ParallelReplicasReadingCoordinator; -using ParallelReplicasReadingCoordinatorPtr = std::shared_ptr; - class PreparedSetsCache; using PreparedSetsCachePtr = std::shared_ptr; @@ -449,8 +446,6 @@ public: KitchenSink kitchen_sink; - ParallelReplicasReadingCoordinatorPtr parallel_reading_coordinator; - private: using SampleBlockCache = std::unordered_map; mutable SampleBlockCache sample_block_cache; diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h index 1f62d05b8de..489d2607849 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index a2486e1eaa1..35ae336a72c 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -5,8 +5,7 @@ #include #include #include -#include -#include "Core/UUID.h" +#include namespace DB { @@ -17,6 +16,9 @@ using ConnectionPoolWithFailoverPtr = std::shared_ptr; +class ParallelReplicasReadingCoordinator; +using ParallelReplicasReadingCoordinatorPtr = std::shared_ptr; + /// Reading step from remote servers. /// Unite query results from several shards. class ReadFromRemote final : public ISourceStep diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index b834870c334..8f4e90ea3d2 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -21,6 +21,7 @@ #include #include #include +#include namespace ProfileEvents diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 8d834eb3f81..9972d4dd45d 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -9,7 +9,6 @@ #include #include #include -#include #include @@ -29,6 +28,8 @@ using ProfileInfoCallback = std::function; class RemoteQueryExecutorReadContext; +class ParallelReplicasReadingCoordinator; + /// This is the same type as StorageS3Source::IteratorWrapper using TaskIterator = std::function; From 97d8e16e8dd0c320f4bfbd708211410c87b5a057 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 11 Oct 2023 15:09:34 +0000 Subject: [PATCH 137/634] Fix MySQL packet row data type Cf. https://github.com/ClickHouse/ClickHouse/pull/55479#discussion_r1355175581 --- src/Core/MySQL/MySQLUtils.cpp | 2 +- src/Core/MySQL/MySQLUtils.h | 2 +- src/Core/MySQL/PacketsProtocolBinary.cpp | 2 +- src/Core/MySQL/PacketsProtocolBinary.h | 4 ++-- src/Core/MySQL/PacketsProtocolText.cpp | 2 +- src/Core/MySQL/PacketsProtocolText.h | 4 ++-- src/Processors/Formats/Impl/MySQLOutputFormat.cpp | 8 ++++---- 7 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Core/MySQL/MySQLUtils.cpp b/src/Core/MySQL/MySQLUtils.cpp index 44d2cace8c9..8dedbdaa630 100644 --- a/src/Core/MySQL/MySQLUtils.cpp +++ b/src/Core/MySQL/MySQLUtils.cpp @@ -13,7 +13,7 @@ namespace MySQLUtils { DecimalUtils::DecimalComponents -getNormalizedDateTime64Components(const DataTypePtr data_type, const ColumnPtr col, const int row_num) +getNormalizedDateTime64Components(DataTypePtr data_type, ColumnPtr col, size_t row_num) { const auto * date_time_type = typeid_cast(data_type.get()); diff --git a/src/Core/MySQL/MySQLUtils.h b/src/Core/MySQL/MySQLUtils.h index 4ca1fc3ddae..48b78fcf4d7 100644 --- a/src/Core/MySQL/MySQLUtils.h +++ b/src/Core/MySQL/MySQLUtils.h @@ -12,7 +12,7 @@ namespace MySQLUtils { /// Splits DateTime64 column data at a certain row number into whole and fractional part /// Additionally, normalizes the fractional part as if it was scale 6 for MySQL compatibility purposes -DecimalUtils::DecimalComponents getNormalizedDateTime64Components(DataTypePtr data_type, ColumnPtr col, int row_num); +DecimalUtils::DecimalComponents getNormalizedDateTime64Components(DataTypePtr data_type, ColumnPtr col, size_t row_num); /// If a column is ColumnSparse/ColumnLowCardinality/ColumnNullable, it is unwrapped in a correct order; /// otherwise, the original column is returned diff --git a/src/Core/MySQL/PacketsProtocolBinary.cpp b/src/Core/MySQL/PacketsProtocolBinary.cpp index 29b36b33ff2..7f5e3900cb0 100644 --- a/src/Core/MySQL/PacketsProtocolBinary.cpp +++ b/src/Core/MySQL/PacketsProtocolBinary.cpp @@ -26,7 +26,7 @@ namespace MySQLProtocol { namespace ProtocolBinary { -ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTypes & data_types_, const Columns & columns_, int row_num_) +ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTypes & data_types_, const Columns & columns_, size_t row_num_) : row_num(row_num_), columns(columns_), data_types(data_types_), serializations(serializations_) { payload_size = 1 + null_bitmap_size; diff --git a/src/Core/MySQL/PacketsProtocolBinary.h b/src/Core/MySQL/PacketsProtocolBinary.h index 5425bf4d667..a440647d267 100644 --- a/src/Core/MySQL/PacketsProtocolBinary.h +++ b/src/Core/MySQL/PacketsProtocolBinary.h @@ -17,7 +17,7 @@ namespace ProtocolBinary class ResultSetRow : public IMySQLWritePacket { protected: - int row_num; + size_t row_num; const Columns & columns; const DataTypes & data_types; const Serializations & serializations; @@ -35,7 +35,7 @@ protected: void writePayloadImpl(WriteBuffer & buffer) const override; public: - ResultSetRow(const Serializations & serializations_, const DataTypes & data_types_, const Columns & columns_, int row_num_); + ResultSetRow(const Serializations & serializations_, const DataTypes & data_types_, const Columns & columns_, size_t row_num_); }; } } diff --git a/src/Core/MySQL/PacketsProtocolText.cpp b/src/Core/MySQL/PacketsProtocolText.cpp index 10e2f13c88b..744227f27dc 100644 --- a/src/Core/MySQL/PacketsProtocolText.cpp +++ b/src/Core/MySQL/PacketsProtocolText.cpp @@ -18,7 +18,7 @@ namespace MySQLProtocol namespace ProtocolText { -ResultSetRow::ResultSetRow(const Serializations & serializations, const DataTypes & data_types, const Columns & columns_, int row_num_) +ResultSetRow::ResultSetRow(const Serializations & serializations, const DataTypes & data_types, const Columns & columns_, size_t row_num_) : columns(columns_), row_num(row_num_) { static FormatSettings format_settings = {.bool_true_representation = "1", .bool_false_representation = "0"}; diff --git a/src/Core/MySQL/PacketsProtocolText.h b/src/Core/MySQL/PacketsProtocolText.h index adca9de8c03..07969a1ed93 100644 --- a/src/Core/MySQL/PacketsProtocolText.h +++ b/src/Core/MySQL/PacketsProtocolText.h @@ -67,7 +67,7 @@ class ResultSetRow : public IMySQLWritePacket { protected: const Columns & columns; - int row_num; + size_t row_num; size_t payload_size = 0; std::vector serialized; @@ -76,7 +76,7 @@ protected: void writePayloadImpl(WriteBuffer & buffer) const override; public: - ResultSetRow(const Serializations & serializations, const DataTypes & data_types, const Columns & columns_, int row_num_); + ResultSetRow(const Serializations & serializations, const DataTypes & data_types, const Columns & columns_, size_t row_num_); }; class ComFieldList : public LimitedReadPacket diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index f3e35733975..7148996cc1d 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -67,17 +67,17 @@ void MySQLOutputFormat::consume(Chunk chunk) { if (!use_binary_result_set) { - for (size_t i = 0; i < chunk.getNumRows(); ++i) + for (size_t row = 0; row < chunk.getNumRows(); ++row) { - ProtocolText::ResultSetRow row_packet(serializations, data_types, chunk.getColumns(), static_cast(i)); + ProtocolText::ResultSetRow row_packet(serializations, data_types, chunk.getColumns(), row); packet_endpoint->sendPacket(row_packet); } } else { - for (size_t i = 0; i < chunk.getNumRows(); ++i) + for (size_t row = 0; row < chunk.getNumRows(); ++row) { - ProtocolBinary::ResultSetRow row_packet(serializations, data_types, chunk.getColumns(), static_cast(i)); + ProtocolBinary::ResultSetRow row_packet(serializations, data_types, chunk.getColumns(), row); packet_endpoint->sendPacket(row_packet); } } From e364ed61a66b3c143374864c63c606e47a6ca32d Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Oct 2023 17:42:44 +0200 Subject: [PATCH 138/634] Proper fix --- src/Analyzer/Utils.cpp | 32 +++++ src/Analyzer/Utils.h | 4 + src/Storages/AlterCommands.cpp | 3 +- src/Storages/MergeTree/MergeTreeData.cpp | 135 ++++++++++-------- src/Storages/MergeTree/MergeTreeData.h | 7 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 16 --- .../MergeTree/ReplicatedMergeTreeQueue.h | 8 +- src/Storages/StorageMergeTree.cpp | 20 --- src/Storages/StorageMergeTree.h | 2 - src/Storages/StorageReplicatedMergeTree.cpp | 5 - src/Storages/StorageReplicatedMergeTree.h | 2 - ...97_column_delete_and_replication.reference | 0 .../02597_column_delete_and_replication.sql | 29 ++++ .../02597_column_update_and_replication.sql | 23 +-- ...tricy_expression_and_replication.reference | 0 ...pdate_tricy_expression_and_replication.sql | 30 ++++ ...projection_materialize_and_replication.sql | 24 +--- ...ge_tree_with_block_number_column.reference | 13 ++ ...ng_merge_tree_with_block_number_column.sql | 23 +++ 19 files changed, 225 insertions(+), 151 deletions(-) create mode 100644 tests/queries/0_stateless/02597_column_delete_and_replication.reference create mode 100644 tests/queries/0_stateless/02597_column_delete_and_replication.sql create mode 100644 tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.reference create mode 100644 tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql create mode 100644 tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.reference create mode 100644 tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.sql diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 6804e85c4cf..5e8f2ab8e61 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -573,4 +573,36 @@ void replaceColumns(QueryTreeNodePtr & node, visitor.visit(node); } +namespace +{ + +class CollectIdentifiersFullNamesVisitor : public ConstInDepthQueryTreeVisitor +{ +public: + explicit CollectIdentifiersFullNamesVisitor(NameSet & used_identifiers_) + : used_identifiers(used_identifiers_) { } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) { return true; } + + void visitImpl(const QueryTreeNodePtr & node) + { + auto * column_node = node->as(); + if (!column_node) + return; + + used_identifiers.insert(column_node->getIdentifier().getFullName()); + } + + NameSet & used_identifiers; +}; + +} + +NameSet collectIdentifiersFullNames(const QueryTreeNodePtr & node) +{ + NameSet out; + CollectIdentifiersFullNamesVisitor visitor(out); + visitor.visit(node); + return out; +} } diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index ea36e17bf11..6756c4d313c 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -83,4 +83,8 @@ void replaceColumns(QueryTreeNodePtr & node, const QueryTreeNodePtr & table_expression_node, const std::unordered_map & column_name_to_node); + +/// Just collect all identifiers from query tree +NameSet collectIdentifiersFullNames(const QueryTreeNodePtr & node); + } diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index fd77e33e97e..c6fa17583b5 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -847,7 +847,8 @@ bool AlterCommand::isRemovingProperty() const bool AlterCommand::isDropSomething() const { - return type == Type::DROP_COLUMN || type == Type::DROP_INDEX || type == Type::DROP_CONSTRAINT; + return type == Type::DROP_COLUMN || type == Type::DROP_INDEX + || type == Type::DROP_CONSTRAINT || type == Type::DROP_PROJECTION; } std::optional AlterCommand::tryConvertToMutationCommand(StorageInMemoryMetadata & metadata, ContextPtr context) const diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1e7c35418ab..e4aa4ca18bc 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1,58 +1,47 @@ #include #include +#include +#include #include #include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include #include #include #include -#include #include #include -#include +#include #include #include +#include #include +#include +#include +#include +#include +#include #include #include -#include -#include +#include #include -#include #include #include #include #include #include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include #include -#include #include +#include #include #include #include @@ -61,24 +50,37 @@ #include #include #include -#include #include #include #include #include +#include #include -#include -#include +#include +#include #include #include #include -#include +#include +#include +#include #include #include #include -#include -#include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -4305,7 +4307,7 @@ void MergeTreeData::delayMutationOrThrowIfNeeded(Poco::Event * until, const Cont if (!num_mutations_to_delay && !num_mutations_to_throw) return; - size_t num_unfinished_mutations = getNumberOfUnfinishedMutations(); + size_t num_unfinished_mutations = getUnfinishedMutationCommands().size(); if (num_mutations_to_throw && num_unfinished_mutations >= num_mutations_to_throw) { ProfileEvents::increment(ProfileEvents::RejectedMutations); @@ -8085,50 +8087,63 @@ bool MergeTreeData::canUsePolymorphicParts() const } -void MergeTreeData::checkDropCommandDoesntAffectInProgressMutations(const AlterCommand & command, const std::map & unfinished_mutations, ContextPtr /*local_context*/) const +void MergeTreeData::checkDropCommandDoesntAffectInProgressMutations(const AlterCommand & command, const std::map & unfinished_mutations, ContextPtr local_context) const { - if (!command.isDropSomething()) + if (!command.isDropSomething() || unfinished_mutations.empty()) return; + auto throw_exception = [] ( + const std::string & mutation_name, + const std::string & entity_name, + const std::string & identifier_name) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot drop {} {} because it's affected by mutation with ID '{}' which is not finished yet. " + "Wait this mutation, or KILL it with command " + "\"KILL MUTATION WHERE mutation_id = '{}'\"", + entity_name, + backQuoteIfNeed(identifier_name), + mutation_name, + mutation_name); + }; + for (const auto & [mutation_name, commands] : unfinished_mutations) { for (const MutationCommand & mutation_command : commands) { if (command.type == AlterCommand::DROP_INDEX && mutation_command.index_name == command.index_name) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Cannot drop index {} because it's affected by mutation with ID '{}' which is not finished yet. " - "Wait this mutation, or KILL it with command " - "\"KILL MUTATION WHERE mutation_id = '{}'\"", - command.index_name, - mutation_name, - mutation_name); + throw_exception(mutation_name, "index", command.index_name); } else if (command.type == AlterCommand::DROP_PROJECTION && mutation_command.projection_name == command.projection_name) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Cannot drop projection {} because it's affected by mutation with ID '{}' which is not finished yet. " - "Wait this mutation, or KILL it with command " - "\"KILL MUTATION WHERE mutation_id = '{}'\"", - command.index_name, - mutation_name, - mutation_name); + throw_exception(mutation_name, "projection", command.projection_name); } else if (command.type == AlterCommand::DROP_COLUMN) { if (mutation_command.column_name == command.column_name) + throw_exception(mutation_name, "column", command.column_name); + + if (mutation_command.predicate) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Cannot drop column {} because it's affected by mutation with ID '{}' which is not finished yet. " - "Wait this mutation, or KILL it with command " - "\"KILL MUTATION WHERE mutation_id = '{}'\"", - command.index_name, - mutation_name, - mutation_name); + auto query_tree = buildQueryTree(mutation_command.predicate, local_context); + auto identifiers = collectIdentifiersFullNames(query_tree); + + if (identifiers.contains(command.column_name)) + throw_exception(mutation_name, "column", command.column_name); + } + + for (const auto & [name, expr] : mutation_command.column_to_update_expression) + { + if (name == command.column_name) + throw_exception(mutation_name, "column", command.column_name); + + auto query_tree = buildQueryTree(expr, local_context); + auto identifiers = collectIdentifiersFullNames(query_tree); + if (identifiers.contains(command.column_name)) + throw_exception(mutation_name, "column", command.column_name); } } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 346bfee076a..9098b6e079d 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -583,9 +583,6 @@ public: /// The decision to delay or throw is made according to settings 'number_of_mutations_to_delay' and 'number_of_mutations_to_throw'. void delayMutationOrThrowIfNeeded(Poco::Event * until, const ContextPtr & query_context) const; - /// Returns number of unfinished mutations (is_done = 0). - virtual size_t getNumberOfUnfinishedMutations() const = 0; - /// Renames temporary part to a permanent part and adds it to the parts set. /// It is assumed that the part does not intersect with existing parts. /// Adds the part in the PreActive state (the part will be added to the active set later with out_transaction->commit()). @@ -722,9 +719,11 @@ public: /// If something is wrong, throws an exception. void checkAlterIsPossible(const AlterCommands & commands, ContextPtr context) const override; + /// Throw exception if command is some kind of DROP command (drop column, drop index, etc) + /// and we have unfinished mutation which need this column to finish. void checkDropCommandDoesntAffectInProgressMutations( const AlterCommand & command, const std::map & unfinished_mutations, ContextPtr context) const; - + /// Return mapping unfinished mutation name -> Mutation command virtual std::map getUnfinishedMutationCommands() const = 0; /// Checks if the Mutation can be performed. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index f993515573a..9f726a7b7c9 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1762,22 +1762,6 @@ size_t ReplicatedMergeTreeQueue::countFinishedMutations() const return count; } -size_t ReplicatedMergeTreeQueue::countUnfinishedMutations() const -{ - std::lock_guard lock(state_mutex); - - size_t count = 0; - for (const auto & [_, status] : mutations_by_znode | std::views::reverse) - { - if (status.is_done) - break; - ++count; - } - - return count; -} - - std::map ReplicatedMergeTreeQueue::getUnfinishedMutations() const { std::map result; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 452dee22ea1..393e9cc48bc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -388,14 +388,12 @@ public: /// Count the total number of active mutations that are finished (is_done = true). size_t countFinishedMutations() const; - /// Count the total number of active mutations that are not finished (is_done = false). - size_t countUnfinishedMutations() const; std::map getUnfinishedMutations() const; - /// Returns functor which used by MergeTreeMergerMutator to select parts for merge - ReplicatedMergeTreeMergePredicate - getMergePredicate(zkutil::ZooKeeperPtr & zookeeper, std::optional && partition_ids_hint); + /// Returns functor which used by MergeTreeMergerMutator to select parts for merge + ReplicatedMergeTreeMergePredicate + getMergePredicate(zkutil::ZooKeeperPtr & zookeeper, std::optional && partition_ids_hint); MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, Strings & mutation_ids) const; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2b1eb6b25d0..f5f8410aa35 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1420,26 +1420,6 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign return scheduled; } -size_t StorageMergeTree::getNumberOfUnfinishedMutations() const -{ - std::unique_lock lock(currently_processing_in_background_mutex); - - size_t count = 0; - for (const auto & [version, _] : current_mutations_by_version | std::views::reverse) - { - auto status = getIncompleteMutationsStatusUnlocked(version, lock, nullptr, true); - if (!status) - continue; - - if (status->is_done) - break; - - ++count; - } - - return count; -} - UInt64 StorageMergeTree::getCurrentMutationVersion( const DataPartPtr & part, std::unique_lock & /*currently_processing_in_background_mutex_lock*/) const diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index a6d2f3a3272..f0759d5c4db 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -112,8 +112,6 @@ public: bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; - size_t getNumberOfUnfinishedMutations() const override; - std::map getUnfinishedMutationCommands() const override; MergeTreeDeduplicationLog * getDeduplicationLog() { return deduplication_log.get(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 421d04087ee..1d5bd1388f2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8710,11 +8710,6 @@ String StorageReplicatedMergeTree::getTableSharedID() const return toString(table_shared_id); } -size_t StorageReplicatedMergeTree::getNumberOfUnfinishedMutations() const -{ - return queue.countUnfinishedMutations(); -} - std::map StorageReplicatedMergeTree::getUnfinishedMutationCommands() const { return queue.getUnfinishedMutations(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index b027735fda2..4e8fae6c6ac 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -349,8 +349,6 @@ public: std::map getUnfinishedMutationCommands() const override; - size_t getNumberOfUnfinishedMutations() const override; - /// Returns the same as getTableSharedID(), but extracts it from a create query. static std::optional tryGetTableSharedIDFromCreateQuery(const IAST & create_query, const ContextPtr & global_context); diff --git a/tests/queries/0_stateless/02597_column_delete_and_replication.reference b/tests/queries/0_stateless/02597_column_delete_and_replication.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02597_column_delete_and_replication.sql b/tests/queries/0_stateless/02597_column_delete_and_replication.sql new file mode 100644 index 00000000000..70dcb95a3a1 --- /dev/null +++ b/tests/queries/0_stateless/02597_column_delete_and_replication.sql @@ -0,0 +1,29 @@ +CREATE TABLE test ( + `c_id` String, + `p_id` String, + `d` String +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1') +ORDER BY (c_id, p_id); + +INSERT INTO test SELECT '1', '11', '111' FROM numbers(3); + +INSERT INTO test SELECT '2', '22', '22' FROM numbers(3); + +set mutations_sync=0; + +ALTER TABLE test UPDATE d = d || toString(sleepEachRow(0.3)) where 1; + +ALTER TABLE test ADD COLUMN x UInt32 default 0; +ALTER TABLE test UPDATE d = d || '1' where x = 42; +ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; --{serverError 36} + +ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync = 2; + +ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; + +SELECT * FROM system.mutations WHERE database=currentDatabase() AND table='test' AND NOT is_done; + +select * from test format Null; + +DROP TABLE test; diff --git a/tests/queries/0_stateless/02597_column_update_and_replication.sql b/tests/queries/0_stateless/02597_column_update_and_replication.sql index 07b13468ccb..f25c177ea38 100644 --- a/tests/queries/0_stateless/02597_column_update_and_replication.sql +++ b/tests/queries/0_stateless/02597_column_update_and_replication.sql @@ -6,37 +6,24 @@ CREATE TABLE test ( ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1') ORDER BY (c_id, p_id); - ----CREATE TABLE test_r2 ( ---- `c_id` String, ---- `p_id` String, ---- `d` String ----) ----ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '2') ----ORDER BY (c_id, p_id); - - INSERT INTO test SELECT '1', '11', '111' FROM numbers(3); INSERT INTO test SELECT '2', '22', '22' FROM numbers(3); -select * from test format Null; -select min(c_id) from test group by d format Null; - set mutations_sync=0; -ALTER TABLE test UPDATE d = d || toString(sleepEachRow(0.1)) where 1; +ALTER TABLE test UPDATE d = d || toString(sleepEachRow(0.3)) where 1; ALTER TABLE test ADD COLUMN x UInt32 default 0; ALTER TABLE test UPDATE x = x + 1 where 1; +ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; --{serverError 36} + +ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync = 2; + ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; SELECT * FROM system.mutations WHERE database=currentDatabase() AND table='test' AND NOT is_done; - - select * from test format Null; - DROP TABLE test; ---DROP TABLE test_r2; diff --git a/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.reference b/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql b/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql new file mode 100644 index 00000000000..ae80cf666f2 --- /dev/null +++ b/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql @@ -0,0 +1,30 @@ +CREATE TABLE test ( + `c_id` String, + `p_id` String, + `d` UInt32 +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1') +ORDER BY (c_id, p_id); + +INSERT INTO test SELECT '1', '11', '111' FROM numbers(3); + +INSERT INTO test SELECT '2', '22', '22' FROM numbers(3); + +set mutations_sync=0; + +ALTER TABLE test UPDATE d = d + sleepEachRow(0.3) where 1; + +ALTER TABLE test ADD COLUMN x UInt32 default 0; +ALTER TABLE test UPDATE d = x + 1 where 1; +ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; --{serverError 36} + +ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync = 2; + +ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; + +SELECT * FROM system.mutations WHERE database=currentDatabase() AND table='test' AND NOT is_done; + +select * from test format Null; + +DROP TABLE test; + diff --git a/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql b/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql index 39f0f9091da..65956dee985 100644 --- a/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql +++ b/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql @@ -6,37 +6,25 @@ CREATE TABLE test ( ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1') ORDER BY (c_id, p_id); - ----CREATE TABLE test_r2 ( ---- `c_id` String, ---- `p_id` String, ---- `d` String ----) ----ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '2') ----ORDER BY (c_id, p_id); - - INSERT INTO test SELECT '1', '11', '111' FROM numbers(3); INSERT INTO test SELECT '2', '22', '22' FROM numbers(3); -select * from test format Null; -select min(c_id) from test group by d format Null; - set mutations_sync=0; -ALTER TABLE test UPDATE d = d || toString(sleepEachRow(0.1)) where 1; +ALTER TABLE test UPDATE d = d || toString(sleepEachRow(0.3)) where 1; ALTER TABLE test ADD PROJECTION d_order ( SELECT min(c_id) GROUP BY `d`); ALTER TABLE test MATERIALIZE PROJECTION d_order; +ALTER TABLE test DROP PROJECTION d_order SETTINGS mutations_sync = 2; --{serverError 36} + +-- just to wait prev mutation +ALTER TABLE test DELETE where d = 'Hello' SETTINGS mutations_sync = 2; + ALTER TABLE test DROP PROJECTION d_order SETTINGS mutations_sync = 2; SELECT * FROM system.mutations WHERE database=currentDatabase() AND table='test' AND NOT is_done; - - select * from test format Null; - DROP TABLE test; ---DROP TABLE test_r2; diff --git a/tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.reference b/tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.reference new file mode 100644 index 00000000000..9c9caa22139 --- /dev/null +++ b/tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.reference @@ -0,0 +1,13 @@ +== Only last version remains after OPTIMIZE W/ CLEANUP == +d1 5 0 +d2 1 0 +d3 1 0 +d4 1 0 +d5 1 0 +d6 3 0 +== OPTIMIZE W/ CLEANUP (remove d6) == +d1 5 0 +d2 1 0 +d3 1 0 +d4 1 0 +d5 1 0 diff --git a/tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.sql b/tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.sql new file mode 100644 index 00000000000..7b78e2900e7 --- /dev/null +++ b/tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version, is_deleted) Order by (uid) SETTINGS vertical_merge_algorithm_min_rows_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 0, + min_rows_for_wide_part = 1, + min_bytes_for_wide_part = 1; + +-- Expect d6 to be version=3 is_deleted=false +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 3, 0); +-- Insert previous version of 'd6' but only v=3 is_deleted=false will remain +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 2, 1); +SELECT '== Only last version remains after OPTIMIZE W/ CLEANUP =='; +OPTIMIZE TABLE test FINAL CLEANUP; +select * from test order by uid; + +-- insert d6 v=3 is_deleted=true (timestamp more recent so this version should be the one take into acount) +INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 3, 1); + +SELECT '== OPTIMIZE W/ CLEANUP (remove d6) =='; +OPTIMIZE TABLE test FINAL CLEANUP; +-- No d6 anymore +select * from test order by uid; + +DROP TABLE IF EXISTS test; From f11fa6a7c3812ff6f2374641f0773b7410d1ed7e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Oct 2023 17:44:15 +0200 Subject: [PATCH 139/634] remove accidentally commited files --- ...ge_tree_with_block_number_column.reference | 13 ----------- ...ng_merge_tree_with_block_number_column.sql | 23 ------------------- 2 files changed, 36 deletions(-) delete mode 100644 tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.reference delete mode 100644 tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.sql diff --git a/tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.reference b/tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.reference deleted file mode 100644 index 9c9caa22139..00000000000 --- a/tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.reference +++ /dev/null @@ -1,13 +0,0 @@ -== Only last version remains after OPTIMIZE W/ CLEANUP == -d1 5 0 -d2 1 0 -d3 1 0 -d4 1 0 -d5 1 0 -d6 3 0 -== OPTIMIZE W/ CLEANUP (remove d6) == -d1 5 0 -d2 1 0 -d3 1 0 -d4 1 0 -d5 1 0 diff --git a/tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.sql b/tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.sql deleted file mode 100644 index 7b78e2900e7..00000000000 --- a/tests/queries/0_stateless/02861_replacing_merge_tree_with_block_number_column.sql +++ /dev/null @@ -1,23 +0,0 @@ -DROP TABLE IF EXISTS test; -CREATE TABLE test (uid String, version UInt32, is_deleted UInt8) ENGINE = ReplacingMergeTree(version, is_deleted) Order by (uid) SETTINGS vertical_merge_algorithm_min_rows_to_activate = 1, - vertical_merge_algorithm_min_columns_to_activate = 0, - min_rows_for_wide_part = 1, - min_bytes_for_wide_part = 1; - --- Expect d6 to be version=3 is_deleted=false -INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 3, 0); --- Insert previous version of 'd6' but only v=3 is_deleted=false will remain -INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 2, 1); -SELECT '== Only last version remains after OPTIMIZE W/ CLEANUP =='; -OPTIMIZE TABLE test FINAL CLEANUP; -select * from test order by uid; - --- insert d6 v=3 is_deleted=true (timestamp more recent so this version should be the one take into acount) -INSERT INTO test (*) VALUES ('d1', 1, 0), ('d1', 2, 1), ('d1', 3, 0), ('d1', 4, 1), ('d1', 5, 0), ('d2', 1, 0), ('d3', 1, 0), ('d4', 1, 0), ('d5', 1, 0), ('d6', 1, 0), ('d6', 3, 1); - -SELECT '== OPTIMIZE W/ CLEANUP (remove d6) =='; -OPTIMIZE TABLE test FINAL CLEANUP; --- No d6 anymore -select * from test order by uid; - -DROP TABLE IF EXISTS test; From e2cde338fbbe354a80462299272736c3c0d5e95e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Oct 2023 17:45:32 +0200 Subject: [PATCH 140/634] better headers --- src/Storages/MergeTree/MergeTreeData.cpp | 28 ++++++++++++------------ 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e4aa4ca18bc..82d09acef2d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8,6 +8,20 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include @@ -67,20 +81,6 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include From 9c1e52e57d34124f55dc0cc87734abdcaee1c723 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 11 Oct 2023 15:43:24 +0000 Subject: [PATCH 141/634] Expand tests --- ...show_columns_mysql_compatibility.reference | 270 ++++++++---------- ...02775_show_columns_mysql_compatibility.sql | 112 +++++--- 2 files changed, 187 insertions(+), 195 deletions(-) diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference index c1844208665..4df75c2bfc9 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference @@ -1,148 +1,122 @@ -aggregate_function AggregateFunction(sum, Int32) NO \N -array_value Array(Int32) NO \N -boolean_value UInt8 NO \N -date32_value Date32 NO \N -date_value Date NO \N -datetime64_value DateTime64(3) NO \N -datetime_value DateTime NO \N -decimal_value Decimal(10, 2) NO \N -enum_value Enum8(\'apple\' = 1, \'banana\' = 2, \'orange\' = 3) NO \N -fixed_string_value FixedString(10) NO \N -float32 Float32 NO \N -float64 Float64 NO \N -int128 Int128 NO \N -int16 Int16 NO \N -int256 Int256 NO \N -int32 Int32 NO \N -int64 Int64 NO \N -int8 Int8 NO \N -ipv4_value IPv4 NO \N -ipv6_value IPv6 NO \N -json_value Object(\'json\') NO \N -low_cardinality LowCardinality(String) NO \N -low_cardinality_date LowCardinality(DateTime) NO \N -map_value Map(String, Int32) NO \N -nested.nested_int Array(Int32) NO \N -nested.nested_string Array(String) NO \N -nint32 Nullable(Int32) YES \N -nullable_value Nullable(Int32) YES \N -string_value String NO \N -tuple_value Tuple(Int32, String) NO \N -uint128 UInt128 NO \N -uint16 UInt16 NO \N -uint256 UInt256 NO \N -uint32 UInt32 NO \N -uint64 UInt64 NO PRI SOR \N -uint8 UInt8 NO \N -uuid_value UUID NO \N -aggregate_function TEXT NO \N -array_value TEXT NO \N -boolean_value TINYINT UNSIGNED NO \N -date32_value DATE NO \N -date_value DATE NO \N -datetime64_value DATETIME NO \N -datetime_value DATETIME NO \N -decimal_value DECIMAL(10, 2) NO \N -enum_value ENUM(\'apple\', \'banana\', \'orange\') NO \N -fixed_string_value TEXT NO \N -float32 FLOAT NO \N -float64 DOUBLE NO \N -int128 TEXT NO \N -int16 SMALLINT NO \N -int256 TEXT NO \N -int32 INTEGER NO \N -int64 BIGINT NO \N -int8 TINYINT NO \N -ipv4_value TEXT NO \N -ipv6_value TEXT NO \N -json_value JSON NO \N -low_cardinality BLOB NO \N -low_cardinality_date DATETIME NO \N -map_value JSON NO \N -nested.nested_int TEXT NO \N -nested.nested_string TEXT NO \N -nint32 INTEGER NO \N -nullable_value INTEGER NO \N -string_value BLOB NO \N -tuple_value JSON NO \N -uint128 TEXT NO \N -uint16 SMALLINT UNSIGNED NO \N -uint256 TEXT NO \N -uint32 INTEGER UNSIGNED NO \N -uint64 BIGINT UNSIGNED NO PRI SOR \N -uint8 TINYINT UNSIGNED NO \N -uuid_value CHAR NO \N -uint8 UInt8 -uint16 UInt16 -uint32 UInt32 -uint64 UInt64 -uint128 UInt128 -uint256 UInt256 -int8 Int8 -int16 Int16 -int32 Int32 -int64 Int64 -int128 Int128 -int256 Int256 -nint32 Nullable(Int32) -float32 Float32 -float64 Float64 -decimal_value Decimal(10, 2) -boolean_value UInt8 -string_value String -fixed_string_value FixedString(10) -date_value Date -date32_value Date32 -datetime_value DateTime -datetime64_value DateTime64(3) -json_value Object(\'json\') -uuid_value UUID -enum_value Enum8(\'apple\' = 1, \'banana\' = 2, \'orange\' = 3) -low_cardinality LowCardinality(String) -low_cardinality_date LowCardinality(DateTime) -aggregate_function AggregateFunction(sum, Int32) -array_value Array(Int32) -map_value Map(String, Int32) -tuple_value Tuple(Int32, String) -nullable_value Nullable(Int32) -ipv4_value IPv4 -ipv6_value IPv6 -nested.nested_int Array(Int32) -nested.nested_string Array(String) -uint8 TINYINT UNSIGNED -uint16 SMALLINT UNSIGNED -uint32 INTEGER UNSIGNED -uint64 BIGINT UNSIGNED -uint128 TEXT -uint256 TEXT -int8 TINYINT -int16 SMALLINT -int32 INTEGER -int64 BIGINT -int128 TEXT -int256 TEXT -nint32 INTEGER -float32 FLOAT -float64 DOUBLE -decimal_value DECIMAL(10, 2) -boolean_value TINYINT UNSIGNED -string_value BLOB -fixed_string_value TEXT -date_value DATE -date32_value DATE -datetime_value DATETIME -datetime64_value DATETIME -json_value JSON -uuid_value CHAR -enum_value ENUM(\'apple\', \'banana\', \'orange\') -low_cardinality BLOB -low_cardinality_date DATETIME -aggregate_function TEXT -array_value TEXT -map_value JSON -tuple_value JSON -nullable_value INTEGER -ipv4_value TEXT -ipv6_value TEXT -nested.nested_int TEXT -nested.nested_string TEXT +-- SHOW COLUMNS with use_mysql_types_in_show_columns = 0 +a Array(String) NO \N +agg AggregateFunction(uniq, UInt64) NO \N +b Bool NO \N +d Date NO \N +d32 Date32 NO \N +dec128 Decimal(38, 2) NO \N +dec128_native Decimal(35, 30) NO \N +dec128_text Decimal(35, 31) NO \N +dec256 Decimal(76, 2) NO \N +dec256_native Decimal(65, 2) NO \N +dec256_text Decimal(66, 2) NO \N +dec32 Decimal(9, 2) NO \N +dec64 Decimal(18, 2) NO \N +dt DateTime NO \N +dt64 DateTime64(3) NO \N +dt64_3_tz1 DateTime64(3, \'UTC\') NO \N +dt64_3_tz2 DateTime64(3, \'Asia/Shanghai\') NO \N +dt64_6 DateTime64(6, \'UTC\') NO \N +dt64_9 DateTime64(9, \'UTC\') NO \N +dt_tz1 DateTime(\'UTC\') NO \N +dt_tz2 DateTime(\'Europe/Amsterdam\') NO \N +enm Enum8(\'hallo\' = 1, \'welt\' = 2) NO \N +f32 Float32 NO \N +f64 Float64 NO \N +fs FixedString(3) NO \N +i128 Int128 NO \N +i16 Int16 NO \N +i256 Int256 NO \N +i32 Int32 NO \N +i64 Int64 NO \N +i8 Int8 NO \N +ip4 IPv4 NO \N +ip6 IPv6 NO \N +lfs LowCardinality(FixedString(3)) NO \N +lnfs LowCardinality(Nullable(FixedString(3))) YES \N +lns LowCardinality(Nullable(String)) YES \N +ls LowCardinality(String) NO \N +m Map(Int32, String) NO \N +m_complex Map(Int32, Map(Int32, LowCardinality(Nullable(String)))) NO \N +mpg MultiPolygon NO \N +ndt64_tz Nullable(DateTime64(3, \'Asia/Shanghai\')) YES \N +ndt_tz Nullable(DateTime64(3)) YES \N +nested.col1 Array(String) NO \N +nested.col2 Array(UInt32) NO \N +nfs Nullable(FixedString(3)) YES \N +ns Nullable(String) YES \N +o Object(\'json\') NO \N +p Point NO \N +pg Polygon NO \N +r Ring NO \N +s String NO \N +sagg SimpleAggregateFunction(sum, Float64) NO \N +t Tuple(Int32, String, Nullable(String), LowCardinality(String), LowCardinality(Nullable(String)), Tuple(Int32, String)) NO \N +ui128 UInt128 NO \N +ui16 UInt16 NO \N +ui256 UInt256 NO \N +ui32 UInt32 NO \N +ui64 UInt64 NO \N +ui8 UInt8 NO \N +uuid UUID NO \N +-- SHOW COLUMNS with use_mysql_types_in_show_columns = 1 +a TEXT NO \N +agg TEXT NO \N +b TINYINT NO \N +d DATE NO \N +d32 DATE NO \N +dec128 DECIMAL(38, 2) NO \N +dec128_native DECIMAL(35, 30) NO \N +dec128_text TEXT NO \N +dec256 TEXT NO \N +dec256_native DECIMAL(65, 2) NO \N +dec256_text TEXT NO \N +dec32 DECIMAL(9, 2) NO \N +dec64 DECIMAL(18, 2) NO \N +dt DATETIME NO \N +dt64 DATETIME NO \N +dt64_3_tz1 DATETIME NO \N +dt64_3_tz2 DATETIME NO \N +dt64_6 DATETIME NO \N +dt64_9 DATETIME NO \N +dt_tz1 DATETIME NO \N +dt_tz2 DATETIME NO \N +enm TEXT NO \N +f32 FLOAT NO \N +f64 DOUBLE NO \N +fs TEXT NO \N +i128 TEXT NO \N +i16 SMALLINT NO \N +i256 TEXT NO \N +i32 INTEGER NO \N +i64 BIGINT NO \N +i8 TINYINT NO \N +ip4 TEXT NO \N +ip6 TEXT NO \N +lfs TEXT NO \N +lnfs TEXT YES \N +lns BLOB YES \N +ls BLOB NO \N +m JSON NO \N +m_complex JSON NO \N +mpg TEXT NO \N +ndt64_tz DATETIME YES \N +ndt_tz DATETIME YES \N +nested.col1 TEXT NO \N +nested.col2 TEXT NO \N +nfs TEXT YES \N +ns BLOB YES \N +o JSON NO \N +p TEXT NO \N +pg TEXT NO \N +r TEXT NO \N +s BLOB NO \N +sagg TEXT NO \N +t JSON NO \N +ui128 TEXT NO \N +ui16 SMALLINT UNSIGNED NO \N +ui256 TEXT NO \N +ui32 INTEGER UNSIGNED NO \N +ui64 BIGINT UNSIGNED NO \N +ui8 TINYINT UNSIGNED NO \N +uuid CHAR NO \N diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql index e93bb8ded34..5947159512d 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql @@ -1,5 +1,5 @@ -- Tags: no-fasttest, no-parallel --- no-fasttest: json type needs rapidjson library +-- no-fasttest: json type needs rapidjson library, geo types need s2 geometry -- no-parallel: can't provide currentDatabase() to SHOW COLUMNS -- Tests setting 'use_mysql_types_in_show_columns' in SHOW COLUMNS and SELECTs on system.columns @@ -9,55 +9,73 @@ DROP TABLE IF EXISTS tab; SET allow_suspicious_low_cardinality_types=1; SET allow_experimental_object_type=1; -CREATE TABLE tab +CREATE OR REPLACE TABLE tab ( - uint8 UInt8, - uint16 UInt16, - uint32 UInt32, - uint64 UInt64, - uint128 UInt128, - uint256 UInt256, - int8 Int8, - int16 Int16, - int32 Int32, - int64 Int64, - int128 Int128, - int256 Int256, - nint32 Nullable(Int32), - float32 Float32, - float64 Float64, - decimal_value Decimal(10, 2), - boolean_value UInt8, - string_value String, - fixed_string_value FixedString(10), - date_value Date, - date32_value Date32, - datetime_value DateTime, - datetime64_value DateTime64(3), - json_value JSON, - uuid_value UUID, - enum_value Enum8('apple' = 1, 'banana' = 2, 'orange' = 3), - low_cardinality LowCardinality(String), - low_cardinality_date LowCardinality(DateTime), - aggregate_function AggregateFunction(sum, Int32), - array_value Array(Int32), - map_value Map(String, Int32), - tuple_value Tuple(Int32, String), - nullable_value Nullable(Int32), - ipv4_value IPv4, - ipv6_value IPv6, - nested Nested - ( - nested_int Int32, - nested_string String - ) -) ENGINE = MergeTree -ORDER BY uint64; + i8 Int8, + i16 Int16, + i32 Int32, + i64 Int64, + i128 Int128, + i256 Int256, + ui8 UInt8, + ui16 UInt16, + ui32 UInt32, + ui64 UInt64, + ui128 UInt128, + ui256 UInt256, + f32 Float32, + f64 Float64, + dec32 Decimal32(2), + dec64 Decimal64(2), + dec128 Decimal128(2), + dec128_native Decimal(35, 30), + dec128_text Decimal(35, 31), + dec256 Decimal256(2), + dec256_native Decimal(65, 2), + dec256_text Decimal(66, 2), + p Point, + r Ring, + pg Polygon, + mpg MultiPolygon, + b Bool, + s String, + fs FixedString(3), + uuid UUID, + d Date, + d32 Date32, + dt DateTime, + dt_tz1 DateTime('UTC'), + dt_tz2 DateTime('Europe/Amsterdam'), + dt64 DateTime64(3), + dt64_3_tz1 DateTime64(3, 'UTC'), + dt64_3_tz2 DateTime64(3, 'Asia/Shanghai'), + dt64_6 DateTime64(6, 'UTC'), + dt64_9 DateTime64(9, 'UTC'), + enm Enum('hallo' = 1, 'welt' = 2), + agg AggregateFunction(uniq, UInt64), + sagg SimpleAggregateFunction(sum, Double), + a Array(String), + o JSON, + t Tuple(Int32, String, Nullable(String), LowCardinality(String), LowCardinality(Nullable(String)), Tuple(Int32, String)), + m Map(Int32, String), + m_complex Map(Int32, Map(Int32, LowCardinality(Nullable(String)))), + nested Nested (col1 String, col2 UInt32), + ip4 IPv4, + ip6 IPv6, + ns Nullable(String), + nfs Nullable(FixedString(3)), + ndt_tz Nullable(DateTime64(3)), + ndt64_tz Nullable(DateTime64(3, 'Asia/Shanghai')), + ls LowCardinality(String), + lfs LowCardinality(FixedString(3)), + lns LowCardinality(Nullable(String)), + lnfs LowCardinality(Nullable(FixedString(3))), +) ENGINE Memory; +SELECT '-- SHOW COLUMNS with use_mysql_types_in_show_columns = 0'; SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 0; + +SELECT '-- SHOW COLUMNS with use_mysql_types_in_show_columns = 1'; SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 1; -SELECT name, type FROM system.columns WHERE database = currentDatabase() AND table = 'tab' SETTINGS use_mysql_types_in_show_columns = 0; -SELECT name, type FROM system.columns WHERE database = currentDatabase() AND table = 'tab' SETTINGS use_mysql_types_in_show_columns = 1; - DROP TABLE tab; From a7c7a25e0fc2dcfbfe6836684bf185724c08f4d5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 11 Oct 2023 15:49:16 +0000 Subject: [PATCH 142/634] Fix a leftover --- src/Interpreters/InterpreterShowColumnsQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index c5fbaf55e4a..78116a2d428 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -24,7 +24,7 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() { const auto & query = query_ptr->as(); - [[maybe_unused]] const bool use_mysql_types = getContext()->getSettingsRef().use_mysql_types_in_show_columns; + const bool use_mysql_types = getContext()->getSettingsRef().use_mysql_types_in_show_columns; WriteBufferFromOwnString buf_database; String resolved_database = getContext()->resolveDatabase(query.database); From db75a41d107fdb75d89f1b4bc9e5fa59107ea5ed Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 11 Oct 2023 15:50:13 +0000 Subject: [PATCH 143/634] + comment --- src/Interpreters/InterpreterShowColumnsQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index 78116a2d428..fe8ac63c214 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -38,6 +38,7 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() String rewritten_query; if (use_mysql_types) /// Cheapskate SQL-based mapping from native types to MySQL types, see https://dev.mysql.com/doc/refman/8.0/en/data-types.html + /// Only used with setting 'use_mysql_types_in_show_columns = 1' /// Known issues: /// - Enums are translated to TEXT rewritten_query += R"( From d00ed1b856dc03f58059dec228392ac4cc3131b3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 11 Oct 2023 15:51:51 +0000 Subject: [PATCH 144/634] Minor test fixup --- .../0_stateless/02775_show_columns_mysql_compatibility.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql index 5947159512d..b31ce720d36 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql @@ -9,7 +9,7 @@ DROP TABLE IF EXISTS tab; SET allow_suspicious_low_cardinality_types=1; SET allow_experimental_object_type=1; -CREATE OR REPLACE TABLE tab +CREATE TABLE tab ( i8 Int8, i16 Int16, From f9033bdf319a6086ebce463879d116cc7ec5555b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 11 Oct 2023 13:17:12 +0300 Subject: [PATCH 145/634] Improve performance of external aggregation with a lot of temporary files --- ...gingAggregatedMemoryEfficientTransform.cpp | 133 +++++++++++------- ...ergingAggregatedMemoryEfficientTransform.h | 8 +- 2 files changed, 84 insertions(+), 57 deletions(-) diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp index a50bdff0de9..d8305d71adb 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp @@ -20,37 +20,9 @@ GroupingAggregatedTransform::GroupingAggregatedTransform( , num_inputs(num_inputs_) , params(std::move(params_)) , last_bucket_number(num_inputs, -1) - , read_from_input(num_inputs, false) { } -void GroupingAggregatedTransform::readFromAllInputs() -{ - auto in = inputs.begin(); - read_from_all_inputs = true; - - for (size_t i = 0; i < num_inputs; ++i, ++in) - { - if (in->isFinished()) - continue; - - if (read_from_input[i]) - continue; - - in->setNeeded(); - - if (!in->hasData()) - { - read_from_all_inputs = false; - continue; - } - - auto chunk = in->pull(); - read_from_input[i] = true; - addChunk(std::move(chunk), i); - } -} - void GroupingAggregatedTransform::pushData(Chunks chunks, Int32 bucket, bool is_overflows) { auto & output = outputs.front(); @@ -119,7 +91,7 @@ bool GroupingAggregatedTransform::tryPushOverflowData() return true; } -IProcessor::Status GroupingAggregatedTransform::prepare() +IProcessor::Status GroupingAggregatedTransform::prepare(const PortNumbers & updated_input_ports, const PortNumbers &) { /// Check can output. auto & output = outputs.front(); @@ -137,30 +109,36 @@ IProcessor::Status GroupingAggregatedTransform::prepare() /// Read first time from each input to understand if we have two-level aggregation. if (!read_from_all_inputs) { - readFromAllInputs(); - if (!read_from_all_inputs) + read_from_all_inputs = true; + auto in = inputs.begin(); + index_to_input.resize(num_inputs); + + for (size_t i = 0; i < num_inputs; ++i, ++in) + { + index_to_input[i] = in; + + if (in->isFinished()) + continue; + + in->setNeeded(); + + if (!in->hasData()) + { + wait_input_ports_numbers.insert(i); + continue; + } + + auto chunk = in->pull(); + addChunk(std::move(chunk), i); + } + + if (has_two_level && !single_level_chunks.empty()) + return Status::Ready; + + if (!wait_input_ports_numbers.empty()) return Status::NeedData; } - /// Convert single level to two levels if have two-level input. - if (has_two_level && !single_level_chunks.empty()) - return Status::Ready; - - /// Check can push (to avoid data caching). - if (!output.canPush()) - { - for (auto & input : inputs) - input.setNotNeeded(); - - return Status::PortFull; - } - - bool pushed_to_output = false; - - /// Output if has data. - if (has_two_level) - pushed_to_output = tryPushTwoLevelData(); - auto need_input = [this](size_t input_num) { if (last_bucket_number[input_num] < current_bucket) @@ -169,6 +147,51 @@ IProcessor::Status GroupingAggregatedTransform::prepare() return expect_several_chunks_for_single_bucket_per_source && last_bucket_number[input_num] == current_bucket; }; + if (!wait_input_ports_numbers.empty()) + { + for (const auto & updated_input_port_number : updated_input_ports) + { + auto & input = index_to_input[updated_input_port_number]; + // input->setNeeded(); + + if (!input->hasData()) + { + wait_input_ports_numbers.erase(updated_input_port_number); + continue; + } + + auto chunk = input->pull(); + addChunk(std::move(chunk), updated_input_port_number); + + if (!input->isFinished() && need_input(updated_input_port_number)) + continue; + + wait_input_ports_numbers.erase(updated_input_port_number); + } + + if (!output.canPush()) + return Status::PortFull; + + if (has_two_level && !single_level_chunks.empty()) + return Status::Ready; + + if (!wait_input_ports_numbers.empty()) + return Status::NeedData; + } + + if (!output.canPush()) + return Status::PortFull; + + /// Convert single level to two levels if have two-level input. + if (has_two_level && !single_level_chunks.empty()) + return Status::Ready; + + bool pushed_to_output = false; + + /// Output if has data. + if (has_two_level) + pushed_to_output = tryPushTwoLevelData(); + /// Read next bucket if can. for (; ; ++current_bucket) { @@ -190,6 +213,7 @@ IProcessor::Status GroupingAggregatedTransform::prepare() if (!in->hasData()) { + wait_input_ports_numbers.insert(input_num); need_data = true; continue; } @@ -197,13 +221,16 @@ IProcessor::Status GroupingAggregatedTransform::prepare() auto chunk = in->pull(); addChunk(std::move(chunk), input_num); - if (has_two_level && !single_level_chunks.empty()) - return Status::Ready; - if (!in->isFinished() && need_input(input_num)) + { + wait_input_ports_numbers.insert(input_num); need_data = true; + } } + if (has_two_level && !single_level_chunks.empty()) + return Status::Ready; + if (finished) { all_inputs_finished = true; diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h index 8ff31782879..7fa9947495a 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h @@ -1,4 +1,5 @@ #pragma once + #include #include #include @@ -67,7 +68,7 @@ public: void allowSeveralChunksForSingleBucketPerSource() { expect_several_chunks_for_single_bucket_per_source = true; } protected: - Status prepare() override; + Status prepare(const PortNumbers & updated_input_ports, const PortNumbers &) override; void work() override; private: @@ -84,15 +85,14 @@ private: bool all_inputs_finished = false; bool read_from_all_inputs = false; - std::vector read_from_input; + std::vector index_to_input; + std::unordered_set wait_input_ports_numbers; /// If we aggregate partitioned data several chunks might be produced for the same bucket: one for each partition. bool expect_several_chunks_for_single_bucket_per_source = true; /// Add chunk read from input to chunks_map, overflow_chunks or single_level_chunks according to it's chunk info. void addChunk(Chunk chunk, size_t input); - /// Read from all inputs first chunk. It is needed to detect if any source has two-level aggregation. - void readFromAllInputs(); /// Push chunks if all inputs has single level. bool tryPushSingleLevelData(); /// Push chunks from ready bucket if has one. From ae94055a29d9ae3f19b2460de26313677c9dad19 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 11 Oct 2023 17:51:59 +0300 Subject: [PATCH 146/634] Updated implementation --- ...gingAggregatedMemoryEfficientTransform.cpp | 43 +++++-------------- ...ergingAggregatedMemoryEfficientTransform.h | 5 ++- 2 files changed, 13 insertions(+), 35 deletions(-) diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp index d8305d71adb..a92e2253314 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp @@ -106,37 +106,14 @@ IProcessor::Status GroupingAggregatedTransform::prepare(const PortNumbers & upda return Status::Finished; } - /// Read first time from each input to understand if we have two-level aggregation. - if (!read_from_all_inputs) + if (!initialized_index_to_input) { - read_from_all_inputs = true; + initialized_index_to_input = true; auto in = inputs.begin(); index_to_input.resize(num_inputs); for (size_t i = 0; i < num_inputs; ++i, ++in) - { index_to_input[i] = in; - - if (in->isFinished()) - continue; - - in->setNeeded(); - - if (!in->hasData()) - { - wait_input_ports_numbers.insert(i); - continue; - } - - auto chunk = in->pull(); - addChunk(std::move(chunk), i); - } - - if (has_two_level && !single_level_chunks.empty()) - return Status::Ready; - - if (!wait_input_ports_numbers.empty()) - return Status::NeedData; } auto need_input = [this](size_t input_num) @@ -151,9 +128,10 @@ IProcessor::Status GroupingAggregatedTransform::prepare(const PortNumbers & upda { for (const auto & updated_input_port_number : updated_input_ports) { - auto & input = index_to_input[updated_input_port_number]; - // input->setNeeded(); + if (!wait_input_ports_numbers.contains(updated_input_port_number)) + continue; + auto & input = index_to_input[updated_input_port_number]; if (!input->hasData()) { wait_input_ports_numbers.erase(updated_input_port_number); @@ -169,18 +147,17 @@ IProcessor::Status GroupingAggregatedTransform::prepare(const PortNumbers & upda wait_input_ports_numbers.erase(updated_input_port_number); } - if (!output.canPush()) - return Status::PortFull; - - if (has_two_level && !single_level_chunks.empty()) - return Status::Ready; - if (!wait_input_ports_numbers.empty()) return Status::NeedData; } if (!output.canPush()) + { + for (auto & input : inputs) + input.setNotNeeded(); + return Status::PortFull; + } /// Convert single level to two levels if have two-level input. if (has_two_level && !single_level_chunks.empty()) diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h index 7fa9947495a..77ee3034ffc 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -84,9 +85,9 @@ private: bool has_two_level = false; bool all_inputs_finished = false; - bool read_from_all_inputs = false; + bool initialized_index_to_input = false; std::vector index_to_input; - std::unordered_set wait_input_ports_numbers; + HashSet wait_input_ports_numbers; /// If we aggregate partitioned data several chunks might be produced for the same bucket: one for each partition. bool expect_several_chunks_for_single_bucket_per_source = true; From d4f9e0de1290acf875928e6c62fc78aa2b99b90a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 11 Oct 2023 17:54:32 +0300 Subject: [PATCH 147/634] Added performance test --- tests/performance/aggregation_external.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) create mode 100644 tests/performance/aggregation_external.xml diff --git a/tests/performance/aggregation_external.xml b/tests/performance/aggregation_external.xml new file mode 100644 index 00000000000..68755a5de4a --- /dev/null +++ b/tests/performance/aggregation_external.xml @@ -0,0 +1,10 @@ + + + 30 + 10485760 + + + SELECT number, count() FROM numbers_mt(5000000) GROUP BY number FORMAT Null; + SELECT number, count() FROM numbers_mt(15000000) GROUP BY number FORMAT Null; + SELECT number, count() FROM numbers_mt(30000000) GROUP BY number FORMAT Null; + From a6f894b99e9f5c5389033f21ab679e901861c79d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 11 Oct 2023 19:00:55 +0300 Subject: [PATCH 148/634] Fixed build --- src/Common/HashTable/HashTable.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 2c22ac39949..0ac00817f85 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -1273,6 +1273,10 @@ public: return !buf[place_value].isZero(*this); } + bool ALWAYS_INLINE contains(const Key & x) const + { + return has(x); + } void write(DB::WriteBuffer & wb) const { From 381ae7292ed13f051359c407f7845e0258a24d60 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 11 Oct 2023 16:12:31 +0000 Subject: [PATCH 149/634] Incorporate feedback --- .../02775_show_columns_mysql_compatibility.reference | 4 ++-- .../0_stateless/02775_show_columns_mysql_compatibility.sql | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference index 4df75c2bfc9..6613e4f170e 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference @@ -39,8 +39,8 @@ ls LowCardinality(String) NO \N m Map(Int32, String) NO \N m_complex Map(Int32, Map(Int32, LowCardinality(Nullable(String)))) NO \N mpg MultiPolygon NO \N +ndt64 Nullable(DateTime64(3)) YES \N ndt64_tz Nullable(DateTime64(3, \'Asia/Shanghai\')) YES \N -ndt_tz Nullable(DateTime64(3)) YES \N nested.col1 Array(String) NO \N nested.col2 Array(UInt32) NO \N nfs Nullable(FixedString(3)) YES \N @@ -100,8 +100,8 @@ ls BLOB NO \N m JSON NO \N m_complex JSON NO \N mpg TEXT NO \N +ndt64 DATETIME YES \N ndt64_tz DATETIME YES \N -ndt_tz DATETIME YES \N nested.col1 TEXT NO \N nested.col2 TEXT NO \N nfs TEXT YES \N diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql index b31ce720d36..34c034fa77f 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql @@ -52,8 +52,8 @@ CREATE TABLE tab dt64_6 DateTime64(6, 'UTC'), dt64_9 DateTime64(9, 'UTC'), enm Enum('hallo' = 1, 'welt' = 2), - agg AggregateFunction(uniq, UInt64), - sagg SimpleAggregateFunction(sum, Double), + agg AggregateFunction(uniq, UInt64), + sagg SimpleAggregateFunction(sum, Double), a Array(String), o JSON, t Tuple(Int32, String, Nullable(String), LowCardinality(String), LowCardinality(Nullable(String)), Tuple(Int32, String)), @@ -64,7 +64,7 @@ CREATE TABLE tab ip6 IPv6, ns Nullable(String), nfs Nullable(FixedString(3)), - ndt_tz Nullable(DateTime64(3)), + ndt64 Nullable(DateTime64(3)), ndt64_tz Nullable(DateTime64(3, 'Asia/Shanghai')), ls LowCardinality(String), lfs LowCardinality(FixedString(3)), From 47ff13056f0a6dbff8c61389f840f4c55216b8a8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 11 Oct 2023 14:18:23 +0200 Subject: [PATCH 150/634] Avoid retrying keeper operations on INSERT after table shutdown Also note, that it should check shutdown_prepared_called, instead of shutdown_called, since shutdown of the database/server firstly calls flushAndPrepareForShutdown() for every storage. And if you have Buffer table that writes to your ReplicatedMergeTree it will not be finished until all retries will be made. Otherwise it can take significantly long time to finish. Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 3f0366a8612..c91f99ce4fd 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -167,6 +167,16 @@ size_t ReplicatedMergeTreeSinkImpl::checkQuorumPrecondition(const { zookeeper->setKeeper(storage.getZooKeeper()); + /// Stop retries if in shutdown, note that we need to check + /// shutdown_prepared_called, not shutdown_called, since the table + /// will be marked as readonly after calling + /// StorageReplicatedMergeTree::flushAndPrepareForShutdown(), and + /// the final shutdown() can not be called if you have Buffer table + /// that writes to this replicated table, until all the retries + /// will be made. + if (storage.is_readonly && storage.shutdown_prepared_called) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to shutdown: replica_path={}", storage.replica_path); + quorum_info.status_path = storage.zookeeper_path + "/quorum/status"; Strings replicas = zookeeper->getChildren(fs::path(storage.zookeeper_path) / "replicas"); @@ -564,7 +574,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: if (storage.is_readonly) { /// stop retries if in shutdown - if (storage.shutdown_called) + if (storage.shutdown_prepared_called) throw Exception( ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to shutdown: replica_path={}", storage.replica_path); From 0879cf9ce79a4639147836bd9bc11b9252b3bc7b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 11 Oct 2023 18:59:54 +0200 Subject: [PATCH 151/634] Fix test and try to fix queries not being run --- .../0_stateless/02784_parallel_replicas_automatic_decision.sh | 2 +- .../02784_parallel_replicas_automatic_decision_join.sh | 2 +- tests/queries/0_stateless/02888_obsolete_settings.reference | 1 + 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 6620495222e..f8249785b67 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -54,7 +54,7 @@ function run_query_with_pure_parallel_replicas () { --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica "$2" \ --send_logs_level "trace" \ - |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' + 2>&1 | grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' } query_id_base="02784_automatic_parallel_replicas-$CLICKHOUSE_DATABASE" diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh index a2ed41bebe3..36d92eda896 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh @@ -69,7 +69,7 @@ function run_query_with_pure_parallel_replicas () { --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica "$2" \ --send_logs_level "trace" \ - |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' + 2>&1 | grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' } query_id_base="02784_automatic_parallel_replicas_join-$CLICKHOUSE_DATABASE" diff --git a/tests/queries/0_stateless/02888_obsolete_settings.reference b/tests/queries/0_stateless/02888_obsolete_settings.reference index 90313977ab4..d16fe07160e 100644 --- a/tests/queries/0_stateless/02888_obsolete_settings.reference +++ b/tests/queries/0_stateless/02888_obsolete_settings.reference @@ -39,6 +39,7 @@ multiple_joins_rewriter_version odbc_max_field_size optimize_duplicate_order_by_and_distinct optimize_fuse_sum_count_avg +parallel_replicas_min_number_of_granules_to_enable partial_merge_join_optimizations query_plan_optimize_projection replication_alter_columns_timeout From 125250b4fdd4759c49f2bbf4f1962ff020883c20 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Wed, 11 Oct 2023 17:24:04 +0000 Subject: [PATCH 152/634] add k-morozov to trusted --- tests/ci/lambda_shared_package/lambda_shared/pr.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index 82fc53b9356..c09b259ffc0 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -63,6 +63,7 @@ TRUSTED_CONTRIBUTORS = { "MikhailBurdukov", "tsolodov", # ClickHouse Employee "kitaisreal", + "k-morozov", # Konstantin Morozov, Yandex Cloud ] } From c04a33a70a3475878aba57e296f95795e9bf009e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 11 Oct 2023 20:38:01 +0300 Subject: [PATCH 153/634] Update pr.py --- .../lambda_shared_package/lambda_shared/pr.py | 41 +++++-------------- 1 file changed, 10 insertions(+), 31 deletions(-) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index c09b259ffc0..60e6dc96440 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -10,58 +10,37 @@ TRUSTED_CONTRIBUTORS = { e.lower() for e in [ "achimbab", - "adevyatova ", # DOCSUP - "Algunenano", # Raúl Marín, Tinybird + "Algunenano", # Raúl Marín, ClickHouse, Inc "amosbird", - "AnaUvarova", # DOCSUP - "anauvarova", # technical writer, Yandex - "annvsh", # technical writer, Yandex - "atereh", # DOCSUP "azat", - "bharatnc", # Newbie, but already with many contributions. + "bharatnc", # Many contributions. "bobrik", # Seasoned contributor, CloudFlare - "BohuTANG", - "codyrobert", # Flickerbox engineer - "cwurm", # Employee - "damozhaeva", # DOCSUP - "den-crane", - "flickerbox-tom", # Flickerbox - "gyuton", # DOCSUP + "cwurm", # ClickHouse, Inc + "den-crane", # Documentation contributor "hagen1778", # Roman Khavronenko, seasoned contributor "hczhcz", "hexiaoting", # Seasoned contributor "ildus", # adjust, ex-pgpro "javisantana", # a Spanish ClickHouse enthusiast, ex-Carto - "ka1bi4", # DOCSUP - "kirillikoff", # DOCSUP "kreuzerkrieg", - "lehasm", # DOCSUP - "michon470", # DOCSUP "nikvas0", - "nvartolomei", - "olgarev", # DOCSUP - "otrazhenia", # Yandex docs contractor - "pdv-ru", # DOCSUP - "podshumok", # cmake expert from QRator Labs - "s-mx", # Maxim Sabyanin, former employee, present contributor - "sevirov", # technical writer, Yandex + "nvartolomei", # Seasoned contributor, CloudFlare "spongedu", # Seasoned contributor "taiyang-li", "ucasFL", # Amos Bird's friend - "vdimir", # Employee - "vzakaznikov", + "vdimir", # ClickHouse, Inc "YiuRULE", "zlobober", # Developer of YT "ilejn", # Arenadata, responsible for Kerberized Kafka - "thomoco", # ClickHouse + "thomoco", # ClickHouse, Inc "BoloniniD", # Seasoned contributor, HSE "tonickkozlov", # Cloudflare - "tylerhannan", # ClickHouse Employee + "tylerhannan", # ClickHouse, Inc "myrrc", # Mike Kot, DoubleCloud - "thevar1able", # ClickHouse Employee + "thevar1able", # ClickHouse, Inc "aalexfvk", "MikhailBurdukov", - "tsolodov", # ClickHouse Employee + "tsolodov", # ClickHouse, Inc "kitaisreal", "k-morozov", # Konstantin Morozov, Yandex Cloud ] From f4ccf9f94ff6ba4be1bd09bee8a3859a891fdc85 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 11 Oct 2023 20:38:19 +0300 Subject: [PATCH 154/634] Update pr.py --- tests/ci/lambda_shared_package/lambda_shared/pr.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index 60e6dc96440..ee7b5f8129f 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -9,7 +9,7 @@ from typing import Tuple TRUSTED_CONTRIBUTORS = { e.lower() for e in [ - "achimbab", + "achimbab", # Kakao corp "Algunenano", # Raúl Marín, ClickHouse, Inc "amosbird", "azat", From 8f541239014aa71455235a7a6e4c3146f0251dca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 11 Oct 2023 20:38:37 +0300 Subject: [PATCH 155/634] Update pr.py --- tests/ci/lambda_shared_package/lambda_shared/pr.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index ee7b5f8129f..4872ecb4d59 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -12,7 +12,7 @@ TRUSTED_CONTRIBUTORS = { "achimbab", # Kakao corp "Algunenano", # Raúl Marín, ClickHouse, Inc "amosbird", - "azat", + "azat", # SEMRush "bharatnc", # Many contributions. "bobrik", # Seasoned contributor, CloudFlare "cwurm", # ClickHouse, Inc From e3719f88b2bf148a0c0f51ce457a59e9ac31cf83 Mon Sep 17 00:00:00 2001 From: Daniel Byta Date: Wed, 11 Oct 2023 20:42:49 +0300 Subject: [PATCH 156/634] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index b43f9cab0f1..1494745c0c9 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1184,6 +1184,7 @@ editDistance byteHammingDistance jaccardIndex levenshteinDistance +clickcache cLoki caConfig cacheSessions From 6a9e19c29f39a0e7d861472e78bd8b6b5c2f7de0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Oct 2023 19:46:32 +0200 Subject: [PATCH 157/634] Fxi distributed race condition --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 14 ++++---- .../MergeTree/ReplicatedMergeTreeQueue.h | 4 +-- src/Storages/StorageReplicatedMergeTree.cpp | 36 +++++++++++++++++-- 3 files changed, 43 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 9f726a7b7c9..69cfd54fa69 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -558,7 +558,7 @@ bool ReplicatedMergeTreeQueue::removeFailedQuorumPart(const MergeTreePartInfo & return virtual_parts.remove(part_info); } -int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback, PullLogsReason reason) +std::pair ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback, PullLogsReason reason) { std::lock_guard lock(pull_logs_to_queue_mutex); @@ -590,7 +590,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper /// in the queue. /// With this we ensure that if you read the log state L1 and then the state of mutations M1, /// then L1 "happened-before" M1. - updateMutations(zookeeper); + int32_t mutations_version = updateMutations(zookeeper); if (index_str.empty()) { @@ -719,7 +719,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper storage.background_operations_assignee.trigger(); } - return stat.version; + return std::pair{stat.version, mutations_version}; } @@ -858,11 +858,12 @@ ActiveDataPartSet getPartNamesToMutate( } -void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallbackPtr watch_callback) +int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallbackPtr watch_callback) { std::lock_guard lock(update_mutations_mutex); - Strings entries_in_zk = zookeeper->getChildrenWatch(fs::path(zookeeper_path) / "mutations", nullptr, watch_callback); + Coordination::Stat mutations_stat; + Strings entries_in_zk = zookeeper->getChildrenWatch(fs::path(zookeeper_path) / "mutations", &mutations_stat, watch_callback); StringSet entries_in_zk_set(entries_in_zk.begin(), entries_in_zk.end()); /// Compare with the local state, delete obsolete entries and determine which new entries to load. @@ -977,6 +978,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C if (some_mutations_are_probably_done) storage.mutations_finalizing_task->schedule(); } + return mutations_stat.version; } @@ -2211,7 +2213,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( committing_blocks = std::make_shared(getCommittingBlocks(zookeeper, queue.zookeeper_path, queue.log)); - merges_version = queue_.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::MERGE_PREDICATE); + std::tie(merges_version, std::ignore) = queue_.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::MERGE_PREDICATE); { /// We avoid returning here a version to be used in a lightweight transaction. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 393e9cc48bc..4860d4df97f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -331,11 +331,11 @@ public: * Additionally loads mutations (so that the set of mutations is always more recent than the queue). * Return the version of "logs" node (that is updated for every merge/mutation/... added to the log) */ - int32_t pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback = {}, PullLogsReason reason = OTHER); + std::pair pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback = {}, PullLogsReason reason = OTHER); /// Load new mutation entries. If something new is loaded, schedule storage.merge_selecting_task. /// If watch_callback is not empty, will call it when new mutations appear in ZK. - void updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallbackPtr watch_callback = {}); + int32_t updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallbackPtr watch_callback = {}); /// Remove a mutation from ZooKeeper and from the local set. Returns the removed entry or nullptr /// if it could not be found. Called during KILL MUTATION query execution. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1d5bd1388f2..0485fe687e8 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5748,6 +5748,7 @@ void StorageReplicatedMergeTree::alter( return; } + auto ast_to_str = [](ASTPtr query) -> String { if (!query) @@ -5762,6 +5763,25 @@ void StorageReplicatedMergeTree::alter( while (true) { + bool pulled_queue = false; + std::optional maybe_mutations_version_after_logs_pull; + std::map unfinished_mutations; + for (const auto & command : commands) + { + if (command.isDropSomething()) + { + if (!pulled_queue) + { + auto [_, mutations_version] = queue.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::SYNC); + maybe_mutations_version_after_logs_pull.emplace(mutations_version); + unfinished_mutations = getUnfinishedMutationCommands(); + pulled_queue = true; + } + + checkDropCommandDoesntAffectInProgressMutations(command, unfinished_mutations, query_context); + } + } + /// Clear nodes from previous iteration alter_entry.emplace(); mutation_znode.reset(); @@ -5875,8 +5895,18 @@ void StorageReplicatedMergeTree::alter( mutation_entry.source_replica = replica_name; mutation_entry.commands = std::move(maybe_mutation_commands); - Coordination::Stat mutations_stat; - zookeeper->get(mutations_path, &mutations_stat); + int32_t mutations_version; + if (maybe_mutations_version_after_logs_pull.has_value()) + { + mutations_version = *maybe_mutations_version_after_logs_pull; + } + else + { + Coordination::Stat mutations_stat; + zookeeper->get(mutations_path, &mutations_stat); + mutations_version = mutations_stat.version; + } + partition_block_numbers_holder = allocateBlockNumbersInAffectedPartitions(mutation_entry.commands, query_context, zookeeper); @@ -5884,7 +5914,7 @@ void StorageReplicatedMergeTree::alter( mutation_entry.block_numbers = partition_block_numbers_holder.getBlockNumbers(); mutation_entry.create_time = time(nullptr); - ops.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_version)); mutation_path_idx = ops.size(); ops.emplace_back( zkutil::makeCreateRequest(fs::path(mutations_path) / "", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential)); From eeacdbae9c15554ca5c05f7e80165d4559ab9abd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 11 Oct 2023 19:46:58 +0200 Subject: [PATCH 158/634] Update replacingmergetree.md --- .../table-engines/mergetree-family/replacingmergetree.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md index 874a43a419f..6de818c130f 100644 --- a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md @@ -101,8 +101,6 @@ The row is deleted when `OPTIMIZE ... FINAL CLEANUP` or `OPTIMIZE ... FINAL` is No matter the operation on the data, the version must be increased. If two inserted rows have the same version number, the last inserted row is the one kept. -Always execute `OPTIMIZE ... FINAL CLEANUP` and `OPTIMIZE ... FINAL` to delete rows with `is_deleted=1` defined, especially when you wish to insert a new row with an older version. Otherwise, the new row with an older version will be replaced and not be persisted. - ::: Example: From 929e400e79ca11bc894d20ffdfa3338dfc016d47 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Oct 2023 23:17:15 +0200 Subject: [PATCH 159/634] Remove potentially flaky query --- .../queries/0_stateless/02597_column_delete_and_replication.sql | 2 -- .../queries/0_stateless/02597_column_update_and_replication.sql | 2 -- .../02597_column_update_tricy_expression_and_replication.sql | 2 -- .../02597_projection_materialize_and_replication.sql | 2 -- 4 files changed, 8 deletions(-) diff --git a/tests/queries/0_stateless/02597_column_delete_and_replication.sql b/tests/queries/0_stateless/02597_column_delete_and_replication.sql index 70dcb95a3a1..b0257f666d9 100644 --- a/tests/queries/0_stateless/02597_column_delete_and_replication.sql +++ b/tests/queries/0_stateless/02597_column_delete_and_replication.sql @@ -22,8 +22,6 @@ ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync = 2; ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; -SELECT * FROM system.mutations WHERE database=currentDatabase() AND table='test' AND NOT is_done; - select * from test format Null; DROP TABLE test; diff --git a/tests/queries/0_stateless/02597_column_update_and_replication.sql b/tests/queries/0_stateless/02597_column_update_and_replication.sql index f25c177ea38..42fe813f8a1 100644 --- a/tests/queries/0_stateless/02597_column_update_and_replication.sql +++ b/tests/queries/0_stateless/02597_column_update_and_replication.sql @@ -22,8 +22,6 @@ ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync = 2; ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; -SELECT * FROM system.mutations WHERE database=currentDatabase() AND table='test' AND NOT is_done; - select * from test format Null; DROP TABLE test; diff --git a/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql b/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql index ae80cf666f2..b07b3b54514 100644 --- a/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql +++ b/tests/queries/0_stateless/02597_column_update_tricy_expression_and_replication.sql @@ -22,8 +22,6 @@ ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync = 2; ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; -SELECT * FROM system.mutations WHERE database=currentDatabase() AND table='test' AND NOT is_done; - select * from test format Null; DROP TABLE test; diff --git a/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql b/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql index 65956dee985..031cb3cb6fb 100644 --- a/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql +++ b/tests/queries/0_stateless/02597_projection_materialize_and_replication.sql @@ -23,8 +23,6 @@ ALTER TABLE test DELETE where d = 'Hello' SETTINGS mutations_sync = 2; ALTER TABLE test DROP PROJECTION d_order SETTINGS mutations_sync = 2; -SELECT * FROM system.mutations WHERE database=currentDatabase() AND table='test' AND NOT is_done; - select * from test format Null; DROP TABLE test; From 360ec55c7ac291a13152b7cbbfc85465d10caca3 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 11 Oct 2023 23:30:38 +0200 Subject: [PATCH 160/634] update table ash --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 30a6dd2e9fa..4a3a24ac8cd 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -127,7 +127,7 @@ function setup_logs_replication do # Calculate hash of its structure. Note: 1 is the version of extra columns - increment it if extra columns are changed: hash=$(clickhouse-client --query " - SELECT sipHash64(1, groupArray((name, type))) + SELECT sipHash64(2, groupArray((name, type))) FROM (SELECT name, type FROM system.columns WHERE database = 'system' AND table = '$table' ORDER BY position) From 30fd4bee841eef3991b64d4aa9f31ba469ce2fd7 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 11 Oct 2023 15:27:16 -0700 Subject: [PATCH 161/634] Fix stack symbolizer parsing DW_FORM_ref_addr incorrectly and sometimes crashing (#55483) --- src/Common/Dwarf.cpp | 87 +++++++++++++++++++++++++------------------- src/Common/Dwarf.h | 21 +++-------- 2 files changed, 55 insertions(+), 53 deletions(-) diff --git a/src/Common/Dwarf.cpp b/src/Common/Dwarf.cpp index 0fa2727086a..a405f73e35e 100644 --- a/src/Common/Dwarf.cpp +++ b/src/Common/Dwarf.cpp @@ -971,6 +971,50 @@ Dwarf::Die Dwarf::getDieAtOffset(const CompilationUnit & cu, uint64_t offset) co return die; } +std::optional, uint64_t>> Dwarf::getReferenceAttribute( + const CompilationUnit & cu, const Die & die, uint64_t attr_name) const +{ + bool found = false; + uint64_t value; + uint64_t form; + forEachAttribute(cu, die, [&](const Attribute & attr) + { + if (attr.spec.name == attr_name) + { + found = true; + value = std::get(attr.attr_value); + form = attr.spec.form; + return false; + } + return true; + }); + if (!found) + return std::nullopt; + switch (form) + { + case DW_FORM_ref1: + case DW_FORM_ref2: + case DW_FORM_ref4: + case DW_FORM_ref8: + case DW_FORM_ref_udata: + return std::make_pair(std::nullopt, cu.offset + value); + + case DW_FORM_ref_addr: + return std::make_pair(findCompilationUnit(value), value); + + case DW_FORM_ref_sig8: + /// Currently we don't use this parser for types, so no need to support this. + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Type signatures are not supported (DIE at 0x{:x}, attr 0x{:x}).", die.offset, attr_name); + + case DW_FORM_ref_sup4: + case DW_FORM_ref_sup8: + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Supplementary object files are not supported (DIE at 0x{:x}, attr 0x{:x}).", die.offset, attr_name); + + default: + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Unexpected form of attribute 0x{:x}: 0x{:x} (DIE at 0x{:x}).", attr_name, form, die.offset); + } +} + /** * Find the @locationInfo for @address in the compilation unit represented * by the @sp .debug_info entry. @@ -1300,51 +1344,24 @@ void Dwarf::findInlinedSubroutineDieForAddress( location.file = line_vm.getFullFileName(*call_file); location.line = *call_line; - /// Something wrong with receiving debug info about inline. - /// If set to true we stop parsing DWARF. - bool die_for_inline_broken = false; - auto get_function_name = [&](const CompilationUnit & srcu, uint64_t die_offset) { - Die decl_die = getDieAtOffset(srcu, die_offset); - auto & die_to_look_for_name = decl_die; + Die die_to_look_for_name = getDieAtOffset(srcu, die_offset); - Die def_die; // Jump to the actual function definition instead of declaration for name // and line info. // DW_AT_specification: Incomplete, non-defining, or separate declaration // corresponding to a declaration - auto offset = getAttribute(srcu, decl_die, DW_AT_specification); - if (offset) + auto def = getReferenceAttribute(srcu, die_to_look_for_name, DW_AT_specification); + if (def.has_value()) { - /// FIXME: actually it's a bug in our DWARF parser. - /// - /// Most of the times compilation unit offset (srcu.offset) is some big number inside .debug_info (like 434782255). - /// Offset of DIE definition is some small relative number to srcu.offset (like 3518). - /// However in some unknown cases offset looks like global, non relative number (like 434672579) and in this - /// case we obviously doing something wrong parsing DWARF. - /// - /// What is important -- this bug? reproduces only with -flto=thin in release mode. - /// Also llvm-dwarfdump --verify ./clickhouse says that our DWARF is ok, so it's another prove - /// that we just doing something wrong. - /// - /// FIXME: Currently we just give up parsing DWARF for inlines when we got into this situation. - if (srcu.offset + offset.value() >= info_.size()) - { - die_for_inline_broken = true; - } - else - { - def_die = getDieAtOffset(srcu, srcu.offset + offset.value()); - die_to_look_for_name = def_die; - } + auto [def_cu, def_offset] = std::move(def.value()); + const CompilationUnit & def_cu_ref = def_cu.has_value() ? def_cu.value() : srcu; + die_to_look_for_name = getDieAtOffset(def_cu_ref, def_offset); } std::string_view name; - if (die_for_inline_broken) - return name; - // The file and line will be set in the next inline subroutine based on // its DW_AT_call_file and DW_AT_call_line. forEachAttribute(srcu, die_to_look_for_name, [&](const Attribute & attr) @@ -1386,10 +1403,6 @@ void Dwarf::findInlinedSubroutineDieForAddress( ? get_function_name(cu, cu.offset + *abstract_origin) : get_function_name(findCompilationUnit(*abstract_origin), *abstract_origin); - /// FIXME: see comment above - if (die_for_inline_broken) - return false; - locations.push_back(location); findInlinedSubroutineDieForAddress(cu, child_die, line_vm, address, base_addr_cu, locations, max_size); diff --git a/src/Common/Dwarf.h b/src/Common/Dwarf.h index ef6364b6b18..3b717305d70 100644 --- a/src/Common/Dwarf.h +++ b/src/Common/Dwarf.h @@ -453,22 +453,11 @@ private: // Finds the Compilation Unit starting at offset. CompilationUnit findCompilationUnit(uint64_t targetOffset) const; - - template - std::optional getAttribute(const CompilationUnit & cu, const Die & die, uint64_t attr_name) const - { - std::optional result; - forEachAttribute(cu, die, [&](const Attribute & attr) - { - if (attr.spec.name == attr_name) - { - result = std::get(attr.attr_value); - return false; - } - return true; - }); - return result; - } + // Parses an attribute of "reference" form class, i.e. a reference to another DIE. + // Returns the unit containing the target DIE (nullopt if it's in the same unit as the source DIE) + // and the offset of the target DIE (relative to .debug_info, not to unit). + std::optional, uint64_t>> getReferenceAttribute( + const CompilationUnit & cu, const Die & die, uint64_t attr_name) const; // Check if the given address is in the range list at the given offset in .debug_ranges. bool isAddrInRangeList( From 8b47bf5e7ef41d2c4523f27901aa35a2b53cb679 Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Thu, 12 Oct 2023 09:58:42 +0800 Subject: [PATCH 162/634] fix CheckSytel --- programs/keeper-client/Commands.cpp | 2 -- tests/integration/helpers/keeper_utils.py | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index c7043baa33c..543ea8f4449 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -326,7 +326,6 @@ void FindBigFamily::execute(const ASTKeeperQuery * query, KeeperClient * client) auto children = client->zookeeper->getChildren(next_path); for (auto & child : children) child = next_path / child; - auto response = client->zookeeper->get(children); for (size_t i = 0; i < response.size(); ++i) @@ -505,7 +504,6 @@ void GetAllChildrenNumberCommand::execute(const ASTKeeperQuery * query, KeeperCl auto children = client->zookeeper->getChildren(next_path); for (auto & child : children) child = next_path / child; - auto response = client->zookeeper->get(children); for (size_t i = 0; i < response.size(); ++i) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 9135ab72c84..80b221811f7 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -148,7 +148,7 @@ class KeeperClient(object): def get_all_children_number(self, path: str, timeout: float = 60.0) -> str: return self.execute_query(f"get_all_children_number {path}", timeout) - + def reconfig( self, joining: tp.Optional[str], From 268ea30c41863ac7ac58d0bfa5e7afbc79d82e11 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 12 Oct 2023 03:48:09 +0000 Subject: [PATCH 163/634] Forbid create inverted index if setting not enabled --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- .../02895_forbid_create_inverted_index.reference | 0 .../0_stateless/02895_forbid_create_inverted_index.sql | 10 ++++++++++ 3 files changed, 12 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02895_forbid_create_inverted_index.reference create mode 100644 tests/queries/0_stateless/02895_forbid_create_inverted_index.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fd5d9f736cc..889dcfa537f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2893,12 +2893,12 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context queryToString(mutation_commands.ast())); } + commands.apply(new_metadata, getContext()); + if (commands.hasInvertedIndex(new_metadata) && !settings.allow_experimental_inverted_index) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental Inverted Index feature is not enabled (turn on setting 'allow_experimental_inverted_index')"); - commands.apply(new_metadata, getContext()); - /// Set of columns that shouldn't be altered. NameSet columns_alter_type_forbidden; diff --git a/tests/queries/0_stateless/02895_forbid_create_inverted_index.reference b/tests/queries/0_stateless/02895_forbid_create_inverted_index.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02895_forbid_create_inverted_index.sql b/tests/queries/0_stateless/02895_forbid_create_inverted_index.sql new file mode 100644 index 00000000000..9bfcf6b88f5 --- /dev/null +++ b/tests/queries/0_stateless/02895_forbid_create_inverted_index.sql @@ -0,0 +1,10 @@ +SET allow_experimental_inverted_index = 0; +CREATE OR REPLACE TABLE tab +( + `key` UInt64, + `str` String +) +ENGINE = MergeTree +ORDER BY key; + +ALTER TABLE tab ADD INDEX inv_idx(str) TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED } From 453de30ad64475b5d2efd3fe5e7da30112616aa6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 12 Oct 2023 06:04:43 +0200 Subject: [PATCH 164/634] Destroy fiber in case of exception in cancelBefore in AsyncTaskExecutor (#55516) * Destroy fiber in case of exception in cancelBefore in AsyncTaskExecutor * Update AsyncTaskExecutor.cpp --------- Co-authored-by: Alexey Milovidov --- src/Common/AsyncTaskExecutor.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Common/AsyncTaskExecutor.cpp b/src/Common/AsyncTaskExecutor.cpp index 923dad1d346..44781ab5b83 100644 --- a/src/Common/AsyncTaskExecutor.cpp +++ b/src/Common/AsyncTaskExecutor.cpp @@ -1,4 +1,6 @@ #include +#include + namespace DB { @@ -46,8 +48,10 @@ void AsyncTaskExecutor::cancel() { std::lock_guard guard(fiber_lock); is_cancelled = true; - cancelBefore(); - destroyFiber(); + { + SCOPE_EXIT({ destroyFiber(); }); + cancelBefore(); + } cancelAfter(); } From 4499dd3e2f5f515343c8a5751cf6a72afd178622 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Mon, 9 Oct 2023 19:03:17 +0800 Subject: [PATCH 165/634] modify style --- docs/en/sql-reference/statements/rename.md | 7 ++- src/Parsers/ParserRenameQuery.cpp | 2 +- ...891_rename_table_without_keyword.reference | 9 ++- .../02891_rename_table_without_keyword.sh | 57 +++++++++++++++++++ .../02891_rename_table_without_keyword.sql | 30 +++++++++- 5 files changed, 98 insertions(+), 7 deletions(-) create mode 100755 tests/queries/0_stateless/02891_rename_table_without_keyword.sh diff --git a/docs/en/sql-reference/statements/rename.md b/docs/en/sql-reference/statements/rename.md index a2d4b15df13..957f2238a84 100644 --- a/docs/en/sql-reference/statements/rename.md +++ b/docs/en/sql-reference/statements/rename.md @@ -16,7 +16,7 @@ The `RENAME` query is supported by the [Atomic](../../engines/database-engines/a **Syntax** ```sql -RENAME DATABASE|TABLE|DICTIONARY name TO new_name [,...] [ON CLUSTER cluster] +RENAME DATABASE|[TABLE|DICTIONARY] name TO new_name [,...] [ON CLUSTER cluster] ``` ## RENAME DATABASE @@ -48,6 +48,11 @@ RENAME TABLE [db1.]name1 TO [db2.]name2 [,...] [ON CLUSTER cluster] RENAME TABLE table_A TO table_A_bak, table_B TO table_B_bak; ``` +And you can use a simpler sql: +```sql +RENAME table_A TO table_A_bak, table_B TO table_B_bak; +``` + ## RENAME DICTIONARY Renames one or several dictionaries. This query can be used to move dictionaries between databases. diff --git a/src/Parsers/ParserRenameQuery.cpp b/src/Parsers/ParserRenameQuery.cpp index 56845d781b3..27f2ed1cd22 100644 --- a/src/Parsers/ParserRenameQuery.cpp +++ b/src/Parsers/ParserRenameQuery.cpp @@ -57,7 +57,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) node = query; return true; } - else if(s_rename_table.ignore(pos, expected)) + else if (s_rename_table.ignore(pos, expected)) ; else if (s_exchange_tables.ignore(pos, expected)) exchange = true; diff --git a/tests/queries/0_stateless/02891_rename_table_without_keyword.reference b/tests/queries/0_stateless/02891_rename_table_without_keyword.reference index 74c047b102a..3116f82ea08 100644 --- a/tests/queries/0_stateless/02891_rename_table_without_keyword.reference +++ b/tests/queries/0_stateless/02891_rename_table_without_keyword.reference @@ -1,3 +1,8 @@ r1 -r2 -r3 +r1_bak +r1 +r1_bak +r2_bak +test_dictionary +test_dictionary_2 +UNKNOWN_TABLE diff --git a/tests/queries/0_stateless/02891_rename_table_without_keyword.sh b/tests/queries/0_stateless/02891_rename_table_without_keyword.sh new file mode 100755 index 00000000000..8d9d73f6ab4 --- /dev/null +++ b/tests/queries/0_stateless/02891_rename_table_without_keyword.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +db_name=db_$(echo $RANDOM |md5sum |cut -c 1-4) + +$CLICKHOUSE_CLIENT --param_db="${db_name}" --multiquery \ + --query="DROP DATABASE IF EXISTS {db:Identifier}; + CREATE DATABASE {db:Identifier}; + CREATE TABLE IF NOT EXISTS {db:Identifier}.r1 (name String) Engine=Memory(); + SHOW TABLES FROM {db:Identifier}" + +$CLICKHOUSE_CLIENT --param_db="${db_name}" --multiquery \ + --query="RENAME TABLE {db:Identifier}.r1 TO {db:Identifier}.r1_bak; + SHOW TABLES FROM {db:Identifier};" + +$CLICKHOUSE_CLIENT --param_db="${db_name}" --multiquery \ + --query="RENAME {db:Identifier}.r1_bak TO {db:Identifier}.r1; + SHOW TABLES FROM {db:Identifier};" + +$CLICKHOUSE_CLIENT --param_db="${db_name}" --multiquery \ + --query="CREATE TABLE IF NOT EXISTS {db:Identifier}.r2 (name String) Engine=Memory(); + RENAME {db:Identifier}.r1 TO {db:Identifier}.r1_bak, {db:Identifier}.r2 TO {db:Identifier}.r2_bak; + SHOW TABLES FROM {db:Identifier};" + + +$CLICKHOUSE_CLIENT --param_db="${db_name}" --multiquery \ + --query="CREATE TABLE IF NOT EXISTS {db:Identifier}.source_table ( + id UInt64, + value String + ) ENGINE = Memory; + + CREATE DICTIONARY IF NOT EXISTS {db:Identifier}.test_dictionary + ( + id UInt64, + value String + ) + PRIMARY KEY id + SOURCE(CLICKHOUSE(TABLE '{db:Identifier}.source_table')) + LAYOUT(FLAT()) + LIFETIME(MIN 0 MAX 1000); + + SHOW DICTIONARIES FROM {db:Identifier};" + + +$CLICKHOUSE_CLIENT --param_db="${db_name}" --multiquery \ + --query="RENAME {db:Identifier}.test_dictionary TO {db:Identifier}.test_dictionary_2; + SHOW DICTIONARIES FROM {db:Identifier};" + +todb_name=db_$(echo $RANDOM |md5sum |cut -c 1-4) + +$CLICKHOUSE_CLIENT --param_todb="${todb_name}" --param_db="${db_name}" --query="RENAME {db:Identifier} TO {todb:Identifier}; -- { serverError 60 }" 2>&1 | grep -o "UNKNOWN_TABLE" | uniq + +$CLICKHOUSE_CLIENT --param_db="${db_name}" --query="DROP DATABASE IF EXISTS {db:Identifier}" + diff --git a/tests/queries/0_stateless/02891_rename_table_without_keyword.sql b/tests/queries/0_stateless/02891_rename_table_without_keyword.sql index d92b3f69e5d..291b5db5c56 100644 --- a/tests/queries/0_stateless/02891_rename_table_without_keyword.sql +++ b/tests/queries/0_stateless/02891_rename_table_without_keyword.sql @@ -1,7 +1,7 @@ DROP DATABASE IF EXISTS rename_db; -CREATE DATABASE rename_db; +CREATE DATABASE IF NOT EXISTS rename_db; -CREATE TABLE rename_db.r1 (name String) Engine=Memory(); +CREATE TABLE IF NOT EXISTS rename_db.r1 (name String) Engine=Memory(); SHOW TABLES FROM rename_db; RENAME TABLE rename_db.r1 TO rename_db.r2; @@ -10,5 +10,29 @@ SHOW TABLES FROM rename_db; RENAME rename_db.r2 TO rename_db.r3; SHOW TABLES FROM rename_db; -DROP DATABASE rename_db; +CREATE TABLE IF NOT EXISTS rename_db.source_table ( + id UInt64, + value String +) ENGINE = Memory; + +CREATE DICTIONARY IF NOT EXISTS rename_db.test_dictionary +( + id UInt64, + value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'rename_db.dictionary_table')) +LAYOUT(FLAT()) +LIFETIME(MIN 0 MAX 1000); + +SHOW DICTIONARIES FROM rename_db; + +RENAME rename_db.test_dictionary TO rename_db.test_dictionary_2; -- { serverError UNKNOWN_TABLE } +SHOW DICTIONARIES FROM rename_db; + +SHOW DATABASES LIKE 'rename_db'; +RENAME rename_db TO rename_db_2; -- { serverError UNKNOWN_TABLE } +SHOW DATABASES LIKE 'rename_db'; + +DROP DATABASE IF EXISTS rename_db; From 89d123d0fdae5fc5fa582a0325b9c7bb08f24141 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Thu, 12 Oct 2023 14:24:59 +0800 Subject: [PATCH 166/634] modify doc --- docs/en/sql-reference/statements/rename.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/rename.md b/docs/en/sql-reference/statements/rename.md index 957f2238a84..bb62cc3af1c 100644 --- a/docs/en/sql-reference/statements/rename.md +++ b/docs/en/sql-reference/statements/rename.md @@ -16,7 +16,7 @@ The `RENAME` query is supported by the [Atomic](../../engines/database-engines/a **Syntax** ```sql -RENAME DATABASE|[TABLE|DICTIONARY] name TO new_name [,...] [ON CLUSTER cluster] +RENAME [DATABASE|TABLE|DICTIONARY] name TO new_name [,...] [ON CLUSTER cluster] ``` ## RENAME DATABASE From ca6975a2653273ce269dbc96bf50c543bf461036 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Thu, 12 Oct 2023 14:25:45 +0800 Subject: [PATCH 167/634] Delete tests/queries/0_stateless/02891_rename_table_without_keyword.sql --- .../02891_rename_table_without_keyword.sql | 38 ------------------- 1 file changed, 38 deletions(-) delete mode 100644 tests/queries/0_stateless/02891_rename_table_without_keyword.sql diff --git a/tests/queries/0_stateless/02891_rename_table_without_keyword.sql b/tests/queries/0_stateless/02891_rename_table_without_keyword.sql deleted file mode 100644 index 291b5db5c56..00000000000 --- a/tests/queries/0_stateless/02891_rename_table_without_keyword.sql +++ /dev/null @@ -1,38 +0,0 @@ -DROP DATABASE IF EXISTS rename_db; -CREATE DATABASE IF NOT EXISTS rename_db; - -CREATE TABLE IF NOT EXISTS rename_db.r1 (name String) Engine=Memory(); -SHOW TABLES FROM rename_db; - -RENAME TABLE rename_db.r1 TO rename_db.r2; -SHOW TABLES FROM rename_db; - -RENAME rename_db.r2 TO rename_db.r3; -SHOW TABLES FROM rename_db; - -CREATE TABLE IF NOT EXISTS rename_db.source_table ( - id UInt64, - value String -) ENGINE = Memory; - -CREATE DICTIONARY IF NOT EXISTS rename_db.test_dictionary -( - id UInt64, - value String -) -PRIMARY KEY id -SOURCE(CLICKHOUSE(TABLE 'rename_db.dictionary_table')) -LAYOUT(FLAT()) -LIFETIME(MIN 0 MAX 1000); - -SHOW DICTIONARIES FROM rename_db; - -RENAME rename_db.test_dictionary TO rename_db.test_dictionary_2; -- { serverError UNKNOWN_TABLE } -SHOW DICTIONARIES FROM rename_db; - -SHOW DATABASES LIKE 'rename_db'; -RENAME rename_db TO rename_db_2; -- { serverError UNKNOWN_TABLE } -SHOW DATABASES LIKE 'rename_db'; - -DROP DATABASE IF EXISTS rename_db; - From a97ce4cbfcc16b98e0179614ad536ef5f621d659 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 12 Oct 2023 09:15:54 +0000 Subject: [PATCH 168/634] update test --- .../0_stateless/02895_forbid_create_inverted_index.sql | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02895_forbid_create_inverted_index.sql b/tests/queries/0_stateless/02895_forbid_create_inverted_index.sql index 9bfcf6b88f5..dc92d9198fb 100644 --- a/tests/queries/0_stateless/02895_forbid_create_inverted_index.sql +++ b/tests/queries/0_stateless/02895_forbid_create_inverted_index.sql @@ -1,5 +1,6 @@ SET allow_experimental_inverted_index = 0; -CREATE OR REPLACE TABLE tab +DROP TABLE IF EXISTS tab; +CREATE TABLE tab ( `key` UInt64, `str` String @@ -8,3 +9,5 @@ ENGINE = MergeTree ORDER BY key; ALTER TABLE tab ADD INDEX inv_idx(str) TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED } + +DROP TABLE tab; From 5d5d4a8443897235a69fe6b90717e6ebb81400ac Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 12 Oct 2023 09:50:04 +0000 Subject: [PATCH 169/634] Check if storage is nullptr --- src/Interpreters/InterpreterSelectQuery.cpp | 10 +++++++++- .../0_stateless/02896_illegal_sampling.reference | 0 tests/queries/0_stateless/02896_illegal_sampling.sql | 1 + 3 files changed, 10 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02896_illegal_sampling.reference create mode 100644 tests/queries/0_stateless/02896_illegal_sampling.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5e7ece5912f..e0aceb1b108 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -684,7 +684,15 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (!options.only_analyze) { if (query.sampleSize() && (input_pipe || !storage || !storage->supportsSampling())) - throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, "Illegal SAMPLE: table {} doesn't support sampling", storage->getStorageID().getNameForLogs()); + { + if (storage) + throw Exception( + ErrorCodes::SAMPLING_NOT_SUPPORTED, + "Storage {} doesn't support sampling", + storage->getStorageID().getNameForLogs()); + else + throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, "Illegal SAMPLE"); + } if (query.final() && (input_pipe || !storage || !storage->supportsFinal())) { diff --git a/tests/queries/0_stateless/02896_illegal_sampling.reference b/tests/queries/0_stateless/02896_illegal_sampling.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02896_illegal_sampling.sql b/tests/queries/0_stateless/02896_illegal_sampling.sql new file mode 100644 index 00000000000..280216b3422 --- /dev/null +++ b/tests/queries/0_stateless/02896_illegal_sampling.sql @@ -0,0 +1 @@ +SELECT * FROM (SELECT 1) SAMPLE 1 / 2; -- { serverError SAMPLING_NOT_SUPPORTED } \ No newline at end of file From 2fd2352addb24a18828c41c75cf4b597499660c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 12 Oct 2023 09:52:06 +0000 Subject: [PATCH 170/634] Use SQL test --- ...891_rename_table_without_keyword.reference | 1 - .../02891_rename_table_without_keyword.sh | 57 ------------------- .../02891_rename_table_without_keyword.sql | 42 ++++++++++++++ 3 files changed, 42 insertions(+), 58 deletions(-) delete mode 100755 tests/queries/0_stateless/02891_rename_table_without_keyword.sh create mode 100755 tests/queries/0_stateless/02891_rename_table_without_keyword.sql diff --git a/tests/queries/0_stateless/02891_rename_table_without_keyword.reference b/tests/queries/0_stateless/02891_rename_table_without_keyword.reference index 3116f82ea08..958c53935c1 100644 --- a/tests/queries/0_stateless/02891_rename_table_without_keyword.reference +++ b/tests/queries/0_stateless/02891_rename_table_without_keyword.reference @@ -5,4 +5,3 @@ r1_bak r2_bak test_dictionary test_dictionary_2 -UNKNOWN_TABLE diff --git a/tests/queries/0_stateless/02891_rename_table_without_keyword.sh b/tests/queries/0_stateless/02891_rename_table_without_keyword.sh deleted file mode 100755 index 8d9d73f6ab4..00000000000 --- a/tests/queries/0_stateless/02891_rename_table_without_keyword.sh +++ /dev/null @@ -1,57 +0,0 @@ -#!/usr/bin/env bash - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -db_name=db_$(echo $RANDOM |md5sum |cut -c 1-4) - -$CLICKHOUSE_CLIENT --param_db="${db_name}" --multiquery \ - --query="DROP DATABASE IF EXISTS {db:Identifier}; - CREATE DATABASE {db:Identifier}; - CREATE TABLE IF NOT EXISTS {db:Identifier}.r1 (name String) Engine=Memory(); - SHOW TABLES FROM {db:Identifier}" - -$CLICKHOUSE_CLIENT --param_db="${db_name}" --multiquery \ - --query="RENAME TABLE {db:Identifier}.r1 TO {db:Identifier}.r1_bak; - SHOW TABLES FROM {db:Identifier};" - -$CLICKHOUSE_CLIENT --param_db="${db_name}" --multiquery \ - --query="RENAME {db:Identifier}.r1_bak TO {db:Identifier}.r1; - SHOW TABLES FROM {db:Identifier};" - -$CLICKHOUSE_CLIENT --param_db="${db_name}" --multiquery \ - --query="CREATE TABLE IF NOT EXISTS {db:Identifier}.r2 (name String) Engine=Memory(); - RENAME {db:Identifier}.r1 TO {db:Identifier}.r1_bak, {db:Identifier}.r2 TO {db:Identifier}.r2_bak; - SHOW TABLES FROM {db:Identifier};" - - -$CLICKHOUSE_CLIENT --param_db="${db_name}" --multiquery \ - --query="CREATE TABLE IF NOT EXISTS {db:Identifier}.source_table ( - id UInt64, - value String - ) ENGINE = Memory; - - CREATE DICTIONARY IF NOT EXISTS {db:Identifier}.test_dictionary - ( - id UInt64, - value String - ) - PRIMARY KEY id - SOURCE(CLICKHOUSE(TABLE '{db:Identifier}.source_table')) - LAYOUT(FLAT()) - LIFETIME(MIN 0 MAX 1000); - - SHOW DICTIONARIES FROM {db:Identifier};" - - -$CLICKHOUSE_CLIENT --param_db="${db_name}" --multiquery \ - --query="RENAME {db:Identifier}.test_dictionary TO {db:Identifier}.test_dictionary_2; - SHOW DICTIONARIES FROM {db:Identifier};" - -todb_name=db_$(echo $RANDOM |md5sum |cut -c 1-4) - -$CLICKHOUSE_CLIENT --param_todb="${todb_name}" --param_db="${db_name}" --query="RENAME {db:Identifier} TO {todb:Identifier}; -- { serverError 60 }" 2>&1 | grep -o "UNKNOWN_TABLE" | uniq - -$CLICKHOUSE_CLIENT --param_db="${db_name}" --query="DROP DATABASE IF EXISTS {db:Identifier}" - diff --git a/tests/queries/0_stateless/02891_rename_table_without_keyword.sql b/tests/queries/0_stateless/02891_rename_table_without_keyword.sql new file mode 100755 index 00000000000..2f32dc94def --- /dev/null +++ b/tests/queries/0_stateless/02891_rename_table_without_keyword.sql @@ -0,0 +1,42 @@ +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; +CREATE DATABASE IF NOT EXISTS {CLICKHOUSE_DATABASE:Identifier}; + +CREATE TABLE IF NOT EXISTS {CLICKHOUSE_DATABASE:Identifier}.r1 (name String) Engine=Memory(); +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier}; + +RENAME TABLE {CLICKHOUSE_DATABASE:Identifier}.r1 TO {CLICKHOUSE_DATABASE:Identifier}.r1_bak; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier}; + +RENAME {CLICKHOUSE_DATABASE:Identifier}.r1_bak TO {CLICKHOUSE_DATABASE:Identifier}.r1; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier}; + +CREATE TABLE IF NOT EXISTS {CLICKHOUSE_DATABASE:Identifier}.r2 (name String) Engine=Memory(); +RENAME {CLICKHOUSE_DATABASE:Identifier}.r1 TO {CLICKHOUSE_DATABASE:Identifier}.r1_bak, + {CLICKHOUSE_DATABASE:Identifier}.r2 TO {CLICKHOUSE_DATABASE:Identifier}.r2_bak; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier}; + +CREATE TABLE IF NOT EXISTS {CLICKHOUSE_DATABASE:Identifier}.source_table ( + id UInt64, + value String + ) ENGINE = Memory; + +CREATE DICTIONARY IF NOT EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_dictionary +( + id UInt64, + value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE '{CLICKHOUSE_DATABASE:String}.dictionary_table')) +LAYOUT(FLAT()) +LIFETIME(MIN 0 MAX 1000); + +SHOW DICTIONARIES FROM {CLICKHOUSE_DATABASE:Identifier}; + +RENAME {CLICKHOUSE_DATABASE:Identifier}.test_dictionary TO {CLICKHOUSE_DATABASE:Identifier}.test_dictionary_2; +SHOW DICTIONARIES FROM {CLICKHOUSE_DATABASE:Identifier}; + +SHOW DATABASES LIKE '{CLICKHOUSE_DATABASE:String}'; +RENAME {CLICKHOUSE_DATABASE:Identifier} TO {CLICKHOUSE_DATABASE_1:Identifier}; -- { serverError UNKNOWN_TABLE } +SHOW DATABASES LIKE '{CLICKHOUSE_DATABASE:String}'; + +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; From f379d9cac5cb63370ecf364ed92d9045207bfc93 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 11 Oct 2023 20:38:40 +0200 Subject: [PATCH 171/634] Fix checking of non handled data for Values format PeekableReadBuffer::hasUnreadData() does not checks the underlying buffer, and so it simply ignore some issues, like: INSERT INTO test_01179_str values ('foo'); ('bar') Signed-off-by: Azat Khuzhin --- .../Formats/Impl/ValuesBlockInputFormat.cpp | 4 ++-- .../01179_insert_values_semicolon.expect | 13 +++++++------ 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 8126c472f70..4a209fba3c5 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -615,12 +615,12 @@ void ValuesBlockInputFormat::readSuffix() { ++buf->position(); skipWhitespaceIfAny(*buf); - if (buf->hasUnreadData()) + if (buf->hasUnreadData() || !buf->eof()) throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read data after semicolon"); return; } - if (buf->hasUnreadData()) + if (buf->hasUnreadData() || !buf->eof()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unread data in PeekableReadBuffer will be lost. Most likely it's a bug."); } diff --git a/tests/queries/0_stateless/01179_insert_values_semicolon.expect b/tests/queries/0_stateless/01179_insert_values_semicolon.expect index cb22fb8265c..072be483e4f 100755 --- a/tests/queries/0_stateless/01179_insert_values_semicolon.expect +++ b/tests/queries/0_stateless/01179_insert_values_semicolon.expect @@ -21,20 +21,21 @@ expect ":) " send -- "DROP TABLE IF EXISTS test_01179\r" expect "Ok." -send -- "CREATE TABLE test_01179 (date DateTime64(3)) ENGINE=Memory()\r" +send -- "CREATE TABLE test_01179 (val String) ENGINE=Memory()\r" expect "Ok." -send -- "INSERT INTO test_01179 values ('2020-01-01')\r" +send -- "INSERT INTO test_01179 values ('foo')\r" + expect "Ok." -send -- "INSERT INTO test_01179 values ('2020-01-01'); \r" +send -- "INSERT INTO test_01179 values ('foo'); \r" expect "Ok." -send -- "INSERT INTO test_01179 values ('2020-01-01 0'); (1) \r" +send -- "INSERT INTO test_01179 values ('foo'); ('bar') \r" expect "Cannot read data after semicolon" -send -- "SELECT date, count() FROM test_01179 GROUP BY date FORMAT TSV\r" -expect "2020-01-01 00:00:00.000\t2" +send -- "SELECT val, count() FROM test_01179 GROUP BY val FORMAT TSV\r" +expect "foo\t2" send -- "DROP TABLE test_01179\r" expect "Ok." From a5cced2b9e031d47d0f3a59f165c46590eaeed99 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 12 Oct 2023 12:24:21 +0200 Subject: [PATCH 172/634] Move comment into correct place Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9ca104ff942..691f488172e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2020,9 +2020,6 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) { bool echo_query = echo_queries; - /// Test tags are started with "--" so they are interpreted as comments anyway. - /// But if the echo is enabled we have to remove the test tags from `all_queries_text` - /// because we don't want test tags to be echoed. { /// disable logs if expects errors TestHint test_hint(all_queries_text); @@ -2030,6 +2027,9 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) processTextAsSingleQuery("SET send_logs_level = 'fatal'"); } + /// Test tags are started with "--" so they are interpreted as comments anyway. + /// But if the echo is enabled we have to remove the test tags from `all_queries_text` + /// because we don't want test tags to be echoed. size_t test_tags_length = getTestTagsLength(all_queries_text); /// Several queries separated by ';'. From a799d4ec62924241a5ef3b49ab63f0986539cb67 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 8 May 2023 14:56:01 +0000 Subject: [PATCH 173/634] Fix 01049_join_low_card_bug_long --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 72 ++-- .../01049_join_low_card_bug_long.reference.j2 | 315 ++++++++++++++++++ .../01049_join_low_card_bug_long.sql.j2 | 6 + 3 files changed, 368 insertions(+), 25 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index d0dd59d99d2..5cdd18b4b2f 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1264,7 +1264,8 @@ private: size_t identifier_bind_size, const QueryTreeNodePtr & compound_expression, String compound_expression_source, - IdentifierResolveScope & scope); + IdentifierResolveScope & scope, + bool can_be_not_found = false); QueryTreeNodePtr tryResolveIdentifierFromExpressionArguments(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); @@ -1313,12 +1314,13 @@ private: IdentifierResolveScope & scope, IdentifierResolveSettings identifier_resolve_settings = {}); - QueryTreeNodePtr resolveIdentifierFromStorageOrThrow( + QueryTreeNodePtr tryResolveIdentifierFromStorage( const Identifier & identifier, const QueryTreeNodePtr & table_expression_node, const TableExpressionData & table_expression_data, IdentifierResolveScope & scope, - size_t identifier_column_qualifier_parts); + size_t identifier_column_qualifier_parts, + bool can_be_not_found = false); /// Resolve query tree nodes functions @@ -2402,11 +2404,13 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con } /// Resolve identifier from compound expression +/// If identifier cannot be resolved throw exception or return nullptr if can_be_not_found is true QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromCompoundExpression(const Identifier & expression_identifier, size_t identifier_bind_size, const QueryTreeNodePtr & compound_expression, String compound_expression_source, - IdentifierResolveScope & scope) + IdentifierResolveScope & scope, + bool can_be_not_found) { Identifier compound_expression_identifier; for (size_t i = 0; i < identifier_bind_size; ++i) @@ -2419,6 +2423,23 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromCompoundExpression(const if (!expression_type->hasSubcolumn(nested_path.getFullName())) { + if (auto * column = compound_expression->as()) + { + const DataTypePtr & column_type = column->getColumn().getTypeInStorage(); + if (column_type->getTypeId() == TypeIndex::Object) + { + const auto * object_type = checkAndGetDataType(column_type.get()); + if (object_type->getSchemaFormat() == "json" && object_type->hasNullableSubcolumns()) + { + QueryTreeNodePtr constant_node_null = std::make_shared(Field()); + return constant_node_null; + } + } + } + + if (can_be_not_found) + return {}; + std::unordered_set valid_identifiers; collectCompoundExpressionValidIdentifiersForTypoCorrection(expression_identifier, expression_type, @@ -2434,20 +2455,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromCompoundExpression(const compound_expression_from_error_message += compound_expression_source; } - if (auto * column = compound_expression->as()) - { - const DataTypePtr & column_type = column->getColumn().getTypeInStorage(); - if (column_type->getTypeId() == TypeIndex::Object) - { - const auto * object_type = checkAndGetDataType(column_type.get()); - if (object_type->getSchemaFormat() == "json" && object_type->hasNullableSubcolumns()) - { - QueryTreeNodePtr constant_node_null = std::make_shared(Field()); - return constant_node_null; - } - } - } - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Identifier {} nested path {} cannot be resolved from type {}{}. In scope {}{}", expression_identifier, @@ -2803,12 +2810,13 @@ bool QueryAnalyzer::tryBindIdentifierToTableExpressions(const IdentifierLookup & return can_bind_identifier_to_table_expression; } -QueryTreeNodePtr QueryAnalyzer::resolveIdentifierFromStorageOrThrow( +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromStorage( const Identifier & identifier, const QueryTreeNodePtr & table_expression_node, const TableExpressionData & table_expression_data, IdentifierResolveScope & scope, - size_t identifier_column_qualifier_parts) + size_t identifier_column_qualifier_parts, + bool can_be_not_found) { auto identifier_without_column_qualifier = identifier; identifier_without_column_qualifier.popFirst(identifier_column_qualifier_parts); @@ -2851,7 +2859,10 @@ QueryTreeNodePtr QueryAnalyzer::resolveIdentifierFromStorageOrThrow( identifier_bind_size, result_expression, table_expression_source, - scope); + scope, + can_be_not_found); + if (can_be_not_found && !result_expression) + return {}; clone_is_needed = false; } @@ -3000,24 +3011,35 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id * 3. Try to bind identifier first parts to database name and table name, if true remove first two parts and try to get full identifier from table or throw exception. */ if (table_expression_data.hasFullIdentifierName(IdentifierView(identifier))) - return resolveIdentifierFromStorageOrThrow(identifier, table_expression_node, table_expression_data, scope, 0 /*identifier_column_qualifier_parts*/); + return tryResolveIdentifierFromStorage(identifier, table_expression_node, table_expression_data, scope, 0 /*identifier_column_qualifier_parts*/); if (table_expression_data.canBindIdentifier(IdentifierView(identifier))) - return resolveIdentifierFromStorageOrThrow(identifier, table_expression_node, table_expression_data, scope, 0 /*identifier_column_qualifier_parts*/); + { + /** This check is insufficient to determine whether and identifier can be resolved from table expression. + * A further check will be performed in `tryResolveIdentifierFromStorage` to see if we have such a subcolumn. + * In cases where the subcolumn cannot be found we want to have `nullptr` instead of exception. + * So, we set `can_be_not_found = true` to have an attempt to resolve the identifier from another table expression. + * Example: `SELECT t.t from (SELECT 1 as t) AS a FULL JOIN (SELECT 1 as t) as t ON a.t = t.t;` + * Initially, we will try to resolve t.t from `a` because `t.` is bound to `1 as t`. However, as it is not a nested column, we will need to resolve it from the second table expression. + */ + auto resolved_identifier = tryResolveIdentifierFromStorage(identifier, table_expression_node, table_expression_data, scope, 0 /*identifier_column_qualifier_parts*/, true /*can_be_not_found*/); + if (resolved_identifier) + return resolved_identifier; + } if (identifier.getPartsSize() == 1) return {}; const auto & table_name = table_expression_data.table_name; if ((!table_name.empty() && path_start == table_name) || (table_expression_node->hasAlias() && path_start == table_expression_node->getAlias())) - return resolveIdentifierFromStorageOrThrow(identifier, table_expression_node, table_expression_data, scope, 1 /*identifier_column_qualifier_parts*/); + return tryResolveIdentifierFromStorage(identifier, table_expression_node, table_expression_data, scope, 1 /*identifier_column_qualifier_parts*/); if (identifier.getPartsSize() == 2) return {}; const auto & database_name = table_expression_data.database_name; if (!database_name.empty() && path_start == database_name && identifier[1] == table_name) - return resolveIdentifierFromStorageOrThrow(identifier, table_expression_node, table_expression_data, scope, 2 /*identifier_column_qualifier_parts*/); + return tryResolveIdentifierFromStorage(identifier, table_expression_node, table_expression_data, scope, 2 /*identifier_column_qualifier_parts*/); return {}; } diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 index 879811f6764..2ebe5c373b2 100644 --- a/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 +++ b/tests/queries/0_stateless/01049_join_low_card_bug_long.reference.j2 @@ -1,4 +1,5 @@ -- { echoOn } +SET allow_experimental_analyzer = 0; {% for join_algorithm in ['default', 'partial_merge', 'parallel_hash'] -%} SET join_algorithm = '{{ join_algorithm }}'; SET join_use_nulls = 0; @@ -312,6 +313,320 @@ SELECT r.lc, materialize(r.lc), toTypeName(r.lc), toTypeName(materialize(r.lc)) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) +SET allow_experimental_analyzer = 1; +{% for join_algorithm in ['default', 'partial_merge', 'parallel_hash'] -%} +SET join_algorithm = '{{ join_algorithm }}'; +SET join_use_nulls = 0; +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, lc; +str LowCardinality(String) + LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, lc; +str LowCardinality(String) +str_r LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, lc; +str LowCardinality(String) + LowCardinality(String) +str_l LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, lc; +str LowCardinality(String) +str_r LowCardinality(String) +str_l LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str LowCardinality(String) LowCardinality(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(String) LowCardinality(String) +LowCardinality(String) LowCardinality(String) LowCardinality(String) LowCardinality(String) str_l str_l +-- + +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x, lc; +str LowCardinality(String) + LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x, lc; +str String +str_r String +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x, lc; +str LowCardinality(String) + LowCardinality(String) +str_l LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x, lc; +str String +str_r String +str_l String +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x, r.lc, l.lc; +String String str str LowCardinality(String) LowCardinality(String) str str +String String str_r str_r LowCardinality(String) LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x, r.lc, l.lc; +String String str str String String str str +String String str_r str_r String String +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x, r.lc, l.lc; +String String str str LowCardinality(String) LowCardinality(String) str str +String String str_r str_r LowCardinality(String) LowCardinality(String) +String String LowCardinality(String) LowCardinality(String) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x, r.lc, l.lc; +String String str str String String str str +String String str_r str_r String String +String String String String str_l str_l +-- + +SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (x) ORDER BY x, lc; +str String + String +SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (lc) ORDER BY x, lc; +str String +str_r String +SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (x) ORDER BY x, lc; +str String + String +str_l String +SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (lc) ORDER BY x, lc; +str String +str_r String +str_l String +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str String String str str +LowCardinality(String) LowCardinality(String) str_r str_r String String +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +String String str str String String str str +String String str_r str_r String String +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str String String str str +LowCardinality(String) LowCardinality(String) str_r str_r String String +LowCardinality(String) LowCardinality(String) String String str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +String String str str String String str str +String String str_r str_r String String +String String String String str_l str_l +-- + +SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (x) ORDER BY x, lc; +str LowCardinality(String) + LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (lc) ORDER BY x, lc; +str Nullable(String) +str_r Nullable(String) +SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (x) ORDER BY x, lc; +str LowCardinality(String) + LowCardinality(String) +str_l LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (lc) ORDER BY x, lc; +str Nullable(String) +str_r Nullable(String) +str_l Nullable(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str +Nullable(String) Nullable(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(String) LowCardinality(String) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(String) LowCardinality(String) +Nullable(String) Nullable(String) \N \N LowCardinality(String) LowCardinality(String) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str +Nullable(String) Nullable(String) str_r str_r Nullable(String) Nullable(String) \N \N +Nullable(String) Nullable(String) \N \N Nullable(String) Nullable(String) str_l str_l +-- + +SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (x) ORDER BY x, lc; +str Nullable(String) +\N Nullable(String) +SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (lc) ORDER BY x, lc; +str Nullable(String) +str_r Nullable(String) +SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (x) ORDER BY x, lc; +str Nullable(String) +\N Nullable(String) +str_l Nullable(String) +SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (lc) ORDER BY x, lc; +str Nullable(String) +str_l Nullable(String) +str_r Nullable(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str +Nullable(String) Nullable(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N +LowCardinality(String) LowCardinality(String) Nullable(String) Nullable(String) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str +Nullable(String) Nullable(String) \N \N Nullable(String) Nullable(String) str_l str_l +Nullable(String) Nullable(String) str_r str_r Nullable(String) Nullable(String) \N \N +SET join_use_nulls = 1; +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +\N LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, lc; +str LowCardinality(String) +str_r LowCardinality(String) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +\N LowCardinality(Nullable(String)) +str_l LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +str_l LowCardinality(Nullable(String)) +str_r LowCardinality(Nullable(String)) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +LowCardinality(String) LowCardinality(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +-- + +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +\N LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x, lc; +str String +str_r String +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +\N LowCardinality(Nullable(String)) +str_l LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x, lc; +str Nullable(String) +str_l Nullable(String) +str_r Nullable(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (x) ORDER BY x, r.lc, l.lc; +String String str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +String String str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN r USING (lc) ORDER BY x, r.lc, l.lc; +String String str str Nullable(String) Nullable(String) str str +String String str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (x) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str +Nullable(String) Nullable(String) \N \N Nullable(String) Nullable(String) str_l str_l +Nullable(String) Nullable(String) str_r str_r Nullable(String) Nullable(String) \N \N +-- + +SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (x) ORDER BY x, lc; +str Nullable(String) +\N Nullable(String) +SELECT lc, toTypeName(lc) FROM l RIGHT JOIN r USING (lc) ORDER BY x, lc; +str String +str_r String +SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (x) ORDER BY x, lc; +str Nullable(String) +\N Nullable(String) +str_l Nullable(String) +SELECT lc, toTypeName(lc) FROM l FULL JOIN r USING (lc) ORDER BY x, lc; +str Nullable(String) +str_l Nullable(String) +str_r Nullable(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +String String str str Nullable(String) Nullable(String) str str +String String str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str +Nullable(String) Nullable(String) \N \N Nullable(String) Nullable(String) str_l str_l +Nullable(String) Nullable(String) str_r str_r Nullable(String) Nullable(String) \N \N +-- + +SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (x) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +\N LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc RIGHT JOIN nr USING (lc) ORDER BY x, lc; +str Nullable(String) +str_r Nullable(String) +SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (x) ORDER BY x, lc; +str LowCardinality(Nullable(String)) +\N LowCardinality(Nullable(String)) +str_l LowCardinality(Nullable(String)) +SELECT lc, toTypeName(lc) FROM l_lc FULL JOIN nr USING (lc) ORDER BY x, lc; +str Nullable(String) +str_l Nullable(String) +str_r Nullable(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str +Nullable(String) Nullable(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (x) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str +Nullable(String) Nullable(String) str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N +Nullable(String) Nullable(String) \N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM l_lc AS l FULL JOIN nr AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str +Nullable(String) Nullable(String) \N \N Nullable(String) Nullable(String) str_l str_l +Nullable(String) Nullable(String) str_r str_r Nullable(String) Nullable(String) \N \N +-- + +SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (x) ORDER BY x, lc; +str Nullable(String) +\N Nullable(String) +SELECT lc, toTypeName(lc) FROM nl RIGHT JOIN r_lc USING (lc) ORDER BY x, lc; +str Nullable(String) +str_r Nullable(String) +SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (x) ORDER BY x, lc; +str Nullable(String) +\N Nullable(String) +str_l Nullable(String) +SELECT lc, toTypeName(lc) FROM nl FULL JOIN r_lc USING (lc) ORDER BY x, lc; +str Nullable(String) +str_l Nullable(String) +str_r Nullable(String) +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(String) LowCardinality(String) str str Nullable(String) Nullable(String) str str +LowCardinality(String) LowCardinality(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l RIGHT JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str +Nullable(String) Nullable(String) str_r str_r Nullable(String) Nullable(String) \N \N +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (x) ORDER BY x, r.lc, l.lc; +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str str Nullable(String) Nullable(String) str str +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) str_r str_r Nullable(String) Nullable(String) \N \N +LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) \N \N Nullable(String) Nullable(String) str_l str_l +SELECT toTypeName(r.lc), toTypeName(materialize(r.lc)), r.lc, materialize(r.lc), toTypeName(l.lc), toTypeName(materialize(l.lc)), l.lc, materialize(l.lc) FROM nl AS l FULL JOIN r_lc AS r USING (lc) ORDER BY x, r.lc, l.lc; +Nullable(String) Nullable(String) str str Nullable(String) Nullable(String) str str +Nullable(String) Nullable(String) \N \N Nullable(String) Nullable(String) str_l str_l +Nullable(String) Nullable(String) str_r str_r Nullable(String) Nullable(String) \N \N +{% endfor -%} +SELECT '--'; +-- +SET join_use_nulls = 0; +SELECT lc, toTypeName(lc) FROM l_lc AS l RIGHT JOIN r_lc AS r USING (x) ORDER BY l.lc; + LowCardinality(String) +str LowCardinality(String) +SET join_algorithm = 'partial_merge'; +SET join_use_nulls = 1; +SELECT r.lc, materialize(r.lc), toTypeName(r.lc), toTypeName(materialize(r.lc)) FROM l_lc AS l FULL OUTER JOIN r_lc AS r USING (x) ORDER BY r.lc; +str str LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) +str_r str_r LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) +\N \N LowCardinality(Nullable(String)) LowCardinality(Nullable(String)) DROP TABLE l; DROP TABLE r; DROP TABLE nl; diff --git a/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 b/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 index aa666079e5a..64ec34ef1bf 100644 --- a/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 +++ b/tests/queries/0_stateless/01049_join_low_card_bug_long.sql.j2 @@ -23,6 +23,10 @@ INSERT INTO nl VALUES (0, 'str'), (2, 'str_l'); INSERT INTO l_lc VALUES (0, 'str'), (2, 'str_l'); -- { echoOn } +{% for allow_experimental_analyzer in [0, 1] -%} + +SET allow_experimental_analyzer = {{ allow_experimental_analyzer }}; + {% for join_algorithm in ['default', 'partial_merge', 'parallel_hash'] -%} SET join_algorithm = '{{ join_algorithm }}'; @@ -101,6 +105,8 @@ SET join_use_nulls = 1; SELECT r.lc, materialize(r.lc), toTypeName(r.lc), toTypeName(materialize(r.lc)) FROM l_lc AS l FULL OUTER JOIN r_lc AS r USING (x) ORDER BY r.lc; +{% endfor -%} + DROP TABLE l; DROP TABLE r; DROP TABLE nl; From 624f38633c8293941799e8e744528515e34b8b3d Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 8 May 2023 15:20:16 +0000 Subject: [PATCH 174/634] Fix 01721_join_implicit_cast_long --- tests/analyzer_tech_debt.txt | 1 - ...01721_join_implicit_cast_long.reference.j2 | 17 ++++++++++++ .../01721_join_implicit_cast_long.sql.j2 | 27 +++++++++++++------ 3 files changed, 36 insertions(+), 9 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 8625c06d33d..95898bc70b0 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -52,7 +52,6 @@ 01681_bloom_filter_nullable_column 01700_system_zookeeper_path_in 01710_projection_additional_filters -01721_join_implicit_cast_long 01739_index_hint 02880_indexHint__partition_id 01747_join_view_filter_dictionary diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 index ae43aa7195c..4d8d4255ce4 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 @@ -1,3 +1,4 @@ +{% for allow_experimental_analyzer in [0, 1] -%} {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} === {{ join_algorithm }} === = full = @@ -72,10 +73,17 @@ 9 9 0 10 10 0 = right = +{% if allow_experimental_analyzer -%} +-4 0 -4 +-3 0 -3 +-2 0 -2 +-1 0 -1 +{% else -%} 0 0 -4 0 0 -3 0 0 -2 0 0 -1 +{% endif -%} 0 0 0 1 1 1 2 2 2 @@ -279,11 +287,19 @@ 3 3 3 4 4 4 5 5 5 +{% if allow_experimental_analyzer -%} +-4 \N -4 +-3 \N -3 +-2 \N -2 +-1 \N -1 +0 \N 0 +{% else -%} \N \N -4 \N \N -3 \N \N -2 \N \N -1 \N \N 0 +{% endif -%} = inner = 1 1 1 2 2 2 @@ -404,3 +420,4 @@ 1 {% endif -%} {% endfor -%} +{% endfor -%} diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 index 38f71f4c5ec..c65665ee2a9 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 @@ -12,6 +12,10 @@ INSERT INTO t2 SELECT number - 5 as a, 200 + number - 5 as b FROM system.numbers {% macro is_implemented(join_algorithm) -%} {% endmacro -%} +{% for allow_experimental_analyzer in [0, 1] -%} + +SET allow_experimental_analyzer = {{ allow_experimental_analyzer }}; + {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} SELECT '=== {{ join_algorithm }} ==='; @@ -61,10 +65,10 @@ SELECT '= inner ='; SELECT * FROM t1 INNER JOIN t2 ON (t1.a + t1.b = t2.a + t2.b - 100) ORDER BY (t1.a, t2.a); -- Int64 and UInt64 has no supertype -SELECT * FROM t1 FULL JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53 } -SELECT * FROM t1 LEFT JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53 } -SELECT * FROM t1 RIGHT JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53 } -SELECT * FROM t1 INNER JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53 } +SELECT * FROM t1 FULL JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53,386 } +SELECT * FROM t1 LEFT JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53,386 } +SELECT * FROM t1 RIGHT JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53,386 } +SELECT * FROM t1 INNER JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53,386 } SELECT '= agg ='; SELECT sum(a) == 7 FROM t1 FULL JOIN t2 USING (a) WHERE b > 102 AND t2.b <= 204; {{ is_implemented(join_algorithm) }} @@ -142,10 +146,10 @@ SELECT '= inner ='; SELECT * FROM t1 INNER JOIN t2 ON (t1.a + t1.b = t2.a + t2.b - 100) ORDER BY (t1.a, t2.a); -- Int64 and UInt64 has no supertype -SELECT * FROM t1 FULL JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53 } -SELECT * FROM t1 LEFT JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53 } -SELECT * FROM t1 RIGHT JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53 } -SELECT * FROM t1 INNER JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53 } +SELECT * FROM t1 FULL JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53,386 } +SELECT * FROM t1 LEFT JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53,386 } +SELECT * FROM t1 RIGHT JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53,386 } +SELECT * FROM t1 INNER JOIN t2 ON (t1.a + t1.b + 100 = t2.a + t2.b) ORDER BY (t1.a, t2.a); -- { serverError 53,386 } SELECT '= agg ='; SELECT sum(a) == 7 FROM t1 FULL JOIN t2 USING (a) WHERE b > 102 AND t2.b <= 204; {{ is_implemented(join_algorithm) }} @@ -163,7 +167,13 @@ SELECT a > 0, sum(a), sum(t2.a), sum(b), sum(t2.b) FROM t1 FULL JOIN t2 ON (t1.a SELECT '= types ='; SELECT any(toTypeName(a)) == 'Nullable(Int32)' AND any(toTypeName(t2.a)) == 'Nullable(Int32)' FROM t1 FULL JOIN t2 USING (a); {{ is_implemented(join_algorithm) }} SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(t2.a)) == 'Nullable(Int32)' FROM t1 LEFT JOIN t2 USING (a); + +{% if allow_experimental_analyzer -%} +SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 RIGHT JOIN t2 USING (a); {{ is_implemented(join_algorithm) }} +{% else -%} SELECT any(toTypeName(a)) == 'Nullable(Int32)' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 RIGHT JOIN t2 USING (a); {{ is_implemented(join_algorithm) }} +{% endif -%} + SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 INNER JOIN t2 USING (a); SELECT toTypeName(any(a)) == 'Nullable(Int32)' AND toTypeName(any(t2.a)) == 'Nullable(Int32)' FROM t1 FULL JOIN t2 USING (a); {{ is_implemented(join_algorithm) }} @@ -178,6 +188,7 @@ SELECT toTypeName(any(a)) == 'Nullable(UInt16)' AND toTypeName(any(t2.a)) == 'Nu SET join_use_nulls = 0; {% endif -%} +{% endfor -%} {% endfor -%} DROP TABLE IF EXISTS t1; From fcb116ad4fab9976e0ffeafd4258c3ccc7afeef7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 11 May 2023 11:19:21 +0000 Subject: [PATCH 175/634] Fix 01062_pm_all_join_with_block_continuation Fixed test file, JOIN using alias from external scope --- ...62_pm_all_join_with_block_continuation.sql | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql b/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql index 15a28e92386..e7488f05fe5 100644 --- a/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql +++ b/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql @@ -2,67 +2,67 @@ SET max_memory_usage = 50000000; SET join_algorithm = 'partial_merge'; SELECT 'defaults'; - -SELECT count(1) FROM ( - SELECT materialize(1) as k, n FROM numbers(10) nums + +SELECT count(ignore(*)) FROM ( + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(10)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); -SELECT count(1) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000) nums +SELECT count(ignore(*)) FROM ( + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); -SELECT count(1), uniqExact(n) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000000) nums +SELECT count(ignore(*)), uniqExact(n) FROM ( + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j USING k); -- errors SET max_joined_block_size_rows = 0; - -SELECT count(1) FROM ( - SELECT materialize(1) as k, n FROM numbers(10) nums + +SELECT count(ignore(*)) FROM ( + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(10)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); -- { serverError 241 } -SELECT count(1) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000) nums +SELECT count(ignore(*)) FROM ( + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); -- { serverError 241 } SELECT 'max_joined_block_size_rows = 2000'; SET max_joined_block_size_rows = 2000; -SELECT count(1) FROM ( - SELECT materialize(1) as k, n FROM numbers(10) nums +SELECT count(ignore(*)) FROM ( + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(10)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); -SELECT count(1), uniqExact(n) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000) nums +SELECT count(ignore(*)), uniqExact(n) FROM ( + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); -SELECT count(1), uniqExact(n) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000000) nums +SELECT count(ignore(*)), uniqExact(n) FROM ( + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j USING k); SELECT 'max_rows_in_join = 1000'; SET max_rows_in_join = 1000; -SELECT count(1) FROM ( - SELECT materialize(1) as k, n FROM numbers(10) nums +SELECT count(ignore(*)) FROM ( + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(10)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); -SELECT count(1), uniqExact(n) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000) nums +SELECT count(ignore(*)), uniqExact(n) FROM ( + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); -SELECT count(1), uniqExact(n) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000000) nums +SELECT count(ignore(*)), uniqExact(n) FROM ( + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j USING k); From 16eeec69bf008168e5c7d8138f611bfae15536a9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 11 May 2023 15:12:46 +0000 Subject: [PATCH 176/634] Fix 02000_join_on_const --- .../0_stateless/02000_join_on_const.reference | 24 +++++++ .../0_stateless/02000_join_on_const.sql | 72 +++++++++++-------- 2 files changed, 68 insertions(+), 28 deletions(-) diff --git a/tests/queries/0_stateless/02000_join_on_const.reference b/tests/queries/0_stateless/02000_join_on_const.reference index 3035351fd87..31fd89d7ec6 100644 --- a/tests/queries/0_stateless/02000_join_on_const.reference +++ b/tests/queries/0_stateless/02000_join_on_const.reference @@ -30,3 +30,27 @@ \N 3 1 \N 2 \N +2 2 +2 2 +-- { echoOn } +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS allow_experimental_analyzer = 1; +1 0 +2 2 +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS allow_experimental_analyzer = 1; +2 2 +0 3 +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS allow_experimental_analyzer = 1; +1 0 +2 2 +0 3 +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS allow_experimental_analyzer = 1; +1 0 +2 0 +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS allow_experimental_analyzer = 1; +0 2 +0 3 +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS allow_experimental_analyzer = 1; +1 0 +2 0 +0 2 +0 3 diff --git a/tests/queries/0_stateless/02000_join_on_const.sql b/tests/queries/0_stateless/02000_join_on_const.sql index cab5a838250..21204796168 100644 --- a/tests/queries/0_stateless/02000_join_on_const.sql +++ b/tests/queries/0_stateless/02000_join_on_const.sql @@ -10,23 +10,23 @@ INSERT INTO t2 VALUES (2), (3); SELECT 70 = 10 * sum(t1.id) + sum(t2.id) AND count() == 4 FROM t1 JOIN t2 ON 1 = 1; SELECT 70 = 10 * sum(t1.id) + sum(t2.id) AND count() == 4 FROM t1 JOIN t2 ON 1; SELECT 70 = 10 * sum(t1.id) + sum(t2.id) AND count() == 4 FROM t1 JOIN t2 ON 2 = 2 AND 3 = 3; -SELECT 70 = 10 * sum(t1.id) + sum(t2.id) AND count() == 4 FROM t1 INNER ANY JOIN t2 ON toNullable(1); -SELECT 70 = 10 * sum(t1.id) + sum(t2.id) AND count() == 4 FROM t1 INNER ANY JOIN t2 ON toLowCardinality(1); -SELECT 70 = 10 * sum(t1.id) + sum(t2.id) AND count() == 4 FROM t1 INNER ANY JOIN t2 ON toLowCardinality(toNullable(1)); -SELECT 70 = 10 * sum(t1.id) + sum(t2.id) AND count() == 4 FROM t1 INNER ANY JOIN t2 ON toNullable(toLowCardinality(1)); +SELECT 70 = 10 * sum(t1.id) + sum(t2.id) AND count() == 4 FROM t1 JOIN t2 ON toNullable(1); +SELECT 70 = 10 * sum(t1.id) + sum(t2.id) AND count() == 4 FROM t1 JOIN t2 ON toLowCardinality(1); +SELECT 70 = 10 * sum(t1.id) + sum(t2.id) AND count() == 4 FROM t1 JOIN t2 ON toLowCardinality(toNullable(1)); +SELECT 70 = 10 * sum(t1.id) + sum(t2.id) AND count() == 4 FROM t1 JOIN t2 ON toNullable(toLowCardinality(1)); -SELECT * FROM t1 INNER ANY JOIN t2 ON toUInt16(1); -- { serverError 403 } -SELECT * FROM t1 INNER ANY JOIN t2 ON toInt8(1); -- { serverError 403 } -SELECT * FROM t1 INNER ANY JOIN t2 ON 256; -- { serverError 403 } -SELECT * FROM t1 INNER ANY JOIN t2 ON -1; -- { serverError 403 } -SELECT * FROM t1 INNER ANY JOIN t2 ON toString(1); -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON toUInt16(1); -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON toInt8(1); -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON 256; -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON -1; -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON toString(1); -- { serverError 403 } SELECT '- ON NULL -'; SELECT '- inner -'; -SELECT * FROM t1 INNER ANY JOIN t2 ON NULL; -SELECT * FROM t1 INNER ANY JOIN t2 ON 0; -SELECT * FROM t1 INNER ANY JOIN t2 ON 1 = 2; +SELECT * FROM t1 JOIN t2 ON NULL; +SELECT * FROM t1 JOIN t2 ON 0; +SELECT * FROM t1 JOIN t2 ON 1 = 2; SELECT '- left -'; SELECT * FROM t1 LEFT JOIN t2 ON NULL ORDER BY t1.id, t2.id; SELECT '- right -'; @@ -35,7 +35,7 @@ SELECT '- full -'; SELECT * FROM t1 FULL JOIN t2 ON NULL ORDER BY t1.id, t2.id; SELECT '- inner -'; -SELECT * FROM t1 INNER ANY JOIN t2 ON NULL ORDER BY t1.id NULLS FIRST, t2.id SETTINGS join_use_nulls = 1; +SELECT * FROM t1 JOIN t2 ON NULL ORDER BY t1.id NULLS FIRST, t2.id SETTINGS join_use_nulls = 1; SELECT '- left -'; SELECT * FROM t1 LEFT JOIN t2 ON NULL ORDER BY t1.id NULLS FIRST, t2.id SETTINGS join_use_nulls = 1; SELECT '- right -'; @@ -43,29 +43,45 @@ SELECT * FROM t1 RIGHT JOIN t2 ON NULL ORDER BY t1.id NULLS FIRST, t2.id SETTING SELECT '- full -'; SELECT * FROM t1 FULL JOIN t2 ON NULL ORDER BY t1.id NULLS FIRST, t2.id SETTINGS join_use_nulls = 1; --- in this cases we have AMBIGUOUS_COLUMN_NAME instead of INVALID_JOIN_ON_EXPRESSION +-- in this cases in old analuyzer we have AMBIGUOUS_COLUMN_NAME instead of INVALID_JOIN_ON_EXPRESSION -- because there's some function in ON expression is not constant itself (result is constant) -SELECT * FROM t1 JOIN t2 ON 1 = 1 SETTINGS join_algorithm = 'full_sorting_merge'; -- { serverError AMBIGUOUS_COLUMN_NAME } -SELECT * FROM t1 JOIN t2 ON 1 = 1 SETTINGS join_algorithm = 'partial_merge'; -- { serverError AMBIGUOUS_COLUMN_NAME } -SELECT * FROM t1 JOIN t2 ON 1 = 1 SETTINGS join_algorithm = 'auto'; -- { serverError AMBIGUOUS_COLUMN_NAME } +SELECT * FROM t1 JOIN t2 ON 1 = 1 SETTINGS join_algorithm = 'full_sorting_merge'; -- { serverError AMBIGUOUS_COLUMN_NAME,NOT_IMPLEMENTED } +SELECT * FROM t1 JOIN t2 ON 1 = 1 SETTINGS join_algorithm = 'partial_merge'; -- { serverError AMBIGUOUS_COLUMN_NAME,NOT_IMPLEMENTED } +SELECT * FROM t1 JOIN t2 ON 1 = 1 SETTINGS join_algorithm = 'auto'; -- { serverError AMBIGUOUS_COLUMN_NAME,NOT_IMPLEMENTED } -SELECT * FROM t1 JOIN t2 ON NULL SETTINGS join_algorithm = 'full_sorting_merge'; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 JOIN t2 ON NULL SETTINGS join_algorithm = 'partial_merge'; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 LEFT JOIN t2 ON NULL SETTINGS join_algorithm = 'partial_merge'; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 RIGHT JOIN t2 ON NULL SETTINGS join_algorithm = 'auto'; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 FULL JOIN t2 ON NULL SETTINGS join_algorithm = 'partial_merge'; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2 ON NULL SETTINGS join_algorithm = 'full_sorting_merge'; -- { serverError INVALID_JOIN_ON_EXPRESSION,NOT_IMPLEMENTED } +SELECT * FROM t1 JOIN t2 ON NULL SETTINGS join_algorithm = 'partial_merge'; -- { serverError INVALID_JOIN_ON_EXPRESSION,NOT_IMPLEMENTED } +SELECT * FROM t1 LEFT JOIN t2 ON NULL SETTINGS join_algorithm = 'partial_merge'; -- { serverError INVALID_JOIN_ON_EXPRESSION,NOT_IMPLEMENTED } +SELECT * FROM t1 RIGHT JOIN t2 ON NULL SETTINGS join_algorithm = 'auto'; -- { serverError INVALID_JOIN_ON_EXPRESSION,NOT_IMPLEMENTED } +SELECT * FROM t1 FULL JOIN t2 ON NULL SETTINGS join_algorithm = 'partial_merge'; -- { serverError INVALID_JOIN_ON_EXPRESSION,NOT_IMPLEMENTED } -- mixing of constant and non-constant expressions in ON is not allowed -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 1; -- { serverError AMBIGUOUS_COLUMN_NAME } -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 2; -- { serverError AMBIGUOUS_COLUMN_NAME } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 1 SETTINGS allow_experimental_analyzer = 0; -- { serverError AMBIGUOUS_COLUMN_NAME } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 1 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 2 SETTINGS allow_experimental_analyzer = 0; -- { serverError AMBIGUOUS_COLUMN_NAME } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 2 SETTINGS allow_experimental_analyzer = 1; -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 != 1; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 != 1 SETTINGS allow_experimental_analyzer = 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 != 1 SETTINGS allow_experimental_analyzer = 1; SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND NULL; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 'aaa'; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 'aaa'; -- { serverError INVALID_JOIN_ON_EXPRESSION,ILLEGAL_TYPE_OF_ARGUMENT } SELECT * FROM t1 JOIN t2 ON 'aaa'; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 0 SETTINGS allow_experimental_analyzer = 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 0 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS allow_experimental_analyzer = 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS allow_experimental_analyzer = 1; + +-- { echoOn } +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS allow_experimental_analyzer = 1; + +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS allow_experimental_analyzer = 1; + +-- { echoOff } DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; From 84e838d038f8b56807cecd4ca2cecee84352b7af Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 1 Jun 2023 10:52:04 +0000 Subject: [PATCH 177/634] Fix 02382_join_and_filtering_set Copy missing corresponding code from Interpreter, adjust test (removed with cube) --- src/Planner/PlannerJoinTree.cpp | 21 ++++++++++++++++++- tests/analyzer_tech_debt.txt | 1 - .../02382_join_and_filtering_set.reference | 3 --- .../02382_join_and_filtering_set.sql | 6 +++--- 4 files changed, 23 insertions(+), 8 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index c95671da6be..597de16cf04 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1248,7 +1248,26 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ const auto & join_clause = table_join->getOnlyClause(); bool kind_allows_filtering = isInner(join_kind) || isLeft(join_kind) || isRight(join_kind); - if (settings.max_rows_in_set_to_optimize_join > 0 && kind_allows_filtering) + + auto has_non_const = [](const Block & block, const auto & keys) + { + for (const auto & key : keys) + { + const auto & column = block.getByName(key).column; + if (column && !isColumnConst(*column)) + return true; + } + return false; + }; + + /// This optimization relies on the sorting that should buffer the whole stream before emitting any rows. + /// It doesn't hold such a guarantee for streams with const keys. + /// Note: it's also doesn't work with the read-in-order optimization. + /// No checks here because read in order is not applied if we have `CreateSetAndFilterOnTheFlyStep` in the pipeline between the reading and sorting steps. + bool has_non_const_keys = has_non_const(left_plan.getCurrentDataStream().header, join_clause.key_names_left) + && has_non_const(right_plan.getCurrentDataStream().header, join_clause.key_names_right); + + if (settings.max_rows_in_set_to_optimize_join > 0 && kind_allows_filtering && has_non_const_keys) { auto * left_set = add_create_set(left_plan, join_clause.key_names_left, JoinTableSide::Left); auto * right_set = add_create_set(right_plan, join_clause.key_names_right, JoinTableSide::Right); diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 95898bc70b0..a100210d36c 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -82,7 +82,6 @@ 02354_annoy 02366_union_decimal_conversion 02375_rocksdb_with_filters -02382_join_and_filtering_set 02402_merge_engine_with_view 02404_memory_bound_merging 02426_orc_bug diff --git a/tests/queries/0_stateless/02382_join_and_filtering_set.reference b/tests/queries/0_stateless/02382_join_and_filtering_set.reference index f81cf99bba7..8d68a1e392e 100644 --- a/tests/queries/0_stateless/02382_join_and_filtering_set.reference +++ b/tests/queries/0_stateless/02382_join_and_filtering_set.reference @@ -7,8 +7,5 @@ 10 bug with constant columns in join keys a a -a a -a a -a a 1 1 diff --git a/tests/queries/0_stateless/02382_join_and_filtering_set.sql b/tests/queries/0_stateless/02382_join_and_filtering_set.sql index b9dd44721dd..69bb8e7c222 100644 --- a/tests/queries/0_stateless/02382_join_and_filtering_set.sql +++ b/tests/queries/0_stateless/02382_join_and_filtering_set.sql @@ -22,20 +22,20 @@ SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.x % 2 == 0 AND t2.x % 2 = SELECT 'bug with constant columns in join keys'; SELECT * FROM ( SELECT 'a' AS key ) AS t1 -INNER JOIN ( SELECT 'a' AS key GROUP BY ignore(1), ignore(2) WITH CUBE ) AS t2 +INNER JOIN ( SELECT 'a' AS key ) AS t2 ON t1.key = t2.key ; SELECT count() > 1 FROM (EXPLAIN PIPELINE SELECT * FROM ( SELECT materialize('a') AS key ) AS t1 - INNER JOIN ( SELECT materialize('a') AS key GROUP BY ignore(1), ignore(2) WITH CUBE ) AS t2 + INNER JOIN ( SELECT materialize('a') AS key ) AS t2 ON t1.key = t2.key ) WHERE explain ilike '%FilterBySetOnTheFlyTransform%' ; SELECT count() == 0 FROM (EXPLAIN PIPELINE SELECT * FROM ( SELECT 'a' AS key ) AS t1 - INNER JOIN ( SELECT 'a' AS key GROUP BY ignore(1), ignore(2) WITH CUBE ) AS t2 + INNER JOIN ( SELECT 'a' AS key ) AS t2 ON t1.key = t2.key ) WHERE explain ilike '%FilterBySetOnTheFlyTransform%' ; From 94fa777b8d02ac1d9b2bb44d68efbc5ba71b177e Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 1 Jun 2023 10:52:05 +0000 Subject: [PATCH 178/634] Fix 01890_materialized_distributed_join Function getHeaderForProcessingStage handle join on top of query tree --- src/Interpreters/IdentifierSemantic.h | 1 - .../getHeaderForProcessingStage.cpp | 27 +++++++++++++++---- src/Storages/HDFS/StorageHDFSCluster.cpp | 3 +++ src/Storages/StorageS3Cluster.cpp | 5 ++++ ...90_materialized_distributed_join.reference | 2 ++ .../01890_materialized_distributed_join.sh | 1 + 6 files changed, 33 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/IdentifierSemantic.h b/src/Interpreters/IdentifierSemantic.h index 178bd291beb..bddaf3ed769 100644 --- a/src/Interpreters/IdentifierSemantic.h +++ b/src/Interpreters/IdentifierSemantic.h @@ -4,7 +4,6 @@ #include #include #include -#include #include namespace DB diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index 3f52b2f5099..58727673fbb 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -1,3 +1,5 @@ +#include +#include #include #include #include @@ -8,6 +10,7 @@ #include #include #include +#include #include namespace DB @@ -124,13 +127,27 @@ Block getHeaderForProcessingStage( ASTPtr query = query_info.query; if (const auto * select = query_info.query->as(); select && hasJoin(*select)) { - /// TODO: Analyzer syntax analyzer result if (!query_info.syntax_analyzer_result) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getHeaderForProcessingStage is unsupported"); + { + if (!query_info.planner_context) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query is not analyzed"); - query = query_info.query->clone(); - TreeRewriterResult new_rewriter_result = *query_info.syntax_analyzer_result; - removeJoin(*query->as(), new_rewriter_result, context); + const auto & query_node = query_info.query_tree->as(); + const auto & join_tree = query_node.getJoinTree(); + auto left_table_expression = extractLeftTableExpression(join_tree); + + auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(left_table_expression); + const auto & query_context = query_info.planner_context->getQueryContext(); + auto columns = table_expression_data.getColumns(); + auto new_query_node = buildSubqueryToReadColumnsFromTableExpression(columns, left_table_expression, query_context); + query = new_query_node->toAST(); + } + else + { + query = query_info.query->clone(); + TreeRewriterResult new_rewriter_result = *query_info.syntax_analyzer_result; + removeJoin(*query->as(), new_rewriter_result, context); + } } Block result; diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 83655b06cc8..e567e4fd87a 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -8,6 +8,9 @@ #include #include #include +#include +#include +#include #include #include diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index c8715938c6f..b1eea324269 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -6,6 +6,11 @@ #include #include +#include +#include +#include +#include +#include #include #include #include diff --git a/tests/queries/0_stateless/01890_materialized_distributed_join.reference b/tests/queries/0_stateless/01890_materialized_distributed_join.reference index 315ebca7e7a..80252c0b586 100644 --- a/tests/queries/0_stateless/01890_materialized_distributed_join.reference +++ b/tests/queries/0_stateless/01890_materialized_distributed_join.reference @@ -6,3 +6,5 @@ 1 1 1 2 2 4 +1 1 2 +1 1 2 diff --git a/tests/queries/0_stateless/01890_materialized_distributed_join.sh b/tests/queries/0_stateless/01890_materialized_distributed_join.sh index 8aca09303e0..64d92d61ac6 100755 --- a/tests/queries/0_stateless/01890_materialized_distributed_join.sh +++ b/tests/queries/0_stateless/01890_materialized_distributed_join.sh @@ -27,6 +27,7 @@ $CLICKHOUSE_CLIENT -nm -q " select * from test_distributed td asof join $CLICKHOUSE_DATABASE.test_local tl on td.k = tl.k and td.v < tl.v order by tl.v; select sum(td.v) from test_distributed td asof join $CLICKHOUSE_DATABASE.test_local tl on td.k = tl.k and td.v < tl.v group by tl.k; select sum(tl.v) from test_distributed td asof join $CLICKHOUSE_DATABASE.test_local tl on td.k = tl.k and td.v < tl.v group by td.k; + select td.k, tl.* from test_distributed td join $CLICKHOUSE_DATABASE.test_local tl on td.k = tl.k; drop table test_distributed; drop table test_source; From dcdcdc5a8697b1d74d265af576c097f49282dee2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 2 Jun 2023 10:56:55 +0000 Subject: [PATCH 179/634] fix style check --- src/Interpreters/getHeaderForProcessingStage.cpp | 1 - src/Storages/StorageS3Cluster.cpp | 1 - 2 files changed, 2 deletions(-) diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index 58727673fbb..3cd35584f79 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int UNSUPPORTED_METHOD; } bool hasJoin(const ASTSelectQuery & select) diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index b1eea324269..4e440cc15fd 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include From ef989fdb2e92f9589324af2ef9be5497eca07204 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 3 Aug 2023 15:23:47 +0000 Subject: [PATCH 180/634] Revert "Fix 01062_pm_all_join_with_block_continuation" This reverts commit 88865ace8efb3c10307ddce088a7c44ebb7009fd. --- ...62_pm_all_join_with_block_continuation.sql | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql b/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql index e7488f05fe5..15a28e92386 100644 --- a/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql +++ b/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql @@ -2,67 +2,67 @@ SET max_memory_usage = 50000000; SET join_algorithm = 'partial_merge'; SELECT 'defaults'; - -SELECT count(ignore(*)) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(10)) nums + +SELECT count(1) FROM ( + SELECT materialize(1) as k, n FROM numbers(10) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); -SELECT count(ignore(*)) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000)) nums +SELECT count(1) FROM ( + SELECT materialize(1) as k, n FROM numbers(1000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); -SELECT count(ignore(*)), uniqExact(n) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000000)) nums +SELECT count(1), uniqExact(n) FROM ( + SELECT materialize(1) as k, n FROM numbers(1000000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j USING k); -- errors SET max_joined_block_size_rows = 0; - -SELECT count(ignore(*)) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(10)) nums + +SELECT count(1) FROM ( + SELECT materialize(1) as k, n FROM numbers(10) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); -- { serverError 241 } -SELECT count(ignore(*)) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000)) nums +SELECT count(1) FROM ( + SELECT materialize(1) as k, n FROM numbers(1000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); -- { serverError 241 } SELECT 'max_joined_block_size_rows = 2000'; SET max_joined_block_size_rows = 2000; -SELECT count(ignore(*)) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(10)) nums +SELECT count(1) FROM ( + SELECT materialize(1) as k, n FROM numbers(10) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); -SELECT count(ignore(*)), uniqExact(n) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000)) nums +SELECT count(1), uniqExact(n) FROM ( + SELECT materialize(1) as k, n FROM numbers(1000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); -SELECT count(ignore(*)), uniqExact(n) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000000)) nums +SELECT count(1), uniqExact(n) FROM ( + SELECT materialize(1) as k, n FROM numbers(1000000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j USING k); SELECT 'max_rows_in_join = 1000'; SET max_rows_in_join = 1000; -SELECT count(ignore(*)) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(10)) nums +SELECT count(1) FROM ( + SELECT materialize(1) as k, n FROM numbers(10) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); -SELECT count(ignore(*)), uniqExact(n) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000)) nums +SELECT count(1), uniqExact(n) FROM ( + SELECT materialize(1) as k, n FROM numbers(1000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); -SELECT count(ignore(*)), uniqExact(n) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000000)) nums +SELECT count(1), uniqExact(n) FROM ( + SELECT materialize(1) as k, n FROM numbers(1000000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j USING k); From 3aac663bd2655360d503d4213fddbde3beaeb1df Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 5 Oct 2023 12:39:17 +0000 Subject: [PATCH 181/634] fix --- .../getHeaderForProcessingStage.cpp | 2 +- src/Planner/PlannerJoinTree.cpp | 4 +- src/Storages/HDFS/StorageHDFSCluster.cpp | 3 -- src/Storages/StorageS3Cluster.cpp | 4 -- ...01721_join_implicit_cast_long.reference.j2 | 38 ++++++++++--------- .../01721_join_implicit_cast_long.sql.j2 | 18 +++------ .../02886_missed_json_subcolumns.sql | 4 +- 7 files changed, 32 insertions(+), 41 deletions(-) diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index 3cd35584f79..d16e01ef2d2 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -129,7 +129,7 @@ Block getHeaderForProcessingStage( if (!query_info.syntax_analyzer_result) { if (!query_info.planner_context) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Query is not analyzed"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query is not analyzed: no planner context"); const auto & query_node = query_info.query_tree->as(); const auto & join_tree = query_node.getJoinTree(); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 597de16cf04..3a4d463f87e 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1260,8 +1260,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ return false; }; - /// This optimization relies on the sorting that should buffer the whole stream before emitting any rows. - /// It doesn't hold such a guarantee for streams with const keys. + /// This optimization relies on the sorting that should buffer data from both streams before emitting any rows. + /// Sorting on a stream with const keys can start returning rows immediately and pipeline may stuck. /// Note: it's also doesn't work with the read-in-order optimization. /// No checks here because read in order is not applied if we have `CreateSetAndFilterOnTheFlyStep` in the pipeline between the reading and sorting steps. bool has_non_const_keys = has_non_const(left_plan.getCurrentDataStream().header, join_clause.key_names_left) diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index e567e4fd87a..83655b06cc8 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -8,9 +8,6 @@ #include #include #include -#include -#include -#include #include #include diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 4e440cc15fd..c8715938c6f 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -6,10 +6,6 @@ #include #include -#include -#include -#include -#include #include #include #include diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 index 4d8d4255ce4..c363a417def 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 @@ -1,4 +1,3 @@ -{% for allow_experimental_analyzer in [0, 1] -%} {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} === {{ join_algorithm }} === = full = @@ -73,17 +72,20 @@ 9 9 0 10 10 0 = right = -{% if allow_experimental_analyzer -%} --4 0 -4 --3 0 -3 --2 0 -2 --1 0 -1 -{% else -%} 0 0 -4 0 0 -3 0 0 -2 0 0 -1 -{% endif -%} +0 0 0 +1 1 1 +2 2 2 +3 3 3 +4 4 4 +5 5 5 +-4 0 -4 +-3 0 -3 +-2 0 -2 +-1 0 -1 0 0 0 1 1 1 2 2 2 @@ -287,19 +289,21 @@ 3 3 3 4 4 4 5 5 5 -{% if allow_experimental_analyzer -%} --4 \N -4 --3 \N -3 --2 \N -2 --1 \N -1 -0 \N 0 -{% else -%} \N \N -4 \N \N -3 \N \N -2 \N \N -1 \N \N 0 -{% endif -%} +1 1 1 +2 2 2 +3 3 3 +4 4 4 +5 5 5 +-4 \N -4 +-3 \N -3 +-2 \N -2 +-1 \N -1 +0 \N 0 = inner = 1 1 1 2 2 2 @@ -418,6 +422,6 @@ 1 1 1 +1 {% endif -%} {% endfor -%} -{% endfor -%} diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 index c65665ee2a9..db004c13d96 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 @@ -12,10 +12,6 @@ INSERT INTO t2 SELECT number - 5 as a, 200 + number - 5 as b FROM system.numbers {% macro is_implemented(join_algorithm) -%} {% endmacro -%} -{% for allow_experimental_analyzer in [0, 1] -%} - -SET allow_experimental_analyzer = {{ allow_experimental_analyzer }}; - {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} SELECT '=== {{ join_algorithm }} ==='; @@ -41,7 +37,8 @@ SELECT a, t1.a, t2.a FROM t1 FULL JOIN t2 USING (a) ORDER BY (t1.a, t2.a); {{ is SELECT '= left ='; SELECT a, t1.a, t2.a FROM t1 LEFT JOIN t2 USING (a) ORDER BY (t1.a, t2.a); SELECT '= right ='; -SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a); {{ is_implemented(join_algorithm) }} +SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a) SETTINGS allow_experimental_analyzer = 0; {{ is_implemented(join_algorithm) }} +SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a) SETTINGS allow_experimental_analyzer = 1; {{ is_implemented(join_algorithm) }} SELECT '= inner ='; SELECT a, t1.a, t2.a FROM t1 INNER JOIN t2 USING (a) ORDER BY (t1.a, t2.a); @@ -122,7 +119,8 @@ SELECT a, t1.a, t2.a FROM t1 FULL JOIN t2 USING (a) ORDER BY (t1.a, t2.a); {{ is SELECT '= left ='; SELECT a, t1.a, t2.a FROM t1 LEFT JOIN t2 USING (a) ORDER BY (t1.a, t2.a); SELECT '= right ='; -SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a); {{ is_implemented(join_algorithm) }} +SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a) SETTINGS allow_experimental_analyzer = 0; {{ is_implemented(join_algorithm) }} +SELECT a, t1.a, t2.a FROM t1 RIGHT JOIN t2 USING (a) ORDER BY (t1.a, t2.a) SETTINGS allow_experimental_analyzer = 1; {{ is_implemented(join_algorithm) }} SELECT '= inner ='; SELECT a, t1.a, t2.a FROM t1 INNER JOIN t2 USING (a) ORDER BY (t1.a, t2.a); @@ -168,11 +166,8 @@ SELECT '= types ='; SELECT any(toTypeName(a)) == 'Nullable(Int32)' AND any(toTypeName(t2.a)) == 'Nullable(Int32)' FROM t1 FULL JOIN t2 USING (a); {{ is_implemented(join_algorithm) }} SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(t2.a)) == 'Nullable(Int32)' FROM t1 LEFT JOIN t2 USING (a); -{% if allow_experimental_analyzer -%} -SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 RIGHT JOIN t2 USING (a); {{ is_implemented(join_algorithm) }} -{% else -%} -SELECT any(toTypeName(a)) == 'Nullable(Int32)' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 RIGHT JOIN t2 USING (a); {{ is_implemented(join_algorithm) }} -{% endif -%} +SELECT any(toTypeName(a)) == 'Nullable(Int32)' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 RIGHT JOIN t2 USING (a) SETTINGS allow_experimental_analyzer = 0; {{ is_implemented(join_algorithm) }} +SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 RIGHT JOIN t2 USING (a) SETTINGS allow_experimental_analyzer = 1; {{ is_implemented(join_algorithm) }} SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(t2.a)) == 'Int32' FROM t1 INNER JOIN t2 USING (a); @@ -188,7 +183,6 @@ SELECT toTypeName(any(a)) == 'Nullable(UInt16)' AND toTypeName(any(t2.a)) == 'Nu SET join_use_nulls = 0; {% endif -%} -{% endfor -%} {% endfor -%} DROP TABLE IF EXISTS t1; diff --git a/tests/queries/0_stateless/02886_missed_json_subcolumns.sql b/tests/queries/0_stateless/02886_missed_json_subcolumns.sql index 90a80509e99..9984809ce21 100644 --- a/tests/queries/0_stateless/02886_missed_json_subcolumns.sql +++ b/tests/queries/0_stateless/02886_missed_json_subcolumns.sql @@ -1,4 +1,4 @@ -DROP TABLE IF EXISTS t_mutations_subcolumns; +DROP TABLE IF EXISTS t_missed_subcolumns; SET allow_experimental_object_type = 1; @@ -14,7 +14,7 @@ OPTIMIZE TABLE t_missed_subcolumns FINAL; SELECT count(), min(id) FROM t_missed_subcolumns; -SELECT * FROM t_missed_subcolumns WHERE obj.k4 = 5 ORDER BY id FORMAT JSONEachRow; +SELECT * FROM t_missed_subcolumns WHERE obj.k4 = 5 ORDER BY id FORMAT JSONEachRow; SELECT * FROM t_missed_subcolumns WHERE obj.k1.k3 = 'fee' ORDER BY id FORMAT JSONEachRow; From 6dbc224fc702b44463d160ff5ed335480f945d17 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 12 Oct 2023 12:50:05 +0200 Subject: [PATCH 182/634] Update table.md --- docs/en/sql-reference/statements/create/table.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 5e94a5fdc6f..c6fad5dc2da 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -487,7 +487,7 @@ ClickHouse supports temporary tables which have the following characteristics: - The DB can’t be specified for a temporary table. It is created outside of databases. - Impossible to create a temporary table with distributed DDL query on all cluster servers (by using `ON CLUSTER`): this table exists only in the current session. - If a temporary table has the same name as another one and a query specifies the table name without specifying the DB, the temporary table will be used. -- For distributed query processing, temporary tables used in a query are passed to remote servers. +- For distributed query processing, temporary tables with Memory engine used in a query are passed to remote servers. To create a temporary table, use the following syntax: From 22f5d6d7085ab9f1b0b8d005bd0d36887a610d49 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 12 Oct 2023 10:57:21 +0000 Subject: [PATCH 183/634] Reset promise_to_build --- src/Processors/Transforms/CreatingSetsTransform.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index bf1f34ba181..e2b7e990685 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -75,6 +75,7 @@ void CreatingSetsTransform::work() try { promise_to_build->set_exception(std::current_exception()); + promise_to_build.reset(); } catch (...) { From 9ab9075c2bc300667e8a0fb4790d1df9ec2e623a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 12 Oct 2023 13:59:33 +0200 Subject: [PATCH 184/634] update again --- docker/test/base/setup_export_logs.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 4a3a24ac8cd..2d6e3f76d33 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -15,7 +15,7 @@ CLICKHOUSE_CI_LOGS_USER=${CLICKHOUSE_CI_LOGS_USER:-ci} # Pre-configured destination cluster, where to export the data CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export} -EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime, check_name String, instance_type String, instance_id String, "} +EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt16, commit_sha String, check_start_time DateTime('UTC'), check_name String, instance_type String, instance_id String, "} EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"0 AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type, '' AS instance_id"} EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "} @@ -127,7 +127,7 @@ function setup_logs_replication do # Calculate hash of its structure. Note: 1 is the version of extra columns - increment it if extra columns are changed: hash=$(clickhouse-client --query " - SELECT sipHash64(2, groupArray((name, type))) + SELECT sipHash64(3, groupArray((name, type))) FROM (SELECT name, type FROM system.columns WHERE database = 'system' AND table = '$table' ORDER BY position) From 6e927325e404084f44cd30805cbc5266f07cf8bb Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Thu, 12 Oct 2023 20:32:36 +0800 Subject: [PATCH 185/634] fix CheckSytel --- tests/integration/test_keeper_client/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 08d0825c118..a00e3d31231 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -217,6 +217,7 @@ def test_quoted_argument_parsing(client: KeeperClient): client.execute_query(f"set '{node_path}' \"value4 with some whitespace\" 3") assert client.get(node_path) == "value4 with some whitespace" + def test_get_all_children_number(client: KeeperClient): client.touch("/test_get_all_children_number") client.touch("/test_get_all_children_number/1") @@ -232,6 +233,7 @@ def test_get_all_children_number(client: KeeperClient): client.touch("/test_get_all_children_number/2/4") response = client.get_all_children_number("/test_get_all_children_number") + assert response == TSV( [ ["11"] From 2cbb069b68f480e1ccfea70dd3120d6dabaf1149 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 12 Oct 2023 12:39:08 +0200 Subject: [PATCH 186/634] Add ability to ignore data after semicolon in Values format This is required for client, to handle comments in multiquery mode. v0: separate context for input format v2: cannot use separate context since params and stuff are changed in global context v3: do not sent this setting to the server (breaks queries for readonly profiles) Signed-off-by: Azat Khuzhin --- programs/client/Client.cpp | 9 +++++++++ programs/local/LocalServer.cpp | 9 +++++++++ src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/ValuesBlockInputFormat.cpp | 4 +++- ...at_values_allow_data_after_semicolon.reference | 6 ++++++ ...ut_format_values_allow_data_after_semicolon.sh | 15 +++++++++++++++ 8 files changed, 45 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.reference create mode 100755 tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.sh diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 7d70b81a178..9f316e54f85 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1264,6 +1264,15 @@ void Client::processConfig() global_context->setQueryKindInitial(); global_context->setQuotaClientKey(config().getString("quota_key", "")); global_context->setQueryKind(query_kind); + + if (is_multiquery && !global_context->getSettingsRef().input_format_values_allow_data_after_semicolon.changed) + { + Settings settings = global_context->getSettings(); + settings.input_format_values_allow_data_after_semicolon = true; + /// Do not send it to the server + settings.input_format_values_allow_data_after_semicolon.changed = false; + global_context->setSettings(settings); + } } diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 9fb629a0871..e074cb638e8 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -783,6 +783,15 @@ void LocalServer::processConfig() global_context->setQueryKindInitial(); global_context->setQueryKind(query_kind); + + if (is_multiquery && !global_context->getSettingsRef().input_format_values_allow_data_after_semicolon.changed) + { + Settings settings = global_context->getSettings(); + settings.input_format_values_allow_data_after_semicolon = true; + /// Do not send it to the server + settings.input_format_values_allow_data_after_semicolon.changed = false; + global_context->setSettings(settings); + } } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c08425c03fd..c60d6511abc 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -963,6 +963,7 @@ class IColumn; M(Bool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \ M(Bool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \ M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \ + M(Bool, input_format_values_allow_data_after_semicolon, false, "For Values format: allow extra data after semicolon (used by client to interpret comments).", 0) \ M(Bool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \ /** This setting is obsolete and do nothing, left for compatibility reasons. */ \ M(Bool, input_format_avro_null_as_default, false, "For Avro/AvroConfluent format: insert default in case of null and non Nullable column", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 862e51aa088..b355d785715 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -170,6 +170,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.tsv.skip_trailing_empty_lines = settings.input_format_tsv_skip_trailing_empty_lines; format_settings.tsv.allow_variable_number_of_columns = settings.input_format_tsv_allow_variable_number_of_columns; format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals; + format_settings.values.allow_data_after_semicolon = settings.input_format_values_allow_data_after_semicolon; format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions; format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; format_settings.with_names_use_header = settings.input_format_with_names_use_header; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index ef8df02b208..655aaa81d35 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -341,6 +341,7 @@ struct FormatSettings bool interpret_expressions = true; bool deduce_templates_of_expressions = true; bool accurate_types_of_literals = true; + bool allow_data_after_semicolon = false; } values; enum class ORCCompression diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 4a209fba3c5..b0ee2f7797a 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -615,8 +615,10 @@ void ValuesBlockInputFormat::readSuffix() { ++buf->position(); skipWhitespaceIfAny(*buf); - if (buf->hasUnreadData() || !buf->eof()) + if (buf->hasUnreadData()) throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read data after semicolon"); + if (!format_settings.values.allow_data_after_semicolon && !buf->eof()) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0)"); return; } diff --git a/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.reference b/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.reference new file mode 100644 index 00000000000..250a673a26b --- /dev/null +++ b/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.reference @@ -0,0 +1,6 @@ +client no multiquery +Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0) +client multiquery +local no multiquery +Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0) +local multiquery diff --git a/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.sh b/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.sh new file mode 100755 index 00000000000..8164c91b2ae --- /dev/null +++ b/tests/queries/0_stateless/02898_input_format_values_allow_data_after_semicolon.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo "client no multiquery" +$CLICKHOUSE_CLIENT -q "insert into function null() values (1); -- { foo }" |& grep -F -o "Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0)" +echo "client multiquery" +$CLICKHOUSE_CLIENT -n -q "insert into function null() values (1); -- { foo }" + +echo "local no multiquery" +$CLICKHOUSE_LOCAL -q "insert into function null() values (1); -- { foo }" |& grep -F -o "Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0)" +echo "local multiquery" +$CLICKHOUSE_LOCAL -n -q "insert into function null() values (1); -- { foo }" From fd16e52164a6a749ce70805cd3bd404b0de93686 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 12 Oct 2023 14:57:03 +0200 Subject: [PATCH 187/634] Update 00626_replace_partition_from_table_zookeeper to avoid using comments after semicolon It does not have multiquery, so better to use shell comments Signed-off-by: Azat Khuzhin --- .../00626_replace_partition_from_table_zookeeper.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index 334025cba28..ffbf4df4ba7 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -96,7 +96,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE src;" $CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2); -- trash part to be deleted" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2);" # trash part to be deleted # Stop replication at the second replica and remove source table to use fetch instead of copying $CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES dst_r2;" @@ -116,7 +116,7 @@ query_with_retry "ALTER TABLE dst_r1 DROP PARTITION 1;" $CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2); -- trash part to be deleted" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2);" # trash part to be deleted $CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES dst_r2;" $CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES dst_r2;" From 369ebf3c4f41275fe9cb878f668b848c23672a26 Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Thu, 12 Oct 2023 21:15:52 +0800 Subject: [PATCH 188/634] fix CheckSytel --- tests/integration/helpers/keeper_utils.py | 6 +++--- tests/integration/test_keeper_client/test.py | 8 +------- 2 files changed, 4 insertions(+), 10 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 80b221811f7..b287bf5d3f7 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -142,13 +142,13 @@ class KeeperClient(object): def find_super_nodes(self, threshold: int, timeout: float = 60.0) -> str: return self.execute_query(f"find_super_nodes {threshold}", timeout) + + def get_all_children_number(self, path: str, timeout: float = 60.0) -> str: + return self.execute_query(f"get_all_children_number {path}", timeout) def delete_stale_backups(self, timeout: float = 60.0) -> str: return self.execute_query("delete_stale_backups", timeout) - def get_all_children_number(self, path: str, timeout: float = 60.0) -> str: - return self.execute_query(f"get_all_children_number {path}", timeout) - def reconfig( self, joining: tp.Optional[str], diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index a00e3d31231..e01c463d94a 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -232,10 +232,4 @@ def test_get_all_children_number(client: KeeperClient): client.touch("/test_get_all_children_number/2/3") client.touch("/test_get_all_children_number/2/4") - response = client.get_all_children_number("/test_get_all_children_number") - - assert response == TSV( - [ - ["11"] - ] - ) + assert client.get_all_children_number("/test_get_all_children_number") == ["11"] From e646fb7613964a45f1c97236b284828ea1679c39 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 12 Oct 2023 13:42:02 +0000 Subject: [PATCH 189/634] Remove executable flag --- tests/queries/0_stateless/02891_rename_table_without_keyword.sql | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/02891_rename_table_without_keyword.sql diff --git a/tests/queries/0_stateless/02891_rename_table_without_keyword.sql b/tests/queries/0_stateless/02891_rename_table_without_keyword.sql old mode 100755 new mode 100644 From 7d5c15dfc9ac34ff2ee8f4cd1bd97420a8c09cd7 Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Thu, 12 Oct 2023 22:01:37 +0800 Subject: [PATCH 190/634] fix python CheckSytel --- tests/integration/helpers/keeper_utils.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index b287bf5d3f7..478bb5ebf72 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -142,13 +142,13 @@ class KeeperClient(object): def find_super_nodes(self, threshold: int, timeout: float = 60.0) -> str: return self.execute_query(f"find_super_nodes {threshold}", timeout) - + def get_all_children_number(self, path: str, timeout: float = 60.0) -> str: return self.execute_query(f"get_all_children_number {path}", timeout) def delete_stale_backups(self, timeout: float = 60.0) -> str: return self.execute_query("delete_stale_backups", timeout) - + def reconfig( self, joining: tp.Optional[str], From 470c858f212456a633e53f7aab67ef289070a92d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 12 Oct 2023 16:06:00 +0200 Subject: [PATCH 191/634] Update src/Interpreters/InterpreterSelectQuery.cpp --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e0aceb1b108..0300cc4c9b4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -691,7 +691,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( "Storage {} doesn't support sampling", storage->getStorageID().getNameForLogs()); else - throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, "Illegal SAMPLE"); + throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, "Illegal SAMPLE: sampling is only allowed with the table engines that support it"); } if (query.final() && (input_pipe || !storage || !storage->supportsFinal())) From 1917a882b77f9361b3ab1401ce8a056ba513c103 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 12 Oct 2023 16:32:56 +0200 Subject: [PATCH 192/634] Better --- .../DatabaseMaterializedPostgreSQL.cpp | 13 +-- .../MaterializedPostgreSQLConsumer.cpp | 5 +- .../PostgreSQLReplicationHandler.cpp | 92 ++++++++++++++++--- .../PostgreSQL/PostgreSQLReplicationHandler.h | 14 +-- .../StorageMaterializedPostgreSQL.cpp | 13 +-- tests/integration/helpers/postgres_utility.py | 19 +++- .../test.py | 43 +++++++++ 7 files changed, 153 insertions(+), 46 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 71a508e818b..c03add1a96f 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -63,20 +63,11 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() if (shutdown_called) return; - String replication_identifier; - if (settings->materialized_postgresql_use_unique_replication_consumer_identifier) - { - replication_identifier = fmt::format("{}_{}", getUUID(), TSA_SUPPRESS_WARNING_FOR_READ(database_name)); - } - else - { - replication_identifier = TSA_SUPPRESS_WARNING_FOR_READ(database_name); - } - replication_handler = std::make_unique( - replication_identifier, remote_database_name, + /* table_name */"", TSA_SUPPRESS_WARNING_FOR_READ(database_name), /// FIXME + toString(getUUID()), connection_info, getContext(), is_attach, diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 502ced12e75..cfd1771aa46 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -574,6 +574,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl void MaterializedPostgreSQLConsumer::syncTables() { + size_t synced_tables = 0; while (!tables_to_sync.empty()) { auto table_name = *tables_to_sync.begin(); @@ -604,6 +605,7 @@ void MaterializedPostgreSQLConsumer::syncTables() CompletedPipelineExecutor executor(io.pipeline); executor.execute(); + ++synced_tables; } } catch (...) @@ -616,7 +618,8 @@ void MaterializedPostgreSQLConsumer::syncTables() tables_to_sync.erase(tables_to_sync.begin()); } - LOG_DEBUG(log, "Table sync end for {} tables, last lsn: {} = {}, (attempted lsn {})", tables_to_sync.size(), current_lsn, getLSNValue(current_lsn), getLSNValue(final_lsn)); + LOG_DEBUG(log, "Table sync end for {} tables, last lsn: {} = {}, (attempted lsn {})", + synced_tables, current_lsn, getLSNValue(current_lsn), getLSNValue(final_lsn)); updateLsn(); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index f57a6a26a62..ee38dcb44d4 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -17,12 +17,14 @@ #include #include #include +#include namespace DB { static const auto CLEANUP_RESCHEDULE_MS = 600000 * 3; /// 30 min +static constexpr size_t replication_slot_name_max_size = 64; namespace ErrorCodes { @@ -56,10 +58,70 @@ private: }; +namespace +{ + /// There can be several replication slots per publication, but one publication per table/database replication. + /// Replication slot might be unique (contain uuid) to allow have multiple replicas for the same PostgreSQL table/database. + + String getPublicationName(const String & postgres_database, const String & postgres_table) + { + return fmt::format( + "{}_ch_publication", + postgres_table.empty() ? postgres_database : fmt::format("{}_{}", postgres_database, postgres_table)); + } + + void checkReplicationSlot(String name) + { + for (const auto & c : name) + { + const bool ok = (std::isalpha(c) && std::islower(c)) || std::isdigit(c) || c == '_'; + if (!ok) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Replication slot can contain lower-case letters, numbers, and the underscore character. " + "Got: {}", name); + } + } + + if (name.size() > replication_slot_name_max_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Too big replication slot size: {}", name); + } + + String normalizeReplicationSlot(String name) + { + name = Poco::toLower(name); + for (auto & c : name) + if (c == '-') + c = '_'; + return name; + } + + String getReplicationSlotName( + const String & postgres_database, + const String & postgres_table, + const String & clickhouse_uuid, + const MaterializedPostgreSQLSettings & replication_settings) + { + String slot_name = replication_settings.materialized_postgresql_replication_slot; + if (slot_name.empty()) + { + if (replication_settings.materialized_postgresql_use_unique_replication_consumer_identifier) + slot_name = clickhouse_uuid; + else + slot_name = postgres_table.empty() ? postgres_database : fmt::format("{}_{}_ch_replication_slot", postgres_database, postgres_table); + + slot_name = normalizeReplicationSlot(slot_name); + } + return slot_name; + } +} + PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( - const String & replication_identifier, const String & postgres_database_, - const String & current_database_name_, + const String & postgres_table_, + const String & clickhouse_database_, + const String & clickhouse_uuid_, const postgres::ConnectionInfo & connection_info_, ContextPtr context_, bool is_attach_, @@ -70,14 +132,18 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , is_attach(is_attach_) , postgres_database(postgres_database_) , postgres_schema(replication_settings.materialized_postgresql_schema) - , current_database_name(current_database_name_) + , current_database_name(clickhouse_database_) , connection_info(connection_info_) , max_block_size(replication_settings.materialized_postgresql_max_block_size) , is_materialized_postgresql_database(is_materialized_postgresql_database_) , tables_list(replication_settings.materialized_postgresql_tables_list) , schema_list(replication_settings.materialized_postgresql_schema_list) , schema_as_a_part_of_table_name(!schema_list.empty() || replication_settings.materialized_postgresql_tables_list_with_schema) + , user_managed_slot(!replication_settings.materialized_postgresql_replication_slot.value.empty()) , user_provided_snapshot(replication_settings.materialized_postgresql_snapshot) + , replication_slot(getReplicationSlotName(postgres_database_, postgres_table_, clickhouse_uuid_, replication_settings)) + , tmp_replication_slot(replication_slot + "_tmp") + , publication_name(getPublicationName(postgres_database_, postgres_table_)) , reschedule_backoff_min_ms(replication_settings.materialized_postgresql_backoff_min_ms) , reschedule_backoff_max_ms(replication_settings.materialized_postgresql_backoff_max_ms) , reschedule_backoff_factor(replication_settings.materialized_postgresql_backoff_factor) @@ -89,13 +155,9 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( if (!schema_list.empty() && !postgres_schema.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot have schema list and common schema at the same time"); - replication_slot = replication_settings.materialized_postgresql_replication_slot; - if (replication_slot.empty()) - { - user_managed_slot = false; - replication_slot = fmt::format("{}_ch_replication_slot", replication_identifier); - } - publication_name = fmt::format("{}_ch_publication", replication_identifier); + checkReplicationSlot(replication_slot); + + LOG_INFO(log, "Using replication slot {} and publication {}", replication_slot, publication_name); startup_task = getContext()->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ checkConnectionAndStart(); }); consumer_task = getContext()->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); @@ -496,7 +558,7 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::nontransactio throw Exception(ErrorCodes::LOGICAL_ERROR, "No table found to be replicated"); /// 'ONLY' means just a table, without descendants. - std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", publication_name, tables_list); + std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", doubleQuoteString(publication_name), tables_list); try { tx.exec(query_str); @@ -519,7 +581,7 @@ bool PostgreSQLReplicationHandler::isReplicationSlotExist(pqxx::nontransaction & { String slot_name; if (temporary) - slot_name = replication_slot + "_tmp"; + slot_name = tmp_replication_slot; else slot_name = replication_slot; @@ -546,11 +608,11 @@ void PostgreSQLReplicationHandler::createReplicationSlot( String query_str, slot_name; if (temporary) - slot_name = replication_slot + "_tmp"; + slot_name = tmp_replication_slot; else slot_name = replication_slot; - query_str = fmt::format("CREATE_REPLICATION_SLOT {} LOGICAL pgoutput EXPORT_SNAPSHOT", slot_name); + query_str = fmt::format("CREATE_REPLICATION_SLOT {} LOGICAL pgoutput EXPORT_SNAPSHOT", doubleQuoteString(slot_name)); try { @@ -573,7 +635,7 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(pqxx::nontransaction & tx std::string slot_name; if (temporary) - slot_name = replication_slot + "_tmp"; + slot_name = tmp_replication_slot; else slot_name = replication_slot; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 4c16ff95692..5d426b3c512 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -21,9 +21,10 @@ public: using ConsumerPtr = std::shared_ptr; PostgreSQLReplicationHandler( - const String & replication_identifier, const String & postgres_database_, - const String & current_database_name_, + const String & postgres_table_, + const String & clickhouse_database_, + const String & clickhouse_uuid_, const postgres::ConnectionInfo & connection_info_, ContextPtr context_, bool is_attach_, @@ -128,10 +129,11 @@ private: /// This is possible to allow replicating tables from multiple schemas in the same MaterializedPostgreSQL database engine. mutable bool schema_as_a_part_of_table_name = false; - bool user_managed_slot = true; - String user_provided_snapshot; - - String replication_slot, publication_name; + const bool user_managed_slot; + const String user_provided_snapshot; + const String replication_slot; + const String tmp_replication_slot; + const String publication_name; /// Replication consumer. Manages decoding of replication stream and syncing into tables. ConsumerPtr consumer; diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index d871b846ece..d83722dba6c 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -74,22 +74,13 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( setInMemoryMetadata(storage_metadata); - String replication_identifier; - if (replication_settings->materialized_postgresql_use_unique_replication_consumer_identifier) - { - replication_identifier = fmt::format("{}_{}_{}", table_id_.uuid, remote_database_name, remote_table_name_); - } - else - { - replication_identifier = fmt::format("{}_{}", remote_database_name, remote_table_name_); - } - replication_settings->materialized_postgresql_tables_list = remote_table_name_; replication_handler = std::make_unique( - replication_identifier, remote_database_name, + remote_table_name_, table_id_.database_name, + toString(table_id_.uuid), connection_info, getContext(), is_attach, diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 3c8a23b15a2..77dc72f8020 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -113,11 +113,19 @@ class PostgresManager: self.created_materialized_postgres_db_list = set() self.created_ch_postgres_db_list = set() - def init(self, instance, ip, port, default_database="postgres_database"): + def init( + self, + instance, + ip, + port, + default_database="postgres_database", + postgres_db_exists=False, + ): self.instance = instance self.ip = ip self.port = port self.default_database = default_database + self.postgres_db_exists = postgres_db_exists self.prepare() def get_default_database(self): @@ -138,7 +146,8 @@ class PostgresManager: self.conn = get_postgres_conn(ip=self.ip, port=self.port) self.cursor = self.conn.cursor() if self.default_database != "": - self.create_postgres_db(self.default_database) + if not self.postgres_db_exists: + self.create_postgres_db(self.default_database) self.conn = get_postgres_conn( ip=self.ip, port=self.port, @@ -364,6 +373,12 @@ def check_tables_are_synchronized( time.sleep(1) result = instance.query(result_query) + if result != expected: + count = int(instance.query(f"select count() from {table_path}")) + expected_count = int( + instance.query(f"select count() from {postgres_database}.{table_name}") + ) + print(f"Having {count}, expected {expected_count}") assert result == expected diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 3f2ec74180b..903d246e1c5 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -719,6 +719,49 @@ def test_too_many_parts(started_cluster): pg_manager2.drop_materialized_db() +def test_replica_consumer(started_cluster): + table = "test_replica_consumer" + + pg_manager_replica = PostgresManager() + pg_manager_replica.init( + instance2, + cluster.postgres_ip, + cluster.postgres_port, + default_database="postgres_database", + postgres_db_exists=True + ) + + for pm in [pg_manager, pg_manager_replica]: + pm.create_and_fill_postgres_table(table) + pm.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = '{table}'", + "materialized_postgresql_backoff_min_ms = 100", + "materialized_postgresql_backoff_max_ms = 100", + "materialized_postgresql_use_unique_replication_consumer_identifier = 1" + ], + ) + + assert 50 == int(instance.query(f"SELECT count() FROM test_database.{table}")) + assert 50 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) + + instance.query(f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(1000, 1000)") + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + check_tables_are_synchronized( + instance2, table, postgres_database=pg_manager_replica.get_default_database() + ) + + assert 1050 == int(instance.query(f"SELECT count() FROM test_database.{table}")) + assert 1050 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) + + pg_manager_replica.clear() + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From a2ee30707df12b738546bc1f706cd832bc5c1c94 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Oct 2023 16:59:03 +0200 Subject: [PATCH 193/634] Better error message in case when merge selecting task failed (#55554) --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index b3a91add879..f78b383e173 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -166,18 +166,20 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( String best_partition_id_to_optimize = getBestPartitionToOptimizeEntire(info.partitions_info); if (!best_partition_id_to_optimize.empty()) { - return selectAllPartsToMergeWithinPartition( - future_part, - can_merge_callback, - best_partition_id_to_optimize, - /*final=*/true, - metadata_snapshot, - txn, - out_disable_reason, - /*optimize_skip_merged_partitions=*/true); + return selectAllPartsToMergeWithinPartition( + future_part, + can_merge_callback, + best_partition_id_to_optimize, + /*final=*/true, + metadata_snapshot, + txn, + out_disable_reason, + /*optimize_skip_merged_partitions=*/true); } - out_disable_reason = "There is no need to merge parts according to merge selector algorithm"; + if (!out_disable_reason.empty()) + out_disable_reason += ". "; + out_disable_reason += "There is no need to merge parts according to merge selector algorithm"; return SelectPartsDecision::CANNOT_SELECT; } From 81bbb0fce1ade9a891c612d267455e2f68907858 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Oct 2023 17:00:25 +0200 Subject: [PATCH 194/634] Improve merge selecting algorithm for big partitions (#55526) Improved overall resilience for ClickHouse in case of many parts within partition (more than 1000). It might reduce the number of TOO_MANY_PARTS errors. --- .../MergeTree/SimpleMergeSelector.cpp | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/SimpleMergeSelector.cpp b/src/Storages/MergeTree/SimpleMergeSelector.cpp index 7e7539f71d5..ee103945342 100644 --- a/src/Storages/MergeTree/SimpleMergeSelector.cpp +++ b/src/Storages/MergeTree/SimpleMergeSelector.cpp @@ -157,12 +157,21 @@ void selectWithinPartition( if (parts_count <= 1) return; - for (size_t begin = 0; begin < parts_count; ++begin) - { - /// If too many parts, select only from first, to avoid complexity. - if (begin > 1000) - break; + /// If the parts in the parts vector are sorted by block number, + /// it may not be ideal to only select parts for merging from the first N ones. + /// This is because if there are more than N parts in the partition, + /// we will not be able to assign a merge for newly created parts. + /// As a result, the total number of parts within the partition could + /// grow uncontrollably, similar to a snowball effect. + /// To address this we will try to assign a merge taking into consideration + /// only last N parts. + static constexpr size_t parts_threshold = 1000; + size_t begin = 0; + if (parts_count >= parts_threshold) + begin = parts_count - parts_threshold; + for (; begin < parts_count; ++begin) + { if (!parts[begin].shall_participate_in_merges) continue; From 20e16be27b022ec29f11f5e9b29560dc7550ef07 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 12 Oct 2023 15:02:08 +0000 Subject: [PATCH 195/634] Fixes & improvements --- .../settings.md | 17 +++ docs/en/sql-reference/statements/system.md | 8 +- programs/server/config.xml | 9 ++ src/Databases/DatabaseReplicated.cpp | 137 +++++++++++------- src/Databases/DatabaseReplicated.h | 17 ++- src/Interpreters/Cluster.cpp | 3 +- src/Interpreters/Cluster.h | 2 + src/Interpreters/InterpreterSystemQuery.cpp | 8 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 12 +- src/Parsers/ASTSystemQuery.h | 1 + src/Parsers/ParserSystemQuery.cpp | 8 + .../configs/backup_group.xml | 2 +- .../test.py | 19 ++- 13 files changed, 170 insertions(+), 73 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index a6ae517e401..ab8210489b2 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1330,6 +1330,23 @@ For more information, see the section [Creating replicated tables](../../engines ``` +## replica_group_name {#replica_group_name} + +Replica group name for database Replicated. + +The cluster created by Replicated database will consist of replicas in the same group. +DDL queries will only wail for the replicas in the same group. + +Empty by default. + +**Example** + +``` xml +backups +``` + +Default value: ``. + ## max_open_files {#max-open-files} The maximum number of open files. diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index fb601cd5d35..c51f7254b6c 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -97,12 +97,12 @@ The fourth one is useful to remove metadata of dead replica when all other repli Dead replicas of `Replicated` databases can be dropped using following syntax: ``` sql -SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] FROM DATABASE database; -SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name']; -SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] FROM ZKPATH '/path/to/table/in/zk'; +SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] [FROM GROUP 'group_name'] FROM DATABASE database; +SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] [FROM GROUP 'group_name']; +SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] [FROM GROUP 'group_name'] FROM ZKPATH '/path/to/table/in/zk'; ``` -Similar to `SYSTEM DROP REPLICA`, but removes the `Replicated` database replica path from ZooKeeper when there's no database to run `DROP DATABASE`. Please note that it does not remove `ReplicatedMergeTree` replicas (so you may need `SYSTEM DROP REPLICA` as well). Shard and replica names are the names that were specified in `Replicated` engine arguments when creating the database. Also, these names can be obtained from `database_shard_name` and `database_replica_name` columns in `system.clusters`. If the `FROM SHARD` clause is missing, then `replica_name` must be a full replica name in `shard_name|replica_name` format. +Similar to `SYSTEM DROP REPLICA`, but removes the `Replicated` database replica path from ZooKeeper when there's no database to run `DROP DATABASE`. Please note that it does not remove `ReplicatedMergeTree` replicas (so you may need `SYSTEM DROP REPLICA` as well). Shard and replica names are the names that were specified in `Replicated` engine arguments when creating the database. Also, these names can be obtained from `database_shard_name` and `database_replica_name` columns in `system.clusters`. Replica group name is the name defined by `replica_group_name` [setting](../../operations/server-configuration-parameters/settings.md#replica_group_name) in the server configuration. If the `FROM SHARD` clause is missing, then `replica_name` must be a full replica name in `shard_name|replica_name` format if replica groups are not used and in `shard_name|replica_name|group_name` otherwise. ## DROP UNCOMPRESSED CACHE diff --git a/programs/server/config.xml b/programs/server/config.xml index 2a7dc1e576a..4a6f0fa8b35 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -914,6 +914,15 @@ --> + + + 3600 diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 8e4460a58bd..c09c24b9fc5 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -117,29 +117,51 @@ DatabaseReplicated::DatabaseReplicated( if (!db_settings.collection_name.value.empty()) fillClusterAuthInfo(db_settings.collection_name.value, context_->getConfigRef()); - cluster_path = zookeeper_path + "/" + getClusterGroup(context_->getConfigRef()); + replica_group_name = context_->getConfigRef().getString("replica_group_name", ""); + + if (replica_group_name.find('/') != std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica group name should not contain '/': {}", replica_group_name); + if (replica_group_name.find('|') != std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica group name should not contain '|': {}", replica_group_name); } -String DatabaseReplicated::getFullReplicaName(const String & shard, const String & replica) +String DatabaseReplicated::getFullReplicaName(const String & shard, const String & replica, const String & replica_group) { - return shard + '|' + replica; + if (replica_group.empty()) + return shard + '|' + replica; + else + return shard + '|' + replica + '|' + replica_group; } String DatabaseReplicated::getFullReplicaName() const { - return getFullReplicaName(shard_name, replica_name); + return getFullReplicaName(shard_name, replica_name, replica_group_name); } -std::pair DatabaseReplicated::parseFullReplicaName(const String & name) +DatabaseReplicated::NameParts DatabaseReplicated::parseFullReplicaName(const String & name) { - String shard; - String replica; - auto pos = name.find('|'); - if (pos == std::string::npos || name.find('|', pos + 1) != std::string::npos) + NameParts parts; + + auto pos_first = name.find('|'); + if (pos_first == std::string::npos) throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect replica identifier: {}", name); - shard = name.substr(0, pos); - replica = name.substr(pos + 1); - return {shard, replica}; + + parts.shard = name.substr(0, pos_first); + + auto pos_second = name.find('|', pos_first + 1); + if (pos_second == std::string::npos) + { + parts.replica = name.substr(pos_first + 1); + return parts; + } + + if (name.find('|', pos_second + 1) != std::string::npos) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect replica identifier: {}", name); + + parts.replica = name.substr(pos_first + 1, pos_second - pos_first - 1); + parts.replica_group = name.substr(pos_second + 1); + + return parts; } ClusterPtr DatabaseReplicated::tryGetCluster() const @@ -177,6 +199,7 @@ void DatabaseReplicated::setCluster(ClusterPtr && new_cluster) ClusterPtr DatabaseReplicated::getClusterImpl() const { + Strings unfiltered_hosts; Strings hosts; Strings host_ids; @@ -188,11 +211,18 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const { host_ids.resize(0); Coordination::Stat stat; - hosts = zookeeper->getChildren(cluster_path, &stat); - if (hosts.empty()) + unfiltered_hosts = zookeeper->getChildren(zookeeper_path + "/replicas", &stat); + if (unfiltered_hosts.empty()) throw Exception(ErrorCodes::NO_ACTIVE_REPLICAS, "No replicas of database {} found. " "It's possible if the first replica is not fully created yet " "or if the last replica was just dropped or due to logical error", zookeeper_path); + + for (const auto & host : unfiltered_hosts) + { + if (replica_group_name == parseFullReplicaName(host).replica_group) + hosts.push_back(host); + } + Int32 cversion = stat.cversion; ::sort(hosts.begin(), hosts.end()); @@ -200,7 +230,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const futures.reserve(hosts.size()); host_ids.reserve(hosts.size()); for (const auto & host : hosts) - futures.emplace_back(zookeeper->asyncTryGet(cluster_path + "/" + host)); + futures.emplace_back(zookeeper->asyncTryGet(zookeeper_path + "/replicas/" + host)); success = true; for (auto & future : futures) @@ -211,7 +241,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const host_ids.emplace_back(res.data); } - zookeeper->get(cluster_path, &stat); + zookeeper->get(zookeeper_path + "/replicas", &stat); if (cversion != stat.cversion) success = false; if (success) @@ -223,7 +253,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const assert(!hosts.empty()); assert(hosts.size() == host_ids.size()); - String current_shard = parseFullReplicaName(hosts.front()).first; + String current_shard = parseFullReplicaName(hosts.front()).shard; std::vector> shards; shards.emplace_back(); for (size_t i = 0; i < hosts.size(); ++i) @@ -231,17 +261,17 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const const auto & id = host_ids[i]; if (id == DROPPED_MARK) continue; - auto [shard, replica] = parseFullReplicaName(hosts[i]); + auto parts = parseFullReplicaName(hosts[i]); auto pos = id.rfind(':'); String host_port = id.substr(0, pos); - if (shard != current_shard) + if (parts.shard != current_shard) { - current_shard = shard; + current_shard = parts.shard; if (!shards.back().empty()) shards.emplace_back(); } String hostname = unescapeForFileName(host_port); - shards.back().push_back(DatabaseReplicaInfo{std::move(hostname), std::move(shard), std::move(replica)}); + shards.back().push_back(DatabaseReplicaInfo{std::move(hostname), std::move(parts.shard), std::move(parts.replica), std::move(parts.replica_group)}); } UInt16 default_port = getContext()->getTCPPort(); @@ -271,8 +301,8 @@ std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr { for (const auto & replica : addresses_with_failover[shard_index]) { - String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); - paths.emplace_back(fs::path(cluster_path) / full_name / "active"); + String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name, replica.database_replica_name); + paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); } } @@ -311,15 +341,6 @@ void DatabaseReplicated::fillClusterAuthInfo(String collection_name, const Poco: cluster_auth_info.cluster_secure_connection = config_ref.getBool(config_prefix + ".cluster_secure_connection", false); } -String DatabaseReplicated::getClusterGroup(const Poco::Util::AbstractConfiguration & config_ref) -{ - const auto cluster_group = config_ref.getString("database_replicated_cluster_group", ""); - - if (cluster_group.empty()) - return "replicas"; - - return "replicas_" + cluster_group; -} void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(LoadingStrictnessLevel mode) { @@ -338,16 +359,7 @@ void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(LoadingStrictnessL createDatabaseNodesInZooKeeper(current_zookeeper); } - if (!current_zookeeper->exists(cluster_path)) - { - /// Create new cluster group, multiple nodes can execute it concurrently - auto code = current_zookeeper->tryCreate(cluster_path, "", zkutil::CreateMode::Persistent); - - if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) - throw Coordination::Exception(code); - } - - replica_path = fs::path(cluster_path) / getFullReplicaName(); + replica_path = fs::path(zookeeper_path) / "replicas" / getFullReplicaName(); bool is_create_query = mode == LoadingStrictnessLevel::CREATE; String replica_host_id; @@ -485,8 +497,26 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt for (int attempts = 10; attempts > 0; --attempts) { - Coordination::Stat stat; - String max_log_ptr_str = current_zookeeper->get(zookeeper_path + "/max_log_ptr", &stat); + Coordination::Stat stat_max_log_ptr; + Coordination::Stat stat_replicas; + String max_log_ptr_str = current_zookeeper->get(zookeeper_path + "/max_log_ptr", &stat_max_log_ptr); + Strings replicas = current_zookeeper->getChildren(zookeeper_path + "/replicas", &stat_replicas); + for (const auto & replica : replicas) + { + NameParts parts = parseFullReplicaName(replica); + if (parts.shard == shard_name && parts.replica == replica_name) + { + throw Exception( + ErrorCodes::REPLICA_ALREADY_EXISTS, + "Replica {} of shard {} of replicated database already exists in the replica group {} at {}", + replica_name, shard_name, parts.replica_group, zookeeper_path); + } + } + + /// This way we make sure that other replica with the same replica_name and shard_name + /// but with a different replica_group_name was not created at the same time. + String replica_value = "Last added replica: " + getFullReplicaName(); + Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent)); @@ -494,7 +524,8 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt /// In addition to creating the replica nodes, we record the max_log_ptr at the instant where /// we declared ourself as an existing replica. We'll need this during recoverLostReplica to /// notify other nodes that issued new queries while this node was recovering. - ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/max_log_ptr", stat.version)); + ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/max_log_ptr", stat_max_log_ptr.version)); + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/replicas", replica_value, stat_replicas.version)); Coordination::Responses responses; const auto code = current_zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZOK) @@ -725,7 +756,15 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex entry.tracing_context = OpenTelemetry::CurrentContext(); String node_path = ddl_worker->tryEnqueueAndExecuteEntry(entry, query_context); - Strings hosts_to_wait = getZooKeeper()->getChildren(cluster_path); + Strings hosts_to_wait; + Strings unfiltered_hosts = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); + + for (const auto & host : unfiltered_hosts) + { + if (replica_group_name == parseFullReplicaName(host).replica_group) + hosts_to_wait.push_back(host); + } + return getDistributedDDLStatus(node_path, entry, query_context, &hosts_to_wait); } @@ -1107,11 +1146,11 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node } void DatabaseReplicated::dropReplica( - DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica) + DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, const String & replica_group) { assert(!database || database_zookeeper_path == database->zookeeper_path); - String full_replica_name = shard.empty() ? replica : getFullReplicaName(shard, replica); + String full_replica_name = shard.empty() ? replica : getFullReplicaName(shard, replica, replica_group); if (full_replica_name.find('/') != std::string::npos) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid replica name, '/' is not allowed: {}", full_replica_name); @@ -1161,7 +1200,7 @@ void DatabaseReplicated::drop(ContextPtr context_) current_zookeeper->tryRemoveRecursive(replica_path); /// TODO it may leave garbage in ZooKeeper if the last node lost connection here - if (current_zookeeper->tryRemove(cluster_path) == Coordination::Error::ZOK) + if (current_zookeeper->tryRemove(zookeeper_path + "/replicas") == Coordination::Error::ZOK) { /// It was the last replica, remove all metadata current_zookeeper->tryRemoveRecursive(zookeeper_path); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 6cb3c98afe3..1622578f3d9 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -54,11 +54,19 @@ public: void stopReplication() override; + struct NameParts + { + String shard; + String replica; + String replica_group; + }; + String getShardName() const { return shard_name; } String getReplicaName() const { return replica_name; } + String getReplicaGroupName() const { return replica_group_name; } String getFullReplicaName() const; - static String getFullReplicaName(const String & shard, const String & replica); - static std::pair parseFullReplicaName(const String & name); + static String getFullReplicaName(const String & shard, const String & replica, const String & replica_group); + static NameParts parseFullReplicaName(const String & name); const String & getZooKeeperPath() const { return zookeeper_path; } @@ -80,7 +88,7 @@ public: bool shouldReplicateQuery(const ContextPtr & query_context, const ASTPtr & query_ptr) const override; - static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica); + static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, const String & replica_group); std::vector tryGetAreReplicasActive(const ClusterPtr & cluster_) const; @@ -101,7 +109,6 @@ private: } cluster_auth_info; void fillClusterAuthInfo(String collection_name, const Poco::Util::AbstractConfiguration & config); - String getClusterGroup(const Poco::Util::AbstractConfiguration & config); void checkQueryValid(const ASTPtr & query, ContextPtr query_context) const; @@ -127,8 +134,8 @@ private: String zookeeper_path; String shard_name; String replica_name; + String replica_group_name; String replica_path; - String cluster_path; DatabaseReplicatedSettings db_settings; zkutil::ZooKeeperPtr getZooKeeper() const; diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 891586d88b6..6a00ec88d8c 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -159,6 +159,7 @@ Cluster::Address::Address( host_name = parsed_host_port.first; database_shard_name = info.shard_name; database_replica_name = info.replica_name; + database_replica_group_name = info.replica_group_name; port = parsed_host_port.second; secure = params.secure ? Protocol::Secure::Enable : Protocol::Secure::Disable; priority = params.priority; @@ -518,7 +519,7 @@ Cluster::Cluster( Addresses current; for (const auto & replica : shard) current.emplace_back( - DatabaseReplicaInfo{replica, "", ""}, + DatabaseReplicaInfo{replica, "", "", ""}, params, current_shard_num, current.size() + 1); diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index b90acd1d576..ba080358a90 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -35,6 +35,7 @@ struct DatabaseReplicaInfo String hostname; String shard_name; String replica_name; + String replica_group_name; }; struct ClusterConnectionParameters @@ -111,6 +112,7 @@ public: String host_name; String database_shard_name; String database_replica_name; + String database_replica_group_name; UInt16 port{0}; String user; String password; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 3207da9941a..fc47101d57e 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -844,7 +844,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) if (!query_.replica_zk_path.empty() && fs::path(replicated->getZooKeeperPath()) != fs::path(query_.replica_zk_path)) return; String full_replica_name = query_.shard.empty() ? query_.replica - : DatabaseReplicated::getFullReplicaName(query_.shard, query_.replica); + : DatabaseReplicated::getFullReplicaName(query_.shard, query_.replica, query_.replica_group); if (replicated->getFullReplicaName() != full_replica_name) return; @@ -860,7 +860,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) if (auto * replicated = dynamic_cast(database.get())) { check_not_local_replica(replicated, query); - DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica); + DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica, query.replica_group); } else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database {} is not Replicated, cannot drop replica", query.getDatabase()); @@ -885,7 +885,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) } check_not_local_replica(replicated, query); - DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica); + DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica, query.replica_group); LOG_TRACE(log, "Dropped replica {} of Replicated database {}", query.replica, backQuoteIfNeed(database->getDatabaseName())); } } @@ -898,7 +898,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) if (auto * replicated = dynamic_cast(elem.second.get())) check_not_local_replica(replicated, query); - DatabaseReplicated::dropReplica(nullptr, query.replica_zk_path, query.shard, query.replica); + DatabaseReplicated::dropReplica(nullptr, query.replica_zk_path, query.shard, query.replica, query.replica_group); LOG_INFO(log, "Dropped replica {} of Replicated database with path {}", query.replica, query.replica_zk_path); } else diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 04c1d545207..efa10aabada 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -357,9 +357,9 @@ Chunk DDLQueryStatusSource::generateChunkWithUnfinishedHosts() const size_t num = 0; if (is_replicated_database) { - auto [shard, replica] = DatabaseReplicated::parseFullReplicaName(host_id); - columns[num++]->insert(shard); - columns[num++]->insert(replica); + auto parts = DatabaseReplicated::parseFullReplicaName(host_id); + columns[num++]->insert(parts.shard); + columns[num++]->insert(parts.replica); if (active_hosts_set.contains(host_id)) columns[num++]->insert(IN_PROGRESS); else @@ -511,9 +511,9 @@ Chunk DDLQueryStatusSource::generate() { if (status.code != 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "There was an error on {}: {} (probably it's a bug)", host_id, status.message); - auto [shard, replica] = DatabaseReplicated::parseFullReplicaName(host_id); - columns[num++]->insert(shard); - columns[num++]->insert(replica); + auto parts = DatabaseReplicated::parseFullReplicaName(host_id); + columns[num++]->insert(parts.shard); + columns[num++]->insert(parts.replica); columns[num++]->insert(OK); } else diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index ebaf357c0ab..0af61796108 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -101,6 +101,7 @@ public: String replica; String shard; String replica_zk_path; + String replica_group; bool is_drop_whole_replica{}; String storage_policy; String volume; diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 9aff0e8879e..eed430ea143 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -165,6 +165,14 @@ enum class SystemQueryTargetType if (!ParserStringLiteral{}.parse(pos, ast, expected)) return false; res->shard = ast->as().value.safeGet(); + + if (database && ParserKeyword{"FROM GROUP"}.ignore(pos, expected)) + { + ASTPtr group_ast; + if (!ParserStringLiteral{}.parse(pos, group_ast, expected)) + return false; + res->replica_group = group_ast->as().value.safeGet(); + } } if (ParserKeyword{"FROM"}.ignore(pos, expected)) diff --git a/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml b/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml index 8e15851a39d..3df343bbc9e 100644 --- a/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml +++ b/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml @@ -1,3 +1,3 @@ - backups + backups diff --git a/tests/integration/test_replicated_database_cluster_groups/test.py b/tests/integration/test_replicated_database_cluster_groups/test.py index 13e51dd6e5f..db43c37bc6e 100644 --- a/tests/integration/test_replicated_database_cluster_groups/test.py +++ b/tests/integration/test_replicated_database_cluster_groups/test.py @@ -72,15 +72,15 @@ def test_cluster_groups(started_cluster): # 1. system.clusters - query = "SELECT host_name from system.clusters WHERE cluster = 'cluster_groups' ORDER BY host_name" + cluster_query = "SELECT host_name from system.clusters WHERE cluster = 'cluster_groups' ORDER BY host_name" expected_main = "main_node_1\nmain_node_2\n" expected_backup = "backup_node_1\nbackup_node_2\n" for node in [main_node_1, main_node_2]: - assert_eq_with_retry(node, query, expected_main) + assert_eq_with_retry(node, cluster_query, expected_main) for node in [backup_node_1, backup_node_2]: - assert_eq_with_retry(node, query, expected_backup) + assert_eq_with_retry(node, cluster_query, expected_backup) # 2. Query execution depends only on your cluster group @@ -114,3 +114,16 @@ def test_cluster_groups(started_cluster): assert_create_query(all_nodes, "cluster_groups.table_1", expected_1) assert_create_query(all_nodes, "cluster_groups.table_2", expected_2) + + # 4. SYSTEM DROP DATABASE REPLICA + backup_node_2.stop_clickhouse() + backup_node_1.query( + "SYSTEM DROP DATABASE REPLICA '4' FROM SHARD '1' FROM GROUP 'backups' FROM DATABASE cluster_groups" + ) + + assert_eq_with_retry(backup_node_1, cluster_query, "backup_node_1\n") + + main_node_2.stop_clickhouse() + main_node_1.query("SYSTEM DROP DATABASE REPLICA '1|2' FROM DATABASE cluster_groups") + + assert_eq_with_retry(main_node_1, cluster_query, "main_node_1\n") From 4c06f79f3a561a063b3177870f285e6c45a3b9e0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 12 Oct 2023 17:30:01 +0200 Subject: [PATCH 196/634] Update src/Common/NamePrompter.h --- src/Common/NamePrompter.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/NamePrompter.h b/src/Common/NamePrompter.h index cc72554657f..97c345414bb 100644 --- a/src/Common/NamePrompter.h +++ b/src/Common/NamePrompter.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include From 049db971e6c7ef7f5c489e8f064cc4da99cce53f Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 12 Oct 2023 17:56:41 +0200 Subject: [PATCH 197/634] Add a test --- .../test.py | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 3f2ec74180b..39a60476347 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -719,6 +719,29 @@ def test_too_many_parts(started_cluster): pg_manager2.drop_materialized_db() +def test_toast(started_cluster): + table = "test_toast" + pg_manager.execute(f"CREATE TABLE {table} (id integer PRIMARY KEY, txt text, other text)") + pg_manager.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = '{table}'", + "materialized_postgresql_backoff_min_ms = 100", + "materialized_postgresql_backoff_max_ms = 100" + ], + ) + + pg_manager.execute(f"""\ +INSERT INTO {table} (id, txt)\ +VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) :: integer) FROM generate_series(1,30000) as t(i)), ''))) + """) + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database(), order_by = "id" + ) + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From c066f6044da36176f383975f92fb391b1e257c2e Mon Sep 17 00:00:00 2001 From: rfraposa Date: Thu, 12 Oct 2023 11:24:57 -0600 Subject: [PATCH 198/634] Add syntax and examples to docs of date/time functions --- .../functions/date-time-functions.md | 817 +++++++++++++++++- 1 file changed, 779 insertions(+), 38 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index aecab38f3c8..0602cf31c58 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -204,6 +204,20 @@ Alias: `timezone`. Type: [String](../../sql-reference/data-types/string.md). +**Example** + +```sql +SELECT timezone() +``` + +Result: + +```response +┌─timezone()─────┐ +│ America/Denver │ +└────────────────┘ +``` + **See also** - [serverTimeZone](#serverTimeZone) @@ -227,6 +241,20 @@ Alias: `serverTimezone`. Type: [String](../../sql-reference/data-types/string.md). +**Example** + +```sql +SELECT serverTimeZone() +``` + +Result: + +```response +┌─serverTimeZone()─┐ +│ UTC │ +└──────────────────┘ +``` + **See also** - [timeZone](#timeZone) @@ -366,37 +394,189 @@ Result: ## toYear -Converts a date or date with time to the year number (AD) as UInt16 value. +Converts a date or date with time to the year number (AD) as `UInt16` value. -Alias: `YEAR`. + +**Syntax** + +```sql +toYear(value) +``` + +Alias: `YEAR` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The year of the given date/time + +Type: `UInt16` + +**Example** + +```sql +SELECT toYear(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toYear(toDateTime('2023-04-21 10:20:30'))─┐ +│ 2023 │ +└───────────────────────────────────────────┘ +``` ## toQuarter -Converts a date or date with time to the quarter number as UInt8 value. +Converts a date or date with time to the quarter number (1-4) as `UInt8` value. + +**Syntax** + +```sql +toQuarter(value) +``` + +Alias: `QUARTER` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The quarter of the year (1, 2, 3 or 4) of the given date/time + +Type: `UInt8` + +**Example** + +```sql +SELECT toQuarter(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toQuarter(toDateTime('2023-04-21 10:20:30'))─┐ +│ 2 │ +└──────────────────────────────────────────────┘ +``` -Alias: `QUARTER`. ## toMonth -Converts a date or date with time to the month number (1-12) as UInt8 value. +Converts a date or date with time to the month number (1-12) as `UInt8` value. -Alias: `MONTH`. +**Syntax** + +```sql +toMonth(value) +``` + +Alias: `MONTH` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The month of the year (1 - 12) of the given date/time + +Type: `UInt8` + +**Example** + +```sql +SELECT toMonth(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toMonth(toDateTime('2023-04-21 10:20:30'))─┐ +│ 4 │ +└────────────────────────────────────────────┘ +``` ## toDayOfYear -Converts a date or date with time to the number of the day of the year (1-366) as UInt16 value. +Converts a date or date with time to the number of the day of the year (1-366) as `UInt16` value. -Alias: `DAYOFYEAR`. +**Syntax** + +```sql +toDayOfYear(value) +``` + +Alias: `DAYOFYEAR` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The day of the year (1 - 366) of the given date/time + +Type: `UInt16` + +**Example** + +```sql +SELECT toDayOfYear(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toDayOfYear(toDateTime('2023-04-21 10:20:30'))─┐ +│ 111 │ +└────────────────────────────────────────────────┘ +``` ## toDayOfMonth -Converts a date or date with time to the number of the day in the month (1-31) as UInt8 value. +Converts a date or date with time to the number of the day in the month (1-31) as `UInt8` value. -Aliases: `DAYOFMONTH`, `DAY`. +**Syntax** + +```sql +toDayOfMonth(value) +``` + +Aliases: `DAYOFMONTH`, `DAY` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The day of the month (1 - 31) of the given date/time + +Type: `UInt8` + +**Example** + +```sql +SELECT toDayOfMonth(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toDayOfMonth(toDateTime('2023-04-21 10:20:30'))─┐ +│ 21 │ +└─────────────────────────────────────────────────┘ +``` ## toDayOfWeek -Converts a date or date with time to the number of the day in the week as UInt8 value. +Converts a date or date with time to the number of the day in the week as `UInt8` value. The two-argument form of `toDayOfWeek()` enables you to specify whether the week starts on Monday or Sunday, and whether the return value should be in the range from 0 to 6 or 1 to 7. If the mode argument is omitted, the default mode is 0. The time zone of the date can be specified as the third argument. @@ -415,25 +595,143 @@ Alias: `DAYOFWEEK`. toDayOfWeek(t[, mode[, timezone]]) ``` +**Arguments** + +- `t` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) +- `mode` - determines what the first day of the week is. Possible values are 0, 1, 2 or 3. See the table above for the differences. +- `timezone` - optional parameter, it behaves like any other conversion function + +**Returned value** + +- The day of the month (1 - 31) of the given date/time + +**Example** + +The following date is April 21, 2023, which was a Friday: + +```sql +SELECT + toDayOfWeek(toDateTime('2023-04-21')), + toDayOfWeek(toDateTime('2023-04-21'), 1) +``` + +Result: + +```response +┌─toDayOfWeek(toDateTime('2023-04-21'))─┬─toDayOfWeek(toDateTime('2023-04-21'), 1)─┐ +│ 5 │ 4 │ +└───────────────────────────────────────┴──────────────────────────────────────────┘ +``` + ## toHour -Converts a date with time the number of the hour in 24-hour time (0-23) as UInt8 value. +Converts a date with time to the number of the hour in 24-hour time (0-23) as `UInt8` value. -Assumes that if clocks are moved ahead, it is by one hour and occurs at 2 a.m., and if clocks are moved back, it is by one hour and occurs at 3 a.m. (which is not always true – even in Moscow the clocks were twice changed at a different time). +Assumes that if clocks are moved ahead, it is by one hour and occurs at 2 a.m., and if clocks are moved back, it is by one hour and occurs at 3 a.m. (which is not always exactly when it occurs - it depends on the timezone). -Alias: `HOUR`. +**Syntax** + +```sql +toHour(value) +``` + +Alias: `HOUR` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The hour of the day (0 - 23) of the given date/time + +Type: `UInt8` + +**Example** + +```sql +SELECT toHour(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toHour(toDateTime('2023-04-21 10:20:30'))─┐ +│ 10 │ +└───────────────────────────────────────────┘ +``` ## toMinute -Converts a date with time to the number of the minute of the hour (0-59) as UInt8 value. +Converts a date with time to the number of the minute of the hour (0-59) as `UInt8` value. -Alias: `MINUTE`. +**Syntax** + +```sql +toMinute(value) +``` + +Alias: `MINUTE` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The minute of the hour (0 - 59) of the given date/time + +Type: `UInt8` + +**Example** + +```sql +SELECT toMinute(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toMinute(toDateTime('2023-04-21 10:20:30'))─┐ +│ 20 │ +└─────────────────────────────────────────────┘ +``` ## toSecond -Converts a date with time to the second in the minute (0-59) as UInt8 value. Leap seconds are not considered. +Converts a date with time to the second in the minute (0-59) as `UInt8` value. Leap seconds are not considered. -Alias: `SECOND`. +**Syntax** + +```sql +toSecond(value) +``` + +Alias: `SECOND` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The second in the minute (0 - 59) of the given date/time + +Type: `UInt8` + +**Example** + +```sql +SELECT toSecond(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toSecond(toDateTime('2023-04-21 10:20:30'))─┐ +│ 30 │ +└─────────────────────────────────────────────┘ +``` ## toUnixTimestamp @@ -496,48 +794,220 @@ Behavior for ## toStartOfYear -Rounds down a date or date with time to the first day of the year. -Returns the date. +Rounds down a date or date with time to the first day of the year. Returns the date as a `Date` object. + +**Syntax** + +```sql +toStartOfYear(value) +``` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The first day of the year of the input date/time + +Type: `Date` + +**Example** + +```sql +SELECT toStartOfYear(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toStartOfYear(toDateTime('2023-04-21 10:20:30'))─┐ +│ 2023-01-01 │ +└──────────────────────────────────────────────────┘ +``` ## toStartOfISOYear -Rounds down a date or date with time to the first day of ISO year. -Returns the date. +Rounds down a date or date with time to the first day of the ISO year, which can be different than a "regular" year. (See [https://en.wikipedia.org/wiki/ISO_week_date](https://en.wikipedia.org/wiki/ISO_week_date).) + +**Syntax** + +```sql +toStartOfISOYear(value) +``` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The first day of the year of the input date/time + +Type: `Date` + +**Example** + +```sql +SELECT toStartOfISOYear(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toStartOfISOYear(toDateTime('2023-04-21 10:20:30'))─┐ +│ 2023-01-02 │ +└─────────────────────────────────────────────────────┘ +``` ## toStartOfQuarter -Rounds down a date or date with time to the first day of the quarter. -The first day of the quarter is either 1 January, 1 April, 1 July, or 1 October. +Rounds down a date or date with time to the first day of the quarter. The first day of the quarter is either 1 January, 1 April, 1 July, or 1 October. Returns the date. +**Syntax** + +```sql +toStartOfQuarter(value) +``` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The first day of the quarter of the given date/time + +Type: `Date` + +**Example** + +```sql +SELECT toStartOfQuarter(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toStartOfQuarter(toDateTime('2023-04-21 10:20:30'))─┐ +│ 2023-04-01 │ +└─────────────────────────────────────────────────────┘ +``` + ## toStartOfMonth -Rounds down a date or date with time to the first day of the month. -Returns the date. +Rounds down a date or date with time to the first day of the month. Returns the date. + +**Syntax** + +```sql +toStartOfMonth(value) +``` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The first day of the month of the given date/time + +Type: `Date` + +**Example** + +```sql +SELECT toStartOfMonth(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toStartOfMonth(toDateTime('2023-04-21 10:20:30'))─┐ +│ 2023-04-01 │ +└───────────────────────────────────────────────────┘ +``` :::note -The behavior of parsing incorrect dates is implementation specific. ClickHouse may return zero date, throw an exception or do “natural” overflow. +The behavior of parsing incorrect dates is implementation specific. ClickHouse may return zero date, throw an exception, or do “natural” overflow. ::: ## toLastDayOfMonth -Rounds a date, or date with time, to the last day of the month. -Returns the date. +Rounds a date or date with time to the last day of the month. Returns the date. -Alias: `LAST_DAY`. +**Syntax** -If `toLastDayOfMonth` is called with an argument of type `Date` greater then 2149-05-31, the result will be calculated from the argument 2149-05-31 instead. +```sql +toLastDayOfMonth(value) +``` + +Alias: `LAST_DAY` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The last day of the month of the given date/time + +Type: `Date` + +**Example** + +```sql +SELECT toLastDayOfMonth(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toLastDayOfMonth(toDateTime('2023-04-21 10:20:30'))─┐ +│ 2023-04-30 │ +└─────────────────────────────────────────────────────┘ +``` ## toMonday -Rounds down a date, or date with time, to the nearest Monday. -Returns the date. +Rounds down a date or date with time to the nearest Monday. Returns the date. + +**Syntax** + +```sql +toMonday(value) +``` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The date of the nearest Monday on or prior to the given date + +Type: `Date` + +**Example** + +```sql +SELECT + toMonday(toDateTime('2023-04-21 10:20:30')), /* a Friday */ + toMonday(toDate('2023-04-24')), /* already a Monday */ +``` + +Result: + +```response +┌─toMonday(toDateTime('2023-04-21 10:20:30'))─┬─toMonday(toDate('2023-04-24'))─┐ +│ 2023-04-17 │ 2023-04-24 │ +└─────────────────────────────────────────────┴────────────────────────────────┘ +``` ## toStartOfWeek -Rounds a date or date with time down to the nearest Sunday or Monday. -Returns the date. -The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, mode is assumed as 0. +Rounds a date or date with time down to the nearest Sunday or Monday. Returns the date. The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, it defaults to 0. **Syntax** @@ -545,10 +1015,43 @@ The mode argument works exactly like the mode argument in function `toWeek()`. I toStartOfWeek(t[, mode[, timezone]]) ``` +**Arguments** + +- `t` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) +- `mode` - determines the first day of the week as described in the [toWeek()](date-time-functions#toweek) function +- `timezone` - Optional parameter, it behaves like any other conversion function + +**Returned value** + +- The date of the nearest Sunday or Monday on or prior to the given date, depending on the mode + +Type: `Date` + +**Example** + +```sql +SELECT + toStartOfWeek(toDateTime('2023-04-21 10:20:30')), /* a Friday */ + toStartOfWeek(toDateTime('2023-04-21 10:20:30'), 1), /* a Friday */ + toStartOfWeek(toDate('2023-04-24')), /* a Monday */ + toStartOfWeek(toDate('2023-04-24'), 1) /* a Monday */ +FORMAT Vertical +``` + +Result: + +```response +Row 1: +────── +toStartOfWeek(toDateTime('2023-04-21 10:20:30')): 2023-04-16 +toStartOfWeek(toDateTime('2023-04-21 10:20:30'), 1): 2023-04-17 +toStartOfWeek(toDate('2023-04-24')): 2023-04-23 +toStartOfWeek(toDate('2023-04-24'), 1): 2023-04-24 +``` + ## toLastDayOfWeek -Rounds a date or date with time up to the nearest Saturday or Sunday. -Returns the date. +Rounds a date or date with time up to the nearest Saturday or Sunday. Returns the date. The mode argument works exactly like the mode argument in function `toWeek()`. If no mode is specified, mode is assumed as 0. **Syntax** @@ -557,18 +1060,148 @@ The mode argument works exactly like the mode argument in function `toWeek()`. I toLastDayOfWeek(t[, mode[, timezone]]) ``` +**Arguments** + +- `t` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) +- `mode` - determines the last day of the week as described in the [toWeek()](date-time-functions#toweek) function +- `timezone` - Optional parameter, it behaves like any other conversion function + +**Returned value** + +- The date of the nearest Sunday or Monday on or after the given date, depending on the mode + +Type: `Date` + +**Example** + +```sql +SELECT + toLastDayOfWeek(toDateTime('2023-04-21 10:20:30')), /* a Friday */ + toLastDayOfWeek(toDateTime('2023-04-21 10:20:30'), 1), /* a Friday */ + toLastDayOfWeek(toDate('2023-04-22')), /* a Saturday */ + toLastDayOfWeek(toDate('2023-04-22'), 1) /* a Saturday */ +FORMAT Vertical +``` + +Result: + +```response +Row 1: +────── +toLastDayOfWeek(toDateTime('2023-04-21 10:20:30')): 2023-04-22 +toLastDayOfWeek(toDateTime('2023-04-21 10:20:30'), 1): 2023-04-23 +toLastDayOfWeek(toDate('2023-04-22')): 2023-04-22 +toLastDayOfWeek(toDate('2023-04-22'), 1): 2023-04-23 +``` + ## toStartOfDay Rounds down a date with time to the start of the day. +**Syntax** + +```sql +toStartOfDay(value) +``` + +**Arguments** + +- `value` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The start of the day of the given date/time + +Type: `DateTime` + +**Example** + +```sql +SELECT toStartOfDay(toDateTime('2023-04-21 10:20:30')) +``` + +Result: + +```response +┌─toStartOfDay(toDateTime('2023-04-21 10:20:30'))─┐ +│ 2023-04-21 00:00:00 │ +└─────────────────────────────────────────────────┘ +``` + ## toStartOfHour Rounds down a date with time to the start of the hour. +**Syntax** + +```sql +toStartOfHour(value) +``` + +**Arguments** + +- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The start of the hour of the given date/time + +Type: `DateTime` + +**Example** + +```sql +SELECT + toStartOfHour(toDateTime('2023-04-21 10:20:30')), + toStartOfHour(toDateTime64('2023-04-21', 6)) +``` + +Result: + +```response +┌─toStartOfHour(toDateTime('2023-04-21 10:20:30'))─┬─toStartOfHour(toDateTime64('2023-04-21', 6))─┐ +│ 2023-04-21 10:00:00 │ 2023-04-21 00:00:00 │ +└──────────────────────────────────────────────────┴──────────────────────────────────────────────┘ +``` + ## toStartOfMinute Rounds down a date with time to the start of the minute. +**Syntax** + +```sql +toStartOfMinute(value) +``` + +**Arguments** + +- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The start of the minute of the given date/time + +Type: `DateTime` + +**Example** + +```sql +SELECT + toStartOfMinute(toDateTime('2023-04-21 10:20:30')), + toStartOfMinute(toDateTime64('2023-04-21 10:20:30.5300', 8)) +FORMAT Vertical +``` + +Result: + +```response +Row 1: +────── +toStartOfMinute(toDateTime('2023-04-21 10:20:30')): 2023-04-21 10:20:00 +toStartOfMinute(toDateTime64('2023-04-21 10:20:30.5300', 8)): 2023-04-21 10:20:00 +``` + ## toStartOfSecond Truncates sub-seconds. @@ -630,14 +1263,122 @@ Result: Rounds down a date with time to the start of the five-minute interval. +**Syntax** + +```sql +toStartOfFiveMinutes(value) +``` + +**Arguments** + +- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The start of the five-minute interval of the given date/time + +Type: `DateTime` + +**Example** + +```sql +SELECT + toStartOfFiveMinutes(toDateTime('2023-04-21 10:17:00')), + toStartOfFiveMinutes(toDateTime('2023-04-21 10:20:00')), + toStartOfFiveMinutes(toDateTime('2023-04-21 10:23:00')) +FORMAT Vertical +``` + +Result: + +```response +Row 1: +────── +toStartOfFiveMinutes(toDateTime('2023-04-21 10:17:00')): 2023-04-21 10:15:00 +toStartOfFiveMinutes(toDateTime('2023-04-21 10:20:00')): 2023-04-21 10:20:00 +toStartOfFiveMinutes(toDateTime('2023-04-21 10:23:00')): 2023-04-21 10:20:00 +``` + ## toStartOfTenMinutes Rounds down a date with time to the start of the ten-minute interval. +**Syntax** + +```sql +toStartOfTenMinutes(value) +``` + +**Arguments** + +- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The start of the ten-minute interval of the given date/time + +Type: `DateTime` + +**Example** + +```sql +SELECT + toStartOfTenMinutes(toDateTime('2023-04-21 10:17:00')), + toStartOfTenMinutes(toDateTime('2023-04-21 10:20:00')), + toStartOfTenMinutes(toDateTime('2023-04-21 10:23:00')) +FORMAT Vertical +``` + +Result: + +```response +Row 1: +────── +toStartOfTenMinutes(toDateTime('2023-04-21 10:17:00')): 2023-04-21 10:10:00 +toStartOfTenMinutes(toDateTime('2023-04-21 10:20:00')): 2023-04-21 10:20:00 +toStartOfTenMinutes(toDateTime('2023-04-21 10:23:00')): 2023-04-21 10:20:00 +``` + ## toStartOfFifteenMinutes Rounds down the date with time to the start of the fifteen-minute interval. +**Syntax** + +```sql +toStartOfFifteenMinutes(value) +``` + +**Arguments** + +- `value` - a [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) + +**Returned value** + +- The start of the fifteen-minute interval of the given date/time + +Type: `DateTime` + +**Example** + +```sql +SELECT + toStartOfFifteenMinutes(toDateTime('2023-04-21 10:17:00')), + toStartOfFifteenMinutes(toDateTime('2023-04-21 10:20:00')), + toStartOfFifteenMinutes(toDateTime('2023-04-21 10:23:00')) +FORMAT Vertical +``` + +Result: + +```response +Row 1: +────── +toStartOfFifteenMinutes(toDateTime('2023-04-21 10:17:00')): 2023-04-21 10:15:00 +toStartOfFifteenMinutes(toDateTime('2023-04-21 10:20:00')): 2023-04-21 10:15:00 +toStartOfFifteenMinutes(toDateTime('2023-04-21 10:23:00')): 2023-04-21 10:15:00 +``` + ## toStartOfInterval(time_or_data, INTERVAL x unit \[, time_zone\]) This function generalizes other `toStartOf*()` functions. For example, From ed520baa066734109f7547dc8411c989521702b6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 12 Oct 2023 19:39:47 +0200 Subject: [PATCH 199/634] Fix black check --- .../test.py | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 39a60476347..1d84ce0e3e9 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -721,24 +721,31 @@ def test_too_many_parts(started_cluster): def test_toast(started_cluster): table = "test_toast" - pg_manager.execute(f"CREATE TABLE {table} (id integer PRIMARY KEY, txt text, other text)") + pg_manager.execute( + f"CREATE TABLE {table} (id integer PRIMARY KEY, txt text, other text)" + ) pg_manager.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, settings=[ f"materialized_postgresql_tables_list = '{table}'", "materialized_postgresql_backoff_min_ms = 100", - "materialized_postgresql_backoff_max_ms = 100" + "materialized_postgresql_backoff_max_ms = 100", ], ) - pg_manager.execute(f"""\ + pg_manager.execute( + f"""\ INSERT INTO {table} (id, txt)\ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) :: integer) FROM generate_series(1,30000) as t(i)), ''))) - """) + """ + ) check_tables_are_synchronized( - instance, table, postgres_database=pg_manager.get_default_database(), order_by = "id" + instance, + table, + postgres_database=pg_manager.get_default_database(), + order_by="id", ) From 34915a8a5e383b402c79960f2fb021b239582300 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 12 Oct 2023 19:47:39 +0200 Subject: [PATCH 200/634] Fix black check --- .../test_postgresql_replica_database_engine_2/test.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 903d246e1c5..21da5bd3ed0 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -728,7 +728,7 @@ def test_replica_consumer(started_cluster): cluster.postgres_ip, cluster.postgres_port, default_database="postgres_database", - postgres_db_exists=True + postgres_db_exists=True, ) for pm in [pg_manager, pg_manager_replica]: @@ -740,14 +740,16 @@ def test_replica_consumer(started_cluster): f"materialized_postgresql_tables_list = '{table}'", "materialized_postgresql_backoff_min_ms = 100", "materialized_postgresql_backoff_max_ms = 100", - "materialized_postgresql_use_unique_replication_consumer_identifier = 1" + "materialized_postgresql_use_unique_replication_consumer_identifier = 1", ], ) assert 50 == int(instance.query(f"SELECT count() FROM test_database.{table}")) assert 50 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) - instance.query(f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(1000, 1000)") + instance.query( + f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(1000, 1000)" + ) check_tables_are_synchronized( instance, table, postgres_database=pg_manager.get_default_database() From 3c6ddb5e1552d9d22a21247e7481914c0a8f81c6 Mon Sep 17 00:00:00 2001 From: Jihyuk Bok Date: Fri, 13 Oct 2023 00:37:55 +0200 Subject: [PATCH 201/634] Add backward compatibility on the obsoleted storage configuration (#55557) * add backward compatibility on the deleted storage configuration * update the reference file --- src/Storages/MergeTree/MergeTreeSettings.h | 2 ++ tests/queries/0_stateless/02888_obsolete_settings.reference | 2 ++ 2 files changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index dbae87b0c5e..53876e77376 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -206,6 +206,8 @@ struct Settings; MAKE_OBSOLETE_MERGE_TREE_SETTING(M, MaxThreads, max_part_loading_threads, 0) \ MAKE_OBSOLETE_MERGE_TREE_SETTING(M, MaxThreads, max_part_removal_threads, 0) \ MAKE_OBSOLETE_MERGE_TREE_SETTING(M, Bool, use_metadata_cache, false) \ + MAKE_OBSOLETE_MERGE_TREE_SETTING(M, UInt64, merge_tree_enable_clear_old_broken_detached, 0) \ + MAKE_OBSOLETE_MERGE_TREE_SETTING(M, UInt64, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds, 1ULL * 3600 * 24 * 30) \ /// Settings that should not change after the creation of a table. /// NOLINTNEXTLINE diff --git a/tests/queries/0_stateless/02888_obsolete_settings.reference b/tests/queries/0_stateless/02888_obsolete_settings.reference index 90313977ab4..9436241838c 100644 --- a/tests/queries/0_stateless/02888_obsolete_settings.reference +++ b/tests/queries/0_stateless/02888_obsolete_settings.reference @@ -50,6 +50,8 @@ in_memory_parts_enable_wal in_memory_parts_insert_sync max_part_loading_threads max_part_removal_threads +merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds +merge_tree_enable_clear_old_broken_detached min_bytes_for_compact_part min_relative_delay_to_yield_leadership min_rows_for_compact_part From 1deb830dab03dea97f47532b10c6b7c14f8e84fd Mon Sep 17 00:00:00 2001 From: bhavuk2002 Date: Fri, 13 Oct 2023 10:57:33 +0530 Subject: [PATCH 202/634] updated dashboard.html --- programs/server/dashboard.html | 168 +++++++++++++----------- src/Functions/fromDaysSinceYearZero.cpp | 26 ++++ 2 files changed, 115 insertions(+), 79 deletions(-) create mode 100644 src/Functions/fromDaysSinceYearZero.cpp diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 555d039cec3..5426be55bd3 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -813,91 +813,101 @@ function insertChart(i) { let edit_buttons = document.createElement('div'); edit_buttons.className = 'chart-buttons'; - let move = document.createElement('a'); - let move_text = document.createTextNode('✥'); - move.appendChild(move_text); + // dragging and maximizing both be added to chart only, when there are more than 1 chart + + if(charts.getElementsByClassName("chart").length != 1) { + + // dragging - let is_dragging = false; - move.addEventListener('mousedown', e => { - const idx = getCurrentIndex(); - is_dragging = true; - chart.className = 'chart chart-moving'; - - let offset_x = e.clientX; - let offset_y = e.clientY; - - let displace_idx = null; - let displace_chart = null; - - function mouseup(e) { - is_dragging = false; - chart.className = 'chart'; - chart.style.left = null; - chart.style.top = null; - - if (displace_idx !== null) { - const elem = queries[idx]; - queries.splice(idx, 1); - queries.splice(displace_idx, 0, elem); - - displace_chart.className = 'chart'; - drawAll(); - } - } - - function mousemove(e) { - if (!is_dragging) { - document.body.removeEventListener('mousemove', mousemove); - document.body.removeEventListener('mouseup', mouseup); - return; - } - - let x = e.clientX - offset_x; - let y = e.clientY - offset_y; - - chart.style.left = `${x}px`; - chart.style.top = `${y}px`; - - displace_idx = null; - displace_chart = null; - let current_idx = -1; - for (const elem of charts.querySelectorAll('.chart')) { - ++current_idx; - if (current_idx == idx) { - continue; - } - - const this_rect = chart.getBoundingClientRect(); - const this_center_x = this_rect.left + this_rect.width / 2; - const this_center_y = this_rect.top + this_rect.height / 2; - - const elem_rect = elem.getBoundingClientRect(); - - if (this_center_x >= elem_rect.left && this_center_x <= elem_rect.right - && this_center_y >= elem_rect.top && this_center_y <= elem_rect.bottom) { - - elem.className = 'chart chart-displaced'; - displace_idx = current_idx; - displace_chart = elem; - } else { - elem.className = 'chart'; + let move = document.createElement('a'); + let move_text = document.createTextNode('✥'); + move.appendChild(move_text); + + let is_dragging = false; + move.addEventListener('mousedown', e => { + const idx = getCurrentIndex(); + is_dragging = true; + chart.className = 'chart chart-moving'; + + let offset_x = e.clientX; + let offset_y = e.clientY; + + let displace_idx = null; + let displace_chart = null; + + function mouseup(e) { + is_dragging = false; + chart.className = 'chart'; + chart.style.left = null; + chart.style.top = null; + + if (displace_idx !== null) { + const elem = queries[idx]; + queries.splice(idx, 1); + queries.splice(displace_idx, 0, elem); + + displace_chart.className = 'chart'; + drawAll(); } } - } + + function mousemove(e) { + if (!is_dragging) { + document.body.removeEventListener('mousemove', mousemove); + document.body.removeEventListener('mouseup', mouseup); + return; + } + + let x = e.clientX - offset_x; + let y = e.clientY - offset_y; + + chart.style.left = `${x}px`; + chart.style.top = `${y}px`; + + displace_idx = null; + displace_chart = null; + let current_idx = -1; + for (const elem of charts.querySelectorAll('.chart')) { + ++current_idx; + if (current_idx == idx) { + continue; + } + + const this_rect = chart.getBoundingClientRect(); + const this_center_x = this_rect.left + this_rect.width / 2; + const this_center_y = this_rect.top + this_rect.height / 2; + + const elem_rect = elem.getBoundingClientRect(); + + if (this_center_x >= elem_rect.left && this_center_x <= elem_rect.right + && this_center_y >= elem_rect.top && this_center_y <= elem_rect.bottom) { + + elem.className = 'chart chart-displaced'; + displace_idx = current_idx; + displace_chart = elem; + } else { + elem.className = 'chart'; + } + } + } + + document.body.addEventListener('mouseup', mouseup); + document.body.addEventListener('mousemove', mousemove); + }); - document.body.addEventListener('mouseup', mouseup); - document.body.addEventListener('mousemove', mousemove); - }); + // maximizing - let maximize = document.createElement('a'); - let maximize_text = document.createTextNode('🗖'); - maximize.appendChild(maximize_text); + let maximize = document.createElement('a'); + let maximize_text = document.createTextNode('🗖'); + maximize.appendChild(maximize_text); + + maximize.addEventListener('click', e => { + const idx = getCurrentIndex(); + chart.className = (chart.className == 'chart' ? 'chart chart-maximized' : 'chart'); + resize(); + }); + } - maximize.addEventListener('click', e => { - const idx = getCurrentIndex(); - chart.className = (chart.className == 'chart' ? 'chart chart-maximized' : 'chart'); - resize(); - }); let edit = document.createElement('a'); let edit_text = document.createTextNode('✎'); diff --git a/src/Functions/fromDaysSinceYearZero.cpp b/src/Functions/fromDaysSinceYearZero.cpp new file mode 100644 index 00000000000..e281524bcb5 --- /dev/null +++ b/src/Functions/fromDaysSinceYearZero.cpp @@ -0,0 +1,26 @@ +#include +#include +#include +#include + + +namespace DB +{ + +using FunctionToDaysSinceYearZero = FunctionDateOrDateTimeToSomething; + +REGISTER_FUNCTION(ToDaysSinceYearZero) +{ + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Returns for the number of days passed since 1 January 0000 in the proleptic Gregorian calendar defined by ISO 8601, a date or date with time. +The calculation is opposite to MySQL's TO_DAYS() function. +)", + .examples{{"typical", "SELECT toDaysSinceYearZero(toDate('2023-09-08'))", "713569"}}, + .categories{"Dates and Times"}}); + + /// MySQL compatibility alias. + // factory.registerAlias("TO_DAYS", FunctionToDaysSinceYearZero::name, FunctionFactory::CaseInsensitive); +} + +} \ No newline at end of file From c4f604dad2ec77532be5448314be9c45bca98df2 Mon Sep 17 00:00:00 2001 From: bhavuk2002 Date: Fri, 13 Oct 2023 11:03:16 +0530 Subject: [PATCH 203/634] files which were i working on were also commited removed them --- src/Functions/fromDaysSinceYearZero.cpp | 26 ------------------------- 1 file changed, 26 deletions(-) delete mode 100644 src/Functions/fromDaysSinceYearZero.cpp diff --git a/src/Functions/fromDaysSinceYearZero.cpp b/src/Functions/fromDaysSinceYearZero.cpp deleted file mode 100644 index e281524bcb5..00000000000 --- a/src/Functions/fromDaysSinceYearZero.cpp +++ /dev/null @@ -1,26 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - -using FunctionToDaysSinceYearZero = FunctionDateOrDateTimeToSomething; - -REGISTER_FUNCTION(ToDaysSinceYearZero) -{ - factory.registerFunction(FunctionDocumentation{ - .description = R"( -Returns for the number of days passed since 1 January 0000 in the proleptic Gregorian calendar defined by ISO 8601, a date or date with time. -The calculation is opposite to MySQL's TO_DAYS() function. -)", - .examples{{"typical", "SELECT toDaysSinceYearZero(toDate('2023-09-08'))", "713569"}}, - .categories{"Dates and Times"}}); - - /// MySQL compatibility alias. - // factory.registerAlias("TO_DAYS", FunctionToDaysSinceYearZero::name, FunctionFactory::CaseInsensitive); -} - -} \ No newline at end of file From 1de89a211bb31193a7534df47c4d67fbd8046aea Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 13 Oct 2023 06:55:39 +0000 Subject: [PATCH 204/634] Fix for analyzer --- tests/queries/0_stateless/02896_illegal_sampling.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02896_illegal_sampling.sql b/tests/queries/0_stateless/02896_illegal_sampling.sql index 280216b3422..9799925e14d 100644 --- a/tests/queries/0_stateless/02896_illegal_sampling.sql +++ b/tests/queries/0_stateless/02896_illegal_sampling.sql @@ -1 +1 @@ -SELECT * FROM (SELECT 1) SAMPLE 1 / 2; -- { serverError SAMPLING_NOT_SUPPORTED } \ No newline at end of file +SELECT * FROM (SELECT 1) SAMPLE 1 / 2; -- { serverError SAMPLING_NOT_SUPPORTED, UNSUPPORTED_METHOD } \ No newline at end of file From 4bf3c4156ecec22806508a9211b95ed80bd174ac Mon Sep 17 00:00:00 2001 From: grantovsky Date: Fri, 13 Oct 2023 12:07:25 +0300 Subject: [PATCH 205/634] Update Russian version replacingmergetree.md. dded is_deleted parameter description Added is_deleted parameter description for russian documentaion version --- .../mergetree-family/replacingmergetree.md | 49 ++++++++++++++++++- 1 file changed, 48 insertions(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md index cd8b55b0259..ec8a9c33c08 100644 --- a/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md @@ -20,7 +20,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], ... -) ENGINE = ReplacingMergeTree([ver]) +) ENGINE = ReplacingMergeTree([ver [, is_deleted]]) [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] @@ -86,6 +86,53 @@ SELECT * FROM mySecondReplacingMT FINAL; │ 1 │ first │ 2020-01-01 01:01:01 │ └─────┴─────────┴─────────────────────┘ ``` +### is_deleted + +`is_deleted` — Имя столбца, который используется во время слияния для обозначения того, нужно ли отображать строку или она подлежит удалению; `1` - для удаления строки, `0` - для отображения строки. + + Тип данных столбца — `UInt8`. + +:::note +`is_deleted` может быть использован, если `ver` используется. + +Строка удаляется при использовании инструкции `OPTIMIZE ... FINAL CLEANUP` или `OPTIMIZE ... FINAL`, или в случае, если параметр `clean_deleted_rows` установлен в значение `Always`. + +Вне зависимости от производимых изменений над данными, версия должна увеличиваться. Если у двух строк одна и та же версия, то остается только последняя вставленная строка. + +::: + +Пример: + +```sql +-- with ver and is_deleted +CREATE OR REPLACE TABLE myThirdReplacingMT +( + `key` Int64, + `someCol` String, + `eventTime` DateTime, + `is_deleted` UInt8 +) +ENGINE = ReplacingMergeTree(eventTime, is_deleted) +ORDER BY key; + +INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 01:01:01', 0); +INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 01:01:01', 1); + +select * from myThirdReplacingMT final; + +0 rows in set. Elapsed: 0.003 sec. + +-- delete rows with is_deleted +OPTIMIZE TABLE myThirdReplacingMT FINAL CLEANUP; + +INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 00:00:00', 0); + +select * from myThirdReplacingMT final; + +┌─key─┬─someCol─┬───────────eventTime─┬─is_deleted─┐ +│ 1 │ first │ 2020-01-01 00:00:00 │ 0 │ +└─────┴─────────┴─────────────────────┴────────────┘ +``` ## Секции запроса From af911dd1864edd00578b84e25aca8ecf64600b66 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 13 Oct 2023 09:26:36 +0000 Subject: [PATCH 206/634] Refactorings --- src/Functions/CustomWeekTransforms.h | 135 +----------------- src/Functions/DateTimeTransforms.h | 122 ++++++++++++++++ .../FunctionCustomWeekToDateOrDate32.h | 20 ++- src/Functions/FunctionCustomWeekToSomething.h | 17 +-- src/Functions/IFunctionCustomWeek.h | 34 +++-- 5 files changed, 158 insertions(+), 170 deletions(-) diff --git a/src/Functions/CustomWeekTransforms.h b/src/Functions/CustomWeekTransforms.h index 2a82912d5d6..dffdd5dc767 100644 --- a/src/Functions/CustomWeekTransforms.h +++ b/src/Functions/CustomWeekTransforms.h @@ -11,9 +11,6 @@ #include #include -/// The default mode value to use for the WEEK() function -#define DEFAULT_WEEK_MODE 0 - namespace DB { @@ -22,132 +19,6 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -/** - * CustomWeek Transformations. - */ - -struct ToYearWeekImpl -{ - static constexpr auto name = "toYearWeek"; - - static inline UInt32 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) - { - // TODO: ditch toDayNum() - YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast(WeekModeFlag::YEAR)); - return yw.first * 100 + yw.second; - } - - static inline UInt32 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) - { - YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast(WeekModeFlag::YEAR)); - return yw.first * 100 + yw.second; - } - static inline UInt32 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) - { - YearWeek yw = time_zone.toYearWeek(ExtendedDayNum (d), week_mode | static_cast(WeekModeFlag::YEAR)); - return yw.first * 100 + yw.second; - } - static inline UInt32 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone) - { - YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode | static_cast(WeekModeFlag::YEAR)); - return yw.first * 100 + yw.second; - } - - using FactorTransform = ZeroTransform; -}; - -struct ToStartOfWeekImpl -{ - static constexpr auto name = "toStartOfWeek"; - - static inline UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) - { - return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); - } - static inline UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) - { - return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); - } - static inline UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) - { - return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode); - } - static inline UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone) - { - return time_zone.toFirstDayNumOfWeek(DayNum(d), week_mode); - } - static inline Int64 executeExtendedResult(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) - { - return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); - } - static inline Int32 executeExtendedResult(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) - { - return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode); - } - - using FactorTransform = ZeroTransform; -}; - -struct ToLastDayOfWeekImpl -{ - static constexpr auto name = "toLastDayOfWeek"; - - static inline UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) - { - return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode); - } - static inline UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) - { - return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode); - } - static inline UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) - { - return time_zone.toLastDayNumOfWeek(ExtendedDayNum(d), week_mode); - } - static inline UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone) - { - return time_zone.toLastDayNumOfWeek(DayNum(d), week_mode); - } - static inline Int64 executeExtendedResult(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) - { - return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode); - } - static inline Int32 executeExtendedResult(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) - { - return time_zone.toLastDayNumOfWeek(ExtendedDayNum(d), week_mode); - } - - using FactorTransform = ZeroTransform; -}; - -struct ToWeekImpl -{ - static constexpr auto name = "toWeek"; - - static inline UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) - { - // TODO: ditch conversion to DayNum, since it doesn't support extended range. - YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode); - return yw.second; - } - static inline UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) - { - YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode); - return yw.second; - } - static inline UInt8 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) - { - YearWeek yw = time_zone.toYearWeek(ExtendedDayNum(d), week_mode); - return yw.second; - } - static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone) - { - YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode); - return yw.second; - } - - using FactorTransform = ToStartOfYearImpl; -}; template struct WeekTransformer @@ -157,8 +28,7 @@ struct WeekTransformer {} template - void - vector(const FromVectorType & vec_from, ToVectorType & vec_to, UInt8 week_mode, const DateLUTImpl & time_zone) const + void vector(const FromVectorType & vec_from, ToVectorType & vec_to, UInt8 week_mode, const DateLUTImpl & time_zone) const { using ValueType = typename ToVectorType::value_type; size_t size = vec_from.size(); @@ -186,7 +56,8 @@ struct CustomWeekTransformImpl { const auto op = WeekTransformer{std::move(transform)}; - UInt8 week_mode = DEFAULT_WEEK_MODE; + static constexpr UInt8 default_week_mode = 0; + UInt8 week_mode = default_week_mode; if (arguments.size() > 1) { if (const auto * week_mode_column = checkAndGetColumnConst(arguments[1].column.get())) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 3a729cfe39f..bbea1d46364 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -305,6 +305,128 @@ struct ToStartOfYearImpl using FactorTransform = ZeroTransform; }; +struct ToYearWeekImpl +{ + static constexpr auto name = "toYearWeek"; + + static UInt32 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + // TODO: ditch toDayNum() + YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast(WeekModeFlag::YEAR)); + return yw.first * 100 + yw.second; + } + + static UInt32 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast(WeekModeFlag::YEAR)); + return yw.first * 100 + yw.second; + } + static UInt32 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + YearWeek yw = time_zone.toYearWeek(ExtendedDayNum (d), week_mode | static_cast(WeekModeFlag::YEAR)); + return yw.first * 100 + yw.second; + } + static UInt32 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode | static_cast(WeekModeFlag::YEAR)); + return yw.first * 100 + yw.second; + } + + using FactorTransform = ZeroTransform; +}; + +struct ToStartOfWeekImpl +{ + static constexpr auto name = "toStartOfWeek"; + + static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + } + static UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + } + static UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode); + } + static UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toFirstDayNumOfWeek(DayNum(d), week_mode); + } + static Int64 executeExtendedResult(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode); + } + static Int32 executeExtendedResult(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode); + } + + using FactorTransform = ZeroTransform; +}; + +struct ToLastDayOfWeekImpl +{ + static constexpr auto name = "toLastDayOfWeek"; + + static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode); + } + static UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode); + } + static UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(ExtendedDayNum(d), week_mode); + } + static UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(DayNum(d), week_mode); + } + static Int64 executeExtendedResult(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(time_zone.toDayNum(t), week_mode); + } + static Int32 executeExtendedResult(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + return time_zone.toLastDayNumOfWeek(ExtendedDayNum(d), week_mode); + } + + using FactorTransform = ZeroTransform; +}; + +struct ToWeekImpl +{ + static constexpr auto name = "toWeek"; + + static UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + // TODO: ditch conversion to DayNum, since it doesn't support extended range. + YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode); + return yw.second; + } + static UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone) + { + YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode); + return yw.second; + } + static UInt8 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + YearWeek yw = time_zone.toYearWeek(ExtendedDayNum(d), week_mode); + return yw.second; + } + static UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone) + { + YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode); + return yw.second; + } + + using FactorTransform = ToStartOfYearImpl; +}; template struct ToStartOfInterval; diff --git a/src/Functions/FunctionCustomWeekToDateOrDate32.h b/src/Functions/FunctionCustomWeekToDateOrDate32.h index cb1ae77de99..0bc46382cbe 100644 --- a/src/Functions/FunctionCustomWeekToDateOrDate32.h +++ b/src/Functions/FunctionCustomWeekToDateOrDate32.h @@ -16,13 +16,13 @@ private: const bool enable_extended_results_for_datetime_functions = false; public: - static FunctionPtr create(ContextPtr context_) + static FunctionPtr create(ContextPtr context) { - return std::make_shared(context_); + return std::make_shared(context); } - explicit FunctionCustomWeekToDateOrDate32(ContextPtr context_) - : enable_extended_results_for_datetime_functions(context_->getSettingsRef().enable_extended_results_for_datetime_functions) + explicit FunctionCustomWeekToDateOrDate32(ContextPtr context) + : enable_extended_results_for_datetime_functions(context->getSettingsRef().enable_extended_results_for_datetime_functions) { } @@ -49,11 +49,9 @@ public: else if (which.isDate32()) { if (enable_extended_results_for_datetime_functions) - return CustomWeekTransformImpl::execute( - arguments, result_type, input_rows_count, Transform{}); + return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, Transform{}); else - return CustomWeekTransformImpl::execute( - arguments, result_type, input_rows_count, Transform{}); + return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, Transform{}); } else if (which.isDateTime()) return CustomWeekTransformImpl::execute( @@ -61,12 +59,10 @@ public: else if (which.isDateTime64()) { if (enable_extended_results_for_datetime_functions) - return CustomWeekTransformImpl::execute( - arguments, result_type, input_rows_count, + return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, TransformDateTime64{assert_cast(from_type)->getScale()}); else - return CustomWeekTransformImpl::execute( - arguments, result_type, input_rows_count, + return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, TransformDateTime64{assert_cast(from_type)->getScale()}); } else diff --git a/src/Functions/FunctionCustomWeekToSomething.h b/src/Functions/FunctionCustomWeekToSomething.h index 7a23747d8a0..1197588fbc6 100644 --- a/src/Functions/FunctionCustomWeekToSomething.h +++ b/src/Functions/FunctionCustomWeekToSomething.h @@ -19,8 +19,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - this->checkArguments(arguments); - + this->checkArguments(arguments, /*is_result_type_date_or_date32*/ false); return std::make_shared(); } @@ -30,20 +29,14 @@ public: WhichDataType which(from_type); if (which.isDate()) - return CustomWeekTransformImpl::execute( - arguments, result_type, input_rows_count, Transform{}); + return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, Transform{}); else if (which.isDate32()) - return CustomWeekTransformImpl::execute( - arguments, result_type, input_rows_count, Transform{}); + return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, Transform{}); else if (which.isDateTime()) - return CustomWeekTransformImpl::execute( - arguments, result_type, input_rows_count, Transform{}); + return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, Transform{}); else if (which.isDateTime64()) - { - return CustomWeekTransformImpl::execute( - arguments, result_type, input_rows_count, + return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, TransformDateTime64{assert_cast(from_type)->getScale()}); - } else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", diff --git a/src/Functions/IFunctionCustomWeek.h b/src/Functions/IFunctionCustomWeek.h index 8f34108dcb4..cb815eff5ae 100644 --- a/src/Functions/IFunctionCustomWeek.h +++ b/src/Functions/IFunctionCustomWeek.h @@ -65,41 +65,47 @@ public: } protected: - void checkArguments(const ColumnsWithTypeAndName & arguments, bool is_result_type_date_or_date32 = false) const + void checkArguments(const ColumnsWithTypeAndName & arguments, bool is_result_type_date_or_date32) const { if (arguments.size() == 1) { - if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + auto type0 = arguments[0].type; + if (!isDate(type0) && !isDate32(type0) && !isDateTime(type0) && !isDateTime64(type0)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. Must be Date, Date32, DateTime or DateTime64.", - arguments[0].type->getName(), getName()); + type0->getName(), getName()); } else if (arguments.size() == 2) { - if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + auto type0 = arguments[0].type; + auto type1 = arguments[1].type; + if (!isDate(type0) && !isDate32(type0) && !isDateTime(type0) && !isDateTime64(type0)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 1st argument of function {}. Must be Date, Date32, DateTime or DateTime64.", - arguments[0].type->getName(), getName()); - if (!isUInt8(arguments[1].type)) + type0->getName(), getName()); + if (!isUInt8(type1)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd (optional) argument of function {}. Must be constant UInt8 (week mode).", - arguments[1].type->getName(), getName()); + type1->getName(), getName()); } else if (arguments.size() == 3) { - if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + auto type0 = arguments[0].type; + auto type1 = arguments[1].type; + auto type2 = arguments[2].type; + if (!isDate(type0) && !isDate32(type0) && !isDateTime(type0) && !isDateTime64(type0)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. Must be Date, Date32, DateTime or DateTime64", - arguments[0].type->getName(), getName()); - if (!isUInt8(arguments[1].type)) + type0->getName(), getName()); + if (!isUInt8(type1)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd (optional) argument of function {}. Must be constant UInt8 (week mode).", - arguments[1].type->getName(), getName()); - if (!isString(arguments[2].type)) + type1->getName(), getName()); + if (!isString(type2)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 3rd (optional) argument of function {}. Must be constant string (timezone name).", - arguments[2].type->getName(), getName()); - if ((isDate(arguments[0].type) || isDate32(arguments[0].type)) && is_result_type_date_or_date32) + type2->getName(), getName()); + if (is_result_type_date_or_date32 && (isDate(type0) || isDate32(type0))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The timezone argument of function {} is allowed only when the 1st argument is DateTime or DateTime64.", getName()); From b13147cd8bc2ce13b76cbec0035be603d57c02c8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Oct 2023 12:07:32 +0200 Subject: [PATCH 207/634] Ping CI From bb0ff98f5ed12e4ea9a603dd31e2a09e2011341d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 13 Oct 2023 10:05:19 +0000 Subject: [PATCH 208/634] String argument support for toDayOfWeek(), toWeek() and toYearWeek() --- .../functions/date-time-functions.md | 6 +++ src/Functions/CustomWeekTransforms.h | 36 ++++++++++++++--- src/Functions/DateTimeTransforms.h | 5 +++ .../FunctionCustomWeekToDateOrDate32.h | 7 ++-- src/Functions/FunctionCustomWeekToSomething.h | 4 +- src/Functions/IFunctionCustomWeek.h | 8 ++-- ...01661_week_functions_string_args.reference | 17 ++++++++ .../01661_week_functions_string_args.sql | 40 +++++++++++++++++++ 8 files changed, 110 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/01661_week_functions_string_args.reference create mode 100644 tests/queries/0_stateless/01661_week_functions_string_args.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 0602cf31c58..33d95c072a0 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -601,6 +601,8 @@ toDayOfWeek(t[, mode[, timezone]]) - `mode` - determines what the first day of the week is. Possible values are 0, 1, 2 or 3. See the table above for the differences. - `timezone` - optional parameter, it behaves like any other conversion function +The first argument can also be specified as [String](../data-types/string.md) in a format supported by [parseDateTime64BestEffort()](type-conversion-functions.md#parsedatetime64besteffort). This was implemented for reasons of compatibility with MySQL which certain 3rd party tools expect. Because string parsing is slow, it is generally recommended to not use string arguments. + **Returned value** - The day of the month (1 - 31) of the given date/time @@ -1495,6 +1497,8 @@ toWeek(t[, mode[, time_zone]]) - `mode` – Optional parameter, Range of values is \[0,9\], default is 0. - `Timezone` – Optional parameter, it behaves like any other conversion function. +The first argument can also be specified as [String](../data-types/string.md) in a format supported by [parseDateTime64BestEffort()](type-conversion-functions.md#parsedatetime64besteffort). This was implemented for reasons of compatibility with MySQL which certain 3rd party tools expect. Because string parsing is slow, it is generally recommended to not use string arguments. + **Example** ``` sql @@ -1525,6 +1529,8 @@ The week number returned by `toYearWeek()` can be different from what the `toWee toYearWeek(t[, mode[, timezone]]) ``` +The first argument can also be specified as [String](../data-types/string.md) in a format supported by [parseDateTime64BestEffort()](type-conversion-functions.md#parsedatetime64besteffort). This was implemented for reasons of compatibility with MySQL which certain 3rd party tools expect. Because string parsing is slow, it is generally recommended to not use string arguments. + **Example** ``` sql diff --git a/src/Functions/CustomWeekTransforms.h b/src/Functions/CustomWeekTransforms.h index dffdd5dc767..f2e637e7a92 100644 --- a/src/Functions/CustomWeekTransforms.h +++ b/src/Functions/CustomWeekTransforms.h @@ -1,15 +1,19 @@ #pragma once +#include #include #include -#include +#include +#include #include +#include #include #include -#include #include -#include -#include +#include +#include +#include +#include namespace DB @@ -66,7 +70,29 @@ struct CustomWeekTransformImpl const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); const ColumnPtr source_col = arguments[0].column; - if (const auto * sources = checkAndGetColumn(source_col.get())) + + if constexpr (std::is_same_v) + { + /// TODO: remove this entire if constexpr branch, move the parsing code into DateTimeTransforms (yes, even if that will + /// duplicate a few lines of code) + + static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); + const auto * sources = checkAndGetColumn(source_col.get()); + + auto col_to = ToDataType::ColumnType::create(); + col_to->getData().resize(sources->size()); + + for (size_t i = 0; i < sources->size(); ++i) + { + DateTime64 dt64; + ReadBufferFromString buf(sources->getDataAt(i).toView()); + parseDateTime64BestEffort(dt64, 0, buf, time_zone, utc_time_zone); + col_to->getData()[i] = static_cast(transform.execute(dt64, week_mode, time_zone)); + } + + return col_to; + } + else if (const auto * sources = checkAndGetColumn(source_col.get())) { auto col_to = ToDataType::ColumnType::create(); op.vector(sources->getData(), col_to->getData(), week_mode, time_zone); diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index bbea1d46364..ad1f6694185 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -308,6 +308,7 @@ struct ToStartOfYearImpl struct ToYearWeekImpl { static constexpr auto name = "toYearWeek"; + static constexpr bool value_may_be_string = true; static UInt32 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { @@ -338,6 +339,7 @@ struct ToYearWeekImpl struct ToStartOfWeekImpl { static constexpr auto name = "toStartOfWeek"; + static constexpr bool value_may_be_string = false; static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { @@ -370,6 +372,7 @@ struct ToStartOfWeekImpl struct ToLastDayOfWeekImpl { static constexpr auto name = "toLastDayOfWeek"; + static constexpr bool value_may_be_string = false; static UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { @@ -402,6 +405,7 @@ struct ToLastDayOfWeekImpl struct ToWeekImpl { static constexpr auto name = "toWeek"; + static constexpr bool value_may_be_string = true; static UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone) { @@ -1298,6 +1302,7 @@ struct ToDayOfMonthImpl struct ToDayOfWeekImpl { static constexpr auto name = "toDayOfWeek"; + static constexpr bool value_may_be_string = true; static UInt8 execute(Int64 t, UInt8 mode, const DateLUTImpl & time_zone) { diff --git a/src/Functions/FunctionCustomWeekToDateOrDate32.h b/src/Functions/FunctionCustomWeekToDateOrDate32.h index 0bc46382cbe..c904bfaec30 100644 --- a/src/Functions/FunctionCustomWeekToDateOrDate32.h +++ b/src/Functions/FunctionCustomWeekToDateOrDate32.h @@ -28,7 +28,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - this->checkArguments(arguments, /*is_result_type_date_or_date32*/ true); + this->checkArguments(arguments, /*is_result_type_date_or_date32*/ true, Transform::value_may_be_string); const IDataType * from_type = arguments[0].type.get(); WhichDataType which(from_type); @@ -44,8 +44,7 @@ public: WhichDataType which(from_type); if (which.isDate()) - return CustomWeekTransformImpl::execute( - arguments, result_type, input_rows_count, Transform{}); + return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, Transform{}); else if (which.isDate32()) { if (enable_extended_results_for_datetime_functions) @@ -65,6 +64,8 @@ public: return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, TransformDateTime64{assert_cast(from_type)->getScale()}); } + else if (Transform::value_may_be_string && which.isString()) + return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, Transform{}); // TODO else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", diff --git a/src/Functions/FunctionCustomWeekToSomething.h b/src/Functions/FunctionCustomWeekToSomething.h index 1197588fbc6..a631a0118cb 100644 --- a/src/Functions/FunctionCustomWeekToSomething.h +++ b/src/Functions/FunctionCustomWeekToSomething.h @@ -19,7 +19,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - this->checkArguments(arguments, /*is_result_type_date_or_date32*/ false); + this->checkArguments(arguments, /*is_result_type_date_or_date32*/ false, Transform::value_may_be_string); return std::make_shared(); } @@ -37,6 +37,8 @@ public: else if (which.isDateTime64()) return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, TransformDateTime64{assert_cast(from_type)->getScale()}); + else if (Transform::value_may_be_string && which.isString()) + return CustomWeekTransformImpl::execute(arguments, result_type, input_rows_count, Transform{}); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", diff --git a/src/Functions/IFunctionCustomWeek.h b/src/Functions/IFunctionCustomWeek.h index cb815eff5ae..51542c9cab1 100644 --- a/src/Functions/IFunctionCustomWeek.h +++ b/src/Functions/IFunctionCustomWeek.h @@ -65,12 +65,12 @@ public: } protected: - void checkArguments(const ColumnsWithTypeAndName & arguments, bool is_result_type_date_or_date32) const + void checkArguments(const ColumnsWithTypeAndName & arguments, bool is_result_type_date_or_date32, bool value_may_be_string) const { if (arguments.size() == 1) { auto type0 = arguments[0].type; - if (!isDate(type0) && !isDate32(type0) && !isDateTime(type0) && !isDateTime64(type0)) + if (!isDate(type0) && !isDate32(type0) && !isDateTime(type0) && !isDateTime64(type0) && !(value_may_be_string && isString(type0))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. Must be Date, Date32, DateTime or DateTime64.", type0->getName(), getName()); @@ -79,7 +79,7 @@ protected: { auto type0 = arguments[0].type; auto type1 = arguments[1].type; - if (!isDate(type0) && !isDate32(type0) && !isDateTime(type0) && !isDateTime64(type0)) + if (!isDate(type0) && !isDate32(type0) && !isDateTime(type0) && !isDateTime64(type0) && !(value_may_be_string && isString(type0))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 1st argument of function {}. Must be Date, Date32, DateTime or DateTime64.", type0->getName(), getName()); @@ -93,7 +93,7 @@ protected: auto type0 = arguments[0].type; auto type1 = arguments[1].type; auto type2 = arguments[2].type; - if (!isDate(type0) && !isDate32(type0) && !isDateTime(type0) && !isDateTime64(type0)) + if (!isDate(type0) && !isDate32(type0) && !isDateTime(type0) && !isDateTime64(type0) && !(value_may_be_string && isString(type0))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. Must be Date, Date32, DateTime or DateTime64", type0->getName(), getName()); diff --git a/tests/queries/0_stateless/01661_week_functions_string_args.reference b/tests/queries/0_stateless/01661_week_functions_string_args.reference new file mode 100644 index 00000000000..9bb1118928a --- /dev/null +++ b/tests/queries/0_stateless/01661_week_functions_string_args.reference @@ -0,0 +1,17 @@ +-- Constant argument +3 3 3 3 +24 24 24 24 +201624 201624 201624 201624 +-- Non-constant argument +3 3 3 3 3 3 +7 7 7 7 7 7 +1 1 1 1 1 1 +4 4 4 4 4 4 +37 37 37 37 37 37 +39 39 39 39 39 39 +4 4 4 4 4 4 +7 7 7 7 7 7 +201737 201737 201737 201737 201737 201737 +201739 201739 201739 201739 201739 201739 +201804 201804 201804 201804 201804 201804 +201907 201907 201907 201907 201907 201907 diff --git a/tests/queries/0_stateless/01661_week_functions_string_args.sql b/tests/queries/0_stateless/01661_week_functions_string_args.sql new file mode 100644 index 00000000000..f0e14d98952 --- /dev/null +++ b/tests/queries/0_stateless/01661_week_functions_string_args.sql @@ -0,0 +1,40 @@ +-- Tests that functions `toDayOfWeek()`, 'toWeek()' and 'toYearWeek()' accepts a date given as string (for compatibility with MySQL) + +SELECT '-- Constant argument'; + +SELECT toDayOfWeek(toDateTime('2016-06-15 23:00:00')), toDayOfWeek('2016-06-15'), toDayOfWeek('2016-06-15 23:00:00'), toDayOfWeek('2016-06-15 23:00:00.123456'); +SELECT toWeek(toDateTime('2016-06-15 23:00:00')), toWeek('2016-06-15'), toWeek('2016-06-15 23:00:00'), toWeek('2016-06-15 23:00:00.123456'); +SELECT toYearWeek(toDateTime('2016-06-15 23:00:00')), toYearWeek('2016-06-15'), toYearWeek('2016-06-15 23:00:00'), toYearWeek('2016-06-15 23:00:00.123456'); + +SELECT toDayOfWeek('invalid'); -- { serverError CANNOT_PARSE_DATETIME } +SELECT toWeek('invalid'); -- { serverError CANNOT_PARSE_DATETIME } +SELECT toYearWeek('invalid'); -- { serverError CANNOT_PARSE_DATETIME } + +SELECT '-- Non-constant argument'; + +DROP TABLE IF EXISTS tab; +CREATE TABLE tab +( + d Date, + dt DateTime('UTC'), + dt64 DateTime64(6, 'UTC'), + str_d String, + str_dt String, + str_dt64 String, + invalid String +) ENGINE MergeTree ORDER BY dt; + +INSERT INTO `tab` VALUES (toDate('2017-09-13'), toDateTime('2017-09-13 19:10:22', 'UTC'), toDateTime64('2017-09-13 19:10:22.123456', 6, 'UTC'), '2017-09-13', '2017-09-13 19:10:22', '2017-09-13 19:10:22.123456', 'foo'); +INSERT INTO `tab` VALUES (toDate('2017-09-24'), toDateTime('2017-09-24 12:05:34', 'UTC'), toDateTime64('2017-09-24 12:05:34.123456', 6, 'UTC'), '2017-09-24', '2017-09-24 12:05:34', '2017-09-24 12:05:34.123456', 'bar'); +INSERT INTO `tab` VALUES (toDate('2018-01-29'), toDateTime('2018-01-29 02:09:48', 'UTC'), toDateTime64('2018-01-29 02:09:48.123456', 6, 'UTC'), '2018-01-29', '2018-01-29 02:09:48', '2018-01-29 02:09:48.123456', 'qaz'); +INSERT INTO `tab` VALUES (toDate('2019-02-21'), toDateTime('2019-02-21 15:07:43', 'UTC'), toDateTime64('2019-02-21 15:07:43.123456', 6, 'UTC'), '2019-02-21', '2019-02-21 15:07:43', '2019-02-21 15:07:43.123456', 'qux'); + +SELECT toDayOfWeek(d), toDayOfWeek(dt), toDayOfWeek(dt64), toDayOfWeek(str_d), toDayOfWeek(str_dt), toDayOfWeek(str_dt64) FROM tab ORDER BY d; +SELECT toWeek(d), toWeek(dt), toWeek(dt64), toWeek(str_d), toWeek(str_dt), toWeek(str_dt64) FROM tab ORDER BY d; +SELECT toYearWeek(d), toYearWeek(dt), toYearWeek(dt64), toYearWeek(str_d), toYearWeek(str_dt), toYearWeek(str_dt64) FROM tab ORDER BY d; + +SELECT toDayOfWeek(invalid) FROM `tab`; -- { serverError CANNOT_PARSE_DATETIME } +SELECT toWeek(invalid) FROM `tab`; -- { serverError CANNOT_PARSE_DATETIME } +SELECT toYearWeek(invalid) FROM `tab`; -- { serverError CANNOT_PARSE_DATETIME } + +DROP TABLE tab; From db1b9408f398634eddbf7c50418872805ee57dd3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 13 Oct 2023 12:25:33 +0200 Subject: [PATCH 209/634] Fix replica groups for Replicated database engine Should fix 02447_drop_database_replica test Signed-off-by: Azat Khuzhin --- src/Databases/DatabaseReplicated.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 1c44a074c96..7234baa818a 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -301,7 +301,7 @@ std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr { for (const auto & replica : addresses_with_failover[shard_index]) { - String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name, replica.database_replica_name); + String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name, replica.database_replica_group_name); paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); } } From bfe174f71b575c8831eee86e57e6307fd74a59c8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Oct 2023 12:57:57 +0200 Subject: [PATCH 210/634] Fix test --- src/Interpreters/Context.cpp | 2 +- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 2 +- src/Storages/S3Queue/S3QueueSource.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 658fb9adbb2..dfa18694858 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3590,7 +3590,7 @@ std::shared_ptr Context::getFilesystemCacheLog() const std::shared_ptr Context::getS3QueueLog() const { - auto lock = getLock(); + auto lock = getGlobalSharedLock(); if (!shared->system_logs) return {}; diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 1856490c92e..535ced7f6df 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -591,11 +591,11 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S const auto node_name_with_retriable_suffix = node_name + ".retriable"; Coordination::Stat stat; std::string res; - auto failed_node_metadata = NodeMetadata::fromString(res); /// Extract the number of already done retries from node_hash.retriable node if it exists. if (zk_client->tryGet(zookeeper_failed_path / node_name_with_retriable_suffix, res, &stat)) { + auto failed_node_metadata = NodeMetadata::fromString(res); node_metadata.retries = failed_node_metadata.retries + 1; std::lock_guard lock(file_status->metadata_lock); diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index d86fc6fe1ce..7140ec9d4f0 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -148,7 +148,7 @@ Chunk StorageS3QueueSource::generate() } catch (const Exception & e) { - LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); + LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", reader.getFile(), e.displayText()); files_metadata->setFileFailed(key_with_info->processing_holder, e.message()); From a9c0c20cad0db7366b1cd5a9d780f73d3c58e38a Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Oct 2023 13:21:50 +0200 Subject: [PATCH 211/634] Update documentation --- .../table-engines/integrations/s3queue.md | 77 +++++++++++++++---- src/Interpreters/S3QueueLog.cpp | 3 +- src/Storages/System/StorageSystemS3Queue.cpp | 3 + 3 files changed, 69 insertions(+), 14 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 7ea848b53b0..4769f573b54 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -248,7 +248,7 @@ If the listing of files contains number ranges with leading zeros, use the const For introspection use `system.s3queue` stateless table and `system.s3queue_log` persistent table. -The `s3queue` system table has the following structure: +1. `system.s3queue`. This table is not persistent and shows in-memory state of `S3Queue`: which files are currently being processed, which files are processed or failed. ``` sql ┌─statement──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ @@ -262,12 +262,62 @@ The `s3queue` system table has the following structure: `processing_start_time` Nullable(DateTime), `processing_end_time` Nullable(DateTime), `ProfileEvents` Map(String, UInt64) + `exception` String ) ENGINE = SystemS3Queue COMMENT 'SYSTEM TABLE is built on the fly.' │ └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` +Example: + +``` sql + +SELECT * +FROM system.s3queue + +Row 1: +────── +zookeeper_path: /clickhouse/s3queue/25ea5621-ae8c-40c7-96d0-cec959c5ab88/3b3f66a1-9866-4c2e-ba78-b6bfa154207e +file_name: wikistat/original/pageviews-20150501-030000.gz +rows_processed: 5068534 +status: Processed +processing_start_time: 2023-10-13 13:09:48 +processing_end_time: 2023-10-13 13:10:31 +ProfileEvents: {'ZooKeeperTransactions':3,'ZooKeeperGet':2,'ZooKeeperMulti':1,'SelectedRows':5068534,'SelectedBytes':198132283,'ContextLock':1,'S3QueueSetFileProcessingMicroseconds':2480,'S3QueueSetFileProcessedMicroseconds':9985,'S3QueuePullMicroseconds':273776,'LogTest':17} +exception: +``` + +2. `system.s3queue_log`. Persistent table. Has the same information as `system.s3queue`, but for `processed` and `failed` files. + +The table has the following structure: + +``` sql +SHOW CREATE TABLE system.s3queue_log + +Query id: 0ad619c3-0f2a-4ee4-8b40-c73d86e04314 + +┌─statement──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ CREATE TABLE system.s3queue_log +( + `event_date` Date, + `event_time` DateTime, + `table_uuid` String, + `file_name` String, + `rows_processed` UInt64, + `status` Enum8('Processed' = 0, 'Failed' = 1), + `processing_start_time` Nullable(DateTime), + `processing_end_time` Nullable(DateTime), + `ProfileEvents` Map(String, UInt64), + `exception` String +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(event_date) +ORDER BY (event_date, event_time) +SETTINGS index_granularity = 8192 │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + In order to use `system.s3queue_log` define its configuration in server config file: ``` xml @@ -280,18 +330,19 @@ In order to use `system.s3queue_log` define its configuration in server config f Example: ``` sql -:) select * from system.s3queue - SELECT * -FROM system.s3queue +FROM system.s3queue_log -Query id: bb41964e-c947-4112-be3a-0f01770a1e84 - -┌─database─┬─table───┬─file_name──────────────────────────────────────────┬─rows_processed─┬─status─────┬─processing_start_time─┬─processing_end_time─┬─ProfileEvents──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ default │ s3queue │ hits_compatible/athena_partitioned/hits_60.parquet │ 0 │ Processing │ 2023-09-25 19:56:51 │ ᴺᵁᴸᴸ │ {'S3QueueSetFileProcessingMicroseconds':516} │ -│ default │ s3queue │ hits_compatible/athena_partitioned/hits_54.parquet │ 1000000 │ Processing │ 2023-09-25 19:56:50 │ ᴺᵁᴸᴸ │ {'SelectedRows':1000000,'SelectedBytes':1284181126,'S3ReadMicroseconds':385274,'S3ReadRequestsCount':3,'S3ReadRequestsErrors':1,'S3GetObject':1,'ReadBufferFromS3Microseconds':405970,'ReadBufferFromS3InitMicroseconds':385790,'ReadBufferFromS3Bytes':65536,'ReadBufferFromS3PreservedSessions':1,'S3QueueSetFileProcessingMicroseconds':567,'S3QueuePullMicroseconds':2475045} │ -└──────────┴─────────┴────────────────────────────────────────────────────┴────────────────┴────────────┴───────────────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ - - -SELECT * FROM system.s3_queue_log; +Row 1: +────── +event_date: 2023-10-13 +event_time: 2023-10-13 13:10:12 +table_uuid: +file_name: wikistat/original/pageviews-20150501-020000.gz +rows_processed: 5112621 +status: Processed +processing_start_time: 2023-10-13 13:09:48 +processing_end_time: 2023-10-13 13:10:12 +ProfileEvents: {'ZooKeeperTransactions':3,'ZooKeeperGet':2,'ZooKeeperMulti':1,'SelectedRows':5112621,'SelectedBytes':198577687,'ContextLock':1,'S3QueueSetFileProcessingMicroseconds':1934,'S3QueueSetFileProcessedMicroseconds':17063,'S3QueuePullMicroseconds':5841972,'LogTest':17} +exception: ``` diff --git a/src/Interpreters/S3QueueLog.cpp b/src/Interpreters/S3QueueLog.cpp index af40d1dd1e8..4e43aae6b0d 100644 --- a/src/Interpreters/S3QueueLog.cpp +++ b/src/Interpreters/S3QueueLog.cpp @@ -42,12 +42,13 @@ void S3QueueLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(table_uuid); columns[i++]->insert(file_name); columns[i++]->insert(rows_processed); - columns[i++]->insert(magic_enum::enum_name(status)); + columns[i++]->insert(status); if (processing_start_time) columns[i++]->insert(processing_start_time); else columns[i++]->insertDefault(); + if (processing_end_time) columns[i++]->insert(processing_end_time); else diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index 235f4ff6f8f..33b3dce4a83 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -30,6 +30,7 @@ NamesAndTypesList StorageSystemS3Queue::getNamesAndTypes() {"processing_start_time", std::make_shared(std::make_shared())}, {"processing_end_time", std::make_shared(std::make_shared())}, {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, + {"exception", std::make_shared()}, }; } @@ -63,6 +64,8 @@ void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr, co res_columns[i++]->insertDefault(); ProfileEvents::dumpToMapColumn(file_status->profile_counters.getPartiallyAtomicSnapshot(), res_columns[i++].get(), true); + + res_columns[i++]->insert(file_status->last_exception); } } } From 48933f52fe66c6217d09d8d5b0ced67dad7bb5a6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 13 Oct 2023 11:30:37 +0000 Subject: [PATCH 212/634] Fix Keeper Context --- src/Coordination/Standalone/Context.cpp | 105 +++++++++++++++--------- src/Coordination/Standalone/Context.h | 39 ++++++--- 2 files changed, 93 insertions(+), 51 deletions(-) diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index f0ba16e50ee..84cd3f6afe2 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -14,6 +15,7 @@ namespace ProfileEvents { extern const Event ContextLock; + extern const Event ContextLockWaitMicroseconds; } namespace CurrentMetrics @@ -40,7 +42,7 @@ struct ContextSharedPart : boost::noncopyable {} /// For access of most of shared objects. Recursive mutex. - mutable std::recursive_mutex mutex; + mutable SharedMutex mutex; mutable std::mutex keeper_dispatcher_mutex; mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); @@ -50,13 +52,16 @@ struct ContextSharedPart : boost::noncopyable String path; /// Path to the data directory, with a slash at the end. ConfigurationPtr config; /// Global configuration settings. MultiVersion macros; /// Substitutions extracted from config. + OnceFlag schedule_pool_initialized; mutable std::unique_ptr schedule_pool; /// A thread pool that can run different jobs in background RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml - /// + + mutable OnceFlag readers_initialized; mutable std::unique_ptr asynchronous_remote_fs_reader; mutable std::unique_ptr asynchronous_local_fs_reader; mutable std::unique_ptr synchronous_local_fs_reader; + mutable OnceFlag threadpool_writer_initialized; mutable std::unique_ptr threadpool_writer; mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads @@ -64,13 +69,14 @@ struct ContextSharedPart : boost::noncopyable mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes - }; +ContextData::ContextData() = default; +ContextData::ContextData(const ContextData &) = default; + Context::Context() = default; +Context::Context(const Context & rhs) : ContextData(rhs), std::enable_shared_from_this(rhs) {} Context::~Context() = default; -Context::Context(const Context &) = default; -Context & Context::operator=(const Context &) = default; SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default; SharedContextHolder & SharedContextHolder::operator=(SharedContextHolder &&) noexcept = default; @@ -87,10 +93,10 @@ void Context::makeGlobalContext() global_context = shared_from_this(); } -ContextMutablePtr Context::createGlobal(ContextSharedPart * shared) +ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part) { auto res = std::shared_ptr(new Context); - res->shared = shared; + res->shared = shared_part; return res; } @@ -105,6 +111,7 @@ SharedContextHolder Context::createShared() return SharedContextHolder(std::make_unique()); } + ContextMutablePtr Context::getGlobalContext() const { auto ptr = global_context.lock(); @@ -112,22 +119,55 @@ ContextMutablePtr Context::getGlobalContext() const return ptr; } -std::unique_lock Context::getLock() const +std::unique_lock Context::getGlobalLock() const { ProfileEvents::increment(ProfileEvents::ContextLock); CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; - return std::unique_lock(shared->mutex); + Stopwatch watch; + auto lock = std::unique_lock(shared->mutex); + ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); + return lock; +} + +std::shared_lock Context::getGlobalSharedLock() const +{ + ProfileEvents::increment(ProfileEvents::ContextLock); + CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; + Stopwatch watch; + auto lock = std::shared_lock(shared->mutex); + ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); + return lock; +} + +std::unique_lock Context::getLocalLock() const +{ + ProfileEvents::increment(ProfileEvents::ContextLock); + CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; + Stopwatch watch; + auto lock = std::unique_lock(mutex); + ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); + return lock; +} + +std::shared_lock Context::getLocalSharedLock() const +{ + ProfileEvents::increment(ProfileEvents::ContextLock); + CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; + Stopwatch watch; + auto lock = std::shared_lock(mutex); + ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); + return lock; } String Context::getPath() const { - auto lock = getLock(); + auto lock = getGlobalSharedLock(); return shared->path; } void Context::setPath(const String & path) { - auto lock = getLock(); + auto lock = getGlobalLock(); shared->path = path; } @@ -143,15 +183,13 @@ void Context::setMacros(std::unique_ptr && macros) BackgroundSchedulePool & Context::getSchedulePool() const { - auto lock = getLock(); - if (!shared->schedule_pool) - { + callOnce(shared->schedule_pool_initialized, [&] { shared->schedule_pool = std::make_unique( shared->server_settings.background_schedule_pool_size, CurrentMetrics::BackgroundSchedulePoolTask, CurrentMetrics::BackgroundSchedulePoolSize, "BgSchPool"); - } + }); return *shared->schedule_pool; } @@ -168,30 +206,21 @@ const RemoteHostFilter & Context::getRemoteHostFilter() const IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const { - auto lock = getLock(); + callOnce(shared->readers_initialized, [&] { + const auto & config = getConfigRef(); + shared->asynchronous_remote_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER, config); + shared->asynchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER, config); + shared->synchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER, config); + }); switch (type) { case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER: - { - if (!shared->asynchronous_remote_fs_reader) - shared->asynchronous_remote_fs_reader = createThreadPoolReader(type, getConfigRef()); return *shared->asynchronous_remote_fs_reader; - } case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER: - { - if (!shared->asynchronous_local_fs_reader) - shared->asynchronous_local_fs_reader = createThreadPoolReader(type, getConfigRef()); - return *shared->asynchronous_local_fs_reader; - } case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER: - { - if (!shared->synchronous_local_fs_reader) - shared->synchronous_local_fs_reader = createThreadPoolReader(type, getConfigRef()); - return *shared->synchronous_local_fs_reader; - } } } @@ -207,19 +236,19 @@ std::shared_ptr Context::getFilesystemReadPrefetche void Context::setConfig(const ConfigurationPtr & config) { - auto lock = getLock(); + auto lock = getGlobalLock(); shared->config = config; } const Poco::Util::AbstractConfiguration & Context::getConfigRef() const { - auto lock = getLock(); + auto lock = getGlobalSharedLock(); return shared->config ? *shared->config : Poco::Util::Application::instance().config(); } std::shared_ptr Context::getAsyncReadCounters() const { - auto lock = getLock(); + auto lock = getLocalLock(); if (!async_read_counters) async_read_counters = std::make_shared(); return async_read_counters; @@ -227,18 +256,14 @@ std::shared_ptr Context::getAsyncReadCounters() const ThreadPool & Context::getThreadPoolWriter() const { - const auto & config = getConfigRef(); - - auto lock = getLock(); - - if (!shared->threadpool_writer) - { + callOnce(shared->threadpool_writer_initialized, [&] { + const auto & config = getConfigRef(); auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100); auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); shared->threadpool_writer = std::make_unique( CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, pool_size, pool_size, queue_size); - } + }); return *shared->threadpool_writer; } diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h index d9bbccea2e9..bcaf8687d71 100644 --- a/src/Coordination/Standalone/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -6,6 +6,7 @@ #include #include +#include #include @@ -44,17 +45,9 @@ private: std::unique_ptr shared; }; - -class Context : public std::enable_shared_from_this +class ContextData { -private: - /// Use copy constructor or createGlobal() instead - Context(); - Context(const Context &); - Context & operator=(const Context &); - - std::unique_lock getLock() const; - +protected: ContextWeakMutablePtr global_context; inline static ContextPtr global_context_instance; ContextSharedPart * shared; @@ -63,9 +56,33 @@ private: mutable std::shared_ptr async_read_counters; Settings settings; /// Setting for query execution. + +public: + /// Use copy constructor or createGlobal() instead + ContextData(); + ContextData(const ContextData &); +}; + +class Context : public ContextData, public std::enable_shared_from_this +{ +private: + /// ContextData mutex + mutable SharedMutex mutex; + + Context(); + Context(const Context &); + + std::unique_lock getGlobalLock() const; + + std::shared_lock getGlobalSharedLock() const; + + std::unique_lock getLocalLock() const; + + std::shared_lock getLocalSharedLock() const; + public: /// Create initial Context with ContextShared and etc. - static ContextMutablePtr createGlobal(ContextSharedPart * shared); + static ContextMutablePtr createGlobal(ContextSharedPart * shared_part); static SharedContextHolder createShared(); ContextMutablePtr getGlobalContext() const; From e60983f5ae263be9f3695f0967ac90ca3d915b31 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 13 Oct 2023 13:32:56 +0200 Subject: [PATCH 213/634] Do not fail if label-to-remove does not exists in PR --- tests/ci/commit_status_helper.py | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index a24eaf4700d..aeeb8531aac 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -8,9 +8,10 @@ import logging import time from github import Github -from github.GithubObject import _NotSetType, NotSet as NotSet from github.Commit import Commit from github.CommitStatus import CommitStatus +from github.GithubException import GithubException +from github.GithubObject import _NotSetType, NotSet as NotSet from github.IssueComment import IssueComment from github.PullRequest import PullRequest from github.Repository import Repository @@ -336,7 +337,18 @@ def remove_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]) -> None: repo = get_repo(gh) pull_request = repo.get_pull(pr_info.number) for label in labels_names: - pull_request.remove_from_labels(label) + try: + pull_request.remove_from_labels(label) + except GithubException as exc: + if not ( + exc.status == 404 + and isinstance(exc.data, dict) + and exc.data.get("message", "") == "Label does not exist" + ): + raise + logging.warning( + "The label '%s' does not exist in PR #%s", pr_info.number, label + ) pr_info.labels.remove(label) From 720ea8441c8f7081dc670a9a7d8119767dde3f8f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 13 Oct 2023 09:47:58 +0200 Subject: [PATCH 214/634] Fix global context for tests with --gtest_filter If you run tests that requires context, but do not initialize it, then it will SIGSEGV, because Context is not initialized. Fix this by using google test envrionment, and instead of gtest_main implement own main function that will initialize it. Signed-off-by: Azat Khuzhin --- contrib/googletest-cmake/CMakeLists.txt | 8 +------- src/CMakeLists.txt | 2 +- src/Common/tests/gtest_main.cpp | 18 ++++++++++++++++++ src/Coordination/tests/gtest_coordination.cpp | 16 +++++++--------- 4 files changed, 27 insertions(+), 17 deletions(-) create mode 100644 src/Common/tests/gtest_main.cpp diff --git a/contrib/googletest-cmake/CMakeLists.txt b/contrib/googletest-cmake/CMakeLists.txt index 5b00096615a..c917a2f31a7 100644 --- a/contrib/googletest-cmake/CMakeLists.txt +++ b/contrib/googletest-cmake/CMakeLists.txt @@ -6,13 +6,7 @@ target_compile_definitions (_gtest PUBLIC GTEST_HAS_POSIX_RE=0) target_include_directories(_gtest SYSTEM PUBLIC "${SRC_DIR}/googletest/include") target_include_directories(_gtest PRIVATE "${SRC_DIR}/googletest") -add_library(_gtest_main "${SRC_DIR}/googletest/src/gtest_main.cc") -set_target_properties(_gtest_main PROPERTIES VERSION "1.0.0") -target_link_libraries(_gtest_main PUBLIC _gtest) - -add_library(_gtest_all INTERFACE) -target_link_libraries(_gtest_all INTERFACE _gtest _gtest_main) -add_library(ch_contrib::gtest_all ALIAS _gtest_all) +add_library(ch_contrib::gtest ALIAS _gtest) add_library(_gmock "${SRC_DIR}/googlemock/src/gmock-all.cc") set_target_properties(_gmock PROPERTIES VERSION "1.0.0") diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 1c93bc5d35e..d13c9cbe9bc 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -592,7 +592,7 @@ if (ENABLE_TESTS) target_link_libraries(unit_tests_dbms PRIVATE ch_contrib::gmock_all - ch_contrib::gtest_all + ch_contrib::gtest clickhouse_functions clickhouse_aggregate_functions clickhouse_parsers diff --git a/src/Common/tests/gtest_main.cpp b/src/Common/tests/gtest_main.cpp new file mode 100644 index 00000000000..81c46950798 --- /dev/null +++ b/src/Common/tests/gtest_main.cpp @@ -0,0 +1,18 @@ +#include + +#include + +class ContextEnvironment : public testing::Environment +{ +public: + void SetUp() override { getContext(); } +}; + +int main(int argc, char ** argv) +{ + testing::InitGoogleTest(&argc, argv); + + testing::AddGlobalTestEnvironment(new ContextEnvironment); + + return RUN_ALL_TESTS(); +} diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index d207fab332e..42bf9d89103 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -71,6 +71,13 @@ protected: DB::KeeperContextPtr keeper_context = std::make_shared(true); Poco::Logger * log{&Poco::Logger::get("CoordinationTest")}; + void SetUp() override + { + Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); + } + void setLogDirectory(const std::string & path) { keeper_context->setLogDisk(std::make_shared("LogDisk", path)); } void setSnapshotDirectory(const std::string & path) @@ -2911,13 +2918,4 @@ INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite, CoordinationTest, ::testing::ValuesIn(std::initializer_list{CompressionParam{true, ".zstd"}, CompressionParam{false, ""}})); -int main(int argc, char ** argv) -{ - Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); - Poco::Logger::root().setChannel(channel); - Poco::Logger::root().setLevel("trace"); - testing::InitGoogleTest(&argc, argv); - return RUN_ALL_TESTS(); -} - #endif From f42cc282ec77d1ed57d8f7da390c58e1d8e53939 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 13 Oct 2023 09:47:11 +0200 Subject: [PATCH 215/634] Do not include headers from tests into client Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9ca104ff942..f584006aa3f 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include From 8b7e6adc380d6dd4abb8d5f12d464c881fe80fb5 Mon Sep 17 00:00:00 2001 From: grantovsky Date: Fri, 13 Oct 2023 15:03:36 +0300 Subject: [PATCH 216/634] Update replacingmergetree.md cleanup+clean_deleted_rows warning MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил формулировку, что CLEANUP + clean_deleted_rows=always - не следует использовать --- .../mergetree-family/replacingmergetree.md | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md index ec8a9c33c08..b4139f1cb77 100644 --- a/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md @@ -95,10 +95,16 @@ SELECT * FROM mySecondReplacingMT FINAL; :::note `is_deleted` может быть использован, если `ver` используется. -Строка удаляется при использовании инструкции `OPTIMIZE ... FINAL CLEANUP` или `OPTIMIZE ... FINAL`, или в случае, если параметр `clean_deleted_rows` установлен в значение `Always`. +Строка удаляется в следующих случаях: + + - при использовании инструкции `OPTIMIZE ... FINAL CLEANUP` + - при использовании инструкции `OPTIMIZE ... FINAL` + - параметр движка `clean_deleted_rows` установлен в значение `Always` (по умолчанию - `Never`) + - есть новые версии строки + +Не рекомендуется одновременно выполнять `CLEANUP` и использовать параметр движка `clean_deleted_rows`, это может привести к неожиданным результатам, например удаленные строки могут внось появиться. Вне зависимости от производимых изменений над данными, версия должна увеличиваться. Если у двух строк одна и та же версия, то остается только последняя вставленная строка. - ::: Пример: From b6b124f5a9a8b93ec21445273e9886818b73fd3b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Oct 2023 13:54:20 +0200 Subject: [PATCH 217/634] Usability improvement --- src/Core/Settings.h | 1 + src/Storages/S3Queue/StorageS3Queue.cpp | 32 ++++++++++++++++--- .../test_storage_s3_queue/configs/users.xml | 1 + 3 files changed, 29 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c08425c03fd..93a15bb80fd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -108,6 +108,7 @@ class IColumn; M(UInt64, s3_http_connection_pool_size, 1000, "How many reusable open connections to keep per S3 endpoint. Only applies to the S3 table engine and table function, not to S3 disks (for disks, use disk config instead). Global setting, can only be set in config, overriding it per session or per query has no effect.", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ M(String, s3queue_default_zookeeper_path, "/clickhouse/s3queue/", "Default zookeeper path prefix for S3Queue engine", 0) \ + M(Bool, s3queue_enable_logging_to_s3queue_log, false, "Enable writing to system.s3queue_log. The value can be overwritten per table with table settings", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 6bed73b0315..c280b909d1a 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -70,6 +70,32 @@ namespace } return zkutil::extractZooKeeperPath(result_zk_path, true); } + + void checkAndAdjustSettings(S3QueueSettings & s3queue_settings, const Settings & settings, Poco::Logger * log) + { + if (s3queue_settings.mode == S3QueueMode::ORDERED && s3queue_settings.s3queue_processing_threads_num > 1) + { + LOG_WARNING(log, "Parallel processing is not yet supported for Ordered mode"); + s3queue_settings.s3queue_processing_threads_num = 1; + } + + if (!s3queue_settings.s3queue_processing_threads_num) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `s3queue_processing_threads_num` cannot be set to zero"); + } + + if (!s3queue_settings.s3queue_enable_logging_to_s3queue_log.changed) + { + s3queue_settings.s3queue_enable_logging_to_s3queue_log = settings.s3queue_enable_logging_to_s3queue_log; + } + + if (s3queue_settings.s3queue_cleanup_interval_min_ms > s3queue_settings.s3queue_cleanup_interval_max_ms) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Setting `s3queue_cleanup_interval_min_ms` ({}) must be less or equal to `s3queue_cleanup_interval_max_ms` ({})", + s3queue_settings.s3queue_cleanup_interval_min_ms, s3queue_settings.s3queue_cleanup_interval_max_ms); + } + } } StorageS3Queue::StorageS3Queue( @@ -101,11 +127,7 @@ StorageS3Queue::StorageS3Queue( throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs"); } - if (s3queue_settings->mode == S3QueueMode::ORDERED && s3queue_settings->s3queue_processing_threads_num > 1) - { - LOG_WARNING(log, "Parallel processing is not yet supported for Ordered mode"); - s3queue_settings->s3queue_processing_threads_num = 1; - } + checkAndAdjustSettings(*s3queue_settings, context_->getSettingsRef(), log); configuration.update(context_); FormatFactory::instance().checkFormatName(configuration.format); diff --git a/tests/integration/test_storage_s3_queue/configs/users.xml b/tests/integration/test_storage_s3_queue/configs/users.xml index 3118ec43654..8e447278752 100644 --- a/tests/integration/test_storage_s3_queue/configs/users.xml +++ b/tests/integration/test_storage_s3_queue/configs/users.xml @@ -3,6 +3,7 @@ 1 1 + 1 From 52480fe092b11b4e634951c3813d4519ba21e805 Mon Sep 17 00:00:00 2001 From: grantovsky Date: Fri, 13 Oct 2023 15:36:34 +0300 Subject: [PATCH 218/634] Update replacingmergetree.md remove simultaneously word correct: remove simultaneously word of using FINAL CLEANUP and clean_deleted_rows --- .../table-engines/mergetree-family/replacingmergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md index b4139f1cb77..e8089b2c42b 100644 --- a/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/replacingmergetree.md @@ -102,7 +102,7 @@ SELECT * FROM mySecondReplacingMT FINAL; - параметр движка `clean_deleted_rows` установлен в значение `Always` (по умолчанию - `Never`) - есть новые версии строки -Не рекомендуется одновременно выполнять `CLEANUP` и использовать параметр движка `clean_deleted_rows`, это может привести к неожиданным результатам, например удаленные строки могут внось появиться. +Не рекомендуется выполнять `FINAL CLEANUP` или использовать параметр движка `clean_deleted_rows` со значением `Always`, это может привести к неожиданным результатам, например удаленные строки могут вновь появиться. Вне зависимости от производимых изменений над данными, версия должна увеличиваться. Если у двух строк одна и та же версия, то остается только последняя вставленная строка. ::: From e3ceca06e065c0429504e0dcfb6595c28c7ca3b0 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 13 Oct 2023 14:52:25 +0200 Subject: [PATCH 219/634] cast extra column expression `pull_request_number` to Int32 --- docker/test/base/setup_export_logs.sh | 8 ++++---- tests/ci/clickhouse_helper.py | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 2d6e3f76d33..b10644bd8db 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -15,8 +15,8 @@ CLICKHOUSE_CI_LOGS_USER=${CLICKHOUSE_CI_LOGS_USER:-ci} # Pre-configured destination cluster, where to export the data CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export} -EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt16, commit_sha String, check_start_time DateTime('UTC'), check_name String, instance_type String, instance_id String, "} -EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"0 AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type, '' AS instance_id"} +EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name String, instance_type String, instance_id String, "} +EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS Int32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type, '' AS instance_id"} EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "} function __set_connection_args @@ -125,9 +125,9 @@ function setup_logs_replication echo 'Create %_log tables' clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table do - # Calculate hash of its structure. Note: 1 is the version of extra columns - increment it if extra columns are changed: + # Calculate hash of its structure. Note: 4 is the version of extra columns - increment it if extra columns are changed: hash=$(clickhouse-client --query " - SELECT sipHash64(3, groupArray((name, type))) + SELECT sipHash64(4, groupArray((name, type))) FROM (SELECT name, type FROM system.columns WHERE database = 'system' AND table = '$table' ORDER BY position) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 1ace0ef1b24..146f0385ba1 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -296,7 +296,7 @@ class CiLogsCredentials: logging.info("Do not use external logs pushing") return "" extra_columns = ( - f"{pr_info.number} AS pull_request_number, '{pr_info.sha}' AS commit_sha, " + f"CAST({pr_info.number} AS Int32) AS pull_request_number, '{pr_info.sha}' AS commit_sha, " f"toDateTime('{check_start_time}', 'UTC') AS check_start_time, '{check_name}' AS check_name, " f"'{get_instance_type()}' AS instance_type, '{get_instance_id()}' AS instance_id" ) From 0092c48af85069ff1cd2946388b5ee3990486177 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 13 Oct 2023 10:18:30 +0200 Subject: [PATCH 220/634] Fix CI --- .../02784_parallel_replicas_automatic_decision.sh | 10 +++++----- .../02784_parallel_replicas_automatic_decision_join.sh | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index f8249785b67..741b51284fe 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -1,9 +1,10 @@ #!/usr/bin/env bash - CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"} + function were_parallel_replicas_used () { # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it $CLICKHOUSE_CLIENT --query " @@ -36,14 +37,14 @@ $CLICKHOUSE_CLIENT --query " " # $1 -> query_id - # $2 -> min rows per replica + # $2 -> parallel_replicas_min_number_of_rows_per_replica # $3 -> query function run_query_with_pure_parallel_replicas () { # Note that we look into the logs to know how many parallel replicas were estimated because, although the coordinator # might decide to use N replicas, one of them might be fast and do all the work before others start up. This means # that those replicas wouldn't log into the system.query_log and the test would be flaky - $CLICKHOUSE_CLIENT \ + $CLICKHOUSE_CLIENT_TRACE \ --query "$3" \ --query_id "${1}_pure" \ --max_parallel_replicas 3 \ @@ -53,8 +54,7 @@ function run_query_with_pure_parallel_replicas () { --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica "$2" \ - --send_logs_level "trace" \ - 2>&1 | grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' + |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' } query_id_base="02784_automatic_parallel_replicas-$CLICKHOUSE_DATABASE" diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh index 36d92eda896..ed68a304b85 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh @@ -1,11 +1,12 @@ #!/usr/bin/env bash - ## Note: The analyzer doesn't support JOIN with parallel replicas yet CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"} + function were_parallel_replicas_used () { # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it $CLICKHOUSE_CLIENT --query " @@ -51,14 +52,14 @@ $CLICKHOUSE_CLIENT --query " " # $1 -> query_id - # $2 -> min rows per replica + # $2 -> parallel_replicas_min_number_of_rows_per_replica # $3 -> query function run_query_with_pure_parallel_replicas () { # Note that we look into the logs to know how many parallel replicas were estimated because, although the coordinator # might decide to use N replicas, one of them might be fast and do all the work before others start up. This means # that those replicas wouldn't log into the system.query_log and the test would be flaky - $CLICKHOUSE_CLIENT \ + $CLICKHOUSE_CLIENT_TRACE \ --query "$3" \ --query_id "${1}_pure" \ --max_parallel_replicas 3 \ @@ -68,8 +69,7 @@ function run_query_with_pure_parallel_replicas () { --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica "$2" \ - --send_logs_level "trace" \ - 2>&1 | grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' + |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' } query_id_base="02784_automatic_parallel_replicas_join-$CLICKHOUSE_DATABASE" From e70b7dd3b73d1ebfb3c86cbf911de6a92ed163d1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 13 Oct 2023 13:27:08 +0200 Subject: [PATCH 221/634] Remove unused protobuf includes --- src/DataTypes/Serializations/SerializationAggregateFunction.cpp | 2 -- src/DataTypes/Serializations/SerializationArray.cpp | 1 - src/DataTypes/Serializations/SerializationDate.cpp | 1 - src/DataTypes/Serializations/SerializationDateTime.cpp | 2 -- src/DataTypes/Serializations/SerializationDateTime64.cpp | 1 - src/DataTypes/Serializations/SerializationDecimal.cpp | 2 -- src/DataTypes/Serializations/SerializationDecimalBase.cpp | 2 -- src/DataTypes/Serializations/SerializationEnum.cpp | 2 -- src/DataTypes/Serializations/SerializationFixedString.cpp | 2 -- src/DataTypes/Serializations/SerializationNumber.cpp | 1 - src/DataTypes/Serializations/SerializationUUID.cpp | 2 -- 11 files changed, 18 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index c482c9623e9..ad71c31a2d7 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -10,8 +10,6 @@ #include #include -#include -#include #include #include diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index d6f36e45e64..c804f58c567 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -11,7 +11,6 @@ #include #include -#include namespace DB { diff --git a/src/DataTypes/Serializations/SerializationDate.cpp b/src/DataTypes/Serializations/SerializationDate.cpp index 1ed48fdd31d..534f599a072 100644 --- a/src/DataTypes/Serializations/SerializationDate.cpp +++ b/src/DataTypes/Serializations/SerializationDate.cpp @@ -4,7 +4,6 @@ #include #include -#include #include diff --git a/src/DataTypes/Serializations/SerializationDateTime.cpp b/src/DataTypes/Serializations/SerializationDateTime.cpp index 2ba24f5351b..5faec2b3833 100644 --- a/src/DataTypes/Serializations/SerializationDateTime.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime.cpp @@ -4,8 +4,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/src/DataTypes/Serializations/SerializationDateTime64.cpp b/src/DataTypes/Serializations/SerializationDateTime64.cpp index c5964f1bd97..1bd9669f07d 100644 --- a/src/DataTypes/Serializations/SerializationDateTime64.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime64.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include diff --git a/src/DataTypes/Serializations/SerializationDecimal.cpp b/src/DataTypes/Serializations/SerializationDecimal.cpp index 9de85d338e9..cf876d785ee 100644 --- a/src/DataTypes/Serializations/SerializationDecimal.cpp +++ b/src/DataTypes/Serializations/SerializationDecimal.cpp @@ -3,8 +3,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/src/DataTypes/Serializations/SerializationDecimalBase.cpp b/src/DataTypes/Serializations/SerializationDecimalBase.cpp index b7f91e6833e..67cf0ae8d79 100644 --- a/src/DataTypes/Serializations/SerializationDecimalBase.cpp +++ b/src/DataTypes/Serializations/SerializationDecimalBase.cpp @@ -2,8 +2,6 @@ #include #include -#include -#include #include #include diff --git a/src/DataTypes/Serializations/SerializationEnum.cpp b/src/DataTypes/Serializations/SerializationEnum.cpp index 71b92bda375..7e84371914c 100644 --- a/src/DataTypes/Serializations/SerializationEnum.cpp +++ b/src/DataTypes/Serializations/SerializationEnum.cpp @@ -4,8 +4,6 @@ #include #include #include -#include -#include namespace DB { diff --git a/src/DataTypes/Serializations/SerializationFixedString.cpp b/src/DataTypes/Serializations/SerializationFixedString.cpp index 1e5ddeb1b19..fa50af52f2f 100644 --- a/src/DataTypes/Serializations/SerializationFixedString.cpp +++ b/src/DataTypes/Serializations/SerializationFixedString.cpp @@ -4,8 +4,6 @@ #include #include -#include -#include #include #include diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 94b44d5cc66..12e6ec88b49 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include diff --git a/src/DataTypes/Serializations/SerializationUUID.cpp b/src/DataTypes/Serializations/SerializationUUID.cpp index 8e8b80cf9d1..5cf17b4c0c8 100644 --- a/src/DataTypes/Serializations/SerializationUUID.cpp +++ b/src/DataTypes/Serializations/SerializationUUID.cpp @@ -1,7 +1,5 @@ #include #include -#include -#include #include #include #include From b9f8f2fc15e9fc21670182b5b7cb36cac6c6ba20 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 13 Oct 2023 13:27:23 +0200 Subject: [PATCH 222/634] Apply clang-format to the changes --- .../Serializations/SerializationAggregateFunction.cpp | 2 +- src/DataTypes/Serializations/SerializationDateTime.cpp | 6 +++--- src/DataTypes/Serializations/SerializationDateTime64.cpp | 6 +++--- src/DataTypes/Serializations/SerializationDecimal.cpp | 4 ++-- .../Serializations/SerializationDecimalBase.cpp | 4 ++-- src/DataTypes/Serializations/SerializationEnum.cpp | 4 ++-- src/DataTypes/Serializations/SerializationNumber.cpp | 9 +++++---- 7 files changed, 18 insertions(+), 17 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index ad71c31a2d7..c9af5d1f838 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -10,8 +10,8 @@ #include #include -#include #include +#include namespace DB { diff --git a/src/DataTypes/Serializations/SerializationDateTime.cpp b/src/DataTypes/Serializations/SerializationDateTime.cpp index 5faec2b3833..77beb0d9b75 100644 --- a/src/DataTypes/Serializations/SerializationDateTime.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime.cpp @@ -1,14 +1,14 @@ #include #include -#include -#include #include #include +#include #include #include #include -#include +#include +#include namespace DB { diff --git a/src/DataTypes/Serializations/SerializationDateTime64.cpp b/src/DataTypes/Serializations/SerializationDateTime64.cpp index 1bd9669f07d..93891886000 100644 --- a/src/DataTypes/Serializations/SerializationDateTime64.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime64.cpp @@ -1,14 +1,14 @@ #include #include -#include -#include #include +#include #include #include #include #include -#include +#include +#include namespace DB { diff --git a/src/DataTypes/Serializations/SerializationDecimal.cpp b/src/DataTypes/Serializations/SerializationDecimal.cpp index cf876d785ee..b576b7a048c 100644 --- a/src/DataTypes/Serializations/SerializationDecimal.cpp +++ b/src/DataTypes/Serializations/SerializationDecimal.cpp @@ -1,11 +1,11 @@ #include #include -#include -#include #include #include #include +#include +#include namespace DB { diff --git a/src/DataTypes/Serializations/SerializationDecimalBase.cpp b/src/DataTypes/Serializations/SerializationDecimalBase.cpp index 67cf0ae8d79..49dc042e872 100644 --- a/src/DataTypes/Serializations/SerializationDecimalBase.cpp +++ b/src/DataTypes/Serializations/SerializationDecimalBase.cpp @@ -1,9 +1,9 @@ #include -#include -#include #include #include +#include +#include #include diff --git a/src/DataTypes/Serializations/SerializationEnum.cpp b/src/DataTypes/Serializations/SerializationEnum.cpp index 7e84371914c..09b0b02714c 100644 --- a/src/DataTypes/Serializations/SerializationEnum.cpp +++ b/src/DataTypes/Serializations/SerializationEnum.cpp @@ -1,9 +1,9 @@ #include #include -#include -#include #include +#include +#include namespace DB { diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 12e6ec88b49..b6c7e4618b8 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -1,13 +1,14 @@ #include -#include + #include +#include +#include +#include #include #include #include -#include #include -#include -#include +#include #include From de62095b98ae6305753f0e148b21d3dd45414f93 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 12 Oct 2023 23:40:26 +0200 Subject: [PATCH 223/634] Test for query success with 'break' mode and timeout > 10 sec --- ...x_execution_time_with_break_overflow_mode.reference | 0 ...896_max_execution_time_with_break_overflow_mode.sql | 10 ++++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.reference create mode 100644 tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.sql diff --git a/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.reference b/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.sql b/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.sql new file mode 100644 index 00000000000..439b8b3f032 --- /dev/null +++ b/tests/queries/0_stateless/02896_max_execution_time_with_break_overflow_mode.sql @@ -0,0 +1,10 @@ +-- Tags: no-fasttest + +-- Query stops after timeout without an error +SELECT * FROM numbers(100000000) SETTINGS max_block_size=1, max_execution_time=13, timeout_overflow_mode='break' FORMAT Null; + +-- Query returns an error when runtime is estimated after 10 sec of execution +SELECT * FROM numbers(100000000) SETTINGS max_block_size=1, max_execution_time=13, timeout_overflow_mode='throw' FORMAT Null; -- { serverError TOO_SLOW } + +-- Query returns timeout error before its full execution time is estimated +SELECT * FROM numbers(100000000) SETTINGS max_block_size=1, max_execution_time=2, timeout_overflow_mode='throw' FORMAT Null; -- { serverError TIMEOUT_EXCEEDED } From 1d7f04208fb1ecb59aec67b89e1ebd108b74404c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 12 Oct 2023 23:42:01 +0200 Subject: [PATCH 224/634] Throw TOO_SLOW error only when overflow mode is 'throw' --- src/QueryPipeline/ExecutionSpeedLimits.cpp | 5 +++-- src/QueryPipeline/ExecutionSpeedLimits.h | 3 ++- src/QueryPipeline/ReadProgressCallback.cpp | 2 +- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/QueryPipeline/ExecutionSpeedLimits.cpp b/src/QueryPipeline/ExecutionSpeedLimits.cpp index 111ba7c9a95..9ceaa4921c7 100644 --- a/src/QueryPipeline/ExecutionSpeedLimits.cpp +++ b/src/QueryPipeline/ExecutionSpeedLimits.cpp @@ -45,7 +45,8 @@ static void limitProgressingSpeed(size_t total_progress_size, size_t max_speed_i void ExecutionSpeedLimits::throttle( size_t read_rows, size_t read_bytes, - size_t total_rows_to_read, UInt64 total_elapsed_microseconds) const + size_t total_rows_to_read, UInt64 total_elapsed_microseconds, + OverflowMode timeout_overflow_mode) const { if ((min_execution_rps != 0 || max_execution_rps != 0 || min_execution_bps != 0 || max_execution_bps != 0 @@ -82,7 +83,7 @@ void ExecutionSpeedLimits::throttle( { double estimated_execution_time_seconds = elapsed_seconds * (static_cast(total_rows_to_read) / read_rows); - if (estimated_execution_time_seconds > max_execution_time.totalSeconds()) + if (timeout_overflow_mode == OverflowMode::THROW && estimated_execution_time_seconds > max_execution_time.totalSeconds()) throw Exception( ErrorCodes::TOO_SLOW, "Estimated query execution time ({} seconds) is too long. Maximum: {}. Estimated rows to process: {}", diff --git a/src/QueryPipeline/ExecutionSpeedLimits.h b/src/QueryPipeline/ExecutionSpeedLimits.h index 63658462c9f..eed8b5c3248 100644 --- a/src/QueryPipeline/ExecutionSpeedLimits.h +++ b/src/QueryPipeline/ExecutionSpeedLimits.h @@ -25,7 +25,8 @@ public: Poco::Timespan timeout_before_checking_execution_speed = 0; /// Pause execution in case if speed limits were exceeded. - void throttle(size_t read_rows, size_t read_bytes, size_t total_rows_to_read, UInt64 total_elapsed_microseconds) const; + void throttle(size_t read_rows, size_t read_bytes, size_t total_rows_to_read, UInt64 total_elapsed_microseconds, + OverflowMode timeout_overflow_mode) const; bool checkTimeLimit(const Stopwatch & stopwatch, OverflowMode overflow_mode) const; }; diff --git a/src/QueryPipeline/ReadProgressCallback.cpp b/src/QueryPipeline/ReadProgressCallback.cpp index 4d7c7aa0f2a..59843d8791d 100644 --- a/src/QueryPipeline/ReadProgressCallback.cpp +++ b/src/QueryPipeline/ReadProgressCallback.cpp @@ -130,7 +130,7 @@ bool ReadProgressCallback::onProgress(uint64_t read_rows, uint64_t read_bytes, c /// TODO: Should be done in PipelineExecutor. for (const auto & limits : storage_limits) - limits.local_limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_stopwatch.elapsedMicroseconds()); + limits.local_limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_stopwatch.elapsedMicroseconds(), limits.local_limits.timeout_overflow_mode); if (quota) quota->used({QuotaType::READ_ROWS, value.read_rows}, {QuotaType::READ_BYTES, value.read_bytes}); From 1957179058e2d57c3cae50e0eba67ff1baa6b0c9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 13 Oct 2023 13:21:38 +0000 Subject: [PATCH 225/634] Make the warning in the docs more strict --- .../sql-reference/functions/date-time-functions.md | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 33d95c072a0..9b6eba9b5f0 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -587,21 +587,21 @@ The two-argument form of `toDayOfWeek()` enables you to specify whether the week | 2 | Sunday | 0-6: Sunday = 0, Monday = 1, ..., Saturday = 6 | | 3 | Sunday | 1-7: Sunday = 1, Monday = 2, ..., Saturday = 7 | -Alias: `DAYOFWEEK`. - **Syntax** ``` sql toDayOfWeek(t[, mode[, timezone]]) ``` +Alias: `DAYOFWEEK`. + **Arguments** - `t` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md) - `mode` - determines what the first day of the week is. Possible values are 0, 1, 2 or 3. See the table above for the differences. - `timezone` - optional parameter, it behaves like any other conversion function -The first argument can also be specified as [String](../data-types/string.md) in a format supported by [parseDateTime64BestEffort()](type-conversion-functions.md#parsedatetime64besteffort). This was implemented for reasons of compatibility with MySQL which certain 3rd party tools expect. Because string parsing is slow, it is generally recommended to not use string arguments. +The first argument can also be specified as [String](../data-types/string.md) in a format supported by [parseDateTime64BestEffort()](type-conversion-functions.md#parsedatetime64besteffort). Support for string arguments exists only for reasons of compatibility with MySQL which is expected by certain 3rd party tools. As string argument support may in future be made dependent on new MySQL-compatibility settings and because string parsing is generally slow, it is recommended to not use it. **Returned value** @@ -1491,13 +1491,15 @@ For mode values with a meaning of “contains January 1”, the week contains Ja toWeek(t[, mode[, time_zone]]) ``` +Alias: `WEEK` + **Arguments** - `t` – Date or DateTime. - `mode` – Optional parameter, Range of values is \[0,9\], default is 0. - `Timezone` – Optional parameter, it behaves like any other conversion function. -The first argument can also be specified as [String](../data-types/string.md) in a format supported by [parseDateTime64BestEffort()](type-conversion-functions.md#parsedatetime64besteffort). This was implemented for reasons of compatibility with MySQL which certain 3rd party tools expect. Because string parsing is slow, it is generally recommended to not use string arguments. +The first argument can also be specified as [String](../data-types/string.md) in a format supported by [parseDateTime64BestEffort()](type-conversion-functions.md#parsedatetime64besteffort). Support for string arguments exists only for reasons of compatibility with MySQL which is expected by certain 3rd party tools. As string argument support may in future be made dependent on new MySQL-compatibility settings and because string parsing is generally slow, it is recommended to not use it. **Example** @@ -1529,7 +1531,9 @@ The week number returned by `toYearWeek()` can be different from what the `toWee toYearWeek(t[, mode[, timezone]]) ``` -The first argument can also be specified as [String](../data-types/string.md) in a format supported by [parseDateTime64BestEffort()](type-conversion-functions.md#parsedatetime64besteffort). This was implemented for reasons of compatibility with MySQL which certain 3rd party tools expect. Because string parsing is slow, it is generally recommended to not use string arguments. +Alias: `YEARWEEK` + +The first argument can also be specified as [String](../data-types/string.md) in a format supported by [parseDateTime64BestEffort()](type-conversion-functions.md#parsedatetime64besteffort). Support for string arguments exists only for reasons of compatibility with MySQL which is expected by certain 3rd party tools. As string argument support may in future be made dependent on new MySQL-compatibility settings and because string parsing is generally slow, it is recommended to not use it. **Example** From 44ea61e57fa5e6898c15c67f51430024d519a8f8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Oct 2023 15:20:54 +0200 Subject: [PATCH 226/634] Improve shutdown --- src/Storages/S3Queue/S3QueueSettings.h | 4 +-- src/Storages/S3Queue/S3QueueSource.cpp | 33 +++++++++++++++++++------ src/Storages/S3Queue/S3QueueSource.h | 7 ++++-- src/Storages/S3Queue/StorageS3Queue.cpp | 2 +- 4 files changed, 33 insertions(+), 13 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index cc9f97182bc..351245e24be 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -27,8 +27,8 @@ class ASTStorage; M(UInt32, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ M(UInt32, s3queue_polling_backoff_ms, 1000, "Polling backoff", 0) \ M(UInt32, s3queue_tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \ - M(UInt32, s3queue_cleanup_interval_min_ms, 10000, "For unordered mode. Polling backoff min for cleanup", 0) \ - M(UInt32, s3queue_cleanup_interval_max_ms, 30000, "For unordered mode. Polling backoff max for cleanup", 0) \ + M(UInt32, s3queue_cleanup_interval_min_ms, 60000, "For unordered mode. Polling backoff min for cleanup", 0) \ + M(UInt32, s3queue_cleanup_interval_max_ms, 60000, "For unordered mode. Polling backoff max for cleanup", 0) \ #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 7140ec9d4f0..2e2435b0d94 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -40,28 +40,30 @@ StorageS3QueueSource::S3QueueKeyWithInfo::S3QueueKeyWithInfo( } StorageS3QueueSource::FileIterator::FileIterator( - std::shared_ptr metadata_, std::unique_ptr glob_iterator_) - : metadata(metadata_) , glob_iterator(std::move(glob_iterator_)) + std::shared_ptr metadata_, + std::unique_ptr glob_iterator_, + std::atomic & shutdown_called_) + : metadata(metadata_) + , glob_iterator(std::move(glob_iterator_)) + , shutdown_called(shutdown_called_) { } StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next() { - /// List results in s3 are always returned in UTF-8 binary order. - /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) - - while (true) + while (!shutdown_called) { KeyWithInfoPtr val = glob_iterator->next(); - if (!val) + if (!val || shutdown_called) return {}; - if (auto processing_holder = metadata->trySetFileAsProcessing(val->key); processing_holder) + if (auto processing_holder = metadata->trySetFileAsProcessing(val->key); processing_holder && !shutdown_called) { return std::make_shared(val->key, val->info, processing_holder); } } + return {}; } size_t StorageS3QueueSource::FileIterator::estimatedKeysCount() @@ -125,6 +127,21 @@ Chunk StorageS3QueueSource::generate() break; } + if (shutdown_called) + { + if (processed_rows_from_file) + { + /// We could delay shutdown until files, which already started processing before the shutdown, finished. + /// But if files are big and `s3queue_processing_threads_num` is not small, it can take a significant time. + /// Anyway we cannot do anything in case of SIGTERM, so destination table must anyway support deduplication, + /// so here we will rely on it here as well. + LOG_WARNING( + log, "Shutdown called, {} rows are already processed, but file is not fully processed", + processed_rows_from_file); + } + break; + } + auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); SCOPE_EXIT({ CurrentThread::get().attachProfileCountersScope(prev_scope); }); diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 9bd2cad9c72..a52bfb5a34f 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -14,7 +14,6 @@ namespace Poco { class Logger; } namespace DB { - class StorageS3QueueSource : public ISource, WithContext { public: @@ -38,8 +37,11 @@ public: class FileIterator : public IIterator { public: - FileIterator(std::shared_ptr metadata_, std::unique_ptr glob_iterator_); + FileIterator(std::shared_ptr metadata_, std::unique_ptr glob_iterator_, std::atomic & shutdown_called_); + /// Note: + /// List results in s3 are always returned in UTF-8 binary order. + /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) KeyWithInfoPtr next() override; size_t estimatedKeysCount() override; @@ -47,6 +49,7 @@ public: private: const std::shared_ptr metadata; const std::unique_ptr glob_iterator; + std::atomic & shutdown_called; std::mutex mutex; }; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index c280b909d1a..37389eb1bd0 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -453,7 +453,7 @@ std::shared_ptr StorageS3Queue::createFileIterator auto glob_iterator = std::make_unique( *configuration.client, configuration.url, query, virtual_columns, local_context, /* read_keys */nullptr, configuration.request_settings); - return std::make_shared(files_metadata, std::move(glob_iterator)); + return std::make_shared(files_metadata, std::move(glob_iterator), shutdown_called); } void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) From 593c3114a152f9eeecede03a1b065cca9bca2ae1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 13 Oct 2023 13:24:26 +0000 Subject: [PATCH 227/634] Remove TODO comment --- src/Functions/CustomWeekTransforms.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Functions/CustomWeekTransforms.h b/src/Functions/CustomWeekTransforms.h index f2e637e7a92..b8d25cc30b6 100644 --- a/src/Functions/CustomWeekTransforms.h +++ b/src/Functions/CustomWeekTransforms.h @@ -73,9 +73,6 @@ struct CustomWeekTransformImpl if constexpr (std::is_same_v) { - /// TODO: remove this entire if constexpr branch, move the parsing code into DateTimeTransforms (yes, even if that will - /// duplicate a few lines of code) - static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); const auto * sources = checkAndGetColumn(source_col.get()); From 2653ace12223ef3852c6b44d97afbd5cef2f0f88 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 13 Oct 2023 13:35:40 +0000 Subject: [PATCH 228/634] Update docs --- docs/en/operations/query-cache.md | 26 +++++++++----- .../operations/system-tables/query_cache.md | 36 +++++++++++++++++++ 2 files changed, 54 insertions(+), 8 deletions(-) create mode 100644 docs/en/operations/system-tables/query_cache.md diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 8405cd2bd36..665ae6cdfdc 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -61,17 +61,17 @@ FROM table SETTINGS use_query_cache = true, enable_writes_to_query_cache = false; ``` -For maximum control, it is generally recommended to provide settings "use_query_cache", "enable_writes_to_query_cache" and -"enable_reads_from_query_cache" only with specific queries. It is also possible to enable caching at user or profile level (e.g. via `SET +For maximum control, it is generally recommended to provide settings `use_query_cache`, `enable_writes_to_query_cache` and +`enable_reads_from_query_cache` only with specific queries. It is also possible to enable caching at user or profile level (e.g. via `SET use_query_cache = true`) but one should keep in mind that all `SELECT` queries including monitoring or debugging queries to system tables may return cached results then. The query cache can be cleared using statement `SYSTEM DROP QUERY CACHE`. The content of the query cache is displayed in system table -`system.query_cache`. The number of query cache hits and misses since database start are shown as events "QueryCacheHits" and -"QueryCacheMisses" in system table [system.events](system-tables/events.md). Both counters are only updated for `SELECT` queries which run -with setting `use_query_cache = true`, other queries do not affect "QueryCacheMisses". Field `query_cache_usage` in system table -[system.query_log](system-tables/query_log.md) shows for each executed query whether the query result was written into or read from the -query cache. Asynchronous metrics "QueryCacheEntries" and "QueryCacheBytes" in system table +[system.query_cache](system-tables/query_cache.md). The number of query cache hits and misses since database start are shown as events +"QueryCacheHits" and "QueryCacheMisses" in system table [system.events](system-tables/events.md). Both counters are only updated for +`SELECT` queries which run with setting `use_query_cache = true`, other queries do not affect "QueryCacheMisses". Field `query_cache_usage` +in system table [system.query_log](system-tables/query_log.md) shows for each executed query whether the query result was written into or +read from the query cache. Asynchronous metrics "QueryCacheEntries" and "QueryCacheBytes" in system table [system.asynchronous_metrics](system-tables/asynchronous_metrics.md) show how many entries / bytes the query cache currently contains. The query cache exists once per ClickHouse server process. However, cache results are by default not shared between users. This can be @@ -86,9 +86,18 @@ If the query was aborted due to an exception or user cancellation, no entry is w The size of the query cache in bytes, the maximum number of cache entries and the maximum size of individual cache entries (in bytes and in records) can be configured using different [server configuration options](server-configuration-parameters/settings.md#server_configuration_parameters_query-cache). +```xml + + 1073741824 + 1024 + 1048576 + 30000000 + +``` + It is also possible to limit the cache usage of individual users using [settings profiles](settings/settings-profiles.md) and [settings constraints](settings/constraints-on-settings.md). More specifically, you can restrict the maximum amount of memory (in bytes) a user may -allocate in the query cache and the the maximum number of stored query results. For that, first provide configurations +allocate in the query cache and the maximum number of stored query results. For that, first provide configurations [query_cache_max_size_in_bytes](settings/settings.md#query-cache-max-size-in-bytes) and [query_cache_max_entries](settings/settings.md#query-cache-size-max-entries) in a user profile in `users.xml`, then make both settings readonly: @@ -158,6 +167,7 @@ Also, results of queries with non-deterministic functions are not cached by defa - functions which depend on the environment: [`currentUser()`](../sql-reference/functions/other-functions.md#currentUser), [`queryID()`](../sql-reference/functions/other-functions.md#queryID), [`getMacro()`](../sql-reference/functions/other-functions.md#getMacro) etc. + To force caching of results of queries with non-deterministic functions regardless, use setting [query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions). diff --git a/docs/en/operations/system-tables/query_cache.md b/docs/en/operations/system-tables/query_cache.md new file mode 100644 index 00000000000..a9f86f5fc2b --- /dev/null +++ b/docs/en/operations/system-tables/query_cache.md @@ -0,0 +1,36 @@ +--- +slug: /en/operations/system-tables/query_cache +--- +# query_cache + +Shows the content of the [query cache](../query-cache.md). + +Columns: + +- `query` ([String](../../sql-reference/data-types/string.md)) — Query string. +- `result_size` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Size of the query cache entry. +- `stale` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is stale. +- `shared` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is shared between multiple users. +- `compressed` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is compressed. +- `expires_at` ([DateTime](../../sql-reference/data-types/datetime.md)) — When the query cache entry becomes stale. +- `key_hash` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — A hash of the query string, used as a key to find query cache entries. + +**Example** + +``` sql +SELECT * FROM system.query_cache FORMAT Vertical; +``` + +``` text +Row 1: +────── +query: SELECT 1 SETTINGS use_query_cache = 1 +result_size: 128 +stale: 0 +shared: 0 +compressed: 1 +expires_at: 2023-10-13 13:35:45 +key_hash: 12188185624808016954 + +1 row in set. Elapsed: 0.004 sec. +``` From 6fde98a33f40b8ef3d5b75578c6ee6b64a609b9c Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Oct 2023 15:56:05 +0200 Subject: [PATCH 229/634] Minor improvement --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 21 +++++++++++-------- src/Storages/S3Queue/S3QueueFilesMetadata.h | 13 ++++++------ src/Storages/S3Queue/S3QueueSource.cpp | 15 +++++++------ src/Storages/S3Queue/S3QueueSource.h | 6 ++++-- 4 files changed, 32 insertions(+), 23 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 535ced7f6df..ac12e8fe70f 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -64,8 +64,7 @@ S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::LocalFileStatuses::getA return file_statuses; } -std::shared_ptr -S3QueueFilesMetadata::LocalFileStatuses::get(const std::string & filename, bool create) +S3QueueFilesMetadata::FileStatusPtr S3QueueFilesMetadata::LocalFileStatuses::get(const std::string & filename, bool create) { auto lk = lock(); auto it = file_statuses.find(filename); @@ -161,7 +160,7 @@ zkutil::ZooKeeperPtr S3QueueFilesMetadata::getZooKeeper() const return Context::getGlobalContextInstance()->getZooKeeper(); } -std::shared_ptr S3QueueFilesMetadata::getFileStatus(const std::string & path) +S3QueueFilesMetadata::FileStatusPtr S3QueueFilesMetadata::getFileStatus(const std::string & path) { /// Return a locally cached file status. return local_file_statuses.get(path, /* create */false); @@ -198,7 +197,8 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( return metadata; } -S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) +std::pair S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessingMicroseconds); auto file_status = local_file_statuses.get(path, /* create */true); @@ -216,13 +216,13 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs case FileStatus::State::Processing: [[fallthrough]]; case FileStatus::State::Processed: { - return nullptr; + return {}; } case FileStatus::State::Failed: { /// If max_loading_retries == 0, file is not retriable. if (max_loading_retries == 0) - return nullptr; + return {}; /// Otherwise file_status->retries is also cached. /// In case file_status->retries >= max_loading_retries we can fully rely that it is true @@ -231,7 +231,7 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs /// (another server could have done a try after we cached retries value), /// so check with zookeeper here. if (file_status->retries >= max_loading_retries) - return nullptr; + return {}; break; } @@ -249,7 +249,7 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs std::unique_lock processing_lock(file_status->processing_lock, std::defer_lock); if (!processing_lock.try_lock()) { - return nullptr; + return {}; } /// Let's go and check metadata in zookeeper and try to create a /processing ephemeral node. @@ -306,7 +306,10 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs } } - return result == SetFileProcessingResult::Success ? processing_node_holder : nullptr; + if (result == SetFileProcessingResult::Success) + return std::pair(processing_node_holder, file_status); + + return {}; } std::pair>; + using FileStatusPtr = std::shared_ptr; + using FileStatuses = std::unordered_map; - std::shared_ptr getFileStatus(const std::string & path); + /// Set file as processing, if it is not alreaty processed, failed or processing. + std::pair trySetFileAsProcessing(const std::string & path); + + FileStatusPtr getFileStatus(const std::string & path); FileStatuses getFileStateses() const { return local_file_statuses.getAll(); } @@ -137,7 +138,7 @@ private: mutable std::mutex mutex; FileStatuses getAll() const; - std::shared_ptr get(const std::string & filename, bool create); + FileStatusPtr get(const std::string & filename, bool create); bool remove(const std::string & filename, bool if_exists); std::unique_lock lock() const; }; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 2e2435b0d94..b3f8f0cf1c3 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -33,9 +33,11 @@ namespace ErrorCodes StorageS3QueueSource::S3QueueKeyWithInfo::S3QueueKeyWithInfo( const std::string & key_, std::optional info_, - Metadata::ProcessingNodeHolderPtr processing_holder_) + Metadata::ProcessingNodeHolderPtr processing_holder_, + FileStatusPtr file_status_) : StorageS3Source::KeyWithInfo(key_, info_) , processing_holder(processing_holder_) + , file_status(file_status_) { } @@ -58,9 +60,10 @@ StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next() if (!val || shutdown_called) return {}; - if (auto processing_holder = metadata->trySetFileAsProcessing(val->key); processing_holder && !shutdown_called) + if (auto [processing_holder, processing_file_status] = metadata->trySetFileAsProcessing(val->key); + processing_holder && !shutdown_called) { - return std::make_shared(val->key, val->info, processing_holder); + return std::make_shared(val->key, val->info, processing_holder, processing_file_status); } } return {}; @@ -100,7 +103,6 @@ StorageS3QueueSource::StorageS3QueueSource( if (reader) { reader_future = std::move(internal_source->reader_future); - file_status = files_metadata->getFileStatus(reader.getFile()); } } @@ -142,11 +144,12 @@ Chunk StorageS3QueueSource::generate() break; } + const auto * key_with_info = dynamic_cast(&reader.getKeyWithInfo()); + auto file_status = key_with_info->file_status; + auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); SCOPE_EXIT({ CurrentThread::get().attachProfileCountersScope(prev_scope); }); - const auto * key_with_info = dynamic_cast(&reader.getKeyWithInfo()); - try { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueuePullMicroseconds); diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index a52bfb5a34f..db3015f129e 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -22,6 +22,7 @@ public: using GlobIterator = StorageS3Source::DisclosedGlobIterator; using ZooKeeperGetter = std::function; using RemoveFileFunc = std::function; + using FileStatusPtr = S3QueueFilesMetadata::FileStatusPtr; using Metadata = S3QueueFilesMetadata; struct S3QueueKeyWithInfo : public StorageS3Source::KeyWithInfo @@ -29,9 +30,11 @@ public: S3QueueKeyWithInfo( const std::string & key_, std::optional info_, - Metadata::ProcessingNodeHolderPtr processing_holder_); + Metadata::ProcessingNodeHolderPtr processing_holder_, + FileStatusPtr file_status_); Metadata::ProcessingNodeHolderPtr processing_holder; + FileStatusPtr file_status; }; class FileIterator : public IIterator @@ -91,7 +94,6 @@ private: ReaderHolder reader; std::future reader_future; size_t processed_rows_from_file = 0; - std::shared_ptr file_status; void applyActionAfterProcessing(const String & path); void appendLogElement(const std::string & filename, S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); From 67068849da835ae8221a131b9335ba380f7ff556 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 13 Oct 2023 15:59:37 +0200 Subject: [PATCH 230/634] Fix comment --- src/Coordination/Standalone/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 84cd3f6afe2..ef23e85e54b 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -41,7 +41,7 @@ struct ContextSharedPart : boost::noncopyable : macros(std::make_unique()) {} - /// For access of most of shared objects. Recursive mutex. + /// For access of most of shared objects. mutable SharedMutex mutex; mutable std::mutex keeper_dispatcher_mutex; From ba5cd016dd661cfc332534664ed17e335d1e9f7c Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 13 Oct 2023 14:02:31 +0000 Subject: [PATCH 231/634] Fix crash in QueryNormalizer with cyclic aliases --- src/Interpreters/QueryNormalizer.cpp | 6 ++++++ .../02896_cyclic_aliases_crash.reference | 1 + .../0_stateless/02896_cyclic_aliases_crash.sql | 17 +++++++++++++++++ 3 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/02896_cyclic_aliases_crash.reference create mode 100644 tests/queries/0_stateless/02896_cyclic_aliases_crash.sql diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 56b81b3d224..6b6ead1f463 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -68,6 +68,10 @@ private: void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) { + /// We do handle cycles via tracking current_asts + /// but in case of bug in that tricky logic we need to prevent stack overflow + checkStackSize(); + auto & current_asts = data.current_asts; String & current_alias = data.current_alias; @@ -116,6 +120,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// In a construct like "a AS b", where a is an alias, you must set alias b to the result of substituting alias a. /// Check size of the alias before cloning too large alias AST alias_node->checkSize(data.settings.max_expanded_ast_elements); + current_asts.insert(alias_node.get()); ast = alias_node->clone(); ast->setAlias(node_alias); @@ -134,6 +139,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// Check size of the alias before cloning too large alias AST alias_node->checkSize(data.settings.max_expanded_ast_elements); auto alias_name = ast->getAliasOrColumnName(); + current_asts.insert(alias_node.get()); ast = alias_node->clone(); ast->setAlias(alias_name); diff --git a/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference b/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference new file mode 100644 index 00000000000..d43017edcc5 --- /dev/null +++ b/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference @@ -0,0 +1 @@ +1 2 3 diff --git a/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql b/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql new file mode 100644 index 00000000000..af54f5df7a7 --- /dev/null +++ b/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql @@ -0,0 +1,17 @@ + +SET max_ast_depth = 10_000_000; + +SELECT + val, + val + 1 as prev, + val + prev as val +FROM ( SELECT 1 as val ) +; -- { serverError CYCLIC_ALIASES } + + +SELECT + val, + val + 1 as prev, + val + prev as val2 +FROM ( SELECT 1 as val ) +; From 47d48d81e09025314e22a1d5e5440d63b2cde592 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Oct 2023 16:22:18 +0200 Subject: [PATCH 232/634] Implement feature --- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Parsers/ASTPartition.cpp | 55 ++++++++--- src/Parsers/ASTPartition.h | 13 ++- src/Parsers/ParserPartition.cpp | 59 +++++++----- src/Storages/IStorage.cpp | 5 +- src/Storages/IStorage.h | 6 +- src/Storages/MergeTree/MergeTreeData.cpp | 32 +++---- src/Storages/MergeTree/MergeTreeData.h | 6 +- src/Storages/MergeTree/MergeTreePartInfo.cpp | 11 ++- src/Storages/MergeTree/MergeTreePartInfo.h | 3 +- src/Storages/StorageMaterializedView.cpp | 5 +- src/Storages/StorageMaterializedView.h | 2 +- src/Storages/StorageProxy.h | 4 +- .../02897_alter_partition_parameters.sql | 94 +++++++++++++++++++ 14 files changed, 227 insertions(+), 70 deletions(-) create mode 100644 tests/queries/0_stateless/02897_alter_partition_parameters.sql diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index a7445f89245..f851607000c 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -172,7 +172,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) if (!partition_commands.empty()) { - table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, getContext()->getSettingsRef()); + table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, getContext()->getSettingsRef(), getContext()); auto partition_commands_pipe = table->alterPartition(metadata_snapshot, partition_commands, getContext()); if (!partition_commands_pipe.empty()) res.pipeline = QueryPipeline(std::move(partition_commands_pipe)); diff --git a/src/Parsers/ASTPartition.cpp b/src/Parsers/ASTPartition.cpp index 87d159b5817..22eecd37565 100644 --- a/src/Parsers/ASTPartition.cpp +++ b/src/Parsers/ASTPartition.cpp @@ -5,12 +5,39 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +void ASTPartition::setPartitionID(const ASTPtr & ast) +{ + if (children.empty()) + { + children.push_back(ast); + id = children[0].get(); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot have multiple children for partition AST"); +} +void ASTPartition::setPartitionValue(const ASTPtr & ast) +{ + if (children.empty()) + { + children.push_back(ast); + value = children[0].get(); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot have multiple children for partition AST"); +} + + String ASTPartition::getID(char delim) const { if (value) return "Partition"; else - return "Partition_ID" + (delim + id); + return "Partition_ID" + (delim + id->getID()); } ASTPtr ASTPartition::clone() const @@ -20,8 +47,14 @@ ASTPtr ASTPartition::clone() const if (value) { - res->value = value->clone(); - res->children.push_back(res->value); + res->children.push_back(children[0]->clone()); + res->value = res->children[0].get(); + } + + if (id) + { + res->children.push_back(children[0]->clone()); + res->id = res->children[0].get(); } return res; @@ -33,18 +66,14 @@ void ASTPartition::formatImpl(const FormatSettings & settings, FormatState & sta { value->formatImpl(settings, state, frame); } + else if (all) + { + settings.ostr << "ALL"; + } else { - if (all) - settings.ostr << "ALL"; - else - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "ID " << (settings.hilite ? hilite_none : ""); - WriteBufferFromOwnString id_buf; - writeQuoted(id, id_buf); - settings.ostr << id_buf.str(); - } + settings.ostr << (settings.hilite ? hilite_keyword : "") << "ID " << (settings.hilite ? hilite_none : ""); + id->formatImpl(settings, state, frame); } } - } diff --git a/src/Parsers/ASTPartition.h b/src/Parsers/ASTPartition.h index d17941a9bb6..b2c87bb446a 100644 --- a/src/Parsers/ASTPartition.h +++ b/src/Parsers/ASTPartition.h @@ -10,15 +10,24 @@ namespace DB class ASTPartition : public IAST { public: - ASTPtr value; + IAST * value{nullptr}; std::optional fields_count; - String id; + IAST * id{nullptr}; bool all = false; String getID(char) const override; ASTPtr clone() const override; + void setPartitionID(const ASTPtr & ast); + void setPartitionValue(const ASTPtr & ast); + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&value)); + f(reinterpret_cast(&id)); + } + protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; }; diff --git a/src/Parsers/ParserPartition.cpp b/src/Parsers/ParserPartition.cpp index ba55847593a..dbb75e03979 100644 --- a/src/Parsers/ParserPartition.cpp +++ b/src/Parsers/ParserPartition.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -16,7 +17,10 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_id("ID"); ParserKeyword s_all("ALL"); ParserStringLiteral parser_string_literal; - ParserExpression parser_expr; + ParserSubstitution parser_substitution; + ParserLiteral literal_parser; + ParserFunction function_parser(false, false); + Pos begin = pos; @@ -25,10 +29,15 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (s_id.ignore(pos, expected)) { ASTPtr partition_id; - if (!parser_string_literal.parse(pos, partition_id, expected)) + if (!parser_string_literal.parse(pos, partition_id, expected) && !parser_substitution.parse(pos, partition_id, expected)) return false; - partition->id = partition_id->as().value.get(); + if (auto * partition_id_literal = partition_id->as(); partition_id_literal != nullptr) + partition->setPartitionID(partition_id); + else if (auto * partition_id_query_parameter = partition_id->as(); partition_id_query_parameter != nullptr) + partition->setPartitionID(partition_id); + else + return false; } else if (s_all.ignore(pos, expected)) { @@ -36,25 +45,12 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else { - ASTPtr value; - if (!parser_expr.parse(pos, value, expected)) - return false; - - std::optional fields_count; - - const auto * tuple_ast = value->as(); bool surrounded_by_parens = false; - if (tuple_ast && tuple_ast->name == "tuple") - { - surrounded_by_parens = true; - const auto * arguments_ast = tuple_ast->arguments->as(); - if (arguments_ast) - fields_count = arguments_ast->children.size(); - else - fields_count = 0; - } - else if (const auto * literal = value->as()) + ASTPtr value; + std::optional fields_count; + if (literal_parser.parse(pos, value, expected)) { + auto * literal = value->as(); if (literal->value.getType() == Field::Types::Tuple) { surrounded_by_parens = true; @@ -65,8 +61,26 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) fields_count = 1; } } - else if (!value->as()) + else if (function_parser.parse(pos, value, expected)) + { + const auto * tuple_ast = value->as(); + if (tuple_ast) + { + if (tuple_ast->name == "tuple") + { + surrounded_by_parens = true; + const auto * arguments_ast = tuple_ast->arguments->as(); + if (arguments_ast) + fields_count = arguments_ast->children.size(); + else + fields_count = 0; + } + } + } + else if (!parser_substitution.parse(pos, value, expected)) + { return false; + } if (surrounded_by_parens) { @@ -84,8 +98,7 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - partition->value = value; - partition->children.push_back(value); + partition->setPartitionValue(value); partition->fields_count = fields_count; } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index ae7659e074f..e98b38438b9 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -211,7 +211,10 @@ void IStorage::checkMutationIsPossible(const MutationCommands & /*commands*/, co } void IStorage::checkAlterPartitionIsPossible( - const PartitionCommands & /*commands*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & /*settings*/) const + const PartitionCommands & /*commands*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + const Settings & /*settings*/, + ContextPtr /*context*/) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitioning", getName()); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index fcf7675d15d..92920c50ba9 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -495,7 +495,11 @@ public: ContextPtr /* context */); /// Checks that partition commands can be applied to storage. - virtual void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const; + virtual void checkAlterPartitionIsPossible( + const PartitionCommands & commands, + const StorageMetadataPtr & metadata_snapshot, + const Settings & settings, + ContextPtr context) const; /** Perform any background work. For example, combining parts in a MergeTree type table. * Returns whether any work has been done. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 89c50793925..d7fc30f14b7 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2969,7 +2969,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context /// Just validate partition expression if (command.partition) { - getPartitionIDFromQuery(command.partition, getContext()); + getPartitionIDFromQuery(command.partition, local_context); } if (command.column_name == merging_params.version_column) @@ -4637,7 +4637,7 @@ void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const D } void MergeTreeData::checkAlterPartitionIsPossible( - const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings) const + const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr local_context) const { for (const auto & command : commands) { @@ -4665,7 +4665,7 @@ void MergeTreeData::checkAlterPartitionIsPossible( throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH PARTITION ALL currently"); } else - getPartitionIDFromQuery(command.partition, getContext()); + getPartitionIDFromQuery(command.partition, local_context); } } } @@ -5338,25 +5338,22 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartRestoredFromBackup(cons String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr local_context, DataPartsLock * acquired_lock) const { const auto & partition_ast = ast->as(); - ASTPtr partition_value_ast = partition_ast.value; if (partition_ast.all) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only Support DETACH PARTITION ALL currently"); - if (!partition_value_ast) + if (!partition_ast.value) { - MergeTreePartInfo::validatePartitionID(partition_ast.id, format_version); - return partition_ast.id; + MergeTreePartInfo::validatePartitionID(partition_ast.id->clone(), format_version); + return partition_ast.id->as()->value.safeGet(); } + ASTPtr partition_value_ast = partition_ast.value->clone(); + size_t partition_ast_fields_count; - if (partition_value_ast->as()) + /// It was query parameter and we didn't know exact parameters + if (!partition_ast.fields_count) { - assert(!partition_ast.fields_count); - - ReplaceQueryParameterVisitor param_visitor(local_context->getQueryParameters()); - param_visitor.visit(partition_value_ast); - if (partition_value_ast->as()) { partition_ast_fields_count = 1; @@ -5381,20 +5378,15 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc } else { - assert(partition_ast.fields_count); partition_ast_fields_count = partition_ast.fields_count.value(); } if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { /// Month-partitioning specific - partition ID can be passed in the partition value. + MergeTreePartInfo::validatePartitionID(partition_value_ast, format_version); const auto * partition_lit = partition_value_ast->as(); - if (partition_lit && partition_lit->value.getType() == Field::Types::String) - { - String partition_id = partition_lit->value.get(); - MergeTreePartInfo::validatePartitionID(partition_id, format_version); - return partition_id; - } + return partition_lit->value.get(); } /// Re-parse partition key fields using the information about expected field types. diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 93b8e69438e..a8dac7cbade 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -723,7 +723,11 @@ public: void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; /// Checks that partition name in all commands is valid - void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override; + void checkAlterPartitionIsPossible( + const PartitionCommands & commands, + const StorageMetadataPtr & metadata_snapshot, + const Settings & settings, + ContextPtr local_context) const override; /// Change MergeTreeSettings void changeSettings( diff --git a/src/Storages/MergeTree/MergeTreePartInfo.cpp b/src/Storages/MergeTree/MergeTreePartInfo.cpp index e1b52d8a7b7..22e93e6cb01 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -2,7 +2,8 @@ #include #include #include -#include "Core/ProtocolDefines.h" +#include +#include namespace DB { @@ -23,8 +24,14 @@ MergeTreePartInfo MergeTreePartInfo::fromPartName(const String & part_name, Merg throw Exception(ErrorCodes::BAD_DATA_PART_NAME, "Unexpected part name: {} for format version: {}", part_name, format_version); } -void MergeTreePartInfo::validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version) +void MergeTreePartInfo::validatePartitionID(const ASTPtr & partition_id_ast, MergeTreeDataFormatVersion format_version) { + std::string partition_id; + if (auto * literal = partition_id_ast->as(); literal != nullptr && literal->value.getType() == Field::Types::String) + partition_id = literal->value.safeGet(); + else + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Partition id must be string literal"); + if (partition_id.empty()) throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Partition id is empty"); diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index 9b14d4a2ace..d359e23901f 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -9,6 +9,7 @@ #include #include #include +#include #include @@ -119,7 +120,7 @@ struct MergeTreePartInfo void deserialize(ReadBuffer & in); /// Simple sanity check for partition ID. Checking that it's not too long or too short, doesn't contain a lot of '_'. - static void validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version); + static void validatePartitionID(const ASTPtr & partition_id_ast, MergeTreeDataFormatVersion format_version); static MergeTreePartInfo fromPartName(const String & part_name, MergeTreeDataFormatVersion format_version); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 7354dd56552..78dc89857b7 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -330,10 +330,11 @@ Pipe StorageMaterializedView::alterPartition( } void StorageMaterializedView::checkAlterPartitionIsPossible( - const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const + const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, + const Settings & settings, ContextPtr local_context) const { checkStatementCanBeForwarded(); - getTargetTable()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings); + getTargetTable()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings, local_context); } void StorageMaterializedView::mutate(const MutationCommands & commands, ContextPtr local_context) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 3ec789aa7e3..95bedcd9ade 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -64,7 +64,7 @@ public: Pipe alterPartition(const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, ContextPtr context) override; - void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override; + void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings, ContextPtr local_context) const override; void mutate(const MutationCommands & commands, ContextPtr context) override; diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index a4304faeaec..d4357ffd3ac 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -115,9 +115,9 @@ public: return getNested()->alterPartition(metadata_snapshot, commands, context); } - void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override + void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings, ContextPtr context) const override { - getNested()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings); + getNested()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings, context); } bool optimize( diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.sql b/tests/queries/0_stateless/02897_alter_partition_parameters.sql new file mode 100644 index 00000000000..c828db4092f --- /dev/null +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.sql @@ -0,0 +1,94 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test +( + EventDate Date +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY toMonday(EventDate); + +INSERT INTO test VALUES(toDate('2023-10-09')); + +SET param_partition='2023-10-09'; + +ALTER TABLE test DROP PARTITION {partition:String}; + +SELECT count() FROM test; + +INSERT INTO test VALUES(toDate('2023-10-09')); + +ALTER TABLE test DROP PARTITION tuple(toMonday({partition:Date})); + +SELECT count() FROM test; + +INSERT INTO test VALUES(toDate('2023-10-09')); + +-- for some reason only tuples are allowed as non-string arguments +ALTER TABLE test DROP PARTITION toMonday({partition:String}); --{serverError 248} + +set param_partition_id = '20231009'; + +ALTER TABLE test DROP PARTITION ID {partition_id:String}; + +SELECT count() FROM test; + +DROP TABLE IF EXISTS test; + +DROP TABLE IF EXISTS test2; + +CREATE TABLE test2 +( + a UInt32, + b Int64 +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY (a * b, b * b); + + +INSERT INTO test2 VALUES(1, 2); + +SET param_first='2'; +SET param_second='4'; + +ALTER TABLE test2 DROP PARTITION tuple({first:UInt32},{second:Int64}); + +SELECT count() FROM test2; + +DROP TABLE IF EXISTS test2; +DROP TABLE IF EXISTS test3; + +CREATE TABLE test3 +( + a UInt32, + b Int64 +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY a; + +INSERT INTO test3 VALUES(1, 2); + +SET param_simple='1'; + +ALTER TABLE test3 DROP PARTITION {simple:String}; + +SELECT count() FROM test3; + +DROP TABLE IF EXISTS test3; + + +DROP TABLE IF EXISTS test4 ON CLUSTER 'test_shard_localhost'; + +CREATE TABLE test4 ON CLUSTER 'test_shard_localhost' (EventDate Date) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY EventDate; + +INSERT INTO test4 VALUES(toDate('2023-10-09')); + +SET param_partition='2023-10-09'; + +ALTER TABLE test4 ON CLUSTER 'test_shard_localhost' DROP PARTITION {partition:String}; + +SELECT count() FROM test4; + +DROP TABLE IF EXISTS test4 ON CLUSTER 'test_shard_localhost'; From a98ffd48014b35a1c864e9d850920024c1d4b4f8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Oct 2023 16:24:26 +0200 Subject: [PATCH 233/634] Forgotten reference --- .../0_stateless/02897_alter_partition_parameters.reference | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/02897_alter_partition_parameters.reference diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.reference b/tests/queries/0_stateless/02897_alter_partition_parameters.reference new file mode 100644 index 00000000000..f7eb44d66e0 --- /dev/null +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.reference @@ -0,0 +1,6 @@ +0 +0 +0 +0 +0 +0 From 3d3cf29bf4c83311bb4d1913096c9d13d683c1f1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 13 Oct 2023 14:35:02 +0000 Subject: [PATCH 234/634] Update croaring to v2.0.4 --- contrib/croaring | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/croaring b/contrib/croaring index e4a7ad55427..9b7cc0ff1c4 160000 --- a/contrib/croaring +++ b/contrib/croaring @@ -1 +1 @@ -Subproject commit e4a7ad5542746103e71ca8b5e56225baf0014c87 +Subproject commit 9b7cc0ff1c41e9457efb6228cfd2c538d0155303 From 80778727d3ca7840c631bbd25bc778a8a2b88ba0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 13 Oct 2023 16:35:42 +0200 Subject: [PATCH 235/634] add back a test that was removed by mistake --- .../test_store_cleanup/__init__.py | 0 .../configs/store_cleanup.xml | 11 + tests/integration/test_store_cleanup/test.py | 212 ++++++++++++++++++ 3 files changed, 223 insertions(+) create mode 100644 tests/integration/test_store_cleanup/__init__.py create mode 100644 tests/integration/test_store_cleanup/configs/store_cleanup.xml create mode 100644 tests/integration/test_store_cleanup/test.py diff --git a/tests/integration/test_store_cleanup/__init__.py b/tests/integration/test_store_cleanup/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_store_cleanup/configs/store_cleanup.xml b/tests/integration/test_store_cleanup/configs/store_cleanup.xml new file mode 100644 index 00000000000..5fbe87cce00 --- /dev/null +++ b/tests/integration/test_store_cleanup/configs/store_cleanup.xml @@ -0,0 +1,11 @@ + + 0 + 60 + 1 + + + + testkeeper + + \ No newline at end of file diff --git a/tests/integration/test_store_cleanup/test.py b/tests/integration/test_store_cleanup/test.py new file mode 100644 index 00000000000..6c5a20a758a --- /dev/null +++ b/tests/integration/test_store_cleanup/test.py @@ -0,0 +1,212 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", stay_alive=True, main_configs=["configs/store_cleanup.xml"] +) + +path_to_data = "/var/lib/clickhouse/" + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_store_cleanup(started_cluster): + node1.query("CREATE DATABASE db UUID '10000000-1000-4000-8000-000000000001'") + node1.query( + "CREATE TABLE db.log UUID '10000000-1000-4000-8000-000000000002' ENGINE=Log AS SELECT 1" + ) + node1.query( + "CREATE TABLE db.mt UUID '10000000-1000-4000-8000-000000000003' ENGINE=MergeTree ORDER BY tuple() AS SELECT 1" + ) + node1.query( + "CREATE TABLE db.mem UUID '10000000-1000-4000-8000-000000000004' ENGINE=Memory AS SELECT 1" + ) + + node1.query("CREATE DATABASE db2 UUID '20000000-1000-4000-8000-000000000001'") + node1.query( + "CREATE TABLE db2.log UUID '20000000-1000-4000-8000-000000000002' ENGINE=Log AS SELECT 1" + ) + node1.query("DETACH DATABASE db2") + + node1.query("CREATE DATABASE db3 UUID '30000000-1000-4000-8000-000000000001'") + node1.query( + "CREATE TABLE db3.log UUID '30000000-1000-4000-8000-000000000002' ENGINE=Log AS SELECT 1" + ) + node1.query( + "CREATE TABLE db3.log2 UUID '30000000-1000-4000-8000-000000000003' ENGINE=Log AS SELECT 1" + ) + node1.query("DETACH TABLE db3.log") + node1.query("DETACH TABLE db3.log2 PERMANENTLY") + + assert "d---------" not in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store"] + ) + assert "d---------" not in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store/100"] + ) + assert "d---------" not in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store/200"] + ) + assert "d---------" not in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store/300"] + ) + + node1.stop_clickhouse(kill=True) + # All dirs related to `db` will be removed + node1.exec_in_container(["rm", f"{path_to_data}/metadata/db.sql"]) + + node1.exec_in_container(["mkdir", f"{path_to_data}/store/kek"]) + node1.exec_in_container(["touch", f"{path_to_data}/store/12"]) + try: + node1.exec_in_container(["mkdir", f"{path_to_data}/store/456"]) + except Exception as e: + print("Failed to create 456/:", str(e)) + node1.exec_in_container(["mkdir", f"{path_to_data}/store/456/testgarbage"]) + node1.exec_in_container( + ["mkdir", f"{path_to_data}/store/456/30000000-1000-4000-8000-000000000003"] + ) + node1.exec_in_container( + ["touch", f"{path_to_data}/store/456/45600000-1000-4000-8000-000000000003"] + ) + node1.exec_in_container( + ["mkdir", f"{path_to_data}/store/456/45600000-1000-4000-8000-000000000004"] + ) + + node1.start_clickhouse() + node1.query("DETACH DATABASE db2") + node1.query("DETACH TABLE db3.log") + + node1.wait_for_log_line( + "Removing access rights for unused directory", + timeout=60, + look_behind_lines=1000000, + ) + node1.wait_for_log_line( + "directories from store", timeout=60, look_behind_lines=1000000 + ) + + store = node1.exec_in_container(["ls", f"{path_to_data}/store"]) + assert "100" in store + assert "200" in store + assert "300" in store + assert "456" in store + assert "kek" in store + assert "12" in store + assert "d---------" in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store"] + ) + assert "d---------" in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store/456"] + ) + + # Metadata is removed, so store/100 contains garbage + store100 = node1.exec_in_container(["ls", f"{path_to_data}/store/100"]) + assert "10000000-1000-4000-8000-000000000001" in store100 + assert "10000000-1000-4000-8000-000000000002" in store100 + assert "10000000-1000-4000-8000-000000000003" in store100 + assert "d---------" in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store/100"] + ) + + # Database is detached, nothing to clean up + store200 = node1.exec_in_container(["ls", f"{path_to_data}/store/200"]) + assert "20000000-1000-4000-8000-000000000001" in store200 + assert "20000000-1000-4000-8000-000000000002" in store200 + assert "d---------" not in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store/200"] + ) + + # Tables are detached, nothing to clean up + store300 = node1.exec_in_container(["ls", f"{path_to_data}/store/300"]) + assert "30000000-1000-4000-8000-000000000001" in store300 + assert "30000000-1000-4000-8000-000000000002" in store300 + assert "30000000-1000-4000-8000-000000000003" in store300 + assert "d---------" not in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store/300"] + ) + + # Manually created garbage + store456 = node1.exec_in_container(["ls", f"{path_to_data}/store/456"]) + assert "30000000-1000-4000-8000-000000000003" in store456 + assert "45600000-1000-4000-8000-000000000003" in store456 + assert "45600000-1000-4000-8000-000000000004" in store456 + assert "testgarbage" in store456 + assert "----------" in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store/456"] + ) + + node1.wait_for_log_line( + "Removing unused directory", timeout=90, look_behind_lines=1000000 + ) + node1.wait_for_log_line( + "directories from store", timeout=90, look_behind_lines=1000000 + ) + node1.wait_for_log_line( + "Nothing to clean up from store/", timeout=90, look_behind_lines=1000000 + ) + + store = node1.exec_in_container(["ls", f"{path_to_data}/store"]) + assert "100" in store + assert "200" in store + assert "300" in store + assert "456" in store + assert "kek" not in store # changed + assert "\n12\n" not in store # changed + assert "d---------" not in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store"] + ) # changed + + # Metadata is removed, so store/100 contains garbage + store100 = node1.exec_in_container(["ls", f"{path_to_data}/store/100"]) # changed + assert "10000000-1000-4000-8000-000000000001" not in store100 # changed + assert "10000000-1000-4000-8000-000000000002" not in store100 # changed + assert "10000000-1000-4000-8000-000000000003" not in store100 # changed + assert "d---------" not in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store/100"] + ) # changed + + # Database is detached, nothing to clean up + store200 = node1.exec_in_container(["ls", f"{path_to_data}/store/200"]) + assert "20000000-1000-4000-8000-000000000001" in store200 + assert "20000000-1000-4000-8000-000000000002" in store200 + assert "d---------" not in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store/200"] + ) + + # Tables are detached, nothing to clean up + store300 = node1.exec_in_container(["ls", f"{path_to_data}/store/300"]) + assert "30000000-1000-4000-8000-000000000001" in store300 + assert "30000000-1000-4000-8000-000000000002" in store300 + assert "30000000-1000-4000-8000-000000000003" in store300 + assert "d---------" not in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store/300"] + ) + + # Manually created garbage + store456 = node1.exec_in_container(["ls", f"{path_to_data}/store/456"]) + assert "30000000-1000-4000-8000-000000000003" not in store456 # changed + assert "45600000-1000-4000-8000-000000000003" not in store456 # changed + assert "45600000-1000-4000-8000-000000000004" not in store456 # changed + assert "testgarbage" not in store456 # changed + assert "---------" not in node1.exec_in_container( + ["ls", "-l", f"{path_to_data}/store/456"] + ) # changed + + node1.query("ATTACH TABLE db3.log2") + node1.query("ATTACH DATABASE db2") + node1.query("ATTACH TABLE db3.log") + + assert "1\n" == node1.query("SELECT * FROM db3.log") + assert "1\n" == node1.query("SELECT * FROM db3.log2") + assert "1\n" == node1.query("SELECT * FROM db2.log") From 78575720d1dcce85b7b62436fac2d841820813cd Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Oct 2023 16:52:04 +0200 Subject: [PATCH 236/634] Simpler and better --- src/Parsers/ASTPartition.h | 2 +- src/Parsers/ParserPartition.cpp | 27 +++++++++------- src/Storages/MergeTree/MergeTreeData.cpp | 32 +------------------ .../02897_alter_partition_parameters.sql | 2 +- 4 files changed, 18 insertions(+), 45 deletions(-) diff --git a/src/Parsers/ASTPartition.h b/src/Parsers/ASTPartition.h index b2c87bb446a..343230fa23e 100644 --- a/src/Parsers/ASTPartition.h +++ b/src/Parsers/ASTPartition.h @@ -11,7 +11,7 @@ class ASTPartition : public IAST { public: IAST * value{nullptr}; - std::optional fields_count; + size_t fields_count; IAST * id{nullptr}; bool all = false; diff --git a/src/Parsers/ParserPartition.cpp b/src/Parsers/ParserPartition.cpp index dbb75e03979..312f65675ef 100644 --- a/src/Parsers/ParserPartition.cpp +++ b/src/Parsers/ParserPartition.cpp @@ -47,7 +47,7 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { bool surrounded_by_parens = false; ASTPtr value; - std::optional fields_count; + size_t fields_count; if (literal_parser.parse(pos, value, expected)) { auto * literal = value->as(); @@ -64,20 +64,23 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) else if (function_parser.parse(pos, value, expected)) { const auto * tuple_ast = value->as(); - if (tuple_ast) + if (tuple_ast && tuple_ast->name == "tuple") { - if (tuple_ast->name == "tuple") - { - surrounded_by_parens = true; - const auto * arguments_ast = tuple_ast->arguments->as(); - if (arguments_ast) - fields_count = arguments_ast->children.size(); - else - fields_count = 0; - } + surrounded_by_parens = true; + const auto * arguments_ast = tuple_ast->arguments->as(); + if (arguments_ast) + fields_count = arguments_ast->children.size(); + else + fields_count = 0; } + else + return false; } - else if (!parser_substitution.parse(pos, value, expected)) + else if (parser_substitution.parse(pos, value, expected)) + { + fields_count = 1; + } + else { return false; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d7fc30f14b7..6676799b70b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5349,37 +5349,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc } ASTPtr partition_value_ast = partition_ast.value->clone(); - - size_t partition_ast_fields_count; - /// It was query parameter and we didn't know exact parameters - if (!partition_ast.fields_count) - { - if (partition_value_ast->as()) - { - partition_ast_fields_count = 1; - } - else if (const auto * tuple_ast = partition_value_ast->as()) - { - if (tuple_ast->name != "tuple") - throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, - "Expected tuple for complex partition key, got {}", tuple_ast->name); - - const auto * arguments_ast = tuple_ast->arguments->as(); - if (arguments_ast) - partition_ast_fields_count = arguments_ast->children.size(); - else - partition_ast_fields_count = 0; - } - else - { - throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, - "Expected literal or tuple for partition key, got {}", partition_value_ast->getID()); - } - } - else - { - partition_ast_fields_count = partition_ast.fields_count.value(); - } + size_t partition_ast_fields_count = partition_ast.fields_count; if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.sql b/tests/queries/0_stateless/02897_alter_partition_parameters.sql index c828db4092f..4509081fc10 100644 --- a/tests/queries/0_stateless/02897_alter_partition_parameters.sql +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.sql @@ -25,7 +25,7 @@ SELECT count() FROM test; INSERT INTO test VALUES(toDate('2023-10-09')); -- for some reason only tuples are allowed as non-string arguments -ALTER TABLE test DROP PARTITION toMonday({partition:String}); --{serverError 248} +ALTER TABLE test DROP PARTITION toMonday({partition:String}); --{clientError 62} set param_partition_id = '20231009'; From 246e29a41a52fb63f44a3cb4e05c015d4ff1e750 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 13 Oct 2023 14:51:13 +0000 Subject: [PATCH 237/634] Add 16/32-byte integer support --- .../sql-reference/functions/arithmetic-functions.md | 2 +- src/Functions/byteSwap.cpp | 11 ++++++++++- tests/queries/0_stateless/02887_byteswap.reference | 4 ++++ tests/queries/0_stateless/02887_byteswap.sql | 8 ++++++-- 4 files changed, 21 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 4efd05c1dbd..6d95f3dc358 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -444,7 +444,7 @@ DB::Exception: Decimal result's scale is less than argument's one: While process ## byteSwap -Reverses the bytes of an integer, i.e. changes its [endianness](https://en.wikipedia.org/wiki/Endianness). Currently, integers of up to 64 bit are supported. +Reverses the bytes of an integer, i.e. changes its [endianness](https://en.wikipedia.org/wiki/Endianness). **Syntax** diff --git a/src/Functions/byteSwap.cpp b/src/Functions/byteSwap.cpp index bdc6eb6c386..2a343a07720 100644 --- a/src/Functions/byteSwap.cpp +++ b/src/Functions/byteSwap.cpp @@ -17,6 +17,15 @@ T byteSwap(T x) return std::byteswap(x); } +template +requires std::is_same_v || std::is_same_v || std::is_same_v || std::is_same_v +T byteSwap(T x) +{ + T dest; + reverseMemcpy(&dest, &x, sizeof(T)); + return dest; +} + template T byteSwap(T) { @@ -55,7 +64,7 @@ REGISTER_FUNCTION(ByteSwap) factory.registerFunction( FunctionDocumentation{ .description = R"( -Reverses the bytes of an integer, i.e. changes its [endianness](https://en.wikipedia.org/wiki/Endianness). Currently, integers of up to 64 bit are supported. +Reverses the bytes of an integer, i.e. changes its [endianness](https://en.wikipedia.org/wiki/Endianness). **Example** diff --git a/tests/queries/0_stateless/02887_byteswap.reference b/tests/queries/0_stateless/02887_byteswap.reference index b55b6f9cd6b..17b1ecf25f6 100644 --- a/tests/queries/0_stateless/02887_byteswap.reference +++ b/tests/queries/0_stateless/02887_byteswap.reference @@ -25,5 +25,9 @@ -549755813889 4039370097989451775 128 +72057594037927936 +-2361183241434822606849 +1329227995784915872903807060280344576 +-43556142965880123323311949751266331066369 0 1 diff --git a/tests/queries/0_stateless/02887_byteswap.sql b/tests/queries/0_stateless/02887_byteswap.sql index 297132c7e7d..e428be82832 100644 --- a/tests/queries/0_stateless/02887_byteswap.sql +++ b/tests/queries/0_stateless/02887_byteswap.sql @@ -33,6 +33,12 @@ SELECT byteSwap(-2147483649::Int64); SELECT byteSwap(-1242525266376::Int64); SELECT byteSwap(-9223372036854775808::Int64); +SELECT byteSwap(18446744073709551616::UInt128); +SELECT byteSwap(-9223372036854775809::Int128); + +SELECT byteSwap(340282366920938463463374607431768211456::UInt256); +SELECT byteSwap(-170141183460469231731687303715884105729::Int256); + -- Booleans are interpreted as UInt8 SELECT byteSwap(false); SELECT byteSwap(true); @@ -52,6 +58,4 @@ SELECT byteSwap(generateUUIDv4()); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT byteSwap(toDecimal32(2, 4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT byteSwap(toFloat32(123.456)); -- { serverError NOT_IMPLEMENTED } SELECT byteSwap(toFloat64(123.456)); -- { serverError NOT_IMPLEMENTED } -SELECT byteSwap(18446744073709551616::UInt128); -- { serverError NOT_IMPLEMENTED } -SELECT byteSwap(-9223372036854775809::Int128); -- { serverError NOT_IMPLEMENTED } From b0f1a4f8c0042b8e32688e777f14c77223faadf3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 13 Oct 2023 17:29:44 +0200 Subject: [PATCH 238/634] Disable wrong optimization --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c08425c03fd..bebe032ead5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -627,7 +627,7 @@ class IColumn; \ M(Bool, optimize_rewrite_sum_if_to_count_if, false, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \ M(Bool, optimize_rewrite_aggregate_function_with_if, true, "Rewrite aggregate functions with if expression as argument when logically equivalent. For example, avg(if(cond, col, null)) can be rewritten to avgIf(cond, col)", 0) \ - M(Bool, optimize_rewrite_array_exists_to_has, true, "Rewrite arrayExists() functions to has() when logically equivalent. For example, arrayExists(x -> x = 1, arr) can be rewritten to has(arr, 1)", 0) \ + M(Bool, optimize_rewrite_array_exists_to_has, false, "Rewrite arrayExists() functions to has() when logically equivalent. For example, arrayExists(x -> x = 1, arr) can be rewritten to has(arr, 1)", 0) \ M(UInt64, insert_shard_id, 0, "If non zero, when insert into a distributed table, the data will be inserted into the shard `insert_shard_id` synchronously. Possible values range from 1 to `shards_number` of corresponding distributed table", 0) \ \ M(Bool, collect_hash_table_stats_during_aggregation, true, "Enable collecting hash table statistics to optimize memory allocation", 0) \ From 9ddad9e88f766b62ab1aae71cc3ddb3da9965d48 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 13 Oct 2023 17:31:25 +0200 Subject: [PATCH 239/634] Add a test --- .../02896_optimize_array_exists_to_has_with_date.reference | 1 + .../0_stateless/02896_optimize_array_exists_to_has_with_date.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02896_optimize_array_exists_to_has_with_date.reference create mode 100644 tests/queries/0_stateless/02896_optimize_array_exists_to_has_with_date.sql diff --git a/tests/queries/0_stateless/02896_optimize_array_exists_to_has_with_date.reference b/tests/queries/0_stateless/02896_optimize_array_exists_to_has_with_date.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02896_optimize_array_exists_to_has_with_date.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02896_optimize_array_exists_to_has_with_date.sql b/tests/queries/0_stateless/02896_optimize_array_exists_to_has_with_date.sql new file mode 100644 index 00000000000..b87a154b46a --- /dev/null +++ b/tests/queries/0_stateless/02896_optimize_array_exists_to_has_with_date.sql @@ -0,0 +1 @@ +SELECT arrayExists(date -> (date = '2022-07-31'), [toDate('2022-07-31')]) AS date_exists; From f80dd06f5290f170790611597ed02d48b18820ee Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Oct 2023 17:36:38 +0200 Subject: [PATCH 240/634] Fix tuple parser --- src/Parsers/ExpressionListParsers.h | 2 +- src/Parsers/ParserPartition.cpp | 27 +++---------------- src/Storages/MergeTree/MergeTreeData.cpp | 9 ++++--- src/Storages/MergeTree/MergeTreePartInfo.cpp | 1 + .../02897_alter_partition_parameters.sql | 6 ++--- 5 files changed, 15 insertions(+), 30 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 040a81bbe27..6dba5a9c31f 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -173,7 +173,7 @@ protected: class ParserExpression : public IParserBase { public: - ParserExpression(bool allow_trailing_commas_ = false) : allow_trailing_commas(allow_trailing_commas_) {} + explicit ParserExpression(bool allow_trailing_commas_ = false) : allow_trailing_commas(allow_trailing_commas_) {} protected: const char * getName() const override { return "lambda expression"; } diff --git a/src/Parsers/ParserPartition.cpp b/src/Parsers/ParserPartition.cpp index 312f65675ef..95e2680aa09 100644 --- a/src/Parsers/ParserPartition.cpp +++ b/src/Parsers/ParserPartition.cpp @@ -19,11 +19,9 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserStringLiteral parser_string_literal; ParserSubstitution parser_substitution; ParserLiteral literal_parser; + ParserTupleOfLiterals tuple_of_literals; ParserFunction function_parser(false, false); - - Pos begin = pos; - auto partition = std::make_shared(); if (s_id.ignore(pos, expected)) @@ -45,15 +43,14 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else { - bool surrounded_by_parens = false; ASTPtr value; size_t fields_count; - if (literal_parser.parse(pos, value, expected)) + if (literal_parser.parse(pos, value, expected) || tuple_of_literals.parse(pos, value, expected)) { auto * literal = value->as(); if (literal->value.getType() == Field::Types::Tuple) { - surrounded_by_parens = true; + //surrounded_by_parens = true; fields_count = literal->value.get().size(); } else @@ -66,7 +63,7 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) const auto * tuple_ast = value->as(); if (tuple_ast && tuple_ast->name == "tuple") { - surrounded_by_parens = true; + //surrounded_by_parens = true; const auto * arguments_ast = tuple_ast->arguments->as(); if (arguments_ast) fields_count = arguments_ast->children.size(); @@ -85,22 +82,6 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - if (surrounded_by_parens) - { - Pos left_paren = begin; - Pos right_paren = pos; - - while (left_paren != right_paren && left_paren->type != TokenType::OpeningRoundBracket) - ++left_paren; - if (left_paren->type != TokenType::OpeningRoundBracket) - return false; - - while (right_paren != left_paren && right_paren->type != TokenType::ClosingRoundBracket) - --right_paren; - if (right_paren->type != TokenType::ClosingRoundBracket) - return false; - } - partition->setPartitionValue(value); partition->fields_count = fields_count; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6676799b70b..3909e90e5cc 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5354,9 +5354,12 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { /// Month-partitioning specific - partition ID can be passed in the partition value. - MergeTreePartInfo::validatePartitionID(partition_value_ast, format_version); - const auto * partition_lit = partition_value_ast->as(); - return partition_lit->value.get(); + const auto * partition_lit = partition_ast.value->as(); + if (partition_lit && partition_lit->value.getType() == Field::Types::String) + { + MergeTreePartInfo::validatePartitionID(partition_ast.value->clone(), format_version); + return partition_lit->value.get(); + } } /// Re-parse partition key fields using the information about expected field types. diff --git a/src/Storages/MergeTree/MergeTreePartInfo.cpp b/src/Storages/MergeTree/MergeTreePartInfo.cpp index 22e93e6cb01..16c119c642a 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -29,6 +29,7 @@ void MergeTreePartInfo::validatePartitionID(const ASTPtr & partition_id_ast, Mer std::string partition_id; if (auto * literal = partition_id_ast->as(); literal != nullptr && literal->value.getType() == Field::Types::String) partition_id = literal->value.safeGet(); + else throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Partition id must be string literal"); diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.sql b/tests/queries/0_stateless/02897_alter_partition_parameters.sql index 4509081fc10..c094281938e 100644 --- a/tests/queries/0_stateless/02897_alter_partition_parameters.sql +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.sql @@ -81,14 +81,14 @@ DROP TABLE IF EXISTS test3; DROP TABLE IF EXISTS test4 ON CLUSTER 'test_shard_localhost'; -CREATE TABLE test4 ON CLUSTER 'test_shard_localhost' (EventDate Date) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY EventDate; +CREATE TABLE test4 (EventDate Date) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY EventDate; INSERT INTO test4 VALUES(toDate('2023-10-09')); SET param_partition='2023-10-09'; -ALTER TABLE test4 ON CLUSTER 'test_shard_localhost' DROP PARTITION {partition:String}; +ALTER TABLE test4 ON CLUSTER 'test_shard_localhost' DROP PARTITION {partition:String} FORMAT Null; SELECT count() FROM test4; -DROP TABLE IF EXISTS test4 ON CLUSTER 'test_shard_localhost'; +DROP TABLE IF EXISTS test4; From bbcd8a42ae0e13df3d47b12b4f022f4f88335e6d Mon Sep 17 00:00:00 2001 From: bhavuk2002 Date: Fri, 13 Oct 2023 21:46:27 +0530 Subject: [PATCH 241/634] rectified if sysntax --- programs/server/dashboard.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 5426be55bd3..7bfa9db752c 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -815,7 +815,7 @@ function insertChart(i) { // dragging and maximizing both be added to chart only, when there are more than 1 chart - if(charts.getElementsByClassName("chart").length != 1) { + if (charts.getElementsByClassName("chart").length != 1) { // dragging From 1571dbcb26e54e25d01830fc8abaa42e0daea107 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Oct 2023 18:16:51 +0200 Subject: [PATCH 242/634] Finally fix it --- src/Parsers/ParserPartition.cpp | 14 +++++------ ...02897_alter_partition_parameters.reference | 1 + .../02897_alter_partition_parameters.sql | 23 +++++++++++++++++++ 3 files changed, 30 insertions(+), 8 deletions(-) diff --git a/src/Parsers/ParserPartition.cpp b/src/Parsers/ParserPartition.cpp index 95e2680aa09..915d8d90f58 100644 --- a/src/Parsers/ParserPartition.cpp +++ b/src/Parsers/ParserPartition.cpp @@ -20,7 +20,7 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserSubstitution parser_substitution; ParserLiteral literal_parser; ParserTupleOfLiterals tuple_of_literals; - ParserFunction function_parser(false, false); + ParserExpression parser_expr; auto partition = std::make_shared(); @@ -50,7 +50,6 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) auto * literal = value->as(); if (literal->value.getType() == Field::Types::Tuple) { - //surrounded_by_parens = true; fields_count = literal->value.get().size(); } else @@ -58,12 +57,15 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) fields_count = 1; } } - else if (function_parser.parse(pos, value, expected)) + else if (parser_substitution.parse(pos, value, expected)) + { + fields_count = 1; + } + else if (parser_expr.parse(pos, value, expected)) { const auto * tuple_ast = value->as(); if (tuple_ast && tuple_ast->name == "tuple") { - //surrounded_by_parens = true; const auto * arguments_ast = tuple_ast->arguments->as(); if (arguments_ast) fields_count = arguments_ast->children.size(); @@ -73,10 +75,6 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) else return false; } - else if (parser_substitution.parse(pos, value, expected)) - { - fields_count = 1; - } else { return false; diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.reference b/tests/queries/0_stateless/02897_alter_partition_parameters.reference index f7eb44d66e0..a21090781f3 100644 --- a/tests/queries/0_stateless/02897_alter_partition_parameters.reference +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.reference @@ -4,3 +4,4 @@ 0 0 0 +0 diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.sql b/tests/queries/0_stateless/02897_alter_partition_parameters.sql index c094281938e..f85e6bc5f54 100644 --- a/tests/queries/0_stateless/02897_alter_partition_parameters.sql +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.sql @@ -92,3 +92,26 @@ ALTER TABLE test4 ON CLUSTER 'test_shard_localhost' DROP PARTITION {partition:St SELECT count() FROM test4; DROP TABLE IF EXISTS test4; + + +DROP TABLE IF EXISTS test5; + +CREATE TABLE test5 +( + a UInt32, + b Int64 +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY (a, b); + +INSERT INTO test5 VALUES(1, 2); + +SET param_f='1'; +SET param_s='2'; + +ALTER TABLE test5 DROP PARTITION ({f:UInt32}, 2); + +SELECT count() FROM test5; + +DROP TABLE IF EXISTS test5; From d440ec5212da09b93e275d6fa4954a742c60101b Mon Sep 17 00:00:00 2001 From: bhavuk2002 Date: Fri, 13 Oct 2023 21:47:29 +0530 Subject: [PATCH 243/634] updated if condition --- programs/server/dashboard.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 7bfa9db752c..10bee73a8dc 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -815,7 +815,7 @@ function insertChart(i) { // dragging and maximizing both be added to chart only, when there are more than 1 chart - if (charts.getElementsByClassName("chart").length != 1) { + if (charts.getElementsByClassName("chart").length > 1) { // dragging From bafc5e7270dc245fb993cd2e5054cbd01da136de Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Oct 2023 18:18:41 +0200 Subject: [PATCH 244/634] Fix --- .../queries/0_stateless/02897_alter_partition_parameters.sql | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.sql b/tests/queries/0_stateless/02897_alter_partition_parameters.sql index f85e6bc5f54..9bd3df0f971 100644 --- a/tests/queries/0_stateless/02897_alter_partition_parameters.sql +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.sql @@ -78,8 +78,7 @@ SELECT count() FROM test3; DROP TABLE IF EXISTS test3; - -DROP TABLE IF EXISTS test4 ON CLUSTER 'test_shard_localhost'; +DROP TABLE IF EXISTS test4; CREATE TABLE test4 (EventDate Date) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY EventDate; @@ -93,7 +92,6 @@ SELECT count() FROM test4; DROP TABLE IF EXISTS test4; - DROP TABLE IF EXISTS test5; CREATE TABLE test5 From fd5c864d194a415c0f027166515036ef5321d21b Mon Sep 17 00:00:00 2001 From: bhavuk2002 Date: Fri, 13 Oct 2023 21:49:59 +0530 Subject: [PATCH 245/634] using queries.length --- programs/server/dashboard.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 10bee73a8dc..132606b728d 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -815,7 +815,7 @@ function insertChart(i) { // dragging and maximizing both be added to chart only, when there are more than 1 chart - if (charts.getElementsByClassName("chart").length > 1) { + if (queries.length > 1) { // dragging From 98c0e54adfca681957e2cd5741eca629ec4fb079 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 13 Oct 2023 18:28:31 +0200 Subject: [PATCH 246/634] Do not initialize EventNotifier explicitly (it is done in ContextHolder) Signed-off-by: Azat Khuzhin --- src/Common/tests/gtest_event_notifier.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/tests/gtest_event_notifier.cpp b/src/Common/tests/gtest_event_notifier.cpp index 39994221812..9c454fa3e17 100644 --- a/src/Common/tests/gtest_event_notifier.cpp +++ b/src/Common/tests/gtest_event_notifier.cpp @@ -9,7 +9,6 @@ TEST(EventNotifier, SimpleTest) using namespace DB; size_t result = 1; - EventNotifier::init(); auto handler3 = EventNotifier::instance().subscribe(Coordination::Error::ZSESSIONEXPIRED, [&result](){ result *= 3; }); From daa8dbbbae760f5072e4f558c1d54396f65fd791 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Oct 2023 18:51:32 +0200 Subject: [PATCH 247/634] Make as flexible as it can be --- src/Parsers/ASTPartition.h | 2 +- src/Parsers/ParserPartition.cpp | 5 +- src/Storages/MergeTree/MergeTreeData.cpp | 63 ++++++++++++++++++- ...02897_alter_partition_parameters.reference | 1 + .../02897_alter_partition_parameters.sql | 21 +++++++ 5 files changed, 87 insertions(+), 5 deletions(-) diff --git a/src/Parsers/ASTPartition.h b/src/Parsers/ASTPartition.h index 343230fa23e..b2c87bb446a 100644 --- a/src/Parsers/ASTPartition.h +++ b/src/Parsers/ASTPartition.h @@ -11,7 +11,7 @@ class ASTPartition : public IAST { public: IAST * value{nullptr}; - size_t fields_count; + std::optional fields_count; IAST * id{nullptr}; bool all = false; diff --git a/src/Parsers/ParserPartition.cpp b/src/Parsers/ParserPartition.cpp index 915d8d90f58..80debc13c67 100644 --- a/src/Parsers/ParserPartition.cpp +++ b/src/Parsers/ParserPartition.cpp @@ -44,7 +44,7 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) else { ASTPtr value; - size_t fields_count; + std::optional fields_count; if (literal_parser.parse(pos, value, expected) || tuple_of_literals.parse(pos, value, expected)) { auto * literal = value->as(); @@ -59,7 +59,8 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else if (parser_substitution.parse(pos, value, expected)) { - fields_count = 1; + /// It can be tuple substitution + fields_count = std::nullopt; } else if (parser_expr.parse(pos, value, expected)) { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3909e90e5cc..ceebed100b2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -53,6 +53,7 @@ #include #include #include +#include #include #include #include @@ -5347,9 +5348,67 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc MergeTreePartInfo::validatePartitionID(partition_ast.id->clone(), format_version); return partition_ast.id->as()->value.safeGet(); } - + size_t partition_ast_fields_count = 0; ASTPtr partition_value_ast = partition_ast.value->clone(); - size_t partition_ast_fields_count = partition_ast.fields_count; + if (!partition_ast.fields_count.has_value()) + { + if (partition_value_ast->as()) + { + partition_ast_fields_count = 1; + } + else if (const auto * tuple_ast = partition_value_ast->as()) + { + if (tuple_ast->name != "tuple") + { + if (isFunctionCast(tuple_ast)) + { + if (tuple_ast->arguments->as()->children.empty()) + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + } + auto first_arg = tuple_ast->arguments->as()->children.at(0); + if (const auto * inner_tuple = first_arg->as(); inner_tuple && inner_tuple->name == "tuple") + { + const auto * arguments_ast = tuple_ast->arguments->as(); + if (arguments_ast) + partition_ast_fields_count = arguments_ast->children.size(); + else + partition_ast_fields_count = 0; + } + else if (const auto * inner_literal_tuple = first_arg->as(); + inner_literal_tuple && inner_literal_tuple->value.getType() == Field::Types::Tuple) + { + partition_ast_fields_count = inner_literal_tuple->value.safeGet().size(); + } + else + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + } + } + else + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + } + else + { + const auto * arguments_ast = tuple_ast->arguments->as(); + if (arguments_ast) + partition_ast_fields_count = arguments_ast->children.size(); + else + partition_ast_fields_count = 0; + } + } + else + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, "Expected literal or tuple for partition key, got {}", partition_value_ast->getID()); + } + } + else + { + partition_ast_fields_count = *partition_ast.fields_count; + } if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.reference b/tests/queries/0_stateless/02897_alter_partition_parameters.reference index a21090781f3..405d3348775 100644 --- a/tests/queries/0_stateless/02897_alter_partition_parameters.reference +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.reference @@ -5,3 +5,4 @@ 0 0 0 +0 diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.sql b/tests/queries/0_stateless/02897_alter_partition_parameters.sql index 9bd3df0f971..df7f7b12550 100644 --- a/tests/queries/0_stateless/02897_alter_partition_parameters.sql +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.sql @@ -113,3 +113,24 @@ ALTER TABLE test5 DROP PARTITION ({f:UInt32}, 2); SELECT count() FROM test5; DROP TABLE IF EXISTS test5; + +DROP TABLE IF EXISTS test6; + +CREATE TABLE test6 +( + a UInt32, + b Int64 +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY (a, b); + +INSERT INTO test6 VALUES(1, 2); + +SET param_tuple=(1, 2); + +ALTER TABLE test6 DROP PARTITION {tuple:Tuple(UInt32, Int64)}; + +SELECT count() FROM test6; + +DROP TABLE IF EXISTS test6; From 713993a210f0727ed5a2ccf3ffa7da40d5d5a26a Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Oct 2023 18:54:38 +0200 Subject: [PATCH 248/634] Finally --- .../0_stateless/02897_alter_partition_parameters.reference | 1 + .../queries/0_stateless/02897_alter_partition_parameters.sql | 5 +++++ 2 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.reference b/tests/queries/0_stateless/02897_alter_partition_parameters.reference index 405d3348775..bc6ff2b709c 100644 --- a/tests/queries/0_stateless/02897_alter_partition_parameters.reference +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.reference @@ -6,3 +6,4 @@ 0 0 0 +0 diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.sql b/tests/queries/0_stateless/02897_alter_partition_parameters.sql index df7f7b12550..62ceb9d9768 100644 --- a/tests/queries/0_stateless/02897_alter_partition_parameters.sql +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.sql @@ -33,6 +33,11 @@ ALTER TABLE test DROP PARTITION ID {partition_id:String}; SELECT count() FROM test; +INSERT INTO test VALUES(toDate('2023-10-09')); + +ALTER TABLE test DROP PARTITION {partition:Date}; +SELECT count() FROM test; + DROP TABLE IF EXISTS test; DROP TABLE IF EXISTS test2; From 83daf766c9450c6f874b1fb3d3a1279734d257b7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 13 Oct 2023 18:55:22 +0200 Subject: [PATCH 249/634] Revert "Merge pull request #55587 from azat/fix-replica-group" This reverts commit ebfad5c8b886aafbea656e7fbecff277b6e801fe, reversing changes made to e64abfc557e7595a7801eae29ccb12153bc7f7f4. --- src/Databases/DatabaseReplicated.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 7234baa818a..1c44a074c96 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -301,7 +301,7 @@ std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr { for (const auto & replica : addresses_with_failover[shard_index]) { - String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name, replica.database_replica_group_name); + String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name, replica.database_replica_name); paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); } } From 3e60451ac7166b6d03b091f8a54a5472e3534e86 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 13 Oct 2023 18:55:51 +0200 Subject: [PATCH 250/634] Revert "Merge pull request #54421 from evillique/cluster-groups" This reverts commit 211c80e1a08910ef9fbec4a4338195ea88ecc917, reversing changes made to b53a98217c33ecc0df3b45f253e281c8ae9092d2. --- .../settings.md | 17 --- docs/en/sql-reference/statements/system.md | 8 +- programs/server/config.xml | 9 -- src/Databases/DatabaseReplicated.cpp | 110 ++++----------- src/Databases/DatabaseReplicated.h | 15 +- src/Interpreters/Cluster.cpp | 3 +- src/Interpreters/Cluster.h | 2 - src/Interpreters/InterpreterSystemQuery.cpp | 8 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 12 +- src/Parsers/ASTSystemQuery.h | 1 - src/Parsers/ParserSystemQuery.cpp | 8 -- .../__init__.py | 0 .../configs/backup_group.xml | 3 - .../configs/settings.xml | 14 -- .../test.py | 129 ------------------ 15 files changed, 43 insertions(+), 296 deletions(-) delete mode 100644 tests/integration/test_replicated_database_cluster_groups/__init__.py delete mode 100644 tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml delete mode 100644 tests/integration/test_replicated_database_cluster_groups/configs/settings.xml delete mode 100644 tests/integration/test_replicated_database_cluster_groups/test.py diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index fa9f8e70692..bd8e1da2f1e 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1396,23 +1396,6 @@ For more information, see the section [Creating replicated tables](../../engines ``` -## replica_group_name {#replica_group_name} - -Replica group name for database Replicated. - -The cluster created by Replicated database will consist of replicas in the same group. -DDL queries will only wail for the replicas in the same group. - -Empty by default. - -**Example** - -``` xml -backups -``` - -Default value: ``. - ## max_open_files {#max-open-files} The maximum number of open files. diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 578ff38574a..1558e64f99b 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -97,12 +97,12 @@ The fourth one is useful to remove metadata of dead replica when all other repli Dead replicas of `Replicated` databases can be dropped using following syntax: ``` sql -SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] [FROM GROUP 'group_name'] FROM DATABASE database; -SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] [FROM GROUP 'group_name']; -SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] [FROM GROUP 'group_name'] FROM ZKPATH '/path/to/table/in/zk'; +SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] FROM DATABASE database; +SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name']; +SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] FROM ZKPATH '/path/to/table/in/zk'; ``` -Similar to `SYSTEM DROP REPLICA`, but removes the `Replicated` database replica path from ZooKeeper when there's no database to run `DROP DATABASE`. Please note that it does not remove `ReplicatedMergeTree` replicas (so you may need `SYSTEM DROP REPLICA` as well). Shard and replica names are the names that were specified in `Replicated` engine arguments when creating the database. Also, these names can be obtained from `database_shard_name` and `database_replica_name` columns in `system.clusters`. Replica group name is the name defined by `replica_group_name` [setting](../../operations/server-configuration-parameters/settings.md#replica_group_name) in the server configuration. If the `FROM SHARD` clause is missing, then `replica_name` must be a full replica name in `shard_name|replica_name` format if replica groups are not used and in `shard_name|replica_name|group_name` otherwise. +Similar to `SYSTEM DROP REPLICA`, but removes the `Replicated` database replica path from ZooKeeper when there's no database to run `DROP DATABASE`. Please note that it does not remove `ReplicatedMergeTree` replicas (so you may need `SYSTEM DROP REPLICA` as well). Shard and replica names are the names that were specified in `Replicated` engine arguments when creating the database. Also, these names can be obtained from `database_shard_name` and `database_replica_name` columns in `system.clusters`. If the `FROM SHARD` clause is missing, then `replica_name` must be a full replica name in `shard_name|replica_name` format. ## DROP UNCOMPRESSED CACHE diff --git a/programs/server/config.xml b/programs/server/config.xml index a1e2907f6b6..1dd527805fd 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -926,15 +926,6 @@ --> - - - 3600 diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 1c44a074c96..91153f2302f 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -116,52 +116,28 @@ DatabaseReplicated::DatabaseReplicated( if (!db_settings.collection_name.value.empty()) fillClusterAuthInfo(db_settings.collection_name.value, context_->getConfigRef()); - - replica_group_name = context_->getConfigRef().getString("replica_group_name", ""); - - if (replica_group_name.find('/') != std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica group name should not contain '/': {}", replica_group_name); - if (replica_group_name.find('|') != std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica group name should not contain '|': {}", replica_group_name); } -String DatabaseReplicated::getFullReplicaName(const String & shard, const String & replica, const String & replica_group) +String DatabaseReplicated::getFullReplicaName(const String & shard, const String & replica) { - if (replica_group.empty()) - return shard + '|' + replica; - else - return shard + '|' + replica + '|' + replica_group; + return shard + '|' + replica; } String DatabaseReplicated::getFullReplicaName() const { - return getFullReplicaName(shard_name, replica_name, replica_group_name); + return getFullReplicaName(shard_name, replica_name); } -DatabaseReplicated::NameParts DatabaseReplicated::parseFullReplicaName(const String & name) +std::pair DatabaseReplicated::parseFullReplicaName(const String & name) { - NameParts parts; - - auto pos_first = name.find('|'); - if (pos_first == std::string::npos) + String shard; + String replica; + auto pos = name.find('|'); + if (pos == std::string::npos || name.find('|', pos + 1) != std::string::npos) throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect replica identifier: {}", name); - - parts.shard = name.substr(0, pos_first); - - auto pos_second = name.find('|', pos_first + 1); - if (pos_second == std::string::npos) - { - parts.replica = name.substr(pos_first + 1); - return parts; - } - - if (name.find('|', pos_second + 1) != std::string::npos) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect replica identifier: {}", name); - - parts.replica = name.substr(pos_first + 1, pos_second - pos_first - 1); - parts.replica_group = name.substr(pos_second + 1); - - return parts; + shard = name.substr(0, pos); + replica = name.substr(pos + 1); + return {shard, replica}; } ClusterPtr DatabaseReplicated::tryGetCluster() const @@ -199,7 +175,6 @@ void DatabaseReplicated::setCluster(ClusterPtr && new_cluster) ClusterPtr DatabaseReplicated::getClusterImpl() const { - Strings unfiltered_hosts; Strings hosts; Strings host_ids; @@ -211,18 +186,11 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const { host_ids.resize(0); Coordination::Stat stat; - unfiltered_hosts = zookeeper->getChildren(zookeeper_path + "/replicas", &stat); - if (unfiltered_hosts.empty()) + hosts = zookeeper->getChildren(zookeeper_path + "/replicas", &stat); + if (hosts.empty()) throw Exception(ErrorCodes::NO_ACTIVE_REPLICAS, "No replicas of database {} found. " "It's possible if the first replica is not fully created yet " "or if the last replica was just dropped or due to logical error", zookeeper_path); - - for (const auto & host : unfiltered_hosts) - { - if (replica_group_name == parseFullReplicaName(host).replica_group) - hosts.push_back(host); - } - Int32 cversion = stat.cversion; ::sort(hosts.begin(), hosts.end()); @@ -253,7 +221,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const assert(!hosts.empty()); assert(hosts.size() == host_ids.size()); - String current_shard = parseFullReplicaName(hosts.front()).shard; + String current_shard = parseFullReplicaName(hosts.front()).first; std::vector> shards; shards.emplace_back(); for (size_t i = 0; i < hosts.size(); ++i) @@ -261,17 +229,17 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const const auto & id = host_ids[i]; if (id == DROPPED_MARK) continue; - auto parts = parseFullReplicaName(hosts[i]); + auto [shard, replica] = parseFullReplicaName(hosts[i]); auto pos = id.rfind(':'); String host_port = id.substr(0, pos); - if (parts.shard != current_shard) + if (shard != current_shard) { - current_shard = parts.shard; + current_shard = shard; if (!shards.back().empty()) shards.emplace_back(); } String hostname = unescapeForFileName(host_port); - shards.back().push_back(DatabaseReplicaInfo{std::move(hostname), std::move(parts.shard), std::move(parts.replica), std::move(parts.replica_group)}); + shards.back().push_back(DatabaseReplicaInfo{std::move(hostname), std::move(shard), std::move(replica)}); } UInt16 default_port = getContext()->getTCPPort(); @@ -301,7 +269,7 @@ std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr { for (const auto & replica : addresses_with_failover[shard_index]) { - String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name, replica.database_replica_name); + String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); } } @@ -341,7 +309,6 @@ void DatabaseReplicated::fillClusterAuthInfo(String collection_name, const Poco: cluster_auth_info.cluster_secure_connection = config_ref.getBool(config_prefix + ".cluster_secure_connection", false); } - void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(LoadingStrictnessLevel mode) { try @@ -497,26 +464,8 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt for (int attempts = 10; attempts > 0; --attempts) { - Coordination::Stat stat_max_log_ptr; - Coordination::Stat stat_replicas; - String max_log_ptr_str = current_zookeeper->get(zookeeper_path + "/max_log_ptr", &stat_max_log_ptr); - Strings replicas = current_zookeeper->getChildren(zookeeper_path + "/replicas", &stat_replicas); - for (const auto & replica : replicas) - { - NameParts parts = parseFullReplicaName(replica); - if (parts.shard == shard_name && parts.replica == replica_name) - { - throw Exception( - ErrorCodes::REPLICA_ALREADY_EXISTS, - "Replica {} of shard {} of replicated database already exists in the replica group {} at {}", - replica_name, shard_name, parts.replica_group, zookeeper_path); - } - } - - /// This way we make sure that other replica with the same replica_name and shard_name - /// but with a different replica_group_name was not created at the same time. - String replica_value = "Last added replica: " + getFullReplicaName(); - + Coordination::Stat stat; + String max_log_ptr_str = current_zookeeper->get(zookeeper_path + "/max_log_ptr", &stat); Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent)); @@ -524,8 +473,7 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt /// In addition to creating the replica nodes, we record the max_log_ptr at the instant where /// we declared ourself as an existing replica. We'll need this during recoverLostReplica to /// notify other nodes that issued new queries while this node was recovering. - ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/max_log_ptr", stat_max_log_ptr.version)); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/replicas", replica_value, stat_replicas.version)); + ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/max_log_ptr", stat.version)); Coordination::Responses responses; const auto code = current_zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZOK) @@ -756,15 +704,7 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex entry.tracing_context = OpenTelemetry::CurrentContext(); String node_path = ddl_worker->tryEnqueueAndExecuteEntry(entry, query_context); - Strings hosts_to_wait; - Strings unfiltered_hosts = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); - - for (const auto & host : unfiltered_hosts) - { - if (replica_group_name == parseFullReplicaName(host).replica_group) - hosts_to_wait.push_back(host); - } - + Strings hosts_to_wait = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); return getDistributedDDLStatus(node_path, entry, query_context, &hosts_to_wait); } @@ -1172,11 +1112,11 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node } void DatabaseReplicated::dropReplica( - DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, const String & replica_group) + DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica) { assert(!database || database_zookeeper_path == database->zookeeper_path); - String full_replica_name = shard.empty() ? replica : getFullReplicaName(shard, replica, replica_group); + String full_replica_name = shard.empty() ? replica : getFullReplicaName(shard, replica); if (full_replica_name.find('/') != std::string::npos) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid replica name, '/' is not allowed: {}", full_replica_name); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 1622578f3d9..7ba91e48085 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -54,19 +54,11 @@ public: void stopReplication() override; - struct NameParts - { - String shard; - String replica; - String replica_group; - }; - String getShardName() const { return shard_name; } String getReplicaName() const { return replica_name; } - String getReplicaGroupName() const { return replica_group_name; } String getFullReplicaName() const; - static String getFullReplicaName(const String & shard, const String & replica, const String & replica_group); - static NameParts parseFullReplicaName(const String & name); + static String getFullReplicaName(const String & shard, const String & replica); + static std::pair parseFullReplicaName(const String & name); const String & getZooKeeperPath() const { return zookeeper_path; } @@ -88,7 +80,7 @@ public: bool shouldReplicateQuery(const ContextPtr & query_context, const ASTPtr & query_ptr) const override; - static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, const String & replica_group); + static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica); std::vector tryGetAreReplicasActive(const ClusterPtr & cluster_) const; @@ -134,7 +126,6 @@ private: String zookeeper_path; String shard_name; String replica_name; - String replica_group_name; String replica_path; DatabaseReplicatedSettings db_settings; diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index fbc760bc486..82c3d48bc05 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -159,7 +159,6 @@ Cluster::Address::Address( host_name = parsed_host_port.first; database_shard_name = info.shard_name; database_replica_name = info.replica_name; - database_replica_group_name = info.replica_group_name; port = parsed_host_port.second; secure = params.secure ? Protocol::Secure::Enable : Protocol::Secure::Disable; priority = params.priority; @@ -517,7 +516,7 @@ Cluster::Cluster( Addresses current; for (const auto & replica : shard) current.emplace_back( - DatabaseReplicaInfo{replica, "", "", ""}, + DatabaseReplicaInfo{replica, "", ""}, params, current_shard_num, current.size() + 1); diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index acda6d9afec..b2bc03dd74d 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -35,7 +35,6 @@ struct DatabaseReplicaInfo String hostname; String shard_name; String replica_name; - String replica_group_name; }; struct ClusterConnectionParameters @@ -112,7 +111,6 @@ public: String host_name; String database_shard_name; String database_replica_name; - String database_replica_group_name; UInt16 port{0}; String user; String password; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index d11c2d9a969..07a1ae7d170 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -927,7 +927,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) if (!query_.replica_zk_path.empty() && fs::path(replicated->getZooKeeperPath()) != fs::path(query_.replica_zk_path)) return; String full_replica_name = query_.shard.empty() ? query_.replica - : DatabaseReplicated::getFullReplicaName(query_.shard, query_.replica, query_.replica_group); + : DatabaseReplicated::getFullReplicaName(query_.shard, query_.replica); if (replicated->getFullReplicaName() != full_replica_name) return; @@ -943,7 +943,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) if (auto * replicated = dynamic_cast(database.get())) { check_not_local_replica(replicated, query); - DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica, query.replica_group); + DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica); } else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database {} is not Replicated, cannot drop replica", query.getDatabase()); @@ -968,7 +968,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) } check_not_local_replica(replicated, query); - DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica, query.replica_group); + DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica); LOG_TRACE(log, "Dropped replica {} of Replicated database {}", query.replica, backQuoteIfNeed(database->getDatabaseName())); } } @@ -981,7 +981,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) if (auto * replicated = dynamic_cast(elem.second.get())) check_not_local_replica(replicated, query); - DatabaseReplicated::dropReplica(nullptr, query.replica_zk_path, query.shard, query.replica, query.replica_group); + DatabaseReplicated::dropReplica(nullptr, query.replica_zk_path, query.shard, query.replica); LOG_INFO(log, "Dropped replica {} of Replicated database with path {}", query.replica, query.replica_zk_path); } else diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 188865cb35c..750affdfe71 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -357,9 +357,9 @@ Chunk DDLQueryStatusSource::generateChunkWithUnfinishedHosts() const size_t num = 0; if (is_replicated_database) { - auto parts = DatabaseReplicated::parseFullReplicaName(host_id); - columns[num++]->insert(parts.shard); - columns[num++]->insert(parts.replica); + auto [shard, replica] = DatabaseReplicated::parseFullReplicaName(host_id); + columns[num++]->insert(shard); + columns[num++]->insert(replica); if (active_hosts_set.contains(host_id)) columns[num++]->insert(IN_PROGRESS); else @@ -511,9 +511,9 @@ Chunk DDLQueryStatusSource::generate() { if (status.code != 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "There was an error on {}: {} (probably it's a bug)", host_id, status.message); - auto parts = DatabaseReplicated::parseFullReplicaName(host_id); - columns[num++]->insert(parts.shard); - columns[num++]->insert(parts.replica); + auto [shard, replica] = DatabaseReplicated::parseFullReplicaName(host_id); + columns[num++]->insert(shard); + columns[num++]->insert(replica); columns[num++]->insert(OK); } else diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 3315d7dd3b6..cc06e0fdcb5 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -107,7 +107,6 @@ public: String replica; String shard; String replica_zk_path; - String replica_group; bool is_drop_whole_replica{}; String storage_policy; String volume; diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 979debeb75f..a26fdc1396b 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -165,14 +165,6 @@ enum class SystemQueryTargetType if (!ParserStringLiteral{}.parse(pos, ast, expected)) return false; res->shard = ast->as().value.safeGet(); - - if (database && ParserKeyword{"FROM GROUP"}.ignore(pos, expected)) - { - ASTPtr group_ast; - if (!ParserStringLiteral{}.parse(pos, group_ast, expected)) - return false; - res->replica_group = group_ast->as().value.safeGet(); - } } if (ParserKeyword{"FROM"}.ignore(pos, expected)) diff --git a/tests/integration/test_replicated_database_cluster_groups/__init__.py b/tests/integration/test_replicated_database_cluster_groups/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml b/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml deleted file mode 100644 index 3df343bbc9e..00000000000 --- a/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml +++ /dev/null @@ -1,3 +0,0 @@ - - backups - diff --git a/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml b/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml deleted file mode 100644 index 5666ffeace8..00000000000 --- a/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml +++ /dev/null @@ -1,14 +0,0 @@ - - - - 1 - 1 - 1 - - - - - default - - - diff --git a/tests/integration/test_replicated_database_cluster_groups/test.py b/tests/integration/test_replicated_database_cluster_groups/test.py deleted file mode 100644 index db43c37bc6e..00000000000 --- a/tests/integration/test_replicated_database_cluster_groups/test.py +++ /dev/null @@ -1,129 +0,0 @@ -import re -import pytest - -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry - -cluster = ClickHouseCluster(__file__) - -main_node_1 = cluster.add_instance( - "main_node_1", - user_configs=["configs/settings.xml"], - with_zookeeper=True, - stay_alive=True, - macros={"shard": 1, "replica": 1}, -) -main_node_2 = cluster.add_instance( - "main_node_2", - user_configs=["configs/settings.xml"], - with_zookeeper=True, - stay_alive=True, - macros={"shard": 1, "replica": 2}, -) -backup_node_1 = cluster.add_instance( - "backup_node_1", - main_configs=["configs/backup_group.xml"], - user_configs=["configs/settings.xml"], - with_zookeeper=True, - stay_alive=True, - macros={"shard": 1, "replica": 3}, -) -backup_node_2 = cluster.add_instance( - "backup_node_2", - main_configs=["configs/backup_group.xml"], - user_configs=["configs/settings.xml"], - with_zookeeper=True, - stay_alive=True, - macros={"shard": 1, "replica": 4}, -) - -all_nodes = [ - main_node_1, - main_node_2, - backup_node_1, - backup_node_2, -] - -uuid_regex = re.compile("[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}") - - -def assert_create_query(nodes, table_name, expected): - replace_uuid = lambda x: re.sub(uuid_regex, "uuid", x) - query = "show create table {}".format(table_name) - for node in nodes: - assert_eq_with_retry(node, query, expected, get_result=replace_uuid) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def test_cluster_groups(started_cluster): - for node in all_nodes: - node.query( - f"CREATE DATABASE cluster_groups ENGINE = Replicated('/test/cluster_groups', '{node.macros['shard']}', '{node.macros['replica']}');" - ) - - # 1. system.clusters - - cluster_query = "SELECT host_name from system.clusters WHERE cluster = 'cluster_groups' ORDER BY host_name" - expected_main = "main_node_1\nmain_node_2\n" - expected_backup = "backup_node_1\nbackup_node_2\n" - - for node in [main_node_1, main_node_2]: - assert_eq_with_retry(node, cluster_query, expected_main) - - for node in [backup_node_1, backup_node_2]: - assert_eq_with_retry(node, cluster_query, expected_backup) - - # 2. Query execution depends only on your cluster group - - backup_node_1.stop_clickhouse() - backup_node_2.stop_clickhouse() - - # OK - main_node_1.query( - "CREATE TABLE cluster_groups.table_1 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);" - ) - - # Exception - main_node_2.stop_clickhouse() - settings = {"distributed_ddl_task_timeout": 5} - assert ( - "There are 1 unfinished hosts (0 of them are currently active)" - in main_node_1.query_and_get_error( - "CREATE TABLE cluster_groups.table_2 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);", - settings=settings, - ) - ) - - # 3. After start both groups are synced - - backup_node_1.start_clickhouse() - backup_node_2.start_clickhouse() - main_node_2.start_clickhouse() - - expected_1 = "CREATE TABLE cluster_groups.table_1\\n(\\n `d` Date,\\n `k` UInt64\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\nPARTITION BY toYYYYMM(d)\\nORDER BY k\\nSETTINGS index_granularity = 8192" - expected_2 = "CREATE TABLE cluster_groups.table_2\\n(\\n `d` Date,\\n `k` UInt64\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\nPARTITION BY toYYYYMM(d)\\nORDER BY k\\nSETTINGS index_granularity = 8192" - - assert_create_query(all_nodes, "cluster_groups.table_1", expected_1) - assert_create_query(all_nodes, "cluster_groups.table_2", expected_2) - - # 4. SYSTEM DROP DATABASE REPLICA - backup_node_2.stop_clickhouse() - backup_node_1.query( - "SYSTEM DROP DATABASE REPLICA '4' FROM SHARD '1' FROM GROUP 'backups' FROM DATABASE cluster_groups" - ) - - assert_eq_with_retry(backup_node_1, cluster_query, "backup_node_1\n") - - main_node_2.stop_clickhouse() - main_node_1.query("SYSTEM DROP DATABASE REPLICA '1|2' FROM DATABASE cluster_groups") - - assert_eq_with_retry(main_node_1, cluster_query, "main_node_1\n") From 25545d504d932f23b466a001bd3567b7483e76de Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Fri, 13 Oct 2023 10:31:21 -0700 Subject: [PATCH 251/634] Replace setting density to max_rows_in postings_list for full text search --- src/Interpreters/GinFilter.cpp | 17 +++-- src/Interpreters/GinFilter.h | 8 ++- .../MergeTree/MergeTreeIndexInverted.cpp | 21 +++--- .../MergeTree/MergeTreeIndexInverted.h | 2 +- .../0_stateless/02346_full_text_search.sql | 71 ------------------- 5 files changed, 27 insertions(+), 92 deletions(-) diff --git a/src/Interpreters/GinFilter.cpp b/src/Interpreters/GinFilter.cpp index 4662128e8ab..e60d54026eb 100644 --- a/src/Interpreters/GinFilter.cpp +++ b/src/Interpreters/GinFilter.cpp @@ -21,14 +21,18 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -GinFilterParameters::GinFilterParameters(size_t ngrams_, Float64 density_) +GinFilterParameters::GinFilterParameters(size_t ngrams_, UInt64 max_rows_) : ngrams(ngrams_) - , density(density_) + , max_rows_in_postings_list(max_rows_) { + /// 0 indicates no limitation of postings list's size + if (max_rows_in_postings_list == 0) + max_rows_in_postings_list = std::numeric_limits::max(); + if (ngrams > 8) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The size of inverted index filter cannot be greater than 8"); - if (density <= 0 || density > 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The density inverted index gin filter must be between 0 and 1"); + if (max_rows_in_postings_list < MIN_ROWS_IN_POSTINGS_LIST) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The maximum rows in postings list must be no less than {}", MIN_ROWS_IN_POSTINGS_LIST); } GinFilter::GinFilter(const GinFilterParameters & params_) @@ -36,7 +40,7 @@ GinFilter::GinFilter(const GinFilterParameters & params_) { } -void GinFilter::add(const char * data, size_t len, UInt32 rowID, GinIndexStorePtr & store, UInt64 limit) const +void GinFilter::add(const char * data, size_t len, UInt32 rowID, GinIndexStorePtr & store) const { if (len > FST::MAX_TERM_LENGTH) return; @@ -51,8 +55,7 @@ void GinFilter::add(const char * data, size_t len, UInt32 rowID, GinIndexStorePt } else { - UInt64 size_limit = std::lround(limit * params.density); - auto builder = std::make_shared(size_limit); + auto builder = std::make_shared(params.max_rows_in_postings_list); builder->add(rowID); store->setPostingsBuilder(term, builder); diff --git a/src/Interpreters/GinFilter.h b/src/Interpreters/GinFilter.h index 8985d84f215..3e57a07ecd4 100644 --- a/src/Interpreters/GinFilter.h +++ b/src/Interpreters/GinFilter.h @@ -8,13 +8,15 @@ namespace DB { static inline constexpr auto INVERTED_INDEX_NAME = "inverted"; +static inline constexpr UInt64 MIN_ROWS_IN_POSTINGS_LIST = 8 * 1024; +static inline constexpr UInt64 DEFAULT_ROWS_IN_POSTINGS_LIST = 64 * 1024; struct GinFilterParameters { - GinFilterParameters(size_t ngrams_, Float64 density_); + GinFilterParameters(size_t ngrams_, UInt64 max_rows_); size_t ngrams; - Float64 density; + UInt64 max_rows_in_postings_list; }; struct GinSegmentWithRowIdRange @@ -42,7 +44,7 @@ public: /// Add term (located at 'data' with length 'len') and its row ID to the postings list builder /// for building inverted index for the given store. - void add(const char * data, size_t len, UInt32 rowID, GinIndexStorePtr & store, UInt64 limit) const; + void add(const char * data, size_t len, UInt32 rowID, GinIndexStorePtr & store) const; /// Accumulate (segmentID, RowIDStart, RowIDEnd) for building skipping index void addRowRangeToGinFilter(UInt32 segmentID, UInt32 rowIDStart, UInt32 rowIDEnd); diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index 04ef7339a0e..169ae768b31 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -109,14 +109,14 @@ MergeTreeIndexGranulePtr MergeTreeIndexAggregatorInverted::getGranuleAndReset() return new_granule; } -void MergeTreeIndexAggregatorInverted::addToGinFilter(UInt32 rowID, const char * data, size_t length, GinFilter & gin_filter, UInt64 limit) +void MergeTreeIndexAggregatorInverted::addToGinFilter(UInt32 rowID, const char * data, size_t length, GinFilter & gin_filter) { size_t cur = 0; size_t token_start = 0; size_t token_len = 0; while (cur < length && token_extractor->nextInStringPadded(data, length, &cur, &token_start, &token_len)) - gin_filter.add(data + token_start, token_len, rowID, store, limit); + gin_filter.add(data + token_start, token_len, rowID, store); } void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos, size_t limit) @@ -150,7 +150,7 @@ void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos, for (size_t row_num = 0; row_num < elements_size; ++row_num) { auto ref = column_key.getDataAt(element_start_row + row_num); - addToGinFilter(row_id, ref.data, ref.size, granule->gin_filters[col], rows_read); + addToGinFilter(row_id, ref.data, ref.size, granule->gin_filters[col]); store->incrementCurrentSizeBy(ref.size); } current_position += 1; @@ -165,7 +165,7 @@ void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos, for (size_t i = 0; i < rows_read; ++i) { auto ref = column->getDataAt(current_position + i); - addToGinFilter(row_id, ref.data, ref.size, granule->gin_filters[col], rows_read); + addToGinFilter(row_id, ref.data, ref.size, granule->gin_filters[col]); store->incrementCurrentSizeBy(ref.size); row_id++; if (store->needToWrite()) @@ -735,8 +735,8 @@ MergeTreeIndexPtr invertedIndexCreator( const IndexDescription & index) { size_t n = index.arguments.empty() ? 0 : index.arguments[0].get(); - Float64 density = index.arguments.size() < 2 ? 1.0 : index.arguments[1].get(); - GinFilterParameters params(n, density); + UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_ROWS_IN_POSTINGS_LIST : index.arguments[1].get(); + GinFilterParameters params(n, max_rows); /// Use SplitTokenExtractor when n is 0, otherwise use NgramTokenExtractor if (n > 0) @@ -780,13 +780,14 @@ void invertedIndexValidator(const IndexDescription & index, bool /*attach*/) if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64) throw Exception(ErrorCodes::INCORRECT_QUERY, "The first Inverted index argument must be positive integer."); - if (index.arguments.size() == 2 && (index.arguments[1].getType() != Field::Types::Float64 || index.arguments[1].get() <= 0 || index.arguments[1].get() > 1)) - throw Exception(ErrorCodes::INCORRECT_QUERY, "The second Inverted index argument must be a float between 0 and 1."); + if (index.arguments.size() == 2 && (index.arguments[1].getType() != Field::Types::UInt64 || + (index.arguments[1].get() != 0 && index.arguments[1].get() < MIN_ROWS_IN_POSTINGS_LIST))) + throw Exception(ErrorCodes::INCORRECT_QUERY, "The maximum rows in postings list must be no less than {}", MIN_ROWS_IN_POSTINGS_LIST); /// Just validate size_t ngrams = index.arguments.empty() ? 0 : index.arguments[0].get(); - Float64 density = index.arguments.size() < 2 ? 1.0 : index.arguments[1].get(); - GinFilterParameters params(ngrams, density); + UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_ROWS_IN_POSTINGS_LIST : index.arguments[1].get(); + GinFilterParameters params(ngrams, max_rows); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.h b/src/Storages/MergeTree/MergeTreeIndexInverted.h index 96a87c2e2ba..96d12128bb4 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.h +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.h @@ -48,7 +48,7 @@ struct MergeTreeIndexAggregatorInverted final : IMergeTreeIndexAggregator void update(const Block & block, size_t * pos, size_t limit) override; - void addToGinFilter(UInt32 rowID, const char * data, size_t length, GinFilter & gin_filter, UInt64 limit); + void addToGinFilter(UInt32 rowID, const char * data, size_t length, GinFilter & gin_filter); GinIndexStorePtr store; Names index_columns; diff --git a/tests/queries/0_stateless/02346_full_text_search.sql b/tests/queries/0_stateless/02346_full_text_search.sql index c8536976377..7d3337b9407 100644 --- a/tests/queries/0_stateless/02346_full_text_search.sql +++ b/tests/queries/0_stateless/02346_full_text_search.sql @@ -277,74 +277,3 @@ SELECT * FROM tab WHERE str == 'b' AND 1.0; -- AND result_rows==1 -- LIMIT 1; -- --- ---------------------------------------------------- --- SELECT 'Test density==1'; --- --- DROP TABLE IF EXISTS tab; --- --- CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(0, 1.0)) --- Engine=MergeTree --- ORDER BY (k) --- SETTINGS max_digestion_size_per_segment = 1, index_granularity = 512 --- AS --- SELECT number, if(number%2, format('happy {}', hex(number)), format('birthday {}', hex(number))) --- FROM numbers(1024); --- --- -- check inverted index was created --- SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1; --- --- -- search inverted index, no row has 'happy birthday' --- SELECT count() == 0 FROM tab WHERE s =='happy birthday'; --- --- -- check the query only skip all granules (0 row total; each granule has 512 rows) --- SYSTEM FLUSH LOGS; --- SELECT read_rows==0 from system.query_log --- WHERE query_kind ='Select' --- AND current_database = currentDatabase() --- AND endsWith(trimRight(query), 'SELECT count() == 0 FROM tab WHERE s ==\'happy birthday\';') --- AND type='QueryFinish' --- AND result_rows==1 --- LIMIT 1; --- --- ---------------------------------------------------- --- SELECT 'Test density==0.1'; --- --- DROP TABLE IF EXISTS tab; --- --- CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(0, 0.1)) --- Engine=MergeTree --- ORDER BY (k) --- SETTINGS max_digestion_size_per_segment = 1, index_granularity = 512 --- AS --- SELECT number, if(number==1023, 'happy new year', if(number%2, format('happy {}', hex(number)), format('birthday {}', hex(number)))) --- FROM numbers(1024); --- --- -- check inverted index was created --- --- SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1; --- --- -- search inverted index, no row has 'happy birthday' --- SELECT count() == 0 FROM tab WHERE s == 'happy birthday'; --- --- -- check the query does not skip any of the 2 granules(1024 rows total; each granule has 512 rows) --- SYSTEM FLUSH LOGS; --- SELECT read_rows==1024 from system.query_log --- WHERE query_kind ='Select' --- AND current_database = currentDatabase() --- AND endsWith(trimRight(query), 'SELECT count() == 0 FROM tab WHERE s == \'happy birthday\';') --- AND type='QueryFinish' --- AND result_rows==1 --- LIMIT 1; --- --- -- search inverted index, no row has 'happy new year' --- SELECT count() == 1 FROM tab WHERE s == 'happy new year'; --- --- -- check the query only read 1 granule because of density (1024 rows total; each granule has 512 rows) --- SYSTEM FLUSH LOGS; --- SELECT read_rows==512 from system.query_log --- WHERE query_kind ='Select' --- AND current_database = currentDatabase() --- AND endsWith(trimRight(query), 'SELECT count() == 1 FROM tab WHERE s == \'happy new year\';') --- AND type='QueryFinish' --- AND result_rows==1 --- LIMIT 1; From 1cd98f3cea25a24bfecebe68eb3032d5db1211d5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 13 Oct 2023 17:43:38 +0000 Subject: [PATCH 252/634] use thread_local for static --- src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp b/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp index 44b08dd0b86..6ef4a52ceff 100644 --- a/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp +++ b/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp @@ -110,7 +110,7 @@ String IParserKQLFunction::generateUniqueIdentifier() // This particular random generator hits each number exactly once before looping over. // Because of this, it's sufficient for queries consisting of up to 2^16 (= 65536) distinct function calls. // Reference: https://www.pcg-random.org/using-pcg-cpp.html#insecure-generators - static pcg32_once_insecure random_generator; + static thread_local pcg32_once_insecure random_generator; return std::to_string(random_generator()); } From cc92359225694fbb977edb923a56ce3577ee5c8c Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Oct 2023 19:46:53 +0200 Subject: [PATCH 253/634] Fxi --- src/Storages/MergeTree/MergeTreeData.cpp | 32 ++++++++++++++++++------ 1 file changed, 24 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ceebed100b2..63214ba671b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5376,10 +5376,12 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc else partition_ast_fields_count = 0; } - else if (const auto * inner_literal_tuple = first_arg->as(); - inner_literal_tuple && inner_literal_tuple->value.getType() == Field::Types::Tuple) + else if (const auto * inner_literal_tuple = first_arg->as(); inner_literal_tuple) { - partition_ast_fields_count = inner_literal_tuple->value.safeGet().size(); + if (inner_literal_tuple->value.getType() == Field::Types::Tuple) + partition_ast_fields_count = inner_literal_tuple->value.safeGet().size(); + else + partition_ast_fields_count = 1; } else { @@ -5447,12 +5449,26 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc } else if (fields_count == 1) { - if (auto * tuple = partition_value_ast->as()) + if (auto * tuple = partition_value_ast->as(); tuple) { - assert(tuple->name == "tuple"); - assert(tuple->arguments); - assert(tuple->arguments->children.size() == 1); - partition_value_ast = tuple->arguments->children[0]; + if (tuple->name == "tuple") + { + assert(tuple->arguments); + assert(tuple->arguments->children.size() == 1); + partition_value_ast = tuple->arguments->children[0]; + } + else if (isFunctionCast(tuple)) + { + assert(tuple->arguments); + assert(tuple->arguments->children.size() == 2); + } + else + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, + "Expected literal or tuple for partition key, got {}", + partition_value_ast->getID()); + } } /// Simple partition key, need to evaluate and cast Field partition_key_value = evaluateConstantExpression(partition_value_ast, local_context).first; From 447631976e2e2364760be25494c87e1bff10e573 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Fri, 13 Oct 2023 20:36:14 +0200 Subject: [PATCH 254/634] Add MySQL String BLOB vs TEXT configuration --- src/Core/Settings.h | 4 +- .../InterpreterShowColumnsQuery.cpp | 52 ++--- ...show_columns_mysql_compatibility.reference | 183 ++++++++++++++++++ ...02775_show_columns_mysql_compatibility.sql | 9 + 4 files changed, 224 insertions(+), 24 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c08425c03fd..41a99ff1319 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -205,7 +205,9 @@ class IColumn; M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ \ M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \ - M(Bool, use_mysql_types_in_show_columns, false, "Show MySQL types in SHOW COLUMNS and system.columns", 0) \ + M(Bool, use_mysql_types_in_show_columns, false, "Show native MySQL types in SHOW [FULL] COLUMNS", 0) \ + M(Bool, mysql_remap_string_as_text_in_show_columns, false, "If enabled, String type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Only works if use_mysql_types_in_show_columns is enabled too", 0) \ + M(Bool, mysql_remap_fixed_string_as_text_in_show_columns, false, "If enabled, FixedString type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Only works if use_mysql_types_in_show_columns is enabled too", 0) \ \ M(UInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ", 0) \ \ diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index fe8ac63c214..a213f27972a 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -24,7 +24,10 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() { const auto & query = query_ptr->as(); - const bool use_mysql_types = getContext()->getSettingsRef().use_mysql_types_in_show_columns; + const auto & settings = getContext()->getSettingsRef(); + const bool use_mysql_types = settings.use_mysql_types_in_show_columns; + const bool remap_string_as_text = settings.mysql_remap_string_as_text_in_show_columns; + const bool remap_fixed_string_as_text = settings.mysql_remap_fixed_string_as_text_in_show_columns; WriteBufferFromOwnString buf_database; String resolved_database = getContext()->resolveDatabase(query.database); @@ -41,28 +44,29 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() /// Only used with setting 'use_mysql_types_in_show_columns = 1' /// Known issues: /// - Enums are translated to TEXT - rewritten_query += R"( + rewritten_query += fmt::format(R"( WITH map( - 'Int8', 'TINYINT', - 'Int16', 'SMALLINT', - 'Int32', 'INTEGER', - 'Int64', 'BIGINT', - 'UInt8', 'TINYINT UNSIGNED', - 'UInt16', 'SMALLINT UNSIGNED', - 'UInt32', 'INTEGER UNSIGNED', - 'UInt64', 'BIGINT UNSIGNED', - 'Float32', 'FLOAT', - 'Float64', 'DOUBLE', - 'String', 'BLOB', - 'UUID', 'CHAR', - 'Bool', 'TINYINT', - 'Date', 'DATE', - 'Date32', 'DATE', - 'DateTime', 'DATETIME', - 'DateTime64', 'DATETIME', - 'Map', 'JSON', - 'Tuple', 'JSON', - 'Object', 'JSON') AS native_to_mysql_mapping, + 'Int8', 'TINYINT', + 'Int16', 'SMALLINT', + 'Int32', 'INTEGER', + 'Int64', 'BIGINT', + 'UInt8', 'TINYINT UNSIGNED', + 'UInt16', 'SMALLINT UNSIGNED', + 'UInt32', 'INTEGER UNSIGNED', + 'UInt64', 'BIGINT UNSIGNED', + 'Float32', 'FLOAT', + 'Float64', 'DOUBLE', + 'String', '{}', + 'FixedString', '{}', + 'UUID', 'CHAR', + 'Bool', 'TINYINT', + 'Date', 'DATE', + 'Date32', 'DATE', + 'DateTime', 'DATETIME', + 'DateTime64', 'DATETIME', + 'Map', 'JSON', + 'Tuple', 'JSON', + 'Object', 'JSON') AS native_to_mysql_mapping, splitByRegexp('\(|\)', type_) AS split, multiIf(startsWith(type_, 'LowCardinality(Nullable'), split[3], startsWith(type_, 'LowCardinality'), split[2], @@ -72,7 +76,9 @@ WITH map( multiIf(inner_type = 'Decimal' AND toInt8(decimal_scale_and_precision[1]) <= 65 AND toInt8(decimal_scale_and_precision[2]) <= 30, concat('DECIMAL(', decimal_scale_and_precision[1], ', ', decimal_scale_and_precision[2], ')'), mapContains(native_to_mysql_mapping, inner_type) = true, native_to_mysql_mapping[inner_type], 'TEXT') AS mysql_type - )"; + )", + remap_string_as_text ? "TEXT" : "BLOB", + remap_fixed_string_as_text ? "TEXT" : "BLOB"); rewritten_query += R"( SELECT diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference index 6613e4f170e..29db3e1b73b 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference @@ -84,6 +84,128 @@ dt_tz2 DATETIME NO \N enm TEXT NO \N f32 FLOAT NO \N f64 DOUBLE NO \N +fs BLOB NO \N +i128 TEXT NO \N +i16 SMALLINT NO \N +i256 TEXT NO \N +i32 INTEGER NO \N +i64 BIGINT NO \N +i8 TINYINT NO \N +ip4 TEXT NO \N +ip6 TEXT NO \N +lfs BLOB NO \N +lnfs BLOB YES \N +lns BLOB YES \N +ls BLOB NO \N +m JSON NO \N +m_complex JSON NO \N +mpg TEXT NO \N +ndt64 DATETIME YES \N +ndt64_tz DATETIME YES \N +nested.col1 TEXT NO \N +nested.col2 TEXT NO \N +nfs BLOB YES \N +ns BLOB YES \N +o JSON NO \N +p TEXT NO \N +pg TEXT NO \N +r TEXT NO \N +s BLOB NO \N +sagg TEXT NO \N +t JSON NO \N +ui128 TEXT NO \N +ui16 SMALLINT UNSIGNED NO \N +ui256 TEXT NO \N +ui32 INTEGER UNSIGNED NO \N +ui64 BIGINT UNSIGNED NO \N +ui8 TINYINT UNSIGNED NO \N +uuid CHAR NO \N +-- SHOW COLUMNS with mysql_remap_string_as_text_in_show_columns = 1 +a TEXT NO \N +agg TEXT NO \N +b TINYINT NO \N +d DATE NO \N +d32 DATE NO \N +dec128 DECIMAL(38, 2) NO \N +dec128_native DECIMAL(35, 30) NO \N +dec128_text TEXT NO \N +dec256 TEXT NO \N +dec256_native DECIMAL(65, 2) NO \N +dec256_text TEXT NO \N +dec32 DECIMAL(9, 2) NO \N +dec64 DECIMAL(18, 2) NO \N +dt DATETIME NO \N +dt64 DATETIME NO \N +dt64_3_tz1 DATETIME NO \N +dt64_3_tz2 DATETIME NO \N +dt64_6 DATETIME NO \N +dt64_9 DATETIME NO \N +dt_tz1 DATETIME NO \N +dt_tz2 DATETIME NO \N +enm TEXT NO \N +f32 FLOAT NO \N +f64 DOUBLE NO \N +fs BLOB NO \N +i128 TEXT NO \N +i16 SMALLINT NO \N +i256 TEXT NO \N +i32 INTEGER NO \N +i64 BIGINT NO \N +i8 TINYINT NO \N +ip4 TEXT NO \N +ip6 TEXT NO \N +lfs BLOB NO \N +lnfs BLOB YES \N +lns TEXT YES \N +ls TEXT NO \N +m JSON NO \N +m_complex JSON NO \N +mpg TEXT NO \N +ndt64 DATETIME YES \N +ndt64_tz DATETIME YES \N +nested.col1 TEXT NO \N +nested.col2 TEXT NO \N +nfs BLOB YES \N +ns TEXT YES \N +o JSON NO \N +p TEXT NO \N +pg TEXT NO \N +r TEXT NO \N +s TEXT NO \N +sagg TEXT NO \N +t JSON NO \N +ui128 TEXT NO \N +ui16 SMALLINT UNSIGNED NO \N +ui256 TEXT NO \N +ui32 INTEGER UNSIGNED NO \N +ui64 BIGINT UNSIGNED NO \N +ui8 TINYINT UNSIGNED NO \N +uuid CHAR NO \N +-- SHOW COLUMNS with mysql_remap_fixed_string_as_text_in_show_columns = 1 +a TEXT NO \N +agg TEXT NO \N +b TINYINT NO \N +d DATE NO \N +d32 DATE NO \N +dec128 DECIMAL(38, 2) NO \N +dec128_native DECIMAL(35, 30) NO \N +dec128_text TEXT NO \N +dec256 TEXT NO \N +dec256_native DECIMAL(65, 2) NO \N +dec256_text TEXT NO \N +dec32 DECIMAL(9, 2) NO \N +dec64 DECIMAL(18, 2) NO \N +dt DATETIME NO \N +dt64 DATETIME NO \N +dt64_3_tz1 DATETIME NO \N +dt64_3_tz2 DATETIME NO \N +dt64_6 DATETIME NO \N +dt64_9 DATETIME NO \N +dt_tz1 DATETIME NO \N +dt_tz2 DATETIME NO \N +enm TEXT NO \N +f32 FLOAT NO \N +f64 DOUBLE NO \N fs TEXT NO \N i128 TEXT NO \N i16 SMALLINT NO \N @@ -120,3 +242,64 @@ ui32 INTEGER UNSIGNED NO \N ui64 BIGINT UNSIGNED NO \N ui8 TINYINT UNSIGNED NO \N uuid CHAR NO \N +-- SHOW COLUMNS with both mysql_remap_string_as_text_in_show_columns = 1 and mysql_remap_fixed_string_as_text_in_show_columns = 1 +a TEXT NO \N +agg TEXT NO \N +b TINYINT NO \N +d DATE NO \N +d32 DATE NO \N +dec128 DECIMAL(38, 2) NO \N +dec128_native DECIMAL(35, 30) NO \N +dec128_text TEXT NO \N +dec256 TEXT NO \N +dec256_native DECIMAL(65, 2) NO \N +dec256_text TEXT NO \N +dec32 DECIMAL(9, 2) NO \N +dec64 DECIMAL(18, 2) NO \N +dt DATETIME NO \N +dt64 DATETIME NO \N +dt64_3_tz1 DATETIME NO \N +dt64_3_tz2 DATETIME NO \N +dt64_6 DATETIME NO \N +dt64_9 DATETIME NO \N +dt_tz1 DATETIME NO \N +dt_tz2 DATETIME NO \N +enm TEXT NO \N +f32 FLOAT NO \N +f64 DOUBLE NO \N +fs TEXT NO \N +i128 TEXT NO \N +i16 SMALLINT NO \N +i256 TEXT NO \N +i32 INTEGER NO \N +i64 BIGINT NO \N +i8 TINYINT NO \N +ip4 TEXT NO \N +ip6 TEXT NO \N +lfs TEXT NO \N +lnfs TEXT YES \N +lns TEXT YES \N +ls TEXT NO \N +m JSON NO \N +m_complex JSON NO \N +mpg TEXT NO \N +ndt64 DATETIME YES \N +ndt64_tz DATETIME YES \N +nested.col1 TEXT NO \N +nested.col2 TEXT NO \N +nfs TEXT YES \N +ns TEXT YES \N +o JSON NO \N +p TEXT NO \N +pg TEXT NO \N +r TEXT NO \N +s TEXT NO \N +sagg TEXT NO \N +t JSON NO \N +ui128 TEXT NO \N +ui16 SMALLINT UNSIGNED NO \N +ui256 TEXT NO \N +ui32 INTEGER UNSIGNED NO \N +ui64 BIGINT UNSIGNED NO \N +ui8 TINYINT UNSIGNED NO \N +uuid CHAR NO \N diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql index 34c034fa77f..5f7d1881702 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql @@ -78,4 +78,13 @@ SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 0; SELECT '-- SHOW COLUMNS with use_mysql_types_in_show_columns = 1'; SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 1; +SELECT '-- SHOW COLUMNS with mysql_remap_string_as_text_in_show_columns = 1'; +SHOW COLUMNS FROM tab SETTINGS mysql_remap_string_as_text_in_show_columns=1; + +SELECT '-- SHOW COLUMNS with mysql_remap_fixed_string_as_text_in_show_columns = 1'; +SHOW COLUMNS FROM tab SETTINGS mysql_remap_fixed_string_as_text_in_show_columns=1; + +SELECT '-- SHOW COLUMNS with both mysql_remap_string_as_text_in_show_columns = 1 and mysql_remap_fixed_string_as_text_in_show_columns = 1'; +SHOW COLUMNS FROM tab SETTINGS mysql_remap_string_as_text_in_show_columns=1, mysql_remap_fixed_string_as_text_in_show_columns=1; + DROP TABLE tab; From 42fc670c2242977219eb541ea1423ab7c3711b9a Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Fri, 13 Oct 2023 20:56:12 +0200 Subject: [PATCH 255/634] Add new settings docs entries --- docs/en/operations/settings/settings.md | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2cb85a61be5..f22badb8c59 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3290,6 +3290,28 @@ Possible values: Default value: `0`. +## mysql_remap_string_as_text_in_show_columns {#mysql_remap_string_as_text_in_show_columns} + +When enabled, [String](../../sql-reference/data-types/string.md) ClickHouse data type will be displayed as `TEXT` in [SHOW COLUMNS](../../sql-reference/statements/show.md#show_columns). + +Has effect only when [use_mysql_types_in_show_columns](#use_mysql_types_in_show_columns) is enabled. + +- 0 - Use `BLOB`. +- 1 - Use `TEXT`. + +Default value: `0`. + +## mysql_remap_fixed_string_as_text_in_show_columns {#mysql_remap_fixed_string_as_text_in_show_columns} + +When enabled, [FixedString](../../sql-reference/data-types/fixedstring.md) ClickHouse data type will be displayed as `TEXT` in [SHOW COLUMNS](../../sql-reference/statements/show.md#show_columns). + +Has effect only when [use_mysql_types_in_show_columns](#use_mysql_types_in_show_columns) is enabled. + +- 0 - Use `BLOB`. +- 1 - Use `TEXT`. + +Default value: `0`. + ## execute_merges_on_single_replica_time_threshold {#execute-merges-on-single-replica-time-threshold} Enables special logic to perform merges on replicas. From 1d65393bff45923cbd2ae488245a6868070636bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 13 Oct 2023 23:28:34 +0200 Subject: [PATCH 256/634] Remove passed tests from `analyzer_tech_debt.txt` --- tests/analyzer_tech_debt.txt | 41 ------------------------------------ 1 file changed, 41 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index a100210d36c..2658c9b427a 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -1,68 +1,46 @@ 00223_shard_distributed_aggregation_memory_efficient -00562_in_subquery_merge_tree 00593_union_all_assert_columns_removed -00673_subquery_prepared_set_performance 00717_merge_and_distributed 00725_memory_tracking 00754_distributed_optimize_skip_select_on_unused_shards 00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere -00927_asof_joins 00940_order_by_read_in_order_query_plan -00945_bloom_filter_index -00981_in_subquery_with_tuple -01049_join_low_card_bug_long 01062_pm_all_join_with_block_continuation 01064_incremental_streaming_from_2_src_with_feedback 01071_force_optimize_skip_unused_shards 01072_optimize_skip_unused_shards_const_expr_eval 01083_expressions_in_engine_arguments -01086_odbc_roundtrip 01155_rename_move_materialized_view 01173_transaction_control_queries -01211_optimize_skip_unused_shards_type_mismatch 01213_optimize_skip_unused_shards_DISTINCT 01214_test_storage_merge_aliases_with_where -01231_distributed_aggregation_memory_efficient_mix_levels 01244_optimize_distributed_group_by_sharding_key 01247_optimize_distributed_group_by_sharding_key_dist_on_dist 01268_mv_scalars 01268_shard_avgweighted 01270_optimize_skip_unused_shards_low_cardinality 01319_optimize_skip_unused_shards_nesting -01428_nullable_asof_join 01455_shard_leaf_max_rows_bytes_to_read 01495_subqueries_in_with_statement 01504_rocksdb -01526_client_start_and_exit 01527_dist_sharding_key_dictGet_reload 01528_allow_nondeterministic_optimize_skip_unused_shards -01540_verbatim_partition_pruning 01560_merge_distributed_join 01563_distributed_query_finish -01576_alias_column_rewrite -01583_const_column_in_set_index 01584_distributed_buffer_cannot_find_column -01585_use_index_for_global_in -01585_use_index_for_global_in_with_null 01586_columns_pruning 01624_soft_constraints 01651_bugs_from_15889 01656_test_query_log_factories_info -01676_clickhouse_client_autocomplete -01681_bloom_filter_nullable_column -01700_system_zookeeper_path_in -01710_projection_additional_filters 01739_index_hint 02880_indexHint__partition_id 01747_join_view_filter_dictionary -01748_partition_id_pruning 01756_optimize_skip_unused_shards_rewrite_in 01757_optimize_skip_unused_shards_limit 01758_optimize_skip_unused_shards_once 01759_optimize_skip_unused_shards_zero_shards 01761_cast_to_enum_nullable 01786_explain_merge_tree -01889_key_condition_function_chains 01890_materialized_distributed_join 01901_in_literal_shard_prune 01925_join_materialized_columns @@ -70,8 +48,6 @@ 01930_optimize_skip_unused_shards_rewrite_in 01947_mv_subquery 01952_optimize_distributed_group_by_sharding_key -02000_join_on_const -02001_shard_num_shard_count 02131_used_row_policies_in_query_log 02139_MV_with_scalar_subquery 02174_cte_scalar_cache_mv @@ -93,35 +69,18 @@ 02554_fix_grouping_sets_predicate_push_down 02575_merge_prewhere_different_default_kind 02713_array_low_cardinality_string -02707_skip_index_with_in -02241_join_rocksdb_bs 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET -01115_join_with_dictionary 01009_global_array_join_names 00917_multiple_joins_denny_crane -00725_join_on_bug_1 00636_partition_key_parts_pruning -00261_storage_aliases_and_array_join 01825_type_json_multiple_files -01281_group_by_limit_memory_tracking -02723_zookeeper_name 00002_log_and_exception_messages_formatting -01646_rewrite_sum_if_bug 02725_agg_projection_resprect_PK -01019_alter_materialized_view_consistent -01600_parts_states_metrics_long -01600_parts_types_metrics_long -01287_max_execution_speed 02703_row_policy_for_database 02721_url_cluster 02534_s3_cluster_insert_select_schema_inference 02765_parallel_replicas_final_modifier -02784_parallel_replicas_automatic_disabling -02581_share_big_sets_between_mutation_tasks_long -02581_share_big_sets_between_multiple_mutations_tasks_long -00992_system_parts_race_condition_zookeeper_long 02818_parameterized_view_with_cte_multiple_usage -02790_optimize_skip_unused_shards_join 01940_custom_tld_sharding_key 02815_range_dict_no_direct_join 02861_join_on_nullsafe_compare From 67182f61953455201b3a457b89b265ce279585eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 13 Oct 2023 23:37:58 +0200 Subject: [PATCH 257/634] Remove passed tests from `analyzer_integration_broken_tests.txt` --- tests/analyzer_integration_broken_tests.txt | 116 ++++++-------------- 1 file changed, 34 insertions(+), 82 deletions(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index d324e4db968..e68f18e01b5 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -1,94 +1,46 @@ test_access_for_functions/test.py::test_access_rights_for_function +test_build_sets_from_multiple_threads/test.py::test_set test_concurrent_backups_s3/test.py::test_concurrent_backups -test_distributed_ddl/test.py::test_default_database[configs] -test_distributed_ddl/test.py::test_default_database[configs_secure] -test_distributed_ddl/test.py::test_on_server_fail[configs] -test_distributed_ddl/test.py::test_on_server_fail[configs_secure] -test_distributed_insert_backward_compatibility/test.py::test_distributed_in_tuple +test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary +test_distributed_backward_compatability/test.py::test_distributed_in_tuple +test_distributed_type_object/test.py::test_distributed_type_object +test_drop_is_lock_free/test.py::test_query_is_lock_free[detach table] +test_executable_table_function/test.py::test_executable_function_input_python test_mask_sensitive_info/test.py::test_encryption_functions test_merge_table_over_distributed/test.py::test_global_in test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_over_distributed +test_merge_tree_s3/test.py::test_heavy_insert_select_check_memory[node] test_mutations_with_merge_tree/test.py::test_mutations_with_merge_background_task -test_passing_max_partitions_to_read_remotely/test.py::test_default_database_on_cluster -test_row_policy/test.py::test_change_of_users_xml_changes_row_policies -test_row_policy/test.py::test_change_of_users_xml_changes_row_policies -test_row_policy/test.py::test_dcl_introspection -test_row_policy/test.py::test_dcl_introspection -test_row_policy/test.py::test_dcl_management -test_row_policy/test.py::test_dcl_management -test_row_policy/test.py::test_dcl_users_with_policies_from_users_xml -test_row_policy/test.py::test_dcl_users_with_policies_from_users_xml -test_row_policy/test.py::test_grant_create_row_policy -test_row_policy/test.py::test_grant_create_row_policy -test_row_policy/test.py::test_introspection -test_row_policy/test.py::test_introspection -test_row_policy/test.py::test_join -test_row_policy/test.py::test_join -test_row_policy/test.py::test_miscellaneous_engines -test_row_policy/test.py::test_miscellaneous_engines -test_row_policy/test.py::test_policy_from_users_xml_affects_only_user_assigned -test_row_policy/test.py::test_policy_from_users_xml_affects_only_user_assigned -test_row_policy/test.py::test_policy_on_distributed_table_via_role -test_row_policy/test.py::test_policy_on_distributed_table_via_role -test_row_policy/test.py::test_reload_users_xml_by_timer -test_row_policy/test.py::test_reload_users_xml_by_timer -test_row_policy/test.py::test_row_policy_filter_with_subquery -test_row_policy/test.py::test_row_policy_filter_with_subquery -test_row_policy/test.py::test_smoke -test_row_policy/test.py::test_smoke -test_row_policy/test.py::test_some_users_without_policies -test_row_policy/test.py::test_some_users_without_policies -test_row_policy/test.py::test_tags_with_db_and_table_names -test_row_policy/test.py::test_tags_with_db_and_table_names -test_row_policy/test.py::test_throwif_error_in_prewhere_with_same_condition_as_filter -test_row_policy/test.py::test_throwif_error_in_prewhere_with_same_condition_as_filter -test_row_policy/test.py::test_throwif_error_in_where_with_same_condition_as_filter -test_row_policy/test.py::test_throwif_error_in_where_with_same_condition_as_filter -test_row_policy/test.py::test_throwif_in_prewhere_doesnt_expose_restricted_data -test_row_policy/test.py::test_throwif_in_prewhere_doesnt_expose_restricted_data -test_row_policy/test.py::test_throwif_in_where_doesnt_expose_restricted_data -test_row_policy/test.py::test_throwif_in_where_doesnt_expose_restricted_data -test_row_policy/test.py::test_users_xml_is_readonly -test_row_policy/test.py::test_users_xml_is_readonly -test_row_policy/test.py::test_with_prewhere -test_row_policy/test.py::test_with_prewhere -test_settings_constraints_distributed/test.py::test_select_clamps_settings -test_compression_codec_read/test.py::test_default_codec_read -test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary -test_distributed_type_object/test.py::test_distributed_type_object -test_materialized_mysql_database/test.py::test_select_without_columns_5_7 -test_materialized_mysql_database/test.py::test_select_without_columns_8_0 -test_shard_level_const_function/test.py::test_remote -test_storage_postgresql/test.py::test_postgres_select_insert -test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view -test_system_merges/test.py::test_mutation_simple[] -test_system_merges/test.py::test_mutation_simple[replicated] -test_disk_over_web_server/test.py::test_cache[node2] -test_disk_over_web_server/test.py::test_incorrect_usage -test_disk_over_web_server/test.py::test_replicated_database -test_disk_over_web_server/test.py::test_unavailable_server -test_disk_over_web_server/test.py::test_usage[node2] -test_distributed_backward_compatability/test.py::test_distributed_in_tuple -test_executable_table_function/test.py::test_executable_function_input_python -test_settings_profile/test.py::test_show_profiles -test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster -test_postgresql_protocol/test.py::test_python_client test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database +test_passing_max_partitions_to_read_remotely/test.py::test_default_database_on_cluster test_profile_events_s3/test.py::test_profile_events -test_user_defined_object_persistence/test.py::test_persistence -test_settings_profile/test.py::test_show_profiles -test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster +test_replicating_constants/test.py::test_different_versions +test_row_policy/test.py::test_change_of_users_xml_changes_row_policies +test_row_policy/test.py::test_dcl_introspection +test_row_policy/test.py::test_dcl_management +test_row_policy/test.py::test_dcl_users_with_policies_from_users_xml +test_row_policy/test.py::test_grant_create_row_policy +test_row_policy/test.py::test_policy_from_users_xml_affects_only_user_assigned +test_row_policy/test.py::test_policy_on_distributed_table_via_role +test_row_policy/test.py::test_reload_users_xml_by_timer +test_row_policy/test.py::test_row_policy_filter_with_subquery +test_row_policy/test.py::test_smoke +test_row_policy/test.py::test_some_users_without_policies +test_row_policy/test.py::test_tags_with_db_and_table_names +test_row_policy/test.py::test_throwif_error_in_prewhere_with_same_condition_as_filter +test_row_policy/test.py::test_throwif_error_in_where_with_same_condition_as_filter +test_row_policy/test.py::test_throwif_in_prewhere_doesnt_expose_restricted_data +test_row_policy/test.py::test_throwif_in_where_doesnt_expose_restricted_data +test_row_policy/test.py::test_users_xml_is_readonly +test_row_policy/test.py::test_with_prewhere test_select_access_rights/test_main.py::test_alias_columns test_select_access_rights/test_main.py::test_select_count test_select_access_rights/test_main.py::test_select_join -test_postgresql_protocol/test.py::test_python_client -test_replicating_constants/test.py::test_different_versions -test_merge_tree_s3/test.py::test_heavy_insert_select_check_memory[node] +test_settings_profile/test.py::test_show_profiles +test_shard_level_const_function/test.py::test_remote +test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster +test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view +test_system_merges/test.py::test_mutation_simple[] +test_system_merges/test.py::test_mutation_simple[replicated] +test_user_defined_object_persistence/test.py::test_persistence test_wrong_db_or_table_name/test.py::test_wrong_table_name -test_drop_is_lock_free/test.py::test_query_is_lock_free[detach table] -test_odbc_interaction/test.py::test_postgres_insert -test_zookeeper_config/test.py::test_chroot_with_different_root -test_zookeeper_config/test.py::test_chroot_with_same_root -test_merge_tree_azure_blob_storage/test.py::test_table_manipulations -test_parallel_replicas_skip_shards/test.py::test_skip_unavailable_shards -test_build_sets_from_multiple_threads/test.py::test_set From 854ad78c3edd4e0ced90193ab243b76c18d709e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 01:48:18 +0200 Subject: [PATCH 258/634] Fix error --- src/Parsers/Lexer.cpp | 23 +++++++++++------------ 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index 5b97a4a939c..0e6db7a40e5 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -303,17 +303,6 @@ Token Lexer::nextTokenImpl() return Token(TokenType::Minus, token_begin, pos); } - case '\xE2': - { - /// Mathematical minus symbol, UTF-8 - if (pos + 3 <= end && pos[1] == '\x88' && pos[2] == '\x92') - { - pos += 3; - return Token(TokenType::Minus, token_begin, pos); - } - /// Other characters starting at E2 can be parsed, see skipWhitespacesUTF8 - [[fallthrough]]; - } case '*': ++pos; return Token(TokenType::Asterisk, token_begin, pos); @@ -437,7 +426,17 @@ Token Lexer::nextTokenImpl() return Token(TokenType::VerticalDelimiter, token_begin, ++pos); return Token(TokenType::Error, token_begin, pos); } - + case '\xE2': + { + /// Mathematical minus symbol, UTF-8 + if (pos + 3 <= end && pos[1] == '\x88' && pos[2] == '\x92') + { + pos += 3; + return Token(TokenType::Minus, token_begin, pos); + } + /// Other characters starting at E2 can be parsed, see skipWhitespacesUTF8 + [[fallthrough]]; + } default: if (*pos == '$') { From 81a78b56294cd3b85a0d263a5bd61442a4d3d5d2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 02:07:35 +0200 Subject: [PATCH 259/634] Fix test --- tests/queries/0_stateless/01196_max_parser_depth.sh | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01196_max_parser_depth.sh b/tests/queries/0_stateless/01196_max_parser_depth.sh index 57283feb7f0..ac968da4d66 100755 --- a/tests/queries/0_stateless/01196_max_parser_depth.sh +++ b/tests/queries/0_stateless/01196_max_parser_depth.sh @@ -6,6 +6,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) { printf "select "; for _ in {1..1000}; do printf "coalesce(null, "; done; printf "1"; for _ in {1..1000}; do printf ")"; done; } > "${CLICKHOUSE_TMP}"/query -cat "${CLICKHOUSE_TMP}"/query | $CLICKHOUSE_CLIENT 2>&1 | grep -o -F 'Code: 167' -cat "${CLICKHOUSE_TMP}"/query | $CLICKHOUSE_LOCAL 2>&1 | grep -o -F 'Code: 167' -cat "${CLICKHOUSE_TMP}"/query | $CLICKHOUSE_CURL --data-binary @- -vsS "$CLICKHOUSE_URL" 2>&1 | grep -o -F 'Code: 167' +echo '-- 1.' +cat "${CLICKHOUSE_TMP}"/query | $CLICKHOUSE_CLIENT 2>&1 | grep -o -m1 -F 'Code: 167' +echo '-- 2.' +cat "${CLICKHOUSE_TMP}"/query | $CLICKHOUSE_LOCAL 2>&1 | grep -o -m1 -F 'Code: 167' +echo '-- 3.' +cat "${CLICKHOUSE_TMP}"/query | $CLICKHOUSE_CURL --data-binary @- -vsS "$CLICKHOUSE_URL" 2>&1 | grep -o -m1 -F 'Code: 167' From 0e13ebfa9560d6375d701dbf3558c445e1b15dde Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 02:08:10 +0200 Subject: [PATCH 260/634] Fix test --- tests/queries/0_stateless/01196_max_parser_depth.reference | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01196_max_parser_depth.reference b/tests/queries/0_stateless/01196_max_parser_depth.reference index d2222a8b895..de8ecef87d7 100644 --- a/tests/queries/0_stateless/01196_max_parser_depth.reference +++ b/tests/queries/0_stateless/01196_max_parser_depth.reference @@ -1,5 +1,6 @@ +-- 1. Code: 167 +-- 2. Code: 167 -Code: 167 -Code: 167 +-- 3. Code: 167 From de8534de706d5e2b36cc38c60ee9843f014df68f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 02:10:13 +0200 Subject: [PATCH 261/634] Fix error --- src/Parsers/parseQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index aa6357ec12f..8f9977c0b8d 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -264,7 +264,7 @@ ASTPtr tryParseQuery( _out_query_end = last_token.end; /// Also check on the AST level, because the generated AST depth can be greater than the recursion depth of the parser. - if (max_parser_depth) + if (res && max_parser_depth) res->checkDepth(max_parser_depth); ASTInsertQuery * insert = nullptr; From bbd67d262a5c51afea7690b9b56bf64636d64ab1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 02:52:53 +0200 Subject: [PATCH 262/634] Merging #52352 --- src/Storages/StorageSet.cpp | 69 +++++++++++++++++-- src/Storages/StorageSet.h | 6 +- .../02867_storage_set_tsan.reference | 0 .../0_stateless/02867_storage_set_tsan.sh | 42 +++++++++++ 4 files changed, 108 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02867_storage_set_tsan.reference create mode 100755 tests/queries/0_stateless/02867_storage_set_tsan.sh diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 79369ab4bcb..c8b38186dfb 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -156,12 +156,62 @@ StorageSet::StorageSet( } -void StorageSet::insertBlock(const Block & block, ContextPtr) { set->insertFromBlock(block.getColumnsWithTypeAndName()); } -void StorageSet::finishInsert() { set->finishInsert(); } +SetPtr StorageSet::getSet() const +{ + std::lock_guard lock(mutex); + return set; +} -size_t StorageSet::getSize(ContextPtr) const { return set->getTotalRowCount(); } -std::optional StorageSet::totalRows(const Settings &) const { return set->getTotalRowCount(); } -std::optional StorageSet::totalBytes(const Settings &) const { return set->getTotalByteCount(); } + +void StorageSet::insertBlock(const Block & block, ContextPtr) +{ + SetPtr current_set; + { + std::lock_guard lock(mutex); + current_set = set; + } + current_set->insertFromBlock(block.getColumnsWithTypeAndName()); +} + +void StorageSet::finishInsert() +{ + SetPtr current_set; + { + std::lock_guard lock(mutex); + current_set = set; + } + current_set->finishInsert(); +} + +size_t StorageSet::getSize(ContextPtr) const +{ + SetPtr current_set; + { + std::lock_guard lock(mutex); + current_set = set; + } + return current_set->getTotalRowCount(); +} + +std::optional StorageSet::totalRows(const Settings &) const +{ + SetPtr current_set; + { + std::lock_guard lock(mutex); + current_set = set; + } + return current_set->getTotalRowCount(); +} + +std::optional StorageSet::totalBytes(const Settings &) const +{ + SetPtr current_set; + { + std::lock_guard lock(mutex); + current_set = set; + } + return current_set->getTotalRowCount(); +} void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) { @@ -176,8 +226,13 @@ void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_sn Block header = metadata_snapshot->getSampleBlock(); increment = 0; - set = std::make_shared(SizeLimits(), 0, true); - set->setHeader(header.getColumnsWithTypeAndName()); + + auto new_set = std::make_shared(SizeLimits(), 0, true); + new_set->setHeader(header.getColumnsWithTypeAndName()); + { + std::lock_guard lock(mutex); + set = new_set; + } } diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index b310f817eb9..67a9528ff5e 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -79,7 +79,7 @@ public: String getName() const override { return "Set"; } /// Access the insides. - SetPtr & getSet() { return set; } + SetPtr getSet() const; void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; @@ -87,7 +87,9 @@ public: std::optional totalBytes(const Settings & settings) const override; private: - SetPtr set; + /// Allows to concurrently truncate the set and work (read/fill) the existing set. + mutable std::mutex mutex; + SetPtr set TSA_GUARDED_BY(mutex); void insertBlock(const Block & block, ContextPtr) override; void finishInsert() override; diff --git a/tests/queries/0_stateless/02867_storage_set_tsan.reference b/tests/queries/0_stateless/02867_storage_set_tsan.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02867_storage_set_tsan.sh b/tests/queries/0_stateless/02867_storage_set_tsan.sh new file mode 100755 index 00000000000..81ae5f0bda8 --- /dev/null +++ b/tests/queries/0_stateless/02867_storage_set_tsan.sh @@ -0,0 +1,42 @@ +#!/usr/bin/env bash +# Tags: race, no-debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -mn -q """ +DROP TABLE IF EXISTS t1_02867; +CREATE TABLE t1_02867 (x UInt64) ENGINE=Set(); +""" + +function repeat_select() { + n=0 + while [ "$n" -lt 20 ]; + do + n=$(( n + 1 )) + $CLICKHOUSE_CLIENT -q "SELECT count() as a FROM numbers(10) WHERE number IN t1_02867" > /dev/null 2> /dev/null || exit + done +} + +function repeat_truncate_insert() { + n=0 + while [ "$n" -lt 20 ]; + do + n=$(( n + 1 )) + $CLICKHOUSE_CLIENT -q "TRUNCATE t1_02867;" > /dev/null 2> /dev/null || exit + done +} + +repeat_select & +repeat_truncate_insert & +repeat_select & +repeat_truncate_insert & +repeat_select & +repeat_truncate_insert & +repeat_select & +repeat_truncate_insert & + +sleep 10 + +$CLICKHOUSE_CLIENT -mn -q "DROP TABLE IF EXISTS t1_02867;" From f5c84b4570e94e41d1071e3977df545de5fa4e8f Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 14 Oct 2023 01:14:32 +0000 Subject: [PATCH 263/634] Enable 02161_addressToLineWithInlines --- tests/queries/0_stateless/02161_addressToLineWithInlines.sql | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02161_addressToLineWithInlines.sql b/tests/queries/0_stateless/02161_addressToLineWithInlines.sql index e4624fffd48..b6b497b4b55 100644 --- a/tests/queries/0_stateless/02161_addressToLineWithInlines.sql +++ b/tests/queries/0_stateless/02161_addressToLineWithInlines.sql @@ -1,5 +1,4 @@ --- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-cpu-aarch64, disabled --- Tag disabled: Parsing inlines may lead to "could not find abbreviation code" (FIXME) +-- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-cpu-aarch64 SET allow_introspection_functions = 0; SELECT addressToLineWithInlines(1); -- { serverError 446 } From 16bf5d294663d0b0e0c32126eba6ed48330425a4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 04:21:07 +0300 Subject: [PATCH 264/634] Revert "Removed "maximize" and "drag" buttons from `dashboard` in case of single chart" --- programs/server/dashboard.html | 168 ++++++++++++++++----------------- 1 file changed, 79 insertions(+), 89 deletions(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 132606b728d..555d039cec3 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -813,101 +813,91 @@ function insertChart(i) { let edit_buttons = document.createElement('div'); edit_buttons.className = 'chart-buttons'; - // dragging and maximizing both be added to chart only, when there are more than 1 chart - - if (queries.length > 1) { - - // dragging + let move = document.createElement('a'); + let move_text = document.createTextNode('✥'); + move.appendChild(move_text); - let move = document.createElement('a'); - let move_text = document.createTextNode('✥'); - move.appendChild(move_text); - - let is_dragging = false; - move.addEventListener('mousedown', e => { - const idx = getCurrentIndex(); - is_dragging = true; - chart.className = 'chart chart-moving'; - - let offset_x = e.clientX; - let offset_y = e.clientY; - - let displace_idx = null; - let displace_chart = null; - - function mouseup(e) { - is_dragging = false; - chart.className = 'chart'; - chart.style.left = null; - chart.style.top = null; - - if (displace_idx !== null) { - const elem = queries[idx]; - queries.splice(idx, 1); - queries.splice(displace_idx, 0, elem); - - displace_chart.className = 'chart'; - drawAll(); + let is_dragging = false; + move.addEventListener('mousedown', e => { + const idx = getCurrentIndex(); + is_dragging = true; + chart.className = 'chart chart-moving'; + + let offset_x = e.clientX; + let offset_y = e.clientY; + + let displace_idx = null; + let displace_chart = null; + + function mouseup(e) { + is_dragging = false; + chart.className = 'chart'; + chart.style.left = null; + chart.style.top = null; + + if (displace_idx !== null) { + const elem = queries[idx]; + queries.splice(idx, 1); + queries.splice(displace_idx, 0, elem); + + displace_chart.className = 'chart'; + drawAll(); + } + } + + function mousemove(e) { + if (!is_dragging) { + document.body.removeEventListener('mousemove', mousemove); + document.body.removeEventListener('mouseup', mouseup); + return; + } + + let x = e.clientX - offset_x; + let y = e.clientY - offset_y; + + chart.style.left = `${x}px`; + chart.style.top = `${y}px`; + + displace_idx = null; + displace_chart = null; + let current_idx = -1; + for (const elem of charts.querySelectorAll('.chart')) { + ++current_idx; + if (current_idx == idx) { + continue; + } + + const this_rect = chart.getBoundingClientRect(); + const this_center_x = this_rect.left + this_rect.width / 2; + const this_center_y = this_rect.top + this_rect.height / 2; + + const elem_rect = elem.getBoundingClientRect(); + + if (this_center_x >= elem_rect.left && this_center_x <= elem_rect.right + && this_center_y >= elem_rect.top && this_center_y <= elem_rect.bottom) { + + elem.className = 'chart chart-displaced'; + displace_idx = current_idx; + displace_chart = elem; + } else { + elem.className = 'chart'; } } - - function mousemove(e) { - if (!is_dragging) { - document.body.removeEventListener('mousemove', mousemove); - document.body.removeEventListener('mouseup', mouseup); - return; - } - - let x = e.clientX - offset_x; - let y = e.clientY - offset_y; - - chart.style.left = `${x}px`; - chart.style.top = `${y}px`; - - displace_idx = null; - displace_chart = null; - let current_idx = -1; - for (const elem of charts.querySelectorAll('.chart')) { - ++current_idx; - if (current_idx == idx) { - continue; - } - - const this_rect = chart.getBoundingClientRect(); - const this_center_x = this_rect.left + this_rect.width / 2; - const this_center_y = this_rect.top + this_rect.height / 2; - - const elem_rect = elem.getBoundingClientRect(); - - if (this_center_x >= elem_rect.left && this_center_x <= elem_rect.right - && this_center_y >= elem_rect.top && this_center_y <= elem_rect.bottom) { - - elem.className = 'chart chart-displaced'; - displace_idx = current_idx; - displace_chart = elem; - } else { - elem.className = 'chart'; - } - } - } - - document.body.addEventListener('mouseup', mouseup); - document.body.addEventListener('mousemove', mousemove); - }); + } - // maximizing + document.body.addEventListener('mouseup', mouseup); + document.body.addEventListener('mousemove', mousemove); + }); - let maximize = document.createElement('a'); - let maximize_text = document.createTextNode('🗖'); - maximize.appendChild(maximize_text); - - maximize.addEventListener('click', e => { - const idx = getCurrentIndex(); - chart.className = (chart.className == 'chart' ? 'chart chart-maximized' : 'chart'); - resize(); - }); - } + let maximize = document.createElement('a'); + let maximize_text = document.createTextNode('🗖'); + maximize.appendChild(maximize_text); + maximize.addEventListener('click', e => { + const idx = getCurrentIndex(); + chart.className = (chart.className == 'chart' ? 'chart chart-maximized' : 'chart'); + resize(); + }); let edit = document.createElement('a'); let edit_text = document.createTextNode('✎'); From a904ee970f637beb64e0e1f8b8c77150991666fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 04:32:25 +0200 Subject: [PATCH 265/634] KeyCondition: preparation --- src/Core/Range.cpp | 176 +++++++++++++++++ src/Core/Range.h | 88 +++++++++ src/Functions/DateTimeTransforms.h | 4 +- .../FunctionDateOrDateTimeToSomething.h | 2 +- src/Functions/IFunction.h | 7 +- src/Functions/IFunctionAdaptors.h | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 20 +- src/Storages/MergeTree/KeyCondition.h | 183 +----------------- 8 files changed, 275 insertions(+), 207 deletions(-) create mode 100644 src/Core/Range.cpp create mode 100644 src/Core/Range.h diff --git a/src/Core/Range.cpp b/src/Core/Range.cpp new file mode 100644 index 00000000000..293c80e70ab --- /dev/null +++ b/src/Core/Range.cpp @@ -0,0 +1,176 @@ +#include +#include +#include +#include + + +namespace DB +{ + + +Range::Range(const FieldRef & point) /// NOLINT + : left(point), right(point), left_included(true), right_included(true) {} + +/// A bounded two-sided range. +Range::Range(const FieldRef & left_, bool left_included_, const FieldRef & right_, bool right_included_) + : left(left_) + , right(right_) + , left_included(left_included_) + , right_included(right_included_) +{ + shrinkToIncludedIfPossible(); +} + +Range Range::createWholeUniverse() +{ + return Range(NEGATIVE_INFINITY, true, POSITIVE_INFINITY, true); +} + +Range Range::createWholeUniverseWithoutNull() +{ + return Range(NEGATIVE_INFINITY, false, POSITIVE_INFINITY, false); +} + +Range Range::createRightBounded(const FieldRef & right_point, bool right_included, bool with_null) +{ + Range r = with_null ? createWholeUniverse() : createWholeUniverseWithoutNull(); + r.right = right_point; + r.right_included = right_included; + r.shrinkToIncludedIfPossible(); + // Special case for [-Inf, -Inf] + if (r.right.isNegativeInfinity() && right_included) + r.left_included = true; + return r; +} + +Range Range::createLeftBounded(const FieldRef & left_point, bool left_included, bool with_null) +{ + Range r = with_null ? createWholeUniverse() : createWholeUniverseWithoutNull(); + r.left = left_point; + r.left_included = left_included; + r.shrinkToIncludedIfPossible(); + // Special case for [+Inf, +Inf] + if (r.left.isPositiveInfinity() && left_included) + r.right_included = true; + return r; +} + +/** Optimize the range. If it has an open boundary and the Field type is "loose" + * - then convert it to closed, narrowing by one. + * That is, for example, turn (0,2) into [1]. + */ +void Range::shrinkToIncludedIfPossible() +{ + if (left.isExplicit() && !left_included) + { + if (left.getType() == Field::Types::UInt64 && left.get() != std::numeric_limits::max()) + { + ++left.get(); + left_included = true; + } + if (left.getType() == Field::Types::Int64 && left.get() != std::numeric_limits::max()) + { + ++left.get(); + left_included = true; + } + } + if (right.isExplicit() && !right_included) + { + if (right.getType() == Field::Types::UInt64 && right.get() != std::numeric_limits::min()) + { + --right.get(); + right_included = true; + } + if (right.getType() == Field::Types::Int64 && right.get() != std::numeric_limits::min()) + { + --right.get(); + right_included = true; + } + } +} + +namespace +{ + inline bool equals(const Field & lhs, const Field & rhs) + { + return applyVisitor(FieldVisitorAccurateEquals(), lhs, rhs); + } + + inline bool less(const Field & lhs, const Field & rhs) + { + return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); + } +} + +bool Range::empty() const +{ + return less(right, left) + || ((!left_included || !right_included) + && !less(left, right)); +} + +/// x contained in the range +bool Range::contains(const FieldRef & x) const +{ + return !leftThan(x) && !rightThan(x); +} + +/// x is to the left +bool Range::rightThan(const FieldRef & x) const +{ + return less(left, x) || (left_included && equals(x, left)); +} + +/// x is to the right +bool Range::leftThan(const FieldRef & x) const +{ + return less(x, right) || (right_included && equals(x, right)); +} + +bool Range::intersectsRange(const Range & r) const +{ + /// r to the left of me. + if (less(r.right, left) || ((!left_included || !r.right_included) && equals(r.right, left))) + return false; + + /// r to the right of me. + if (less(right, r.left) || ((!right_included || !r.left_included) && equals(r.left, right))) + return false; + + return true; +} + +bool Range::containsRange(const Range & r) const +{ + /// r starts to the left of me. + if (less(r.left, left) || (r.left_included && !left_included && equals(r.left, left))) + return false; + + /// r ends right of me. + if (less(right, r.right) || (r.right_included && !right_included && equals(r.right, right))) + return false; + + return true; +} + +void Range::invert() +{ + std::swap(left, right); + if (left.isPositiveInfinity()) + left = NEGATIVE_INFINITY; + if (right.isNegativeInfinity()) + right = POSITIVE_INFINITY; + std::swap(left_included, right_included); +} + +String Range::toString() const +{ + WriteBufferFromOwnString str; + + str << (left_included ? '[' : '(') << applyVisitor(FieldVisitorToString(), left) << ", "; + str << applyVisitor(FieldVisitorToString(), right) << (right_included ? ']' : ')'); + + return str.str(); +} + +} diff --git a/src/Core/Range.h b/src/Core/Range.h new file mode 100644 index 00000000000..a4344c2fa66 --- /dev/null +++ b/src/Core/Range.h @@ -0,0 +1,88 @@ +#pragma once + +#include +#include +#include + +/** Range between fields, used for index analysis + * (various arithmetics on intervals of various forms). + */ + +namespace DB +{ + +/** A field, that can be stored in two representations: + * - A standalone field. + * - A field with reference to its position in a block. + * It's needed for execution of functions on ranges during + * index analysis. If function was executed once for field, + * its result would be cached for whole block for which field's reference points to. + */ +struct FieldRef : public Field +{ + FieldRef() = default; + + /// Create as explicit field without block. + template + FieldRef(T && value) : Field(std::forward(value)) {} /// NOLINT + + /// Create as reference to field in block. + FieldRef(ColumnsWithTypeAndName * columns_, size_t row_idx_, size_t column_idx_) + : Field((*(*columns_)[column_idx_].column)[row_idx_]), + columns(columns_), row_idx(row_idx_), column_idx(column_idx_) {} + + bool isExplicit() const { return columns == nullptr; } + + ColumnsWithTypeAndName * columns = nullptr; + size_t row_idx = 0; + size_t column_idx = 0; +}; + +/** Range with open or closed ends; possibly unbounded. + */ +struct Range +{ +public: + FieldRef left; /// the left border + FieldRef right; /// the right border + bool left_included; /// includes the left border + bool right_included; /// includes the right border + + /// One point. + Range(const FieldRef & point); /// NOLINT + + /// A bounded two-sided range. + Range(const FieldRef & left_, bool left_included_, const FieldRef & right_, bool right_included_); + + static Range createWholeUniverse(); + static Range createWholeUniverseWithoutNull(); + static Range createRightBounded(const FieldRef & right_point, bool right_included, bool with_null = false); + static Range createLeftBounded(const FieldRef & left_point, bool left_included, bool with_null = false); + + /** Optimize the range. If it has an open boundary and the Field type is "loose" + * - then convert it to closed, narrowing by one. + * That is, for example, turn (0,2) into [1]. + */ + void shrinkToIncludedIfPossible(); + + bool empty() const; + + /// x contained in the range + bool contains(const FieldRef & x) const; + + /// x is to the left + bool rightThan(const FieldRef & x) const; + + /// x is to the right + bool leftThan(const FieldRef & x) const; + + bool intersectsRange(const Range & r) const; + + bool containsRange(const Range & r) const; + + void invert(); + + String toString() const; +}; + +} diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 3a729cfe39f..d70391d516f 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1025,7 +1025,7 @@ struct ToYearImpl static constexpr bool hasPreimage() { return true; } - static RangeOrNull getPreimage(const IDataType & type, const Field & point) + static OptionalFieldInterval getPreimage(const IDataType & type, const Field & point) { if (point.getType() != Field::Types::UInt64) return std::nullopt; @@ -1759,7 +1759,7 @@ struct ToYYYYMMImpl } static constexpr bool hasPreimage() { return true; } - static RangeOrNull getPreimage(const IDataType & type, const Field & point) + static OptionalFieldInterval getPreimage(const IDataType & type, const Field & point) { if (point.getType() != Field::Types::UInt64) return std::nullopt; diff --git a/src/Functions/FunctionDateOrDateTimeToSomething.h b/src/Functions/FunctionDateOrDateTimeToSomething.h index e5e12ec6e92..166e5e7ca9b 100644 --- a/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -80,7 +80,7 @@ public: bool hasInformationAboutPreimage() const override { return Transform::hasPreimage(); } - RangeOrNull getPreimage(const IDataType & type, const Field & point) const override + OptionalFieldInterval getPreimage(const IDataType & type, const Field & point) const override { if constexpr (Transform::hasPreimage()) return Transform::getPreimage(type, point); diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 09758d59e4a..18932bf93fb 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -37,7 +37,8 @@ namespace ErrorCodes } /// A left-closed and right-open interval representing the preimage of a function. -using RangeOrNull = std::optional>; +using FieldInterval = std::pair; +using OptionalFieldInterval = std::optional; /// The simplest executable object. /// Motivation: @@ -297,7 +298,7 @@ public: /** Get the preimage of a function in the form of a left-closed and right-open interval. Call only if hasInformationAboutPreimage. * std::nullopt might be returned if the point (a single value) is invalid for this function. */ - virtual RangeOrNull getPreimage(const IDataType & /*type*/, const Field & /*point*/) const + virtual OptionalFieldInterval getPreimage(const IDataType & /*type*/, const Field & /*point*/) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its preimage", getName()); } @@ -498,7 +499,7 @@ public: { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName()); } - virtual RangeOrNull getPreimage(const IDataType & /*type*/, const Field & /*point*/) const + virtual OptionalFieldInterval getPreimage(const IDataType & /*type*/, const Field & /*point*/) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its preimage", getName()); } diff --git a/src/Functions/IFunctionAdaptors.h b/src/Functions/IFunctionAdaptors.h index 123fdbc2f50..0cb3b7456e4 100644 --- a/src/Functions/IFunctionAdaptors.h +++ b/src/Functions/IFunctionAdaptors.h @@ -97,7 +97,7 @@ public: return function->getMonotonicityForRange(type, left, right); } - RangeOrNull getPreimage(const IDataType & type, const Field & point) const override + OptionalFieldInterval getPreimage(const IDataType & type, const Field & point) const override { return function->getPreimage(type, point); } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 0aa475d23e6..b5f93b80ab6 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -24,12 +24,9 @@ #include #include #include -#include #include #include -#include #include - #include #include @@ -48,17 +45,6 @@ namespace ErrorCodes } -String Range::toString() const -{ - WriteBufferFromOwnString str; - - str << (left_included ? '[' : '(') << applyVisitor(FieldVisitorToString(), left) << ", "; - str << applyVisitor(FieldVisitorToString(), right) << (right_included ? ']' : ')'); - - return str.str(); -} - - /// Returns the prefix of like_pattern before the first wildcard, e.g. 'Hello\_World% ...' --> 'Hello\_World' /// We call a pattern "perfect prefix" if: /// - (1) the pattern has a wildcard @@ -705,10 +691,6 @@ static ActionsDAGPtr cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs } -inline bool Range::equals(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateEquals(), lhs, rhs); } -inline bool Range::less(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); } - - /** Calculate expressions, that depend only on constants. * For index to work when something like "WHERE Date = toDate(now())" is written. */ @@ -966,7 +948,7 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & * CREATE TABLE (x String) ORDER BY toDate(x) * SELECT ... WHERE x LIKE 'Hello%' * we want to apply the function to the constant for index analysis, - * but should modify it to pass on unparsable values. + * but should modify it to pass on un-parsable values. */ static std::set date_time_parsing_functions = { "toDate", diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index f29ace57e32..496c7a13966 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -3,6 +3,7 @@ #include #include +#include #include @@ -13,6 +14,7 @@ #include #include + namespace DB { @@ -24,187 +26,6 @@ class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; struct ActionDAGNodes; -/** A field, that can be stored in two representations: - * - A standalone field. - * - A field with reference to its position in a block. - * It's needed for execution of functions on ranges during - * index analysis. If function was executed once for field, - * its result would be cached for whole block for which field's reference points to. - */ -struct FieldRef : public Field -{ - FieldRef() = default; - - /// Create as explicit field without block. - template - FieldRef(T && value) : Field(std::forward(value)) {} /// NOLINT - - /// Create as reference to field in block. - FieldRef(ColumnsWithTypeAndName * columns_, size_t row_idx_, size_t column_idx_) - : Field((*(*columns_)[column_idx_].column)[row_idx_]), - columns(columns_), row_idx(row_idx_), column_idx(column_idx_) {} - - bool isExplicit() const { return columns == nullptr; } - - ColumnsWithTypeAndName * columns = nullptr; - size_t row_idx = 0; - size_t column_idx = 0; -}; - -/** Range with open or closed ends; possibly unbounded. - */ -struct Range -{ -private: - static bool equals(const Field & lhs, const Field & rhs); - static bool less(const Field & lhs, const Field & rhs); - -public: - FieldRef left; /// the left border - FieldRef right; /// the right border - bool left_included; /// includes the left border - bool right_included; /// includes the right border - - /// One point. - Range(const FieldRef & point) /// NOLINT - : left(point), right(point), left_included(true), right_included(true) {} - - /// A bounded two-sided range. - Range(const FieldRef & left_, bool left_included_, const FieldRef & right_, bool right_included_) - : left(left_) - , right(right_) - , left_included(left_included_) - , right_included(right_included_) - { - shrinkToIncludedIfPossible(); - } - - static Range createWholeUniverse() - { - return Range(NEGATIVE_INFINITY, true, POSITIVE_INFINITY, true); - } - - static Range createWholeUniverseWithoutNull() - { - return Range(NEGATIVE_INFINITY, false, POSITIVE_INFINITY, false); - } - - static Range createRightBounded(const FieldRef & right_point, bool right_included, bool with_null = false) - { - Range r = with_null ? createWholeUniverse() : createWholeUniverseWithoutNull(); - r.right = right_point; - r.right_included = right_included; - r.shrinkToIncludedIfPossible(); - // Special case for [-Inf, -Inf] - if (r.right.isNegativeInfinity() && right_included) - r.left_included = true; - return r; - } - - static Range createLeftBounded(const FieldRef & left_point, bool left_included, bool with_null = false) - { - Range r = with_null ? createWholeUniverse() : createWholeUniverseWithoutNull(); - r.left = left_point; - r.left_included = left_included; - r.shrinkToIncludedIfPossible(); - // Special case for [+Inf, +Inf] - if (r.left.isPositiveInfinity() && left_included) - r.right_included = true; - return r; - } - - /** Optimize the range. If it has an open boundary and the Field type is "loose" - * - then convert it to closed, narrowing by one. - * That is, for example, turn (0,2) into [1]. - */ - void shrinkToIncludedIfPossible() - { - if (left.isExplicit() && !left_included) - { - if (left.getType() == Field::Types::UInt64 && left.get() != std::numeric_limits::max()) - { - ++left.get(); - left_included = true; - } - if (left.getType() == Field::Types::Int64 && left.get() != std::numeric_limits::max()) - { - ++left.get(); - left_included = true; - } - } - if (right.isExplicit() && !right_included) - { - if (right.getType() == Field::Types::UInt64 && right.get() != std::numeric_limits::min()) - { - --right.get(); - right_included = true; - } - if (right.getType() == Field::Types::Int64 && right.get() != std::numeric_limits::min()) - { - --right.get(); - right_included = true; - } - } - } - - bool empty() const { return less(right, left) || ((!left_included || !right_included) && !less(left, right)); } - - /// x contained in the range - bool contains(const FieldRef & x) const - { - return !leftThan(x) && !rightThan(x); - } - - /// x is to the left - bool rightThan(const FieldRef & x) const - { - return less(left, x) || (left_included && equals(x, left)); - } - - /// x is to the right - bool leftThan(const FieldRef & x) const - { - return less(x, right) || (right_included && equals(x, right)); - } - - bool intersectsRange(const Range & r) const - { - /// r to the left of me. - if (less(r.right, left) || ((!left_included || !r.right_included) && equals(r.right, left))) - return false; - - /// r to the right of me. - if (less(right, r.left) || ((!right_included || !r.left_included) && equals(r.left, right))) - return false; - - return true; - } - - bool containsRange(const Range & r) const - { - /// r starts to the left of me. - if (less(r.left, left) || (r.left_included && !left_included && equals(r.left, left))) - return false; - - /// r ends right of me. - if (less(right, r.right) || (r.right_included && !right_included && equals(r.right, right))) - return false; - - return true; - } - - void invert() - { - std::swap(left, right); - if (left.isPositiveInfinity()) - left = NEGATIVE_INFINITY; - if (right.isNegativeInfinity()) - right = POSITIVE_INFINITY; - std::swap(left_included, right_included); - } - - String toString() const; -}; /** Condition on the index. * From d6984a2819f37fbe54f4675ca9289fbd8c0a37c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 04:46:22 +0200 Subject: [PATCH 266/634] Preparation --- src/Core/Range.h | 4 ++++ src/Storages/MergeTree/KeyCondition.cpp | 9 ++++----- src/Storages/MergeTree/KeyCondition.h | 2 +- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Core/Range.h b/src/Core/Range.h index a4344c2fa66..b19dac066e9 100644 --- a/src/Core/Range.h +++ b/src/Core/Range.h @@ -85,4 +85,8 @@ public: String toString() const; }; +/** Hyperrectangle is a product of ranges: each range across each coordinate. + */ +using Hyperrectangle = std::vector; + } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index b5f93b80ab6..2622b63a2bf 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include @@ -2083,7 +2082,7 @@ static BoolMask forAnyHyperrectangle( const FieldRef * right_keys, bool left_bounded, bool right_bounded, - std::vector & hyperrectangle, + Hyperrectangle & hyperrectangle, const DataTypes & data_types, size_t prefix_size, BoolMask initial_mask, @@ -2186,7 +2185,7 @@ BoolMask KeyCondition::checkInRange( const DataTypes & data_types, BoolMask initial_mask) const { - std::vector key_ranges; + Hyperrectangle key_ranges; key_ranges.reserve(used_key_size); for (size_t i = 0; i < used_key_size; ++i) @@ -2207,7 +2206,7 @@ BoolMask KeyCondition::checkInRange( // std::cerr << "]\n"; return forAnyHyperrectangle(used_key_size, left_keys, right_keys, true, true, key_ranges, data_types, 0, initial_mask, - [&] (const std::vector & key_ranges_hyperrectangle) + [&] (const Hyperrectangle & key_ranges_hyperrectangle) { auto res = checkInHyperrectangle(key_ranges_hyperrectangle, data_types); @@ -2342,7 +2341,7 @@ bool KeyCondition::matchesExactContinuousRange() const } BoolMask KeyCondition::checkInHyperrectangle( - const std::vector & hyperrectangle, + const Hyperrectangle & hyperrectangle, const DataTypes & data_types) const { std::vector rpn_stack; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 496c7a13966..93b07b1b8fd 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -74,7 +74,7 @@ public: /// Whether the condition and its negation are feasible in the direct product of single column ranges specified by `hyperrectangle`. BoolMask checkInHyperrectangle( - const std::vector & hyperrectangle, + const Hyperrectangle & hyperrectangle, const DataTypes & data_types) const; /// Whether the condition and its negation are (independently) feasible in the key range. From 0a3e431c8753e9cfbbc20ad26730756f95eacba2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 04:50:30 +0200 Subject: [PATCH 267/634] Additional changes --- src/Interpreters/MergeJoin.cpp | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 6f0c8f1bff4..e14cb544b10 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -217,8 +217,6 @@ struct MergeJoinEqualRange bool empty() const { return !left_length && !right_length; } }; -using Range = MergeJoinEqualRange; - class MergeJoinCursor { @@ -248,7 +246,7 @@ public: } } - Range getNextEqualRange(MergeJoinCursor & rhs) + MergeJoinEqualRange getNextEqualRange(MergeJoinCursor & rhs) { if (has_left_nullable && has_right_nullable) return getNextEqualRangeImpl(rhs); @@ -293,7 +291,7 @@ private: bool has_right_nullable = false; template - Range getNextEqualRangeImpl(MergeJoinCursor & rhs) + MergeJoinEqualRange getNextEqualRangeImpl(MergeJoinCursor & rhs) { while (!atEnd() && !rhs.atEnd()) { @@ -303,10 +301,10 @@ private: else if (cmp > 0) rhs.impl.next(); else if (!cmp) - return Range{impl.getRow(), rhs.impl.getRow(), getEqualLength(), rhs.getEqualLength()}; + return MergeJoinEqualRange{impl.getRow(), rhs.impl.getRow(), getEqualLength(), rhs.getEqualLength()}; } - return Range{impl.getRow(), rhs.impl.getRow(), 0, 0}; + return MergeJoinEqualRange{impl.getRow(), rhs.impl.getRow(), 0, 0}; } template @@ -404,14 +402,14 @@ void copyRightRange(const Block & right_block, const Block & right_columns_to_ad } } -void joinEqualsAnyLeft(const Block & right_block, const Block & right_columns_to_add, MutableColumns & right_columns, const Range & range) +void joinEqualsAnyLeft(const Block & right_block, const Block & right_columns_to_add, MutableColumns & right_columns, const MergeJoinEqualRange & range) { copyRightRange(right_block, right_columns_to_add, right_columns, range.right_start, range.left_length); } template bool joinEquals(const Block & left_block, const Block & right_block, const Block & right_columns_to_add, - MutableColumns & left_columns, MutableColumns & right_columns, Range & range, size_t max_rows [[maybe_unused]]) + MutableColumns & left_columns, MutableColumns & right_columns, MergeJoinEqualRange & range, size_t max_rows [[maybe_unused]]) { bool one_more = true; @@ -876,7 +874,7 @@ bool MergeJoin::leftJoin(MergeJoinCursor & left_cursor, const Block & left_block size_t left_unequal_position = left_cursor.position() + left_key_tail; left_key_tail = 0; - Range range = left_cursor.getNextEqualRange(right_cursor); + MergeJoinEqualRange range = left_cursor.getNextEqualRange(right_cursor); joinInequalsLeft(left_block, left_columns, right_columns_to_add, right_columns, left_unequal_position, range.left_start); @@ -930,7 +928,7 @@ bool MergeJoin::allInnerJoin(MergeJoinCursor & left_cursor, const Block & left_b while (!left_cursor.atEnd() && !right_cursor.atEnd()) { - Range range = left_cursor.getNextEqualRange(right_cursor); + MergeJoinEqualRange range = left_cursor.getNextEqualRange(right_cursor); if (range.empty()) break; @@ -968,7 +966,7 @@ bool MergeJoin::semiLeftJoin(MergeJoinCursor & left_cursor, const Block & left_b while (!left_cursor.atEnd() && !right_cursor.atEnd()) { - Range range = left_cursor.getNextEqualRange(right_cursor); + MergeJoinEqualRange range = left_cursor.getNextEqualRange(right_cursor); if (range.empty()) break; From e3976a5e5008fad5c8689a87cc1bba17ab648256 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Oct 2023 18:52:20 +0200 Subject: [PATCH 268/634] Do not allow tests with state ERROR be overwritten by PASSED In the ERROR case there can be sanitizers issues, that should not be hidden, like right now it is doing for `test_s3_table_functions/test.py::test_s3_table_functions_timeouts` test [1], but I was lucky enough to trigger this in [2]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/55247/e99b0f46961733fa8ba10e490279dbcb0cdd67ad/integration_tests__asan__[5_6].html [2]: https://s3.amazonaws.com/clickhouse-test-reports/55245/918d65d6707c69ab541cdb56a076cdb83845d3ed/integration_tests__asan__[5_6].html Though there could be also other issues, like with dependent services, but still, let's try. Signed-off-by: Azat Khuzhin --- tests/integration/ci-runner.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 5c3a7695119..177a43cc591 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -492,8 +492,6 @@ class ClickhouseIntegrationTestsRunner: if test not in main_counters["PASSED"]: if test in main_counters["FAILED"]: main_counters["FAILED"].remove(test) - if test in main_counters["ERROR"]: - main_counters["ERROR"].remove(test) if test in main_counters["BROKEN"]: main_counters["BROKEN"].remove(test) @@ -506,7 +504,6 @@ class ClickhouseIntegrationTestsRunner: for test in current_counters[state]: if test in main_counters["PASSED"]: main_counters["PASSED"].remove(test) - continue if test not in broken_tests: if test not in main_counters[state]: main_counters[state].append(test) From 0fb6ed5af7610968b779d90fc9fd3a893f7db3a4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 14 Oct 2023 10:28:23 +0200 Subject: [PATCH 269/634] Fix flakiness of test_system_merges (by increasing sleep interval properly) CI: https://s3.amazonaws.com/clickhouse-test-reports/55418/769ed2e19d46fcb9cb6a678a0da6d6f2fc5d239e/integration_tests__tsan__[4_6].html Signed-off-by: Azat Khuzhin --- .../test_system_merges/configs/user_overrides.xml | 7 +++++++ tests/integration/test_system_merges/test.py | 6 ++++-- 2 files changed, 11 insertions(+), 2 deletions(-) create mode 100644 tests/integration/test_system_merges/configs/user_overrides.xml diff --git a/tests/integration/test_system_merges/configs/user_overrides.xml b/tests/integration/test_system_merges/configs/user_overrides.xml new file mode 100644 index 00000000000..ca0a435aee7 --- /dev/null +++ b/tests/integration/test_system_merges/configs/user_overrides.xml @@ -0,0 +1,7 @@ + + + + 10G + + + diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index ff303afe19e..d0fa7a1d426 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -10,6 +10,7 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", main_configs=["configs/logs_config.xml"], + user_configs=["configs/user_overrides.xml"], with_zookeeper=True, macros={"shard": 0, "replica": 1}, ) @@ -17,6 +18,7 @@ node1 = cluster.add_instance( node2 = cluster.add_instance( "node2", main_configs=["configs/logs_config.xml"], + user_configs=["configs/user_overrides.xml"], with_zookeeper=True, macros={"shard": 0, "replica": 2}, ) @@ -183,10 +185,10 @@ def test_mutation_simple(started_cluster, replicated): starting_block, starting_block, starting_block + 1 ) - # ALTER will sleep for 3s * 3 (rows) = 9s + # ALTER will sleep for 9s def alter(): node1.query( - f"ALTER TABLE {name} UPDATE a = 42 WHERE sleep(3) OR 1", + f"ALTER TABLE {name} UPDATE a = 42 WHERE sleep(9) OR 1", settings=settings, ) From 0167b76e98d03d4d8c22f01cdf51fd07d1136034 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 14 Oct 2023 16:01:32 +0200 Subject: [PATCH 270/634] fix `structure does not match` logs again --- docker/test/base/setup_export_logs.sh | 2 +- tests/ci/clickhouse_helper.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index b10644bd8db..0ff79e24bf8 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -16,7 +16,7 @@ CLICKHOUSE_CI_LOGS_USER=${CLICKHOUSE_CI_LOGS_USER:-ci} CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export} EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name String, instance_type String, instance_id String, "} -EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS Int32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type, '' AS instance_id"} +EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type, '' AS instance_id"} EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "} function __set_connection_args diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 146f0385ba1..3ae20bd871d 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -296,7 +296,7 @@ class CiLogsCredentials: logging.info("Do not use external logs pushing") return "" extra_columns = ( - f"CAST({pr_info.number} AS Int32) AS pull_request_number, '{pr_info.sha}' AS commit_sha, " + f"CAST({pr_info.number} AS UInt32) AS pull_request_number, '{pr_info.sha}' AS commit_sha, " f"toDateTime('{check_start_time}', 'UTC') AS check_start_time, '{check_name}' AS check_name, " f"'{get_instance_type()}' AS instance_type, '{get_instance_id()}' AS instance_id" ) From 13ac2ed1c910f1b52b6ef3e9bfefd479ef6d4730 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 18:48:21 +0300 Subject: [PATCH 271/634] Update analyzer_integration_broken_tests.txt --- tests/analyzer_integration_broken_tests.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index e68f18e01b5..421ae81e9fe 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -44,3 +44,5 @@ test_system_merges/test.py::test_mutation_simple[] test_system_merges/test.py::test_mutation_simple[replicated] test_user_defined_object_persistence/test.py::test_persistence test_wrong_db_or_table_name/test.py::test_wrong_table_name +test_zookeeper_config/test.py::test_chroot_with_same_root +test_zookeeper_config/test.py::test_chroot_with_different_root From bafea39ce3f66d7ce793a6822db22af3838bd0ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 18:52:30 +0300 Subject: [PATCH 272/634] Update Range.h --- src/Core/Range.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Range.h b/src/Core/Range.h index b19dac066e9..89113e960be 100644 --- a/src/Core/Range.h +++ b/src/Core/Range.h @@ -5,7 +5,7 @@ #include /** Range between fields, used for index analysis - * (various arithmetics on intervals of various forms). + * (various arithmetic on intervals of various forms). */ namespace DB From bafabf1a56e6a84f75deba131d182b86d29eee34 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 19:31:46 +0200 Subject: [PATCH 273/634] Fix typo in comment --- src/Storages/MergeTree/KeyCondition.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 2622b63a2bf..6c38d568b29 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1579,7 +1579,7 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme bool is_set_const = false; bool is_constant_transformed = false; - /// We don't look for inversed key transformations when strict is true, which is required for trivial count(). + /// We don't look for inverted key transformations when strict is true, which is required for trivial count(). /// Consider the following test case: /// /// create table test1(p DateTime, k int) engine MergeTree partition by toDate(p) order by k; From c1fba94d67dc9bcb2e6449ade1a98e0d54fafc13 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 14 Oct 2023 19:04:01 +0000 Subject: [PATCH 274/634] Resubmit #54421 This reverts commit 3e60451ac7166b6d03b091f8a54a5472e3534e86. --- .../settings.md | 17 +++ docs/en/sql-reference/statements/system.md | 8 +- programs/server/config.xml | 9 ++ src/Databases/DatabaseReplicated.cpp | 110 +++++++++++---- src/Databases/DatabaseReplicated.h | 15 +- src/Interpreters/Cluster.cpp | 3 +- src/Interpreters/Cluster.h | 2 + src/Interpreters/InterpreterSystemQuery.cpp | 8 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 12 +- src/Parsers/ASTSystemQuery.h | 1 + src/Parsers/ParserSystemQuery.cpp | 8 ++ .../__init__.py | 0 .../configs/backup_group.xml | 3 + .../configs/settings.xml | 14 ++ .../test.py | 129 ++++++++++++++++++ 15 files changed, 296 insertions(+), 43 deletions(-) create mode 100644 tests/integration/test_replicated_database_cluster_groups/__init__.py create mode 100644 tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml create mode 100644 tests/integration/test_replicated_database_cluster_groups/configs/settings.xml create mode 100644 tests/integration/test_replicated_database_cluster_groups/test.py diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index bd8e1da2f1e..fa9f8e70692 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1396,6 +1396,23 @@ For more information, see the section [Creating replicated tables](../../engines ``` +## replica_group_name {#replica_group_name} + +Replica group name for database Replicated. + +The cluster created by Replicated database will consist of replicas in the same group. +DDL queries will only wail for the replicas in the same group. + +Empty by default. + +**Example** + +``` xml +backups +``` + +Default value: ``. + ## max_open_files {#max-open-files} The maximum number of open files. diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 1558e64f99b..578ff38574a 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -97,12 +97,12 @@ The fourth one is useful to remove metadata of dead replica when all other repli Dead replicas of `Replicated` databases can be dropped using following syntax: ``` sql -SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] FROM DATABASE database; -SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name']; -SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] FROM ZKPATH '/path/to/table/in/zk'; +SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] [FROM GROUP 'group_name'] FROM DATABASE database; +SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] [FROM GROUP 'group_name']; +SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] [FROM GROUP 'group_name'] FROM ZKPATH '/path/to/table/in/zk'; ``` -Similar to `SYSTEM DROP REPLICA`, but removes the `Replicated` database replica path from ZooKeeper when there's no database to run `DROP DATABASE`. Please note that it does not remove `ReplicatedMergeTree` replicas (so you may need `SYSTEM DROP REPLICA` as well). Shard and replica names are the names that were specified in `Replicated` engine arguments when creating the database. Also, these names can be obtained from `database_shard_name` and `database_replica_name` columns in `system.clusters`. If the `FROM SHARD` clause is missing, then `replica_name` must be a full replica name in `shard_name|replica_name` format. +Similar to `SYSTEM DROP REPLICA`, but removes the `Replicated` database replica path from ZooKeeper when there's no database to run `DROP DATABASE`. Please note that it does not remove `ReplicatedMergeTree` replicas (so you may need `SYSTEM DROP REPLICA` as well). Shard and replica names are the names that were specified in `Replicated` engine arguments when creating the database. Also, these names can be obtained from `database_shard_name` and `database_replica_name` columns in `system.clusters`. Replica group name is the name defined by `replica_group_name` [setting](../../operations/server-configuration-parameters/settings.md#replica_group_name) in the server configuration. If the `FROM SHARD` clause is missing, then `replica_name` must be a full replica name in `shard_name|replica_name` format if replica groups are not used and in `shard_name|replica_name|group_name` otherwise. ## DROP UNCOMPRESSED CACHE diff --git a/programs/server/config.xml b/programs/server/config.xml index 1dd527805fd..a1e2907f6b6 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -926,6 +926,15 @@ --> + + + 3600 diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 91153f2302f..1c44a074c96 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -116,28 +116,52 @@ DatabaseReplicated::DatabaseReplicated( if (!db_settings.collection_name.value.empty()) fillClusterAuthInfo(db_settings.collection_name.value, context_->getConfigRef()); + + replica_group_name = context_->getConfigRef().getString("replica_group_name", ""); + + if (replica_group_name.find('/') != std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica group name should not contain '/': {}", replica_group_name); + if (replica_group_name.find('|') != std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica group name should not contain '|': {}", replica_group_name); } -String DatabaseReplicated::getFullReplicaName(const String & shard, const String & replica) +String DatabaseReplicated::getFullReplicaName(const String & shard, const String & replica, const String & replica_group) { - return shard + '|' + replica; + if (replica_group.empty()) + return shard + '|' + replica; + else + return shard + '|' + replica + '|' + replica_group; } String DatabaseReplicated::getFullReplicaName() const { - return getFullReplicaName(shard_name, replica_name); + return getFullReplicaName(shard_name, replica_name, replica_group_name); } -std::pair DatabaseReplicated::parseFullReplicaName(const String & name) +DatabaseReplicated::NameParts DatabaseReplicated::parseFullReplicaName(const String & name) { - String shard; - String replica; - auto pos = name.find('|'); - if (pos == std::string::npos || name.find('|', pos + 1) != std::string::npos) + NameParts parts; + + auto pos_first = name.find('|'); + if (pos_first == std::string::npos) throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect replica identifier: {}", name); - shard = name.substr(0, pos); - replica = name.substr(pos + 1); - return {shard, replica}; + + parts.shard = name.substr(0, pos_first); + + auto pos_second = name.find('|', pos_first + 1); + if (pos_second == std::string::npos) + { + parts.replica = name.substr(pos_first + 1); + return parts; + } + + if (name.find('|', pos_second + 1) != std::string::npos) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect replica identifier: {}", name); + + parts.replica = name.substr(pos_first + 1, pos_second - pos_first - 1); + parts.replica_group = name.substr(pos_second + 1); + + return parts; } ClusterPtr DatabaseReplicated::tryGetCluster() const @@ -175,6 +199,7 @@ void DatabaseReplicated::setCluster(ClusterPtr && new_cluster) ClusterPtr DatabaseReplicated::getClusterImpl() const { + Strings unfiltered_hosts; Strings hosts; Strings host_ids; @@ -186,11 +211,18 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const { host_ids.resize(0); Coordination::Stat stat; - hosts = zookeeper->getChildren(zookeeper_path + "/replicas", &stat); - if (hosts.empty()) + unfiltered_hosts = zookeeper->getChildren(zookeeper_path + "/replicas", &stat); + if (unfiltered_hosts.empty()) throw Exception(ErrorCodes::NO_ACTIVE_REPLICAS, "No replicas of database {} found. " "It's possible if the first replica is not fully created yet " "or if the last replica was just dropped or due to logical error", zookeeper_path); + + for (const auto & host : unfiltered_hosts) + { + if (replica_group_name == parseFullReplicaName(host).replica_group) + hosts.push_back(host); + } + Int32 cversion = stat.cversion; ::sort(hosts.begin(), hosts.end()); @@ -221,7 +253,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const assert(!hosts.empty()); assert(hosts.size() == host_ids.size()); - String current_shard = parseFullReplicaName(hosts.front()).first; + String current_shard = parseFullReplicaName(hosts.front()).shard; std::vector> shards; shards.emplace_back(); for (size_t i = 0; i < hosts.size(); ++i) @@ -229,17 +261,17 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const const auto & id = host_ids[i]; if (id == DROPPED_MARK) continue; - auto [shard, replica] = parseFullReplicaName(hosts[i]); + auto parts = parseFullReplicaName(hosts[i]); auto pos = id.rfind(':'); String host_port = id.substr(0, pos); - if (shard != current_shard) + if (parts.shard != current_shard) { - current_shard = shard; + current_shard = parts.shard; if (!shards.back().empty()) shards.emplace_back(); } String hostname = unescapeForFileName(host_port); - shards.back().push_back(DatabaseReplicaInfo{std::move(hostname), std::move(shard), std::move(replica)}); + shards.back().push_back(DatabaseReplicaInfo{std::move(hostname), std::move(parts.shard), std::move(parts.replica), std::move(parts.replica_group)}); } UInt16 default_port = getContext()->getTCPPort(); @@ -269,7 +301,7 @@ std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr { for (const auto & replica : addresses_with_failover[shard_index]) { - String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); + String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name, replica.database_replica_name); paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); } } @@ -309,6 +341,7 @@ void DatabaseReplicated::fillClusterAuthInfo(String collection_name, const Poco: cluster_auth_info.cluster_secure_connection = config_ref.getBool(config_prefix + ".cluster_secure_connection", false); } + void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(LoadingStrictnessLevel mode) { try @@ -464,8 +497,26 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt for (int attempts = 10; attempts > 0; --attempts) { - Coordination::Stat stat; - String max_log_ptr_str = current_zookeeper->get(zookeeper_path + "/max_log_ptr", &stat); + Coordination::Stat stat_max_log_ptr; + Coordination::Stat stat_replicas; + String max_log_ptr_str = current_zookeeper->get(zookeeper_path + "/max_log_ptr", &stat_max_log_ptr); + Strings replicas = current_zookeeper->getChildren(zookeeper_path + "/replicas", &stat_replicas); + for (const auto & replica : replicas) + { + NameParts parts = parseFullReplicaName(replica); + if (parts.shard == shard_name && parts.replica == replica_name) + { + throw Exception( + ErrorCodes::REPLICA_ALREADY_EXISTS, + "Replica {} of shard {} of replicated database already exists in the replica group {} at {}", + replica_name, shard_name, parts.replica_group, zookeeper_path); + } + } + + /// This way we make sure that other replica with the same replica_name and shard_name + /// but with a different replica_group_name was not created at the same time. + String replica_value = "Last added replica: " + getFullReplicaName(); + Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent)); @@ -473,7 +524,8 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt /// In addition to creating the replica nodes, we record the max_log_ptr at the instant where /// we declared ourself as an existing replica. We'll need this during recoverLostReplica to /// notify other nodes that issued new queries while this node was recovering. - ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/max_log_ptr", stat.version)); + ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/max_log_ptr", stat_max_log_ptr.version)); + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/replicas", replica_value, stat_replicas.version)); Coordination::Responses responses; const auto code = current_zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZOK) @@ -704,7 +756,15 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex entry.tracing_context = OpenTelemetry::CurrentContext(); String node_path = ddl_worker->tryEnqueueAndExecuteEntry(entry, query_context); - Strings hosts_to_wait = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); + Strings hosts_to_wait; + Strings unfiltered_hosts = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); + + for (const auto & host : unfiltered_hosts) + { + if (replica_group_name == parseFullReplicaName(host).replica_group) + hosts_to_wait.push_back(host); + } + return getDistributedDDLStatus(node_path, entry, query_context, &hosts_to_wait); } @@ -1112,11 +1172,11 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node } void DatabaseReplicated::dropReplica( - DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica) + DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, const String & replica_group) { assert(!database || database_zookeeper_path == database->zookeeper_path); - String full_replica_name = shard.empty() ? replica : getFullReplicaName(shard, replica); + String full_replica_name = shard.empty() ? replica : getFullReplicaName(shard, replica, replica_group); if (full_replica_name.find('/') != std::string::npos) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid replica name, '/' is not allowed: {}", full_replica_name); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 7ba91e48085..1622578f3d9 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -54,11 +54,19 @@ public: void stopReplication() override; + struct NameParts + { + String shard; + String replica; + String replica_group; + }; + String getShardName() const { return shard_name; } String getReplicaName() const { return replica_name; } + String getReplicaGroupName() const { return replica_group_name; } String getFullReplicaName() const; - static String getFullReplicaName(const String & shard, const String & replica); - static std::pair parseFullReplicaName(const String & name); + static String getFullReplicaName(const String & shard, const String & replica, const String & replica_group); + static NameParts parseFullReplicaName(const String & name); const String & getZooKeeperPath() const { return zookeeper_path; } @@ -80,7 +88,7 @@ public: bool shouldReplicateQuery(const ContextPtr & query_context, const ASTPtr & query_ptr) const override; - static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica); + static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, const String & replica_group); std::vector tryGetAreReplicasActive(const ClusterPtr & cluster_) const; @@ -126,6 +134,7 @@ private: String zookeeper_path; String shard_name; String replica_name; + String replica_group_name; String replica_path; DatabaseReplicatedSettings db_settings; diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 82c3d48bc05..fbc760bc486 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -159,6 +159,7 @@ Cluster::Address::Address( host_name = parsed_host_port.first; database_shard_name = info.shard_name; database_replica_name = info.replica_name; + database_replica_group_name = info.replica_group_name; port = parsed_host_port.second; secure = params.secure ? Protocol::Secure::Enable : Protocol::Secure::Disable; priority = params.priority; @@ -516,7 +517,7 @@ Cluster::Cluster( Addresses current; for (const auto & replica : shard) current.emplace_back( - DatabaseReplicaInfo{replica, "", ""}, + DatabaseReplicaInfo{replica, "", "", ""}, params, current_shard_num, current.size() + 1); diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index b2bc03dd74d..acda6d9afec 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -35,6 +35,7 @@ struct DatabaseReplicaInfo String hostname; String shard_name; String replica_name; + String replica_group_name; }; struct ClusterConnectionParameters @@ -111,6 +112,7 @@ public: String host_name; String database_shard_name; String database_replica_name; + String database_replica_group_name; UInt16 port{0}; String user; String password; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 07a1ae7d170..d11c2d9a969 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -927,7 +927,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) if (!query_.replica_zk_path.empty() && fs::path(replicated->getZooKeeperPath()) != fs::path(query_.replica_zk_path)) return; String full_replica_name = query_.shard.empty() ? query_.replica - : DatabaseReplicated::getFullReplicaName(query_.shard, query_.replica); + : DatabaseReplicated::getFullReplicaName(query_.shard, query_.replica, query_.replica_group); if (replicated->getFullReplicaName() != full_replica_name) return; @@ -943,7 +943,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) if (auto * replicated = dynamic_cast(database.get())) { check_not_local_replica(replicated, query); - DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica); + DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica, query.replica_group); } else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database {} is not Replicated, cannot drop replica", query.getDatabase()); @@ -968,7 +968,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) } check_not_local_replica(replicated, query); - DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica); + DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica, query.replica_group); LOG_TRACE(log, "Dropped replica {} of Replicated database {}", query.replica, backQuoteIfNeed(database->getDatabaseName())); } } @@ -981,7 +981,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) if (auto * replicated = dynamic_cast(elem.second.get())) check_not_local_replica(replicated, query); - DatabaseReplicated::dropReplica(nullptr, query.replica_zk_path, query.shard, query.replica); + DatabaseReplicated::dropReplica(nullptr, query.replica_zk_path, query.shard, query.replica, query.replica_group); LOG_INFO(log, "Dropped replica {} of Replicated database with path {}", query.replica, query.replica_zk_path); } else diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 750affdfe71..188865cb35c 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -357,9 +357,9 @@ Chunk DDLQueryStatusSource::generateChunkWithUnfinishedHosts() const size_t num = 0; if (is_replicated_database) { - auto [shard, replica] = DatabaseReplicated::parseFullReplicaName(host_id); - columns[num++]->insert(shard); - columns[num++]->insert(replica); + auto parts = DatabaseReplicated::parseFullReplicaName(host_id); + columns[num++]->insert(parts.shard); + columns[num++]->insert(parts.replica); if (active_hosts_set.contains(host_id)) columns[num++]->insert(IN_PROGRESS); else @@ -511,9 +511,9 @@ Chunk DDLQueryStatusSource::generate() { if (status.code != 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "There was an error on {}: {} (probably it's a bug)", host_id, status.message); - auto [shard, replica] = DatabaseReplicated::parseFullReplicaName(host_id); - columns[num++]->insert(shard); - columns[num++]->insert(replica); + auto parts = DatabaseReplicated::parseFullReplicaName(host_id); + columns[num++]->insert(parts.shard); + columns[num++]->insert(parts.replica); columns[num++]->insert(OK); } else diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index cc06e0fdcb5..3315d7dd3b6 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -107,6 +107,7 @@ public: String replica; String shard; String replica_zk_path; + String replica_group; bool is_drop_whole_replica{}; String storage_policy; String volume; diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index a26fdc1396b..979debeb75f 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -165,6 +165,14 @@ enum class SystemQueryTargetType if (!ParserStringLiteral{}.parse(pos, ast, expected)) return false; res->shard = ast->as().value.safeGet(); + + if (database && ParserKeyword{"FROM GROUP"}.ignore(pos, expected)) + { + ASTPtr group_ast; + if (!ParserStringLiteral{}.parse(pos, group_ast, expected)) + return false; + res->replica_group = group_ast->as().value.safeGet(); + } } if (ParserKeyword{"FROM"}.ignore(pos, expected)) diff --git a/tests/integration/test_replicated_database_cluster_groups/__init__.py b/tests/integration/test_replicated_database_cluster_groups/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml b/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml new file mode 100644 index 00000000000..3df343bbc9e --- /dev/null +++ b/tests/integration/test_replicated_database_cluster_groups/configs/backup_group.xml @@ -0,0 +1,3 @@ + + backups + diff --git a/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml b/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml new file mode 100644 index 00000000000..5666ffeace8 --- /dev/null +++ b/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml @@ -0,0 +1,14 @@ + + + + 1 + 1 + 1 + + + + + default + + + diff --git a/tests/integration/test_replicated_database_cluster_groups/test.py b/tests/integration/test_replicated_database_cluster_groups/test.py new file mode 100644 index 00000000000..db43c37bc6e --- /dev/null +++ b/tests/integration/test_replicated_database_cluster_groups/test.py @@ -0,0 +1,129 @@ +import re +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) + +main_node_1 = cluster.add_instance( + "main_node_1", + user_configs=["configs/settings.xml"], + with_zookeeper=True, + stay_alive=True, + macros={"shard": 1, "replica": 1}, +) +main_node_2 = cluster.add_instance( + "main_node_2", + user_configs=["configs/settings.xml"], + with_zookeeper=True, + stay_alive=True, + macros={"shard": 1, "replica": 2}, +) +backup_node_1 = cluster.add_instance( + "backup_node_1", + main_configs=["configs/backup_group.xml"], + user_configs=["configs/settings.xml"], + with_zookeeper=True, + stay_alive=True, + macros={"shard": 1, "replica": 3}, +) +backup_node_2 = cluster.add_instance( + "backup_node_2", + main_configs=["configs/backup_group.xml"], + user_configs=["configs/settings.xml"], + with_zookeeper=True, + stay_alive=True, + macros={"shard": 1, "replica": 4}, +) + +all_nodes = [ + main_node_1, + main_node_2, + backup_node_1, + backup_node_2, +] + +uuid_regex = re.compile("[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}") + + +def assert_create_query(nodes, table_name, expected): + replace_uuid = lambda x: re.sub(uuid_regex, "uuid", x) + query = "show create table {}".format(table_name) + for node in nodes: + assert_eq_with_retry(node, query, expected, get_result=replace_uuid) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_cluster_groups(started_cluster): + for node in all_nodes: + node.query( + f"CREATE DATABASE cluster_groups ENGINE = Replicated('/test/cluster_groups', '{node.macros['shard']}', '{node.macros['replica']}');" + ) + + # 1. system.clusters + + cluster_query = "SELECT host_name from system.clusters WHERE cluster = 'cluster_groups' ORDER BY host_name" + expected_main = "main_node_1\nmain_node_2\n" + expected_backup = "backup_node_1\nbackup_node_2\n" + + for node in [main_node_1, main_node_2]: + assert_eq_with_retry(node, cluster_query, expected_main) + + for node in [backup_node_1, backup_node_2]: + assert_eq_with_retry(node, cluster_query, expected_backup) + + # 2. Query execution depends only on your cluster group + + backup_node_1.stop_clickhouse() + backup_node_2.stop_clickhouse() + + # OK + main_node_1.query( + "CREATE TABLE cluster_groups.table_1 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);" + ) + + # Exception + main_node_2.stop_clickhouse() + settings = {"distributed_ddl_task_timeout": 5} + assert ( + "There are 1 unfinished hosts (0 of them are currently active)" + in main_node_1.query_and_get_error( + "CREATE TABLE cluster_groups.table_2 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);", + settings=settings, + ) + ) + + # 3. After start both groups are synced + + backup_node_1.start_clickhouse() + backup_node_2.start_clickhouse() + main_node_2.start_clickhouse() + + expected_1 = "CREATE TABLE cluster_groups.table_1\\n(\\n `d` Date,\\n `k` UInt64\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\nPARTITION BY toYYYYMM(d)\\nORDER BY k\\nSETTINGS index_granularity = 8192" + expected_2 = "CREATE TABLE cluster_groups.table_2\\n(\\n `d` Date,\\n `k` UInt64\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\nPARTITION BY toYYYYMM(d)\\nORDER BY k\\nSETTINGS index_granularity = 8192" + + assert_create_query(all_nodes, "cluster_groups.table_1", expected_1) + assert_create_query(all_nodes, "cluster_groups.table_2", expected_2) + + # 4. SYSTEM DROP DATABASE REPLICA + backup_node_2.stop_clickhouse() + backup_node_1.query( + "SYSTEM DROP DATABASE REPLICA '4' FROM SHARD '1' FROM GROUP 'backups' FROM DATABASE cluster_groups" + ) + + assert_eq_with_retry(backup_node_1, cluster_query, "backup_node_1\n") + + main_node_2.stop_clickhouse() + main_node_1.query("SYSTEM DROP DATABASE REPLICA '1|2' FROM DATABASE cluster_groups") + + assert_eq_with_retry(main_node_1, cluster_query, "main_node_1\n") From b737866d024da0d0fd4335fa8105067562d4f3d6 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 14 Oct 2023 19:29:21 +0000 Subject: [PATCH 275/634] Fixes --- src/Databases/DatabaseReplicated.cpp | 2 +- src/Parsers/ASTSystemQuery.cpp | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 1c44a074c96..7234baa818a 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -301,7 +301,7 @@ std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr { for (const auto & replica : addresses_with_failover[shard_index]) { - String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name, replica.database_replica_name); + String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name, replica.database_replica_group_name); paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); } } diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 77235dfb6c2..b6817bb8a3a 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -116,6 +116,9 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, if (!shard.empty()) print_keyword(" FROM SHARD ") << quoteString(shard); + if (!replica_group.empty()) + print_keyword(" FROM GROUP ") << quoteString(replica_group); + if (table) { print_keyword(" FROM TABLE "); From 25efd188d5968911774ce901838cd19927560c68 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 22:17:36 +0200 Subject: [PATCH 276/634] KeyCondition: small changes --- src/Storages/MergeTree/KeyCondition.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 6c38d568b29..133325c2cb4 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -451,7 +451,8 @@ const KeyCondition::AtomMap KeyCondition::atom_map }; -static const std::map inverse_relations = { +static const std::map inverse_relations = +{ {"equals", "notEquals"}, {"notEquals", "equals"}, {"less", "greaterOrEquals"}, @@ -475,7 +476,7 @@ static const std::map inverse_relations = { }; -bool isLogicalOperator(const String & func_name) +static bool isLogicalOperator(const String & func_name) { return (func_name == "and" || func_name == "or" || func_name == "not" || func_name == "indexHint"); } @@ -954,10 +955,10 @@ static std::set date_time_parsing_functions = { "toDate32", "toDateTime", "toDateTime64", - "ParseDateTimeBestEffort", - "ParseDateTimeBestEffortUS", - "ParseDateTime32BestEffort", - "ParseDateTime64BestEffort", + "parseDateTimeBestEffort", + "parseDateTimeBestEffortUS", + "parseDateTime32BestEffort", + "parseDateTime64BestEffort", "parseDateTime", "parseDateTimeInJodaSyntax", }; @@ -1482,7 +1483,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( */ const auto & sample_block = key_expr->getSampleBlock(); - // Key columns should use canonical names for index analysis + /// Key columns should use canonical names for the index analysis. String name = node.getColumnName(); if (array_joined_column_names.contains(name)) From 7764dedb8e34bed7af80c8ccd579e803e09ff573 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 22:44:44 +0200 Subject: [PATCH 277/634] Something strange --- src/Storages/MergeTree/KeyCondition.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 133325c2cb4..95a58ef83f8 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1493,7 +1493,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( if (key_columns.end() != it) { out_key_column_num = it->second; - out_key_column_type = sample_block.getByName(it->first).type; + out_key_column_type = sample_block.getByPosition(out_key_column_num).type; return true; } From 1ef7bc41d4af39bf70f1813110c81fa5ccbe0035 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Oct 2023 00:07:28 +0200 Subject: [PATCH 278/634] Fix garbage --- src/Storages/MergeTree/KeyCondition.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 95a58ef83f8..80a6c381ecd 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1686,7 +1686,7 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme func_name = "greaterOrEquals"; else if (func_name == "in" || func_name == "notIn" || func_name == "like" || func_name == "notLike" || - func_name == "ilike" || func_name == "notIlike" || + func_name == "ilike" || func_name == "notILike" || func_name == "startsWith" || func_name == "match") { /// "const IN data_column" doesn't make sense (unlike "data_column IN const") From 17cbc128315abc6753db70c624cb07bbd4ab893b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 15 Oct 2023 07:20:04 +0300 Subject: [PATCH 279/634] Update KeyCondition.cpp --- src/Storages/MergeTree/KeyCondition.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 80a6c381ecd..2d643454ecd 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1493,7 +1493,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( if (key_columns.end() != it) { out_key_column_num = it->second; - out_key_column_type = sample_block.getByPosition(out_key_column_num).type; + out_key_column_type = sample_block.getByName(it->first).type; return true; } From 395a54915dc94c3a50126ec145055d4d59ddb393 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sun, 15 Oct 2023 10:40:37 +0200 Subject: [PATCH 280/634] Fix totalBytes() --- src/Storages/StorageSet.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index c8b38186dfb..1b0db1da800 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -210,7 +210,7 @@ std::optional StorageSet::totalBytes(const Settings &) const std::lock_guard lock(mutex); current_set = set; } - return current_set->getTotalRowCount(); + return current_set->getTotalByteCount(); } void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) From e0668e9ea01c879ac175786006ddba2eaf26844b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 15 Oct 2023 14:19:44 +0200 Subject: [PATCH 281/634] Update test.py --- .../test.py | 97 ++++++++++--------- 1 file changed, 52 insertions(+), 45 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 2c1a8a0a3b6..7bdec76e4eb 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -719,51 +719,6 @@ def test_too_many_parts(started_cluster): pg_manager2.drop_materialized_db() -def test_replica_consumer(started_cluster): - table = "test_replica_consumer" - - pg_manager_replica = PostgresManager() - pg_manager_replica.init( - instance2, - cluster.postgres_ip, - cluster.postgres_port, - default_database="postgres_database", - postgres_db_exists=True, - ) - - for pm in [pg_manager, pg_manager_replica]: - pm.create_and_fill_postgres_table(table) - pm.create_materialized_db( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - settings=[ - f"materialized_postgresql_tables_list = '{table}'", - "materialized_postgresql_backoff_min_ms = 100", - "materialized_postgresql_backoff_max_ms = 100", - "materialized_postgresql_use_unique_replication_consumer_identifier = 1", - ], - ) - - assert 50 == int(instance.query(f"SELECT count() FROM test_database.{table}")) - assert 50 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) - - instance.query( - f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(1000, 1000)" - ) - - check_tables_are_synchronized( - instance, table, postgres_database=pg_manager.get_default_database() - ) - check_tables_are_synchronized( - instance2, table, postgres_database=pg_manager_replica.get_default_database() - ) - - assert 1050 == int(instance.query(f"SELECT count() FROM test_database.{table}")) - assert 1050 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) - - pg_manager_replica.clear() - - def test_toast(started_cluster): table = "test_toast" pg_manager.execute( @@ -794,6 +749,58 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) ) +def test_replica_consumer(started_cluster): + table = "test_replica_consumer" + + pg_manager_replica = PostgresManager() + pg_manager_replica.init( + instance2, + cluster.postgres_ip, + cluster.postgres_port, + default_database="postgres_database", + postgres_db_exists=True, + ) + + for pm in [pg_manager, pg_manager_replica]: + pm.create_and_fill_postgres_table(table) + pm.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = '{table}'", + "materialized_postgresql_backoff_min_ms = 100", + "materialized_postgresql_backoff_max_ms = 100", + "materialized_postgresql_use_unique_replication_consumer_identifier = 1", + ], + ) + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + check_tables_are_synchronized( + instance2, table, postgres_database=pg_manager_replica.get_default_database() + ) + + assert 50 == int(instance.query(f"SELECT count() FROM test_database.{table}")) + assert 50 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) + + instance.query( + f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(1000, 1000)" + ) + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + check_tables_are_synchronized( + instance2, table, postgres_database=pg_manager_replica.get_default_database() + ) + + assert 1050 == int(instance.query(f"SELECT count() FROM test_database.{table}")) + assert 1050 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) + + pg_manager_replica.clear() + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 0ddee6ad6c2d201534761944ce182ac529194837 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 15 Oct 2023 14:44:57 +0200 Subject: [PATCH 282/634] Fix --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 33 ++++++++++++++----- src/Storages/S3Queue/S3QueueSource.cpp | 1 + 2 files changed, 25 insertions(+), 9 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index ac12e8fe70f..150174aabcb 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -326,8 +326,13 @@ std::pairerror != Coordination::Error::ZOK) + { return std::pair{SetFileProcessingResult::AlreadyProcessed, nullptr}; - - if (responses[1]->error != Coordination::Error::ZOK) + } + else if (responses[2]->error != Coordination::Error::ZOK) + { return std::pair{SetFileProcessingResult::AlreadyFailed, nullptr}; - - chassert(responses[2]->error != Coordination::Error::ZOK); - return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; + } + else if (responses[4]->error != Coordination::Error::ZOK) + { + return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected state of zookeeper transaction: {}", magic_enum::enum_name(code)); + } } std::pairerror != Coordination::Error::ZOK) + else if (responses[2]->error != Coordination::Error::ZOK) { LOG_TEST(log, "Skipping file `{}`: already processing", path); return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index b3f8f0cf1c3..1a44fe9cff8 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -149,6 +149,7 @@ Chunk StorageS3QueueSource::generate() auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); SCOPE_EXIT({ CurrentThread::get().attachProfileCountersScope(prev_scope); }); + /// FIXME: if files are compressed, profile counters update does not work fully (s3 related counters are not saved). Why? try { From 41479b21e60f85f953b4b5b52697efd1238ead31 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 15 Oct 2023 15:33:33 +0200 Subject: [PATCH 283/634] Restore only top-level parts instead of unexpected --- src/Storages/MergeTree/ActiveDataPartSet.cpp | 74 +++++++++-- src/Storages/MergeTree/ActiveDataPartSet.h | 11 ++ src/Storages/MergeTree/MergeTreeData.cpp | 130 +++++++++---------- 3 files changed, 136 insertions(+), 79 deletions(-) diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index 5b7965bc3a0..aac1705c3b9 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -21,18 +21,46 @@ ActiveDataPartSet::ActiveDataPartSet(MergeTreeDataFormatVersion format_version_, add(name); } -bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) +ActiveDataPartSet::AddPartOutcome ActiveDataPartSet::tryAddPart(const MergeTreePartInfo & part_info, String * out_reason) { - auto part_info = MergeTreePartInfo::fromPartName(name, format_version); - return add(part_info, name, out_replaced_parts); + return addImpl(part_info, part_info.getPartNameAndCheckFormat(format_version), nullptr, out_reason); } bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, const String & name, Strings * out_replaced_parts) +{ + String out_reason; + AddPartOutcome outcome = addImpl(part_info, name, out_replaced_parts, &out_reason); + if (outcome == AddPartOutcome::HasIntersectingPart) + { + chassert(out_reason.empty()); + throw Exception(ErrorCodes::LOGICAL_ERROR, fmt::runtime(out_reason)); + } + + return outcome == AddPartOutcome::Added; +} + + +bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) +{ + auto part_info = MergeTreePartInfo::fromPartName(name, format_version); + String out_reason; + AddPartOutcome outcome = addImpl(part_info, name, out_replaced_parts, &out_reason); + if (outcome == AddPartOutcome::HasIntersectingPart) + { + chassert(out_reason.empty()); + throw Exception(ErrorCodes::LOGICAL_ERROR, fmt::runtime(out_reason)); + } + + return outcome == AddPartOutcome::Added; +} + + +ActiveDataPartSet::AddPartOutcome ActiveDataPartSet::addImpl(const MergeTreePartInfo & part_info, const String & name, Strings * out_replaced_parts, String * out_reason) { /// TODO make it exception safe (out_replaced_parts->push_back(...) may throw) if (getContainingPartImpl(part_info) != part_info_to_name.end()) - return false; + return AddPartOutcome::HasCovering; /// Parts contained in `part` are located contiguously in `part_info_to_name`, overlapping with the place where the part itself would be inserted. auto it = part_info_to_name.lower_bound(part_info); @@ -47,10 +75,15 @@ bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, const String & if (!part_info.contains(it->first)) { if (!part_info.isDisjoint(it->first)) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Part {} intersects previous part {}. " - "It is a bug or a result of manual intervention in the ZooKeeper data.", - part_info.getPartNameForLogs(), it->first.getPartNameForLogs()); + { + if (out_reason != nullptr) + *out_reason = fmt::format( + "Part {} intersects previous part {}. " + "It is a bug or a result of manual intervention in the ZooKeeper data.", + part_info.getPartNameForLogs(), + it->first.getPartNameForLogs()); + return AddPartOutcome::HasIntersectingPart; + } ++it; break; } @@ -73,18 +106,33 @@ bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, const String & } if (it != part_info_to_name.end() && !part_info.isDisjoint(it->first)) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Part {} intersects part {}. It is a bug or a result of manual intervention " - "in the ZooKeeper data.", name, it->first.getPartNameForLogs()); + { + if (out_reason != nullptr) + *out_reason = fmt::format( + "Part {} intersects part {}. It is a bug or a result of manual intervention " + "in the ZooKeeper data.", + name, + it->first.getPartNameForLogs()); + + return AddPartOutcome::HasIntersectingPart; + } part_info_to_name.emplace(part_info, name); - return true; + return AddPartOutcome::Added; } bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, Strings * out_replaced_parts) { - return add(part_info, part_info.getPartNameAndCheckFormat(format_version), out_replaced_parts); + String out_reason; + AddPartOutcome outcome = addImpl(part_info, part_info.getPartNameAndCheckFormat(format_version), out_replaced_parts, &out_reason); + if (outcome == AddPartOutcome::HasIntersectingPart) + { + chassert(out_reason.empty()); + throw Exception(ErrorCodes::LOGICAL_ERROR, fmt::runtime(out_reason)); + } + + return outcome == AddPartOutcome::Added; } diff --git a/src/Storages/MergeTree/ActiveDataPartSet.h b/src/Storages/MergeTree/ActiveDataPartSet.h index f3cd6b0019d..662309f64cf 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.h +++ b/src/Storages/MergeTree/ActiveDataPartSet.h @@ -22,6 +22,13 @@ using Strings = std::vector; class ActiveDataPartSet { public: + enum class AddPartOutcome + { + Added, + HasCovering, + HasIntersectingPart, + }; + explicit ActiveDataPartSet(MergeTreeDataFormatVersion format_version_) : format_version(format_version_) {} ActiveDataPartSet(MergeTreeDataFormatVersion format_version_, const Strings & names); @@ -43,6 +50,8 @@ public: bool add(const MergeTreePartInfo & part_info, const String & name, Strings * out_replaced_parts = nullptr); bool add(const MergeTreePartInfo & part_info, Strings * out_replaced_parts = nullptr); + AddPartOutcome tryAddPart(const MergeTreePartInfo & part_info, String * out_reason = nullptr); + bool remove(const MergeTreePartInfo & part_info) { return part_info_to_name.erase(part_info) > 0; @@ -97,6 +106,8 @@ public: MergeTreeDataFormatVersion getFormatVersion() const { return format_version; } private: + + AddPartOutcome addImpl(const MergeTreePartInfo & part_info, const String & name, Strings * out_replaced_parts = nullptr, String * out_reason = nullptr); MergeTreeDataFormatVersion format_version; std::map part_info_to_name; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 889dcfa537f..fcfbe27c741 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -73,6 +73,7 @@ #include #include #include +#include #include #include #include @@ -3912,25 +3913,17 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT return; } + /// Let's restore some parts covered by unexpected to avoid partial data if (restore_covered) { Strings restored; - bool error = false; - String error_parts; - - Int64 pos = part->info.min_block; + Strings error_parts; auto is_appropriate_state = [] (DataPartState state) { return state == DataPartState::Active || state == DataPartState::Outdated; }; - auto update_error = [&] (DataPartIteratorByInfo it) - { - error = true; - error_parts += (*it)->getNameWithState() + " "; - }; - auto activate_part = [this, &restored_active_part](auto it) { /// It's not clear what to do if we try to activate part that was removed in transaction. @@ -3948,68 +3941,73 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT restored_active_part = true; }; - auto it_middle = data_parts_by_info.lower_bound(part->info); - - /// Restore the leftmost part covered by the part - if (it_middle != data_parts_by_info.begin()) + /// ActiveDataPartSet allows to restore most top-level parts instead of unexpected. + /// It can be important in case of assigned merges. If unexpected part is result of some + /// finished, but not committed merge we should restore closest ancestors for the + /// unexpected part to be able to execute it. + ActiveDataPartSet parts_for_replacement(format_version); + for (const auto & part_candidate_in_partition : getDataPartsPartitionRange(part->info.partition_id)) { - auto it = std::prev(it_middle); - - if (part->contains(**it) && is_appropriate_state((*it)->getState())) + if (part->info.contains(part_candidate_in_partition->info) + && is_appropriate_state(part_candidate_in_partition->getState())) { - /// Maybe, we must consider part level somehow - if ((*it)->info.min_block != part->info.min_block) - update_error(it); - - if ((*it)->getState() != DataPartState::Active) - activate_part(it); - - pos = (*it)->info.max_block + 1; - restored.push_back((*it)->name); + String out_reason; + /// Outdated parts can itersect legally (because of DROP_PART) here it's okay, we + /// are trying to do out best to restore covered parts. + auto outcome = parts_for_replacement.tryAddPart(part_candidate_in_partition->info, &out_reason); + if (outcome == ActiveDataPartSet::AddPartOutcome::HasIntersectingPart) + { + error_parts.push_back(part->name); + LOG_ERROR(log, "Failed to restore part {}, because of intersection reason '{}'", part->name, out_reason); + } + } + } + + if (parts_for_replacement.size() > 0) + { + std::vector> holes_list; + auto part_infos = parts_for_replacement.getPartInfos(); + int64_t current_right_block = part_infos[0].min_block; + for (const auto & top_level_part_to_replace : part_infos) + { + auto data_part_it = data_parts_by_info.find(top_level_part_to_replace); + if (data_part_it == data_parts_by_info.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find part {} in own set", top_level_part_to_replace.getPartNameForLogs()); + activate_part(data_part_it); + restored.push_back((*data_part_it)->name); + if (top_level_part_to_replace.min_block - current_right_block > 1) + holes_list.emplace_back(current_right_block, top_level_part_to_replace.min_block); + current_right_block = top_level_part_to_replace.max_block; + } + if (part->info.max_block != current_right_block) + holes_list.emplace_back(current_right_block, part->info.max_block); + + for (const String & name : restored) + LOG_INFO(log, "Activated part {} in place of unexpected {}", name, part->name); + + if (!error_parts.empty() || !holes_list.empty()) + { + std::string error_parts_message, holes_list_message; + if (!error_parts.empty()) + error_parts_message = fmt::format(" Parts failed to restore because of intersection: [{}]", fmt::join(error_parts, ", ")); + if (!holes_list.empty()) + { + if (!error_parts.empty()) + holes_list_message = "."; + + Strings holes_list_pairs; + for (const auto & [left_side, right_side] : holes_list) + holes_list_pairs.push_back(fmt::format("({}, {})", left_side + 1, right_side - 1)); + holes_list_message += fmt::format(" Block ranges failed to restore: [{}]", fmt::join(holes_list_pairs, ", ")); + } + LOG_WARNING(log, "The set of parts restored in place of {} looks incomplete. " + "SELECT queries may observe gaps in data until this replica is synchronized with other replicas.{}{}", + part->name, error_parts_message, holes_list_message); } - else if ((*it)->info.partition_id == part->info.partition_id) - update_error(it); - else - error = true; } else - error = true; - - /// Restore "right" parts - for (auto it = it_middle; it != data_parts_by_info.end() && part->contains(**it); ++it) { - if ((*it)->info.min_block < pos) - continue; - - if (!is_appropriate_state((*it)->getState())) - { - update_error(it); - continue; - } - - if ((*it)->info.min_block > pos) - update_error(it); - - if ((*it)->getState() != DataPartState::Active) - activate_part(it); - - pos = (*it)->info.max_block + 1; - restored.push_back((*it)->name); - } - - if (pos != part->info.max_block + 1) - error = true; - - for (const String & name : restored) - { - LOG_INFO(log, "Activated part {}", name); - } - - if (error) - { - LOG_WARNING(log, "The set of parts restored in place of {} looks incomplete. " - "SELECT queries may observe gaps in data until this replica is synchronized with other replicas.{}", - part->name, (error_parts.empty() ? "" : " Suspicious parts: " + error_parts)); + LOG_INFO(log, "Don't find any parts for replacement instead of unexpected {}", part->name); } } From 3b02748cb644f97c402bc3f4851adf52c0162927 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 15 Oct 2023 15:43:02 +0200 Subject: [PATCH 284/634] Fix some typos --- programs/server/Server.cpp | 2 +- src/Common/CaresPTRResolver.cpp | 2 +- src/Common/OptimizedRegularExpression.cpp | 2 +- src/Common/SpaceSaving.h | 2 +- src/Coordination/ZooKeeperDataReader.cpp | 2 +- src/DataTypes/ObjectUtils.cpp | 2 +- src/Disks/ObjectStorages/DiskObjectStorageMetadata.h | 2 +- src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp | 3 +-- src/Interpreters/Access/InterpreterGrantQuery.cpp | 2 +- src/Interpreters/ConvertStringsToEnumVisitor.cpp | 2 +- src/Interpreters/QueryAliasesVisitor.cpp | 2 +- src/Processors/Formats/Impl/Parquet/Write.cpp | 2 +- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 2 +- src/Storages/MergeTree/MergeTreeReaderCompact.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeQueue.h | 2 +- 16 files changed, 16 insertions(+), 17 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 57db5da6552..af460ccc7d9 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1474,7 +1474,7 @@ try { std::lock_guard lock(servers_lock); - /// We should start interserver communications before (and more imporant shutdown after) tables. + /// We should start interserver communications before (and more important shutdown after) tables. /// Because server can wait for a long-running queries (for example in tcp_handler) after interserver handler was already shut down. /// In this case we will have replicated tables which are unable to send any parts to other replicas, but still can /// communicate with zookeeper, execute merges, etc. diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp index bf61e940745..0261f4a130f 100644 --- a/src/Common/CaresPTRResolver.cpp +++ b/src/Common/CaresPTRResolver.cpp @@ -223,7 +223,7 @@ namespace DB void CaresPTRResolver::process_possible_timeout(ares_channel channel) { - /* Call ares_process() unconditonally here, even if we simply timed out + /* Call ares_process() unconditionally here, even if we simply timed out above, as otherwise the ares name resolve won't timeout! */ ares_process_fd(channel, ARES_SOCKET_BAD, ARES_SOCKET_BAD); } diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index d64b26a28a3..b566ffc6045 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -418,7 +418,7 @@ finish: /// this two vals are useless, xxx|xxx cannot be trivial nor prefix. bool next_is_trivial = true; pos = analyzeImpl(regexp, pos, required_substring, next_is_trivial, next_alternatives); - /// For xxx|xxx|xxx, we only conbine the alternatives and return a empty required_substring. + /// For xxx|xxx|xxx, we only combine the alternatives and return a empty required_substring. if (next_alternatives.empty() || shortest_literal_length(next_alternatives) < required_substring.literal.size()) { global_alternatives.push_back(required_substring); diff --git a/src/Common/SpaceSaving.h b/src/Common/SpaceSaving.h index 5d3e8fa321b..8649e703b46 100644 --- a/src/Common/SpaceSaving.h +++ b/src/Common/SpaceSaving.h @@ -321,7 +321,7 @@ protected: percolate(ptr); } - // This is equivallent to one step of bubble sort + // This is equivalent to one step of bubble sort void percolate(Counter * counter) { while (counter->slot > 0) diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 79929c4e66e..3c1550f08c8 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -284,7 +284,7 @@ void deserializeLogMagic(ReadBuffer & in) /// strange, that this 550 bytes obviously was a part of Create transaction, /// but the operation code was -1. We have added debug prints to original /// zookeeper (3.6.3) and found that it just reads 550 bytes of this "Error" -/// transaction, tooks the first 4 bytes as an error code (it was 79, non +/// transaction, took the first 4 bytes as an error code (it was 79, non /// existing code) and skip all remaining 546 bytes. NOTE: it looks like a bug /// in ZooKeeper. /// diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 28f000b6f0d..23d29136c85 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -339,7 +339,7 @@ static DataTypePtr getLeastCommonTypeForObject(const DataTypes & types, bool che for (size_t i = 1; i < subtypes.size(); ++i) if (first_dim != getNumberOfDimensions(*subtypes[i])) throw Exception(ErrorCodes::TYPE_MISMATCH, - "Uncompatible types of subcolumn '{}': {} and {}", + "Incompatible types of subcolumn '{}': {} and {}", key.getPath(), subtypes[0]->getName(), subtypes[i]->getName()); tuple_paths.emplace_back(key); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h index 6dced85d0b1..1abb829c12a 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h @@ -33,7 +33,7 @@ private: /// Number of references (hardlinks) to this metadata file. /// - /// FIXME: Why we are tracking it explicetly, without + /// FIXME: Why we are tracking it explicitly, without /// info from filesystem???? uint32_t ref_count = 0; diff --git a/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp b/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp index 6dd6ea67125..83c20701353 100644 --- a/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp +++ b/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp @@ -182,7 +182,7 @@ INSTANTIATE_TEST_SUITE_P(Basic, DateLUT::instance("Europe/Minsk") }, { - "When scale is 0, subsecond part (and separtor) is missing from string", + "When scale is 0, subsecond part (and separator) is missing from string", "2019-09-16 19:20:17", 1568650817ULL, 0, @@ -197,4 +197,3 @@ INSTANTIATE_TEST_SUITE_P(Basic, } }) ); - diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index 1a8268b9b1b..45e8ba9ea0d 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -166,7 +166,7 @@ namespace access_to_revoke.grant(elements_to_revoke); access_to_revoke.makeIntersection(all_granted_access); - /// Build more accurate list of elements to revoke, now we use an intesection of the initial list of elements to revoke + /// Build more accurate list of elements to revoke, now we use an intersection of the initial list of elements to revoke /// and all the granted access rights to these grantees. bool grant_option = !elements_to_revoke.empty() && elements_to_revoke[0].grant_option; elements_to_revoke.clear(); diff --git a/src/Interpreters/ConvertStringsToEnumVisitor.cpp b/src/Interpreters/ConvertStringsToEnumVisitor.cpp index b141f75fbec..7cc95dc521b 100644 --- a/src/Interpreters/ConvertStringsToEnumVisitor.cpp +++ b/src/Interpreters/ConvertStringsToEnumVisitor.cpp @@ -11,7 +11,7 @@ namespace DB namespace { -/// @note We place strings in ascending order here under the assumption it colud speed up String to Enum conversion. +/// @note We place strings in ascending order here under the assumption it could speed up String to Enum conversion. String makeStringsEnum(const std::set & values) { String enum_string = "Enum8("; diff --git a/src/Interpreters/QueryAliasesVisitor.cpp b/src/Interpreters/QueryAliasesVisitor.cpp index 148f1d305cc..345a27bc3ab 100644 --- a/src/Interpreters/QueryAliasesVisitor.cpp +++ b/src/Interpreters/QueryAliasesVisitor.cpp @@ -140,7 +140,7 @@ void QueryAliasesMatcher::visitOther(const ASTPtr & ast, Data & data) information for our ast node with query string. And this alias will be dropped because prefer_alias_to_column_name for ASTWIthAlias by default is false. - It is imporant that subquery can be converted to literal during ExecuteScalarSubqueriesVisitor. + It is important that subquery can be converted to literal during ExecuteScalarSubqueriesVisitor. And code below check if we previously set for subquery alias as _subquery, and if it is true then set prefer_alias_to_column_name = true for node that was optimized during ExecuteScalarSubqueriesVisitor. */ diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index 978787d07cd..340ed5f510c 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -226,7 +226,7 @@ struct StatisticsStringRef /// or [element of ColumnString] -> std::string_view. /// We do this conversion in small batches rather than all at once, just before encoding the batch, /// in hopes of getting better performance through cache locality. -/// The Coverter* structs below are responsible for that. +/// The Converter* structs below are responsible for that. /// When conversion is not needed, getBatch() will just return pointer into original data. template znode_name; switch (entry->type) { - /// explicetely specify all types of entries without default, so if + /// explicitly specify all types of entries without default, so if /// someone decide to add new type it will produce a compiler warning (error in our case) case LogEntryType::GET_PART: case LogEntryType::ATTACH_PART: diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index d5d85e58cb5..29204611200 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -240,7 +240,7 @@ private: /// by first argument. If remove_part == true, than also remove part itself. /// Both negative flags will throw exception. /// - /// Part removed from mutations which satisfy contitions: + /// Part removed from mutations which satisfy conditions: /// block_number > part.getDataVersion() /// or block_number == part.getDataVersion() /// ^ (this may happen if we downloaded mutated part from other replica) From f3056b16434acf7edb34044a073bc99ac04a1114 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 15 Oct 2023 18:02:34 +0200 Subject: [PATCH 285/634] Add a test --- .../DataLakes/IcebergMetadataParser.cpp | 7 ++- tests/integration/helpers/cluster.py | 2 +- .../integration/test_storage_iceberg/test.py | 54 +++++++++++++++++++ 3 files changed, 61 insertions(+), 2 deletions(-) diff --git a/src/Storages/DataLakes/IcebergMetadataParser.cpp b/src/Storages/DataLakes/IcebergMetadataParser.cpp index 666e046c169..250363d9210 100644 --- a/src/Storages/DataLakes/IcebergMetadataParser.cpp +++ b/src/Storages/DataLakes/IcebergMetadataParser.cpp @@ -32,6 +32,8 @@ namespace ErrorCodes template struct IcebergMetadataParser::Impl { + Poco::Logger * log = &Poco::Logger::get("IcebergMetadataParser"); + /** * Useful links: * - https://iceberg.apache.org/spec/ @@ -299,7 +301,10 @@ struct IcebergMetadataParser::Impl throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected to find {} in data path: {}", configuration.url.key, data_path); if (status == 2) - keys.erase(file_path); + { + LOG_TEST(log, "Got delete file for {}", file_path); + chassert(!keys.contains(file_path)); + } else keys.insert(file_path); } diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index dfd964b04d2..6054001bd94 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -32,7 +32,6 @@ try: import nats import ssl import meilisearch - import pyspark from confluent_kafka.avro.cached_schema_registry_client import ( CachedSchemaRegistryClient, ) @@ -631,6 +630,7 @@ class ClickHouseCluster: logging.debug(f"Removed :{self.instances_dir}") if with_spark: + import pyspark # if you change packages, don't forget to update them in docker/test/integration/runner/dockerd-entrypoint.sh ( pyspark.sql.SparkSession.builder.appName("spark_test") diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index c22b8cda9b5..3a5f1724793 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -313,3 +313,57 @@ def test_types(started_cluster, format_version): ["e", "Nullable(Bool)"], ] ) + + +@pytest.mark.parametrize("format_version", ["1", "2"]) +def test_delete_files(started_cluster, format_version): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + bucket = started_cluster.minio_bucket + TABLE_NAME = "test_delete_files_" + format_version + + write_iceberg_from_df( + spark, + generate_data(spark, 0, 100), + TABLE_NAME, + mode="overwrite", + format_version=format_version, + ) + + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + create_iceberg_table(instance, TABLE_NAME) + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + + spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 0") + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 0 + assert instance.contains_in_log("Got delete file for") + + write_iceberg_from_df( + spark, + generate_data(spark, 100, 200), + TABLE_NAME, + mode="upsert", + format_version=format_version, + ) + + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + + spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 150") + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 50 From 17ce951ed79debc5757de6807d71111b629d54a3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 15 Oct 2023 18:12:01 +0200 Subject: [PATCH 286/634] Update test.py --- .../test_postgresql_replica_database_engine_2/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 7bdec76e4eb..82730c34664 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -798,6 +798,8 @@ def test_replica_consumer(started_cluster): assert 1050 == int(instance.query(f"SELECT count() FROM test_database.{table}")) assert 1050 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) + for pm in [pg_manager, pg_manager_replica]: + pm.drop_materialized_db() pg_manager_replica.clear() From 6ac790e91831ece6b34718cc717aef7bf645f7e4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 15 Oct 2023 20:17:46 +0200 Subject: [PATCH 287/634] Update src/Storages/MergeTree/ActiveDataPartSet.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/ActiveDataPartSet.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index aac1705c3b9..c192e3484b9 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -128,7 +128,7 @@ bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, Strings * out_r AddPartOutcome outcome = addImpl(part_info, part_info.getPartNameAndCheckFormat(format_version), out_replaced_parts, &out_reason); if (outcome == AddPartOutcome::HasIntersectingPart) { - chassert(out_reason.empty()); + chassert(!out_reason.empty()); throw Exception(ErrorCodes::LOGICAL_ERROR, fmt::runtime(out_reason)); } From 13ca29649706d396c1b595f1fab2e41c05315819 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 15 Oct 2023 20:17:50 +0200 Subject: [PATCH 288/634] Update src/Storages/MergeTree/ActiveDataPartSet.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/ActiveDataPartSet.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index c192e3484b9..67f7004cd70 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -47,7 +47,7 @@ bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) AddPartOutcome outcome = addImpl(part_info, name, out_replaced_parts, &out_reason); if (outcome == AddPartOutcome::HasIntersectingPart) { - chassert(out_reason.empty()); + chassert(!out_reason.empty()); throw Exception(ErrorCodes::LOGICAL_ERROR, fmt::runtime(out_reason)); } From bdafe17e9a486caac402067869375636fb1bf7b7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 15 Oct 2023 20:17:55 +0200 Subject: [PATCH 289/634] Update src/Storages/MergeTree/ActiveDataPartSet.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/ActiveDataPartSet.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index 67f7004cd70..a1746cc1746 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -32,7 +32,7 @@ bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, const String & AddPartOutcome outcome = addImpl(part_info, name, out_replaced_parts, &out_reason); if (outcome == AddPartOutcome::HasIntersectingPart) { - chassert(out_reason.empty()); + chassert(!out_reason.empty()); throw Exception(ErrorCodes::LOGICAL_ERROR, fmt::runtime(out_reason)); } From 98af58d7cf5d6e22999c2ba121ffe03080e507e8 Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Mon, 16 Oct 2023 10:02:31 +0800 Subject: [PATCH 290/634] fix test error --- tests/integration/test_keeper_client/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index e01c463d94a..92b5b95dc50 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -232,4 +232,4 @@ def test_get_all_children_number(client: KeeperClient): client.touch("/test_get_all_children_number/2/3") client.touch("/test_get_all_children_number/2/4") - assert client.get_all_children_number("/test_get_all_children_number") == ["11"] + assert client.get_all_children_number("/test_get_all_children_number") == "11" From 95be5c7c07a5a397752b3e3e7aa574be5e7209f2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 16 Oct 2023 04:57:42 +0200 Subject: [PATCH 291/634] Show move/maximize only if there is more than a single chart --- programs/server/dashboard.html | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 555d039cec3..123ab7b9cc4 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -420,6 +420,10 @@ color: var(--auth-error-color); } + #charts > div:only-child .display-only-if-more-than-one-chart { + display: none; + } + /* Source: https://cdn.jsdelivr.net/npm/uplot@1.6.21/dist/uPlot.min.css * It is copy-pasted to lower the number of requests. */ @@ -931,6 +935,9 @@ function insertChart(i) { saveState(); }); + move.classList.add('display-only-if-more-than-one-chart'); + maximize.classList.add('display-only-if-more-than-one-chart'); + edit_buttons.appendChild(move); edit_buttons.appendChild(maximize); edit_buttons.appendChild(edit); From 750d45bad381388400f6994295f69a97c51e5775 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 16 Oct 2023 05:25:59 +0200 Subject: [PATCH 292/634] Add support for touch devices --- programs/server/dashboard.html | 144 +++++++++++++++++---------------- 1 file changed, 76 insertions(+), 68 deletions(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 555d039cec3..0c311fb8250 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -817,77 +817,85 @@ function insertChart(i) { let move_text = document.createTextNode('✥'); move.appendChild(move_text); - let is_dragging = false; - move.addEventListener('mousedown', e => { - const idx = getCurrentIndex(); - is_dragging = true; + let drag_state = { + is_dragging: false, + idx: null, + offset_x: null, + offset_y: null, + displace_idx: null, + displace_chart: null + }; + + function dragStop(e) { + drag_state.is_dragging = false; + chart.className = 'chart'; + chart.style.left = null; + chart.style.top = null; + + if (drag_state.displace_idx !== null) { + const elem = queries[drag_state.idx]; + queries.splice(drag_state.idx, 1); + queries.splice(drag_state.displace_idx, 0, elem); + + drag_state.displace_chart.className = 'chart'; + drawAll(); + } + } + + function dragMove(e) { + if (!drag_state.is_dragging) return; + + let x = e.clientX - drag_state.offset_x; + let y = e.clientY - drag_state.offset_y; + + chart.style.left = `${x}px`; + chart.style.top = `${y}px`; + + drag_state.displace_idx = null; + drag_state.displace_chart = null; + let current_idx = -1; + for (const elem of charts.querySelectorAll('.chart')) { + ++current_idx; + if (current_idx == drag_state.idx) { + continue; + } + + const this_rect = chart.getBoundingClientRect(); + const this_center_x = this_rect.left + this_rect.width / 2; + const this_center_y = this_rect.top + this_rect.height / 2; + + const elem_rect = elem.getBoundingClientRect(); + + if (this_center_x >= elem_rect.left && this_center_x <= elem_rect.right + && this_center_y >= elem_rect.top && this_center_y <= elem_rect.bottom) { + + elem.className = 'chart chart-displaced'; + drag_state.displace_idx = current_idx; + drag_state.displace_chart = elem; + } else { + elem.className = 'chart'; + } + } + } + + function dragStart(e) { + if (e.button !== 0) return; /// left button only + move.setPointerCapture(e.pointerId); + + drag_state.is_dragging = true; + drag_state.idx = getCurrentIndex(); chart.className = 'chart chart-moving'; - let offset_x = e.clientX; - let offset_y = e.clientY; + drag_state.offset_x = e.clientX; + drag_state.offset_y = e.clientY; + } - let displace_idx = null; - let displace_chart = null; - - function mouseup(e) { - is_dragging = false; - chart.className = 'chart'; - chart.style.left = null; - chart.style.top = null; - - if (displace_idx !== null) { - const elem = queries[idx]; - queries.splice(idx, 1); - queries.splice(displace_idx, 0, elem); - - displace_chart.className = 'chart'; - drawAll(); - } - } - - function mousemove(e) { - if (!is_dragging) { - document.body.removeEventListener('mousemove', mousemove); - document.body.removeEventListener('mouseup', mouseup); - return; - } - - let x = e.clientX - offset_x; - let y = e.clientY - offset_y; - - chart.style.left = `${x}px`; - chart.style.top = `${y}px`; - - displace_idx = null; - displace_chart = null; - let current_idx = -1; - for (const elem of charts.querySelectorAll('.chart')) { - ++current_idx; - if (current_idx == idx) { - continue; - } - - const this_rect = chart.getBoundingClientRect(); - const this_center_x = this_rect.left + this_rect.width / 2; - const this_center_y = this_rect.top + this_rect.height / 2; - - const elem_rect = elem.getBoundingClientRect(); - - if (this_center_x >= elem_rect.left && this_center_x <= elem_rect.right - && this_center_y >= elem_rect.top && this_center_y <= elem_rect.bottom) { - - elem.className = 'chart chart-displaced'; - displace_idx = current_idx; - displace_chart = elem; - } else { - elem.className = 'chart'; - } - } - } - - document.body.addEventListener('mouseup', mouseup); - document.body.addEventListener('mousemove', mousemove); - }); + /// Read https://www.redblobgames.com/making-of/draggable/ + move.addEventListener('pointerdown', dragStart); + move.addEventListener('pointermove', dragMove); + move.addEventListener('pointerup', dragStop); + move.addEventListener('pointerancel', dragStop); + move.addEventListener('touchstart', (e) => e.preventDefault()); let maximize = document.createElement('a'); let maximize_text = document.createTextNode('🗖'); From fb901c24a19a41de9bc1b27bcfc6bcdcc8b03057 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 16 Oct 2023 09:45:49 +0200 Subject: [PATCH 293/634] Revert "Fix filtering by virtual columns with OR filter in query" --- src/Storages/VirtualColumnUtils.cpp | 55 ++++--------------- .../02840_merge__table_or_filter.sql.j2 | 5 -- .../0_stateless/02896_multiple_OR.reference | 14 ----- .../queries/0_stateless/02896_multiple_OR.sql | 28 ---------- 4 files changed, 10 insertions(+), 92 deletions(-) delete mode 100644 tests/queries/0_stateless/02896_multiple_OR.reference delete mode 100644 tests/queries/0_stateless/02896_multiple_OR.sql diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 219043f25c6..d0d6233728e 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -82,33 +81,14 @@ bool extractFunctions(const ASTPtr & expression, const std::functionname == "or") { - bool ret = false; + bool ret = true; ASTs or_args; for (const auto & child : function->arguments->children) - ret |= extractFunctions(child, is_constant, or_args); - - if (!or_args.empty()) - { - /// In case of there are less number of arguments for which - /// is_constant() == true, we need to add always-true - /// implicitly to avoid breaking AND invariant. - /// - /// Consider the following: - /// - /// ((value = 10) OR (_table = 'v2')) AND ((_table = 'v1') OR (value = 20)) - /// - /// Without implicit always-true: - /// - /// (_table = 'v2') AND (_table = 'v1') - /// - /// With: - /// - /// (_table = 'v2' OR 1) AND (_table = 'v1' OR 1) -> (_table = 'v2') OR (_table = 'v1') - /// - if (or_args.size() != function->arguments->children.size()) - or_args.push_back(std::make_shared(Field(1))); + ret &= extractFunctions(child, is_constant, or_args); + /// We can keep condition only if it still OR condition (i.e. we + /// have dependent conditions for columns at both sides) + if (or_args.size() == 2) result.push_back(makeASTForLogicalOr(std::move(or_args))); - } return ret; } } @@ -185,10 +165,8 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block if (!select.where() && !select.prewhere()) return unmodified; - // Provide input columns as constant columns to check if an expression is - // constant and depends on the columns from provided block (the last is - // required to allow skipping some conditions for handling OR). - std::function is_constant = [&block, &context](const ASTPtr & expr) + // Provide input columns as constant columns to check if an expression is constant. + std::function is_constant = [&block, &context](const ASTPtr & node) { auto actions = std::make_shared(block.getColumnsWithTypeAndName()); PreparedSetsPtr prepared_sets = std::make_shared(); @@ -200,26 +178,13 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block context, SizeLimits{}, 1, source_columns, std::move(actions), prepared_sets, true, true, true, { aggregation_keys, grouping_set_keys, GroupByKind::NONE }); - ActionsVisitor(visitor_data).visit(expr); + ActionsVisitor(visitor_data).visit(node); actions = visitor_data.getActions(); - auto expr_column_name = expr->getColumnName(); - - const auto * expr_const_node = actions->tryFindInOutputs(expr_column_name); - if (!expr_const_node) - return false; - auto filter_actions = ActionsDAG::buildFilterActionsDAG({expr_const_node}, {}, context); - const auto & nodes = filter_actions->getNodes(); - bool has_dependent_columns = std::any_of(nodes.begin(), nodes.end(), [&](const auto & node) - { - return block.has(node.result_name); - }); - if (!has_dependent_columns) - return false; - auto expression_actions = std::make_shared(actions); auto block_with_constants = block; expression_actions->execute(block_with_constants); - return block_with_constants.has(expr_column_name) && isColumnConst(*block_with_constants.getByName(expr_column_name).column); + auto column_name = node->getColumnName(); + return block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column); }; /// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. diff --git a/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 b/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 index 286e4545ef7..a87ef7302c6 100644 --- a/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 +++ b/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 @@ -18,11 +18,6 @@ create view v2 as select * from d2; create table m as v1 engine=Merge(currentDatabase(), '^(v1|v2)$'); -{# -- FIXME: -select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') or 0 or 0 settings {{ settings }}; -select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') or 0 or 0 settings {{ settings }}; -#} - -- avoid reorder set max_threads=1; -- { echoOn } diff --git a/tests/queries/0_stateless/02896_multiple_OR.reference b/tests/queries/0_stateless/02896_multiple_OR.reference deleted file mode 100644 index 96480a75d11..00000000000 --- a/tests/queries/0_stateless/02896_multiple_OR.reference +++ /dev/null @@ -1,14 +0,0 @@ --- { echoOn } -SELECT * FROM or_bug WHERE (key = 1) OR false OR false; -1 -SELECT * FROM or_bug WHERE (key = 1) OR false; -1 -SELECT * FROM or_bug WHERE (key = 1); -1 --- { echoOn } -select * from forms where text_field like '%this%' or 0 = 1 or 0 = 1; -5840ead423829c1eab29fa97 this is a test -select * from forms where text_field like '%this%' or 0 = 1; -5840ead423829c1eab29fa97 this is a test -select * from forms where text_field like '%this%'; -5840ead423829c1eab29fa97 this is a test diff --git a/tests/queries/0_stateless/02896_multiple_OR.sql b/tests/queries/0_stateless/02896_multiple_OR.sql deleted file mode 100644 index 653ddebca1f..00000000000 --- a/tests/queries/0_stateless/02896_multiple_OR.sql +++ /dev/null @@ -1,28 +0,0 @@ --- https://github.com/ClickHouse/ClickHouse/pull/52653 -DROP TABLE IF EXISTS or_bug; -CREATE TABLE or_bug (key UInt8) ENGINE=MergeTree ORDER BY key; -INSERT INTO TABLE or_bug VALUES (0), (1); - --- { echoOn } -SELECT * FROM or_bug WHERE (key = 1) OR false OR false; -SELECT * FROM or_bug WHERE (key = 1) OR false; -SELECT * FROM or_bug WHERE (key = 1); --- { echoOff } - --- https://github.com/ClickHouse/ClickHouse/issues/55288 -DROP TABLE IF EXISTS forms; -CREATE TABLE forms -( - `form_id` FixedString(24), - `text_field` String -) -ENGINE = MergeTree -PRIMARY KEY form_id -ORDER BY form_id; -insert into forms values ('5840ead423829c1eab29fa97','this is a test'); - --- { echoOn } -select * from forms where text_field like '%this%' or 0 = 1 or 0 = 1; -select * from forms where text_field like '%this%' or 0 = 1; -select * from forms where text_field like '%this%'; --- { echoOff } From 26a8aa441cf81a4dd5a329e0fc0a0de0b6f6dbe3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Oct 2023 10:24:54 +0200 Subject: [PATCH 294/634] Better shutdown handling --- src/Storages/StorageReplicatedMergeTree.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0485fe687e8..91c004e0883 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5763,6 +5763,9 @@ void StorageReplicatedMergeTree::alter( while (true) { + if (shutdown_called || partial_shutdown_called) + throw Exception(ErrorCodes::ABORTED, "Cannot assing alter because shutdown called"); + bool pulled_queue = false; std::optional maybe_mutations_version_after_logs_pull; std::map unfinished_mutations; @@ -5770,6 +5773,9 @@ void StorageReplicatedMergeTree::alter( { if (command.isDropSomething()) { + if (shutdown_called || partial_shutdown_called) + throw Exception(ErrorCodes::ABORTED, "Cannot assing alter because shutdown called"); + if (!pulled_queue) { auto [_, mutations_version] = queue.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::SYNC); From 2d9812b9e9611e7083a082dca0134db76a4933ef Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Oct 2023 10:43:25 +0200 Subject: [PATCH 295/634] Fix headers --- src/Storages/MergeTree/MergeTreeData.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d00da850a59..808ee8533a2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -50,13 +50,11 @@ #include #include #include -#include -#include #include -#include #include -#include #include +#include +#include #include #include #include @@ -65,6 +63,7 @@ #include #include #include +#include #include #include #include From dbebdc3dcad9521d48d9141748ae198c0338375c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Oct 2023 11:05:59 +0200 Subject: [PATCH 296/634] Fix comment --- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 91c004e0883..f76a9a95ba2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5764,7 +5764,7 @@ void StorageReplicatedMergeTree::alter( while (true) { if (shutdown_called || partial_shutdown_called) - throw Exception(ErrorCodes::ABORTED, "Cannot assing alter because shutdown called"); + throw Exception(ErrorCodes::ABORTED, "Cannot assign alter because shutdown called"); bool pulled_queue = false; std::optional maybe_mutations_version_after_logs_pull; @@ -5774,7 +5774,7 @@ void StorageReplicatedMergeTree::alter( if (command.isDropSomething()) { if (shutdown_called || partial_shutdown_called) - throw Exception(ErrorCodes::ABORTED, "Cannot assing alter because shutdown called"); + throw Exception(ErrorCodes::ABORTED, "Cannot assign alter because shutdown called"); if (!pulled_queue) { From d7c77420b6539e73946fd58f4591ae93a3cc1a9f Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 16 Oct 2023 10:33:04 +0000 Subject: [PATCH 297/634] QueryNormalizer only checks stack size --- src/Interpreters/QueryNormalizer.cpp | 2 -- tests/queries/0_stateless/02896_cyclic_aliases_crash.sql | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 6b6ead1f463..f47635a3c3f 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -120,7 +120,6 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// In a construct like "a AS b", where a is an alias, you must set alias b to the result of substituting alias a. /// Check size of the alias before cloning too large alias AST alias_node->checkSize(data.settings.max_expanded_ast_elements); - current_asts.insert(alias_node.get()); ast = alias_node->clone(); ast->setAlias(node_alias); @@ -139,7 +138,6 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// Check size of the alias before cloning too large alias AST alias_node->checkSize(data.settings.max_expanded_ast_elements); auto alias_name = ast->getAliasOrColumnName(); - current_asts.insert(alias_node.get()); ast = alias_node->clone(); ast->setAlias(alias_name); diff --git a/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql b/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql index af54f5df7a7..76eff95cf31 100644 --- a/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql +++ b/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql @@ -6,7 +6,7 @@ SELECT val + 1 as prev, val + prev as val FROM ( SELECT 1 as val ) -; -- { serverError CYCLIC_ALIASES } +; -- { serverError CYCLIC_ALIASES, TOO_DEEP_RECURSION } SELECT From f1c316b79b302a075fbf13332408505b27167b53 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 11 Oct 2023 11:30:48 +0000 Subject: [PATCH 298/634] ci script to use git hub api urls, readme for test --- tests/README.md | 218 ++++++++++++++++++++++++++++++ tests/ci/build_download_helper.py | 1 + tests/ci/pr_info.py | 35 ++--- tests/ci/worker/prepare-ci-ami.sh | 4 +- 4 files changed, 235 insertions(+), 23 deletions(-) create mode 100644 tests/README.md diff --git a/tests/README.md b/tests/README.md new file mode 100644 index 00000000000..9273ee00c58 --- /dev/null +++ b/tests/README.md @@ -0,0 +1,218 @@ +# [WIP] Running CI tests & checks locally + +## Style Check + +#### in docker +```sh +# use docker to run all checks at once and avoid package installation: +mkdir /tmp/test_output +docker run --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE clickhouse/style-test + +# run certain check (e.g.: check-duplicate-includes.sh) +docker run --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE --entrypoint= -w/ClickHouse/utils/check-style clickhouse/style-test ./check-duplicate-includes.sh +``` + +#### on a host +```sh +# refer to ./docker/test/style/Dockerfile whenever needed, to check required packages and installation commands + +cd ./utils/check-style + +# Check duplicate includes: +./check-duplicate-includes.sh + +# Check style +./check-style + +# Check python formatting with black +./check-black + +# Check python type hinting with mypy +./check-mypy + +# Check typos +./check-typos + +# Check docs spelling +./check-doc-aspell + +# Check whitespaces +./check-whitespaces + +# Check workflows +./check-workflows + +# Check submodules +./check-submodules + +# Check shell scripts with shellcheck +./shellcheck-run.sh +``` + +## Fast Tests: + +FastTest CI job builds minimal CH version without extra libraries and runs limited number of Functional tests to check for a smoke + +#### Building minimal CH +```sh +# following git commands suggests updating only necesarry submodule for minimal CH build +# you can use your normal CH build for Fast Tests, though these commands give the idea of how CH is being built for Fast Tests in CI +git clone --depth 1 https://github.com/ClickHouse/ClickHouse.git +SUBMODULES_TO_UPDATE=( + contrib/sysroot + contrib/magic_enum + contrib/abseil-cpp + contrib/boost + contrib/zlib-ng + contrib/libxml2 + contrib/libunwind + contrib/fmtlib + contrib/aklomp-base64 + contrib/cctz + contrib/libcpuid + contrib/libdivide + contrib/double-conversion + contrib/llvm-project + contrib/lz4 + contrib/zstd + contrib/fastops + contrib/rapidjson + contrib/re2 + contrib/sparsehash-c11 + contrib/croaring + contrib/miniselect + contrib/xz + contrib/dragonbox + contrib/fast_float + contrib/NuRaft + contrib/jemalloc + contrib/replxx + contrib/wyhash + contrib/c-ares + contrib/morton-nd + contrib/xxHash + contrib/simdjson + contrib/liburing + contrib/libfiu + contrib/incbin + contrib/yaml-cpp +) +git submodule sync +git submodule init +# --jobs does not work as fast as real parallel running +printf '%s\0' "${SUBMODULES_TO_UPDATE[@]}" | \ + xargs --max-procs=100 --null --no-run-if-empty --max-args=1 \ + git submodule update --depth 1 --single-branch + +# configuring build +LLVM_VERSION=${LLVM_VERSION:-17} + +CMAKE_LIBS_CONFIG=( + "-DENABLE_LIBRARIES=0" + "-DENABLE_TESTS=0" + "-DENABLE_UTILS=0" + "-DENABLE_EMBEDDED_COMPILER=0" + "-DENABLE_THINLTO=0" + "-DENABLE_NURAFT=1" + "-DENABLE_SIMDJSON=1" + "-DENABLE_JEMALLOC=1" + "-DENABLE_LIBURING=1" + "-DENABLE_YAML_CPP=1" +) +mkdir build +cd build +cmake -DCMAKE_CXX_COMPILER="clang++-${LLVM_VERSION}" -DCMAKE_C_COMPILER="clang-${LLVM_VERSION}" "${CMAKE_LIBS_CONFIG[@]}" .. +ninja clickhouse-bundle +cd .. +``` + +#### Running Fast (Smoke, Functional) Tests +```sh +# prepare test CH configuration - adjust file locations if necessary +mkdir -p /tmp/etc/server +mkdir -p /tmp/etc/client +cp programs/server/{config,users}.xml /tmp/etc/server/ +./tests/config/install.sh /tmp/etc/server/ /tmp/etc/client/ +cp programs/server/config.d/log_to_console.xml /tmp/etc/server/config.d/ +rm /tmp/etc/server/config.d/secure_ports.xml +sudo mkdir /var/lib/clickhouse +sudo chmod 777 /var/lib/clickhouse + +# prepare server command options +opts=( + --config-file /tmp/etc/server/config.xml + --pid-file /tmp/etc/server/clickhouse-server.pid + -- + --path /tmp/etc/server + --user_files_path /tmp/etc/server/user_files + --top_level_domains_path /tmp/etc/server/top_level_domains + --keeper_server.storage_path /tmp/etc/server/coordination +) +# start server +./build/programs/clickhouse-server "${opts[@]}" + +# start a new terminal for tests +# prepare test runner command options +test_opts=( + --hung-check + --fast-tests-only + --no-random-settings + --no-random-merge-tree-settings + --no-long + --testname + --shard + --zookeeper + --check-zookeeper-session + --order random + --print-time + --report-logs-stats +) +# start test runner +cd ./tests; +time ./clickhouse-test -c ../build/programs "${test_opts[@]}" + +``` + + +## Functional Tests + +Tests are comprised in Test Suites (TS) and located in `./tests/queries/_/*`. + +Each Test Case (TC) represented by two obligatory files: + * test script or queries: `./_.` + * test expected values: `./_.reference` + +To run tests use a test runner: +```sh +cd ./tests +./clickhouse-test [--help] [test ...] +# e.g.: +# Running all tests: +# ./clickhouse-test +# Running tests that match given pattern +# ./clickhouse-test hash_table +# See all runner options in help menu +``` + +## Static analysis with clang-tidy build + +#### in docker +```sh +mkdir build_tidy + +# configure cmake +docker run --rm --volume=.:/build -u $(id -u ${USER}):$(id -g ${USER}) --entrypoint= -w /build/build-tidy clickhouse/binary-builder:latest cmake -DCMAKE_C_COMPILER=clang-17 -DCMAKE_CXX_COMPILER=clang++-17 -LA -DCMAKE_BUILD_TYPE=Debug -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DCOMPILER_CACHE=ccache -DENABLE_CLANG_TIDY=1 -DENABLE_TESTS=1 -DENABLE_EXAMPLES=1 -DENABLE_UTILS=1 -DENABLE_BUILD_PROFILING=1 -DCLICKHOUSE_OFFICIAL_BUILD=1 .. + +# build +docker run --rm --volume=.:/build -u $(id -u ${USER}):$(id -g ${USER}) --entrypoint= -w /build/build-tidy clickhouse/binary-builder:latest ninja -k0 all +``` + +#### on a host +```sh +# install dependencies if not yet +# sudo apt-get install clang-tidy-17 + +mkdir build_tidy && cd build_tidy +cmake --debug-trycompile -DCMAKE_CXX_COMPILER="clang++-17" -DCMAKE_C_COMPILER="clang-17" -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=Debug -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DCMAKE_C_COMPILER=clang-17 -DCMAKE_CXX_COMPILER=clang++-17 -DCOMPILER_CACHE=ccache -DENABLE_CLANG_TIDY=1 -DENABLE_TESTS=1 -DENABLE_EXAMPLES=1 -DENABLE_UTILS=1 -DENABLE_BUILD_PROFILING=1 -DCLICKHOUSE_OFFICIAL_BUILD=1 .. +ninja all +``` diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index b76c5433142..50d82512369 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -89,6 +89,7 @@ def get_gh_api( set_auth_header() need_retry = True break + raise exc if need_retry: return get_with_retries(url, retries, sleep, **kwargs) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 830aa936bea..a3604f4c573 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -4,8 +4,6 @@ import logging import os from typing import Dict, List, Set, Union, Literal -from unidiff import PatchSet # type: ignore - from build_download_helper import get_gh_api from env_helper import ( GITHUB_REPOSITORY, @@ -171,7 +169,10 @@ class PRInfo: response_json = user_orgs_response.json() self.user_orgs = set(org["id"] for org in response_json) - self.diff_urls.append(github_event["pull_request"]["diff_url"]) + self.diff_urls.append( + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/master...{self.head_ref}" + ) elif "commits" in github_event: # `head_commit` always comes with `commits` commit_message = github_event["head_commit"]["message"] # type: str @@ -215,12 +216,12 @@ class PRInfo: # files changed in upstream AND master...{self.head_ref} # to get files, changed in current HEAD self.diff_urls.append( - f"https://github.com/{GITHUB_REPOSITORY}/" - f"compare/master...{self.head_ref}.diff" + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/master...{self.head_ref}" ) self.diff_urls.append( - f"https://github.com/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master.diff" + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/{self.head_ref}...master" ) # Get release PR number. self.release_pr = get_pr_for_commit(self.base_ref, self.base_ref)[ @@ -232,8 +233,8 @@ class PRInfo: # For release PRs we must get not only files changed in the PR # itself, but as well files changed since we branched out self.diff_urls.append( - f"https://github.com/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master.diff" + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/{self.head_ref}...master" ) else: print("event.json does not match pull_request or push:") @@ -261,19 +262,11 @@ class PRInfo: raise TypeError("The event does not have diff URLs") for diff_url in self.diff_urls: - response = get_gh_api( - diff_url, - sleep=RETRY_SLEEP, - ) + response = get_gh_api(diff_url, sleep=RETRY_SLEEP) response.raise_for_status() - if "commits" in self.event and self.number == 0: - diff = response.json() - - if "files" in diff: - self.changed_files = {f["filename"] for f in diff["files"]} - else: - diff_object = PatchSet(response.text) - self.changed_files.update({f.path for f in diff_object}) + diff = response.json() + if "files" in diff: + self.changed_files = {f["filename"] for f in diff["files"]} print(f"Fetched info about {len(self.changed_files)} changed files") def get_dict(self): diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index 20e7e3fd53e..dc180207be3 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -9,7 +9,7 @@ set -xeuo pipefail echo "Running prepare script" export DEBIAN_FRONTEND=noninteractive -export RUNNER_VERSION=2.304.0 +export RUNNER_VERSION=2.310.1 export RUNNER_HOME=/home/ubuntu/actions-runner deb_arch() { @@ -90,7 +90,7 @@ systemctl restart docker sudo -u ubuntu docker buildx version sudo -u ubuntu docker buildx create --use --name default-builder -pip install boto3 pygithub requests urllib3 unidiff dohq-artifactory +pip install boto3 pygithub requests urllib3 dohq-artifactory mkdir -p $RUNNER_HOME && cd $RUNNER_HOME From 346e8a01626a90653e2d7b2d433a04b9ffac41ba Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 11 Oct 2023 14:46:00 +0200 Subject: [PATCH 299/634] Remove recursion in favor of reset retry loop --- tests/ci/build_download_helper.py | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 50d82512369..123d0e40731 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -71,28 +71,30 @@ def get_gh_api( if grt.ROBOT_TOKEN is not None: set_auth_header() - need_retry = False - for _ in range(retries): + header_is_set = "Authorization" in kwargs.get("headers", {}) + retry = 1 + exc = Exception("placeholder") + while retry <= retries: try: + retry += 1 response = get_with_retries(url, 1, sleep, **kwargs) response.raise_for_status() return response - except requests.HTTPError as exc: + except requests.HTTPError as e: + exc = e if ( - exc.response.status_code == 403 + e.response.status_code == 403 and b"rate limit exceeded" - in exc.response._content # pylint:disable=protected-access + in e.response._content # pylint:disable=protected-access + and not header_is_set ): logging.warning( "Received rate limit exception, setting the auth header and retry" ) set_auth_header() - need_retry = True - break - raise exc + retry = 1 - if need_retry: - return get_with_retries(url, retries, sleep, **kwargs) + raise exc def get_build_name_for_check(check_name: str) -> str: From c5ebd6f072abe495ea6fc193d90c2340ef20cf6a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 11 Oct 2023 16:31:58 +0200 Subject: [PATCH 300/634] Use requests.gte in get_gh_api --- tests/ci/build_download_helper.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 123d0e40731..eeb91cf5258 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -77,7 +77,7 @@ def get_gh_api( while retry <= retries: try: retry += 1 - response = get_with_retries(url, 1, sleep, **kwargs) + response = requests.get(url, **kwargs) response.raise_for_status() return response except requests.HTTPError as e: @@ -93,6 +93,12 @@ def get_gh_api( ) set_auth_header() retry = 1 + elif retry < retries: + time.sleep(sleep) + except Exception as e: + exc = e + if retry < retries: + time.sleep(sleep) raise exc From 3f3fcce7e5f317c454baf4b33db45e3009517444 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 11 Oct 2023 16:00:48 +0000 Subject: [PATCH 301/634] gh api request func update --- docker/test/style/Dockerfile | 2 +- tests/ci/build_download_helper.py | 31 +++++++++++++------------------ tests/ci/pr_info.py | 9 +++++---- 3 files changed, 19 insertions(+), 23 deletions(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index a4feae27c67..de2c1c64938 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -19,7 +19,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ shellcheck \ yamllint \ locales \ - && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub unidiff pylint==2.6.2 \ + && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub pylint==2.6.2 \ && apt-get clean \ && rm -rf /root/.cache/pip diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index eeb91cf5258..f856ff4e30d 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -51,10 +51,8 @@ def get_gh_api( sleep: int = 3, **kwargs: Any, ) -> requests.Response: - """It's a wrapper around get_with_retries that requests GH api w/o auth by - default, and falls back to the get_best_robot_token in case of receiving - "403 rate limit exceeded" error - It sets auth automatically when ROBOT_TOKEN is already set by get_best_robot_token + """ + get gh api with retries and failover to another token if ratelimit is exceeded """ def set_auth_header(): @@ -71,12 +69,10 @@ def get_gh_api( if grt.ROBOT_TOKEN is not None: set_auth_header() - header_is_set = "Authorization" in kwargs.get("headers", {}) - retry = 1 - exc = Exception("placeholder") - while retry <= retries: + token_is_set = "Authorization" in kwargs.get("headers", {}) + exc = Exception("A placeholder to satisfy typing and avoid nesting") + for i in range(retries): try: - retry += 1 response = requests.get(url, **kwargs) response.raise_for_status() return response @@ -84,21 +80,20 @@ def get_gh_api( exc = e if ( e.response.status_code == 403 - and b"rate limit exceeded" - in e.response._content # pylint:disable=protected-access - and not header_is_set + and b"rate limit exceeded" in e.response._content # pylint:disable=protected-access + and token_is_set ): logging.warning( - "Received rate limit exception, setting the auth header and retry" + "Received rate limit exception, re-setting the auth header and retry" ) set_auth_header() - retry = 1 - elif retry < retries: - time.sleep(sleep) + continue except Exception as e: exc = e - if retry < retries: - time.sleep(sleep) + + if i + 1 < retries: + logging.info("Exception '%s' while getting, retry %i", exc, i + 1) + time.sleep(sleep) raise exc diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index a3604f4c573..7dbfe124760 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -169,10 +169,11 @@ class PRInfo: response_json = user_orgs_response.json() self.user_orgs = set(org["id"] for org in response_json) - self.diff_urls.append( - f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/master...{self.head_ref}" - ) + self.diff_urls.append( + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/master...{self.head_ref}" + ) + elif "commits" in github_event: # `head_commit` always comes with `commits` commit_message = github_event["head_commit"]["message"] # type: str From 7c2fee3280cab16dde06ea4a25d24248c9a12e9d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 11 Oct 2023 16:15:22 +0000 Subject: [PATCH 302/634] Automatic style fix --- tests/ci/build_download_helper.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index f856ff4e30d..f09f03fc3d0 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -80,7 +80,8 @@ def get_gh_api( exc = e if ( e.response.status_code == 403 - and b"rate limit exceeded" in e.response._content # pylint:disable=protected-access + and b"rate limit exceeded" + in e.response._content # pylint:disable=protected-access and token_is_set ): logging.warning( From 60ad1c79bd078aab0c903b9e403cac12146631bf Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 11 Oct 2023 17:25:36 +0000 Subject: [PATCH 303/634] fix docs --- docs/en/development/continuous-integration.md | 137 +++++++++++ tests/README.md | 219 +----------------- tests/ci/build_download_helper.py | 23 +- 3 files changed, 152 insertions(+), 227 deletions(-) diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index eec5ccbb9dc..94ffa43c956 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -67,6 +67,51 @@ This check means that the CI system started to process the pull request. When it Performs some simple regex-based checks of code style, using the [`utils/check-style/check-style`](https://github.com/ClickHouse/ClickHouse/blob/master/utils/check-style/check-style) binary (note that it can be run locally). If it fails, fix the style errors following the [code style guide](style.md). +#### Running localy in docker: +```sh +mkdir /tmp/test_output + +# running all checks +docker run --rm --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE clickhouse/style-test + +# run certain check (e.g.: ./check-mypy) +docker run --rm --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE --entrypoint= -w/ClickHouse/utils/check-style clickhouse/style-test ./check-mypy +``` + +#### Running localy on a host: +```sh +cd ./utils/check-style + +# Check duplicate includes +./check-duplicate-includes.sh + +# Check style +./check-style + +# Check python formatting with black +./check-black + +# Check python type hinting with mypy +./check-mypy + +# Check typos +./check-typos + +# Check docs spelling +./check-doc-aspell + +# Check whitespaces +./check-whitespaces + +# Check workflows +./check-workflows + +# Check submodules +./check-submodules + +# Check shell scripts with shellcheck +./shellcheck-run.sh +``` ## Fast Test Normally this is the first check that is ran for a PR. It builds ClickHouse and @@ -75,6 +120,77 @@ some. If it fails, further checks are not started until it is fixed. Look at the report to see which tests fail, then reproduce the failure locally as described [here](tests.md#functional-test-locally). +Fast test job builds minimal CH, if you for watever reason need the same ClickHouse build, follow the instructions: +```sh +git clone --depth 1 https://github.com/ClickHouse/ClickHouse.git +SUBMODULES_TO_UPDATE=( + contrib/sysroot + contrib/magic_enum + contrib/abseil-cpp + contrib/boost + contrib/zlib-ng + contrib/libxml2 + contrib/libunwind + contrib/fmtlib + contrib/aklomp-base64 + contrib/cctz + contrib/libcpuid + contrib/libdivide + contrib/double-conversion + contrib/llvm-project + contrib/lz4 + contrib/zstd + contrib/fastops + contrib/rapidjson + contrib/re2 + contrib/sparsehash-c11 + contrib/croaring + contrib/miniselect + contrib/xz + contrib/dragonbox + contrib/fast_float + contrib/NuRaft + contrib/jemalloc + contrib/replxx + contrib/wyhash + contrib/c-ares + contrib/morton-nd + contrib/xxHash + contrib/simdjson + contrib/liburing + contrib/libfiu + contrib/incbin + contrib/yaml-cpp +) +git submodule sync +git submodule init +# --jobs does not work as fast as real parallel running +printf '%s\0' "${SUBMODULES_TO_UPDATE[@]}" | \ + xargs --max-procs=100 --null --no-run-if-empty --max-args=1 \ + git submodule update --depth 1 --single-branch + +# configuring build +LLVM_VERSION=${LLVM_VERSION:-17} + +CMAKE_LIBS_CONFIG=( + "-DENABLE_LIBRARIES=0" + "-DENABLE_TESTS=0" + "-DENABLE_UTILS=0" + "-DENABLE_EMBEDDED_COMPILER=0" + "-DENABLE_THINLTO=0" + "-DENABLE_NURAFT=1" + "-DENABLE_SIMDJSON=1" + "-DENABLE_JEMALLOC=1" + "-DENABLE_LIBURING=1" + "-DENABLE_YAML_CPP=1" +) +mkdir build +cd build +cmake -DCMAKE_CXX_COMPILER="clang++-${LLVM_VERSION}" -DCMAKE_C_COMPILER="clang-${LLVM_VERSION}" "${CMAKE_LIBS_CONFIG[@]}" .. +ninja clickhouse-bundle +cd .. +``` + #### Status Page Files - `runlog.out.log` is the general log that includes all other logs. @@ -122,6 +238,27 @@ Builds ClickHouse in various configurations for use in further steps. You have t ## Special Build Check Performs static analysis and code style checks using `clang-tidy`. The report is similar to the [build check](#build-check). Fix the errors found in the build log. +#### Running localy in docker: +```sh +mkdir build_tidy + +# configure cmake +docker run --rm --volume=.:/build -u $(id -u ${USER}):$(id -g ${USER}) --entrypoint= -w /build/build-tidy clickhouse/binary-builder:latest cmake -DCMAKE_C_COMPILER=clang-17 -DCMAKE_CXX_COMPILER=clang++-17 -LA -DCMAKE_BUILD_TYPE=Debug -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DCOMPILER_CACHE=ccache -DENABLE_CLANG_TIDY=1 -DENABLE_TESTS=1 -DENABLE_EXAMPLES=1 -DENABLE_UTILS=1 -DENABLE_BUILD_PROFILING=1 -DCLICKHOUSE_OFFICIAL_BUILD=1 .. + +# build +docker run --rm --volume=.:/build -u $(id -u ${USER}):$(id -g ${USER}) --entrypoint= -w /build/build-tidy clickhouse/binary-builder:latest ninja -k0 all +``` + +#### Running localy on a host: +```sh +# install dependencies if not yet +# sudo apt-get install clang-tidy-17 + +mkdir build_tidy && cd build_tidy +cmake --debug-trycompile -DCMAKE_CXX_COMPILER="clang++-17" -DCMAKE_C_COMPILER="clang-17" -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=Debug -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DCMAKE_C_COMPILER=clang-17 -DCMAKE_CXX_COMPILER=clang++-17 -DCOMPILER_CACHE=ccache -DENABLE_CLANG_TIDY=1 -DENABLE_TESTS=1 -DENABLE_EXAMPLES=1 -DENABLE_UTILS=1 -DENABLE_BUILD_PROFILING=1 -DCLICKHOUSE_OFFICIAL_BUILD=1 .. +ninja all +``` + ## Functional Stateless Tests Runs [stateless functional tests](tests.md#functional-tests) for ClickHouse diff --git a/tests/README.md b/tests/README.md index 9273ee00c58..0ca9b110f28 100644 --- a/tests/README.md +++ b/tests/README.md @@ -1,218 +1 @@ -# [WIP] Running CI tests & checks locally - -## Style Check - -#### in docker -```sh -# use docker to run all checks at once and avoid package installation: -mkdir /tmp/test_output -docker run --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE clickhouse/style-test - -# run certain check (e.g.: check-duplicate-includes.sh) -docker run --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE --entrypoint= -w/ClickHouse/utils/check-style clickhouse/style-test ./check-duplicate-includes.sh -``` - -#### on a host -```sh -# refer to ./docker/test/style/Dockerfile whenever needed, to check required packages and installation commands - -cd ./utils/check-style - -# Check duplicate includes: -./check-duplicate-includes.sh - -# Check style -./check-style - -# Check python formatting with black -./check-black - -# Check python type hinting with mypy -./check-mypy - -# Check typos -./check-typos - -# Check docs spelling -./check-doc-aspell - -# Check whitespaces -./check-whitespaces - -# Check workflows -./check-workflows - -# Check submodules -./check-submodules - -# Check shell scripts with shellcheck -./shellcheck-run.sh -``` - -## Fast Tests: - -FastTest CI job builds minimal CH version without extra libraries and runs limited number of Functional tests to check for a smoke - -#### Building minimal CH -```sh -# following git commands suggests updating only necesarry submodule for minimal CH build -# you can use your normal CH build for Fast Tests, though these commands give the idea of how CH is being built for Fast Tests in CI -git clone --depth 1 https://github.com/ClickHouse/ClickHouse.git -SUBMODULES_TO_UPDATE=( - contrib/sysroot - contrib/magic_enum - contrib/abseil-cpp - contrib/boost - contrib/zlib-ng - contrib/libxml2 - contrib/libunwind - contrib/fmtlib - contrib/aklomp-base64 - contrib/cctz - contrib/libcpuid - contrib/libdivide - contrib/double-conversion - contrib/llvm-project - contrib/lz4 - contrib/zstd - contrib/fastops - contrib/rapidjson - contrib/re2 - contrib/sparsehash-c11 - contrib/croaring - contrib/miniselect - contrib/xz - contrib/dragonbox - contrib/fast_float - contrib/NuRaft - contrib/jemalloc - contrib/replxx - contrib/wyhash - contrib/c-ares - contrib/morton-nd - contrib/xxHash - contrib/simdjson - contrib/liburing - contrib/libfiu - contrib/incbin - contrib/yaml-cpp -) -git submodule sync -git submodule init -# --jobs does not work as fast as real parallel running -printf '%s\0' "${SUBMODULES_TO_UPDATE[@]}" | \ - xargs --max-procs=100 --null --no-run-if-empty --max-args=1 \ - git submodule update --depth 1 --single-branch - -# configuring build -LLVM_VERSION=${LLVM_VERSION:-17} - -CMAKE_LIBS_CONFIG=( - "-DENABLE_LIBRARIES=0" - "-DENABLE_TESTS=0" - "-DENABLE_UTILS=0" - "-DENABLE_EMBEDDED_COMPILER=0" - "-DENABLE_THINLTO=0" - "-DENABLE_NURAFT=1" - "-DENABLE_SIMDJSON=1" - "-DENABLE_JEMALLOC=1" - "-DENABLE_LIBURING=1" - "-DENABLE_YAML_CPP=1" -) -mkdir build -cd build -cmake -DCMAKE_CXX_COMPILER="clang++-${LLVM_VERSION}" -DCMAKE_C_COMPILER="clang-${LLVM_VERSION}" "${CMAKE_LIBS_CONFIG[@]}" .. -ninja clickhouse-bundle -cd .. -``` - -#### Running Fast (Smoke, Functional) Tests -```sh -# prepare test CH configuration - adjust file locations if necessary -mkdir -p /tmp/etc/server -mkdir -p /tmp/etc/client -cp programs/server/{config,users}.xml /tmp/etc/server/ -./tests/config/install.sh /tmp/etc/server/ /tmp/etc/client/ -cp programs/server/config.d/log_to_console.xml /tmp/etc/server/config.d/ -rm /tmp/etc/server/config.d/secure_ports.xml -sudo mkdir /var/lib/clickhouse -sudo chmod 777 /var/lib/clickhouse - -# prepare server command options -opts=( - --config-file /tmp/etc/server/config.xml - --pid-file /tmp/etc/server/clickhouse-server.pid - -- - --path /tmp/etc/server - --user_files_path /tmp/etc/server/user_files - --top_level_domains_path /tmp/etc/server/top_level_domains - --keeper_server.storage_path /tmp/etc/server/coordination -) -# start server -./build/programs/clickhouse-server "${opts[@]}" - -# start a new terminal for tests -# prepare test runner command options -test_opts=( - --hung-check - --fast-tests-only - --no-random-settings - --no-random-merge-tree-settings - --no-long - --testname - --shard - --zookeeper - --check-zookeeper-session - --order random - --print-time - --report-logs-stats -) -# start test runner -cd ./tests; -time ./clickhouse-test -c ../build/programs "${test_opts[@]}" - -``` - - -## Functional Tests - -Tests are comprised in Test Suites (TS) and located in `./tests/queries/_/*`. - -Each Test Case (TC) represented by two obligatory files: - * test script or queries: `./_.` - * test expected values: `./_.reference` - -To run tests use a test runner: -```sh -cd ./tests -./clickhouse-test [--help] [test ...] -# e.g.: -# Running all tests: -# ./clickhouse-test -# Running tests that match given pattern -# ./clickhouse-test hash_table -# See all runner options in help menu -``` - -## Static analysis with clang-tidy build - -#### in docker -```sh -mkdir build_tidy - -# configure cmake -docker run --rm --volume=.:/build -u $(id -u ${USER}):$(id -g ${USER}) --entrypoint= -w /build/build-tidy clickhouse/binary-builder:latest cmake -DCMAKE_C_COMPILER=clang-17 -DCMAKE_CXX_COMPILER=clang++-17 -LA -DCMAKE_BUILD_TYPE=Debug -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DCOMPILER_CACHE=ccache -DENABLE_CLANG_TIDY=1 -DENABLE_TESTS=1 -DENABLE_EXAMPLES=1 -DENABLE_UTILS=1 -DENABLE_BUILD_PROFILING=1 -DCLICKHOUSE_OFFICIAL_BUILD=1 .. - -# build -docker run --rm --volume=.:/build -u $(id -u ${USER}):$(id -g ${USER}) --entrypoint= -w /build/build-tidy clickhouse/binary-builder:latest ninja -k0 all -``` - -#### on a host -```sh -# install dependencies if not yet -# sudo apt-get install clang-tidy-17 - -mkdir build_tidy && cd build_tidy -cmake --debug-trycompile -DCMAKE_CXX_COMPILER="clang++-17" -DCMAKE_C_COMPILER="clang-17" -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=Debug -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DCMAKE_C_COMPILER=clang-17 -DCMAKE_CXX_COMPILER=clang++-17 -DCOMPILER_CACHE=ccache -DENABLE_CLANG_TIDY=1 -DENABLE_TESTS=1 -DENABLE_EXAMPLES=1 -DENABLE_UTILS=1 -DENABLE_BUILD_PROFILING=1 -DCLICKHOUSE_OFFICIAL_BUILD=1 .. -ninja all -``` +Find CI documents and instructions on running CI checks localy in [here](https://clickhouse.com/docs/en/development/continuous-integration). \ No newline at end of file diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index f09f03fc3d0..5c4019d0cea 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -52,7 +52,9 @@ def get_gh_api( **kwargs: Any, ) -> requests.Response: """ - get gh api with retries and failover to another token if ratelimit is exceeded + Request GH api w/o auth by default, and failover to the get_best_robot_token in case of receiving + "403 rate limit exceeded" or "404 not found" error + It sets auth automatically when ROBOT_TOKEN is already set by get_best_robot_token """ def set_auth_header(): @@ -71,29 +73,32 @@ def get_gh_api( token_is_set = "Authorization" in kwargs.get("headers", {}) exc = Exception("A placeholder to satisfy typing and avoid nesting") - for i in range(retries): + try_cnt = 0 + while try_cnt < retries: + try_cnt += 1 try: response = requests.get(url, **kwargs) response.raise_for_status() return response except requests.HTTPError as e: exc = e - if ( - e.response.status_code == 403 - and b"rate limit exceeded" + ratelimit_exceeded = ( + e.response.status_code == 403 and b"rate limit exceeded" in e.response._content # pylint:disable=protected-access - and token_is_set - ): + ) + try_auth = e.response.status_code == 404 + if (ratelimit_exceeded or try_auth) and not token_is_set: logging.warning( "Received rate limit exception, re-setting the auth header and retry" ) set_auth_header() + try_cnt = 0 continue except Exception as e: exc = e - if i + 1 < retries: - logging.info("Exception '%s' while getting, retry %i", exc, i + 1) + if try_cnt < retries: + logging.info("Exception '%s' while getting, retry %i", exc, try_cnt) time.sleep(sleep) raise exc From b0b6e7fd47d86a4de7a05501260ab9d05f060fc4 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 11 Oct 2023 17:32:45 +0000 Subject: [PATCH 304/634] fix docs --- tests/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/README.md b/tests/README.md index 0ca9b110f28..a1fc0f530f2 100644 --- a/tests/README.md +++ b/tests/README.md @@ -1 +1 @@ -Find CI documents and instructions on running CI checks localy in [here](https://clickhouse.com/docs/en/development/continuous-integration). \ No newline at end of file +Find CI documents and instructions on running CI checks localy [here](https://clickhouse.com/docs/en/development/continuous-integration). \ No newline at end of file From 683437c61c994e545ffe5a285b7ba689a947c395 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 12 Oct 2023 17:32:11 +0000 Subject: [PATCH 305/634] fix docs and script --- docs/en/development/continuous-integration.md | 111 +++--------------- tests/ci/build_download_helper.py | 6 +- 2 files changed, 20 insertions(+), 97 deletions(-) diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index 94ffa43c956..46a30f56f11 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -67,25 +67,22 @@ This check means that the CI system started to process the pull request. When it Performs some simple regex-based checks of code style, using the [`utils/check-style/check-style`](https://github.com/ClickHouse/ClickHouse/blob/master/utils/check-style/check-style) binary (note that it can be run locally). If it fails, fix the style errors following the [code style guide](style.md). -#### Running localy in docker: +#### Running style check locally: ```sh -mkdir /tmp/test_output - +mkdir -p /tmp/test_output # running all checks docker run --rm --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE clickhouse/style-test -# run certain check (e.g.: ./check-mypy) +# run specified check script (e.g.: ./check-mypy) docker run --rm --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE --entrypoint= -w/ClickHouse/utils/check-style clickhouse/style-test ./check-mypy -``` -#### Running localy on a host: -```sh +# find all style check scripts under the directory: cd ./utils/check-style # Check duplicate includes ./check-duplicate-includes.sh -# Check style +# Check c++ formatiing ./check-style # Check python formatting with black @@ -94,7 +91,7 @@ cd ./utils/check-style # Check python type hinting with mypy ./check-mypy -# Check typos +# Check code with codespell ./check-typos # Check docs spelling @@ -103,7 +100,7 @@ cd ./utils/check-style # Check whitespaces ./check-whitespaces -# Check workflows +# Check github actions workflows ./check-workflows # Check submodules @@ -120,75 +117,13 @@ some. If it fails, further checks are not started until it is fixed. Look at the report to see which tests fail, then reproduce the failure locally as described [here](tests.md#functional-test-locally). -Fast test job builds minimal CH, if you for watever reason need the same ClickHouse build, follow the instructions: +#### Running Fast Test locally: ```sh -git clone --depth 1 https://github.com/ClickHouse/ClickHouse.git -SUBMODULES_TO_UPDATE=( - contrib/sysroot - contrib/magic_enum - contrib/abseil-cpp - contrib/boost - contrib/zlib-ng - contrib/libxml2 - contrib/libunwind - contrib/fmtlib - contrib/aklomp-base64 - contrib/cctz - contrib/libcpuid - contrib/libdivide - contrib/double-conversion - contrib/llvm-project - contrib/lz4 - contrib/zstd - contrib/fastops - contrib/rapidjson - contrib/re2 - contrib/sparsehash-c11 - contrib/croaring - contrib/miniselect - contrib/xz - contrib/dragonbox - contrib/fast_float - contrib/NuRaft - contrib/jemalloc - contrib/replxx - contrib/wyhash - contrib/c-ares - contrib/morton-nd - contrib/xxHash - contrib/simdjson - contrib/liburing - contrib/libfiu - contrib/incbin - contrib/yaml-cpp -) -git submodule sync -git submodule init -# --jobs does not work as fast as real parallel running -printf '%s\0' "${SUBMODULES_TO_UPDATE[@]}" | \ - xargs --max-procs=100 --null --no-run-if-empty --max-args=1 \ - git submodule update --depth 1 --single-branch - -# configuring build -LLVM_VERSION=${LLVM_VERSION:-17} - -CMAKE_LIBS_CONFIG=( - "-DENABLE_LIBRARIES=0" - "-DENABLE_TESTS=0" - "-DENABLE_UTILS=0" - "-DENABLE_EMBEDDED_COMPILER=0" - "-DENABLE_THINLTO=0" - "-DENABLE_NURAFT=1" - "-DENABLE_SIMDJSON=1" - "-DENABLE_JEMALLOC=1" - "-DENABLE_LIBURING=1" - "-DENABLE_YAML_CPP=1" -) -mkdir build -cd build -cmake -DCMAKE_CXX_COMPILER="clang++-${LLVM_VERSION}" -DCMAKE_C_COMPILER="clang-${LLVM_VERSION}" "${CMAKE_LIBS_CONFIG[@]}" .. -ninja clickhouse-bundle -cd .. +mkdir -p /tmp/test_output +mkdir -p /tmp/fasttest-workspace +cd ClickHouse +# this docker command performs minimal ClickHouse build and run FastTests against it +docker run --rm --cap-add=SYS_PTRACE -u $(id -u ${USER}):$(id -g ${USER}) --network=host -e FASTTEST_WORKSPACE=/fasttest-workspace -e FASTTEST_OUTPUT=/test_output -e FASTTEST_SOURCE=/ClickHouse --cap-add=SYS_PTRACE -e stage=clone_submodules --volume=/tmp/fasttest-workspace:/fasttest-workspace --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output clickhouse/fasttest ``` @@ -238,25 +173,11 @@ Builds ClickHouse in various configurations for use in further steps. You have t ## Special Build Check Performs static analysis and code style checks using `clang-tidy`. The report is similar to the [build check](#build-check). Fix the errors found in the build log. -#### Running localy in docker: +#### Running clang-tidy locally: +There is a convenience `packager` script that runs the clang-tidy build in docker ```sh mkdir build_tidy - -# configure cmake -docker run --rm --volume=.:/build -u $(id -u ${USER}):$(id -g ${USER}) --entrypoint= -w /build/build-tidy clickhouse/binary-builder:latest cmake -DCMAKE_C_COMPILER=clang-17 -DCMAKE_CXX_COMPILER=clang++-17 -LA -DCMAKE_BUILD_TYPE=Debug -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DCOMPILER_CACHE=ccache -DENABLE_CLANG_TIDY=1 -DENABLE_TESTS=1 -DENABLE_EXAMPLES=1 -DENABLE_UTILS=1 -DENABLE_BUILD_PROFILING=1 -DCLICKHOUSE_OFFICIAL_BUILD=1 .. - -# build -docker run --rm --volume=.:/build -u $(id -u ${USER}):$(id -g ${USER}) --entrypoint= -w /build/build-tidy clickhouse/binary-builder:latest ninja -k0 all -``` - -#### Running localy on a host: -```sh -# install dependencies if not yet -# sudo apt-get install clang-tidy-17 - -mkdir build_tidy && cd build_tidy -cmake --debug-trycompile -DCMAKE_CXX_COMPILER="clang++-17" -DCMAKE_C_COMPILER="clang-17" -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=Debug -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DCMAKE_C_COMPILER=clang-17 -DCMAKE_CXX_COMPILER=clang++-17 -DCOMPILER_CACHE=ccache -DENABLE_CLANG_TIDY=1 -DENABLE_TESTS=1 -DENABLE_EXAMPLES=1 -DENABLE_UTILS=1 -DENABLE_BUILD_PROFILING=1 -DCLICKHOUSE_OFFICIAL_BUILD=1 .. -ninja all +./docker/packager/packager --output-dir=./build_tidy --package-type=binary --compiler=clang-17 --debug-build --clang-tidy ``` diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 5c4019d0cea..21012f6337d 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -83,15 +83,17 @@ def get_gh_api( except requests.HTTPError as e: exc = e ratelimit_exceeded = ( - e.response.status_code == 403 and b"rate limit exceeded" + e.response.status_code == 403 + and b"rate limit exceeded" in e.response._content # pylint:disable=protected-access ) try_auth = e.response.status_code == 404 if (ratelimit_exceeded or try_auth) and not token_is_set: logging.warning( - "Received rate limit exception, re-setting the auth header and retry" + "Received rate limit exception, setting the auth header and retry" ) set_auth_header() + token_is_set = True try_cnt = 0 continue except Exception as e: From 836e35b6c45a38093d6d2cc6594a18471aa73ce3 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 16 Oct 2023 12:38:10 +0200 Subject: [PATCH 306/634] Fix progress bar for s3 and azure Cluster functions with url without globs --- src/Storages/StorageS3.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 95f0dc308a6..eb15ee36ced 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -207,6 +207,16 @@ private: { auto answer = *buffer_iter; ++buffer_iter; + + /// If url doesn't contain globs, we didn't list s3 bucket and didn't get object info for the key. + /// So we get object info lazily here on 'next()' request. + if (!answer.info) + { + answer.info = S3::getObjectInfo(*client, globbed_uri.bucket, answer.key, globbed_uri.version_id, request_settings); + if (file_progress_callback) + file_progress_callback(FileProgress(0, answer.info->size)); + } + return answer; } From 6dcc75694860073c3a2b4436ab78c65e3ea4abbe Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 16 Oct 2023 12:38:55 +0200 Subject: [PATCH 307/634] Fix for azure --- src/Storages/StorageAzureBlob.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 9ad76657024..a4a686b2691 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -847,6 +847,8 @@ StorageAzureBlobSource::GlobIterator::GlobIterator( blobs_with_metadata.emplace_back(blob_path_with_globs, object_metadata); if (outer_blobs) outer_blobs->emplace_back(blobs_with_metadata.back()); + if (file_progress_callback) + file_progress_callback(FileProgress(0, object_metadata.size_bytes)); is_finished = true; return; } From cf1deb7bd5dc7fa1e36e18ac66e8d11149d54e47 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 12 Oct 2023 14:40:41 +0000 Subject: [PATCH 308/634] Fix 'Invalid cursor state' in odbc interacting with MS SQL Server --- programs/odbc-bridge/ColumnInfoHandler.cpp | 8 ++++++++ programs/odbc-bridge/ODBCPooledConnectionFactory.h | 13 +++++++------ 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index 147ba43a51d..77ca46c9c88 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -146,6 +146,10 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ { catalog_name = tables.table_catalog(); LOG_TRACE(log, "Will fetch info for table '{}.{}'", catalog_name, table_name); + while (tables.next()) + { + /// drain the iterator before next operation to avoid "Invalid cursor state" + } return catalog.find_columns(/* column = */ "", table_name, /* schema = */ "", catalog_name); } @@ -154,6 +158,10 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ { catalog_name = tables.table_catalog(); LOG_TRACE(log, "Will fetch info for table '{}.{}.{}'", catalog_name, schema_name, table_name); + while (tables.next()) + { + /// drain the iterator before next operation to avoid "Invalid cursor state" + } return catalog.find_columns(/* column = */ "", table_name, schema_name, catalog_name); } diff --git a/programs/odbc-bridge/ODBCPooledConnectionFactory.h b/programs/odbc-bridge/ODBCPooledConnectionFactory.h index e425dea47f7..a10055c6659 100644 --- a/programs/odbc-bridge/ODBCPooledConnectionFactory.h +++ b/programs/odbc-bridge/ODBCPooledConnectionFactory.h @@ -91,16 +91,17 @@ T execute(nanodbc::ConnectionHolderPtr connection_holder, std::functionupdateConnection(); return query_func(connection_holder->get()); From aae3894c23ff8bccac282a40cdadb71808fb3127 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 13 Oct 2023 10:50:36 +0000 Subject: [PATCH 309/634] Throw an exception in odbc-bridge if more than one table matched a single query --- programs/odbc-bridge/ColumnInfoHandler.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index 77ca46c9c88..434abf0bf14 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -145,11 +145,11 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ if (tables.next()) { catalog_name = tables.table_catalog(); + /// `tables.next()` call is mandatory to drain the iterator before next operation and avoid "Invalid cursor state" + if (tables.next()) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Driver returned more than one table for '{}': '{}' and '{}'", + table_name, catalog_name, tables.table_schema()); LOG_TRACE(log, "Will fetch info for table '{}.{}'", catalog_name, table_name); - while (tables.next()) - { - /// drain the iterator before next operation to avoid "Invalid cursor state" - } return catalog.find_columns(/* column = */ "", table_name, /* schema = */ "", catalog_name); } @@ -157,11 +157,11 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ if (tables.next()) { catalog_name = tables.table_catalog(); + /// `tables.next()` call is mandatory to drain the iterator before next operation and avoid "Invalid cursor state" + if (tables.next()) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Driver returned more than one table for '{}': '{}' and '{}'", + table_name, catalog_name, tables.table_schema()); LOG_TRACE(log, "Will fetch info for table '{}.{}.{}'", catalog_name, schema_name, table_name); - while (tables.next()) - { - /// drain the iterator before next operation to avoid "Invalid cursor state" - } return catalog.find_columns(/* column = */ "", table_name, schema_name, catalog_name); } From 3cd175d11d355dbbae64665f9693ff3dfa8096fc Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 16 Oct 2023 13:01:36 +0200 Subject: [PATCH 310/634] Fix test --- .../DatabaseMaterializedPostgreSQL.cpp | 33 ++++++++++++------ .../DatabaseMaterializedPostgreSQL.h | 1 + tests/integration/helpers/postgres_utility.py | 2 +- .../test.py | 34 ++++++++++--------- 4 files changed, 42 insertions(+), 28 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index c03add1a96f..6400d1e45f9 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -52,10 +52,29 @@ DatabaseMaterializedPostgreSQL::DatabaseMaterializedPostgreSQL( , remote_database_name(postgres_database_name) , connection_info(connection_info_) , settings(std::move(settings_)) - , startup_task(getContext()->getSchedulePool().createTask("MaterializedPostgreSQLDatabaseStartup", [this]{ startSynchronization(); })) + , startup_task(getContext()->getSchedulePool().createTask("MaterializedPostgreSQLDatabaseStartup", [this]{ tryStartSynchronization(); })) { } +void DatabaseMaterializedPostgreSQL::tryStartSynchronization() +{ + if (shutdown_called) + return; + + try + { + startSynchronization(); + LOG_INFO(log, "Successfully loaded tables from PostgreSQL and started replication"); + } + catch (...) + { + LOG_ERROR(log, "Failed to start replication from PostgreSQL, " + "will retry. Error: {}", getCurrentExceptionMessage(true)); + + if (!shutdown_called) + startup_task->scheduleAfter(5000); + } +} void DatabaseMaterializedPostgreSQL::startSynchronization() { @@ -115,15 +134,7 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() LOG_TRACE(log, "Loaded {} tables. Starting synchronization", materialized_tables.size()); - try - { - replication_handler->startup(/* delayed */false); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - throw; - } + replication_handler->startup(/* delayed */false); } @@ -402,6 +413,7 @@ void DatabaseMaterializedPostgreSQL::detachTablePermanently(ContextPtr, const St void DatabaseMaterializedPostgreSQL::shutdown() { + shutdown_called = true; startup_task->deactivate(); stopReplication(); DatabaseAtomic::shutdown(); @@ -414,7 +426,6 @@ void DatabaseMaterializedPostgreSQL::stopReplication() if (replication_handler) replication_handler->shutdown(); - shutdown_called = true; /// Clear wrappers over nested, all access is not done to nested tables directly. materialized_tables.clear(); } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index edbef281da4..838516fa6f6 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -73,6 +73,7 @@ protected: ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr local_context, bool throw_on_error) const override; private: + void tryStartSynchronization(); void startSynchronization(); ASTPtr createAlterSettingsQuery(const SettingChange & new_setting); diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 77dc72f8020..9b431317b4a 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -193,7 +193,7 @@ class PostgresManager: def drop_postgres_db(self, database_name=""): database_name = self.database_or_default(database_name) - self.cursor.execute(f"DROP DATABASE IF EXISTS {database_name}") + self.cursor.execute(f"DROP DATABASE IF EXISTS {database_name} WITH (FORCE)") if database_name in self.created_postgres_db_list: self.created_postgres_db_list.remove(database_name) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 82730c34664..e72e5f179f5 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -57,6 +57,7 @@ instance2 = cluster.add_instance( pg_manager = PostgresManager() pg_manager2 = PostgresManager() +pg_manager_instance2 = PostgresManager() @pytest.fixture(scope="module") @@ -69,9 +70,17 @@ def started_cluster(): cluster.postgres_port, default_database="postgres_database", ) + pg_manager_instance2.init( + instance2, + cluster.postgres_ip, + cluster.postgres_port, + default_database="postgres_database", + postgres_db_exists=True, + ) pg_manager2.init( instance2, cluster.postgres_ip, cluster.postgres_port, "postgres_database2" ) + yield cluster finally: @@ -721,8 +730,10 @@ def test_too_many_parts(started_cluster): def test_toast(started_cluster): table = "test_toast" - pg_manager.execute( - f"CREATE TABLE {table} (id integer PRIMARY KEY, txt text, other text)" + pg_manager.create_postgres_table( + table, + "", + """CREATE TABLE "{}" (id integer PRIMARY KEY, txt text, other text)""", ) pg_manager.create_materialized_db( ip=started_cluster.postgres_ip, @@ -752,16 +763,7 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) def test_replica_consumer(started_cluster): table = "test_replica_consumer" - pg_manager_replica = PostgresManager() - pg_manager_replica.init( - instance2, - cluster.postgres_ip, - cluster.postgres_port, - default_database="postgres_database", - postgres_db_exists=True, - ) - - for pm in [pg_manager, pg_manager_replica]: + for pm in [pg_manager, pg_manager_instance2]: pm.create_and_fill_postgres_table(table) pm.create_materialized_db( ip=started_cluster.postgres_ip, @@ -778,7 +780,7 @@ def test_replica_consumer(started_cluster): instance, table, postgres_database=pg_manager.get_default_database() ) check_tables_are_synchronized( - instance2, table, postgres_database=pg_manager_replica.get_default_database() + instance2, table, postgres_database=pg_manager_instance2.get_default_database() ) assert 50 == int(instance.query(f"SELECT count() FROM test_database.{table}")) @@ -792,15 +794,15 @@ def test_replica_consumer(started_cluster): instance, table, postgres_database=pg_manager.get_default_database() ) check_tables_are_synchronized( - instance2, table, postgres_database=pg_manager_replica.get_default_database() + instance2, table, postgres_database=pg_manager_instance2.get_default_database() ) assert 1050 == int(instance.query(f"SELECT count() FROM test_database.{table}")) assert 1050 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) - for pm in [pg_manager, pg_manager_replica]: + for pm in [pg_manager, pg_manager_instance2]: pm.drop_materialized_db() - pg_manager_replica.clear() + pg_manager_instance2.clear() if __name__ == "__main__": From 34d804686a4b5fd4cb546faef33346a92504a503 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 16 Oct 2023 13:03:12 +0200 Subject: [PATCH 311/634] Fix black check --- tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index ed3e355ef49..c25a5cb761d 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -691,6 +691,7 @@ class ClickHouseCluster: if with_spark: import pyspark + # if you change packages, don't forget to update them in docker/test/integration/runner/dockerd-entrypoint.sh ( pyspark.sql.SparkSession.builder.appName("spark_test") From 31dc46a8898e3db905c2d64cc168f3e71f144ff0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Oct 2023 13:04:45 +0200 Subject: [PATCH 312/634] Better test --- src/Storages/MergeTree/MergeTreeData.cpp | 25 +++++- .../test_merge_session_expired/__init__.py | 0 .../configs/disks.xml | 21 ----- .../configs/keeper_config.xml | 5 -- .../test_merge_session_expired/test.py | 90 ------------------- ...tore_parts_replicated_merge_tree.reference | 6 ++ ...899_restore_parts_replicated_merge_tree.sh | 54 +++++++++++ 7 files changed, 82 insertions(+), 119 deletions(-) delete mode 100644 tests/integration/test_merge_session_expired/__init__.py delete mode 100644 tests/integration/test_merge_session_expired/configs/disks.xml delete mode 100644 tests/integration/test_merge_session_expired/configs/keeper_config.xml delete mode 100644 tests/integration/test_merge_session_expired/test.py create mode 100644 tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference create mode 100755 tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fcfbe27c741..f9935671d9c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -94,6 +95,7 @@ #include #include #include +#include #include #include #include @@ -3943,10 +3945,26 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT /// ActiveDataPartSet allows to restore most top-level parts instead of unexpected. /// It can be important in case of assigned merges. If unexpected part is result of some - /// finished, but not committed merge we should restore closest ancestors for the - /// unexpected part to be able to execute it. + /// finished, but not committed merge then we should restore (at least try to restore) + /// closest ancestors for the unexpected part to be able to execute it. + /// However it's not guaranteed because outdated parts can intersect ActiveDataPartSet parts_for_replacement(format_version); - for (const auto & part_candidate_in_partition : getDataPartsPartitionRange(part->info.partition_id)) + auto range = getDataPartsPartitionRange(part->info.partition_id); + DataPartsVector parts_candidates(range.begin(), range.end()); + + /// In case of intersecting outdated parts we want to add bigger parts (with higher level) first + auto comparator = [] (const DataPartPtr left, const DataPartPtr right) -> bool + { + if (left->info.level < right->info.level) + return true; + else if (left->info.level > right->info.level) + return false; + else + return left->info.mutation < right->info.mutation; + }; + std::sort(parts_candidates.begin(), parts_candidates.end(), comparator); + /// From larger to smaller parts + for (const auto & part_candidate_in_partition : parts_candidates | std::views::reverse) { if (part->info.contains(part_candidate_in_partition->info) && is_appropriate_state(part_candidate_in_partition->getState())) @@ -3966,6 +3984,7 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT if (parts_for_replacement.size() > 0) { std::vector> holes_list; + /// Most part of the code bellow is just to write pretty message auto part_infos = parts_for_replacement.getPartInfos(); int64_t current_right_block = part_infos[0].min_block; for (const auto & top_level_part_to_replace : part_infos) diff --git a/tests/integration/test_merge_session_expired/__init__.py b/tests/integration/test_merge_session_expired/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_merge_session_expired/configs/disks.xml b/tests/integration/test_merge_session_expired/configs/disks.xml deleted file mode 100644 index 94ac83b32ac..00000000000 --- a/tests/integration/test_merge_session_expired/configs/disks.xml +++ /dev/null @@ -1,21 +0,0 @@ - - - - - s3 - http://minio1:9001/root/data/ - minio - minio123 - - - - - -
- s3 -
-
-
-
-
-
diff --git a/tests/integration/test_merge_session_expired/configs/keeper_config.xml b/tests/integration/test_merge_session_expired/configs/keeper_config.xml deleted file mode 100644 index 4258475f7b5..00000000000 --- a/tests/integration/test_merge_session_expired/configs/keeper_config.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - 3000 - - diff --git a/tests/integration/test_merge_session_expired/test.py b/tests/integration/test_merge_session_expired/test.py deleted file mode 100644 index 61e8ff3c627..00000000000 --- a/tests/integration/test_merge_session_expired/test.py +++ /dev/null @@ -1,90 +0,0 @@ -import logging -import pytest -import time -from helpers.cluster import ClickHouseCluster -from helpers.network import PartitionManager -from helpers.client import QueryRuntimeException - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", - main_configs=["configs/keeper_config.xml", "configs/disks.xml"], - stay_alive=True, - with_zookeeper=True, - with_minio=True, -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - finally: - cluster.shutdown() - - -def test_merge_session_expired(started_cluster): - node1.query("drop table if exists tab") - node1.query( - "create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple() settings old_parts_lifetime=3" - ) - node1.query("insert into tab select number from numbers(10)") - node1.query("insert into tab select number + 10 from numbers(10)") - node1.query("alter table tab delete where x = 12 settings mutations_sync=2") - node1.query("alter table tab delete where x = 14 settings mutations_sync=2") - node1.query("alter table tab delete where x = 16 settings mutations_sync=2") - node1.query("system stop merges") - node1.query("optimize table tab final settings alter_sync=0") - - with PartitionManager() as pm: - pm.drop_instance_zk_connections(node1) - node1.query("system start merges") - node1.query("select sleep(1)") - node1.restart_clickhouse() - pm.restore_instance_zk_connections(node1) - - node1.query("system restart replica tab") - assert node1.query("select count() from tab") == "17\n" - - -def test_merge_session_expired_zero_copy(started_cluster): - node1.query("drop table if exists tab") - node1.query( - """ - create table tab (x UInt64, y UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab2/', '0') order by tuple() - settings old_parts_lifetime=1, storage_policy='s3', allow_remote_fs_zero_copy_replication=1, replicated_max_ratio_of_wrong_parts=1, min_bytes_for_wide_part=1 - """ - ) - - node1.query("insert into tab select number, number from numbers(10)") - node1.query("insert into tab select number + 10, number + 10 from numbers(10)") - node1.query("alter table tab update x = x + 1 where 1 settings mutations_sync=2") - node1.query("select * from tab") - node1.query( - "alter table tab update x = x + 1, y = y + 1 where 1 settings mutations_sync=2" - ) - node1.query("select * from tab") - node1.query("alter table tab update x = x + 1 where 1 settings mutations_sync=2") - node1.query("select * from tab") - - node1.query( - "alter table tab add column z UInt64 materialized x + sleepEachRow(0.05) settings mutations_sync=2" - ) - node1.query("optimize table tab final settings alter_sync=0") - - with PartitionManager() as pm: - pm.drop_instance_zk_connections(node1) - # Wait some time for merge to start - # Part should be merged and stayed on disk, but not commited into zk - node1.query("select sleep(2)") - node1.restart_clickhouse() - pm.restore_instance_zk_connections(node1) - - node1.query("system restart replica tab") - # Wait for outdated parts to be removed - node1.query("select sleep(3)") - node1.query("select * from tab") - node1.query("system sync replica tab") - assert node1.query("select count() from tab") == "20\n" diff --git a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference new file mode 100644 index 00000000000..3f4a9dbc3a9 --- /dev/null +++ b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference @@ -0,0 +1,6 @@ +all_0_1_11_5 +all_4_4_0_5 +all_0_1_11_5 +all_4_4_0_5 +all_0_1_11_5 +all_4_4_0_5 diff --git a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh new file mode 100755 index 00000000000..c40996851f5 --- /dev/null +++ b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh @@ -0,0 +1,54 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_with_unsuccessful_commits" + +$CLICKHOUSE_CLIENT --query "CREATE TABLE table_with_unsuccessful_commits (key UInt64, value String) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/unsuccessful', '1') ORDER BY tuple() SETTINGS cleanup_delay_period=1000, max_cleanup_delay_period=1000, old_parts_lifetime = 1697460529, remove_rolled_back_parts_immediately=0, replicated_max_ratio_of_wrong_parts=1, max_suspicious_broken_parts=1000000, max_suspicious_broken_parts_bytes=10000000000" + +$CLICKHOUSE_CLIENT --query "INSERT INTO table_with_unsuccessful_commits SELECT rand(), toString(rand()) FROM numbers(10)" + +$CLICKHOUSE_CLIENT --query "INSERT INTO table_with_unsuccessful_commits SELECT rand(), toString(rand()) FROM numbers(10)" + + +for i in {0..10}; do + $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE table_with_unsuccessful_commits FINAL" +done + +$CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_unsuccessful_commits DELETE WHERE value = 'hello' SETTINGS mutations_sync=2" +$CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_unsuccessful_commits DELETE WHERE value = 'hello' SETTINGS mutations_sync=2" + +$CLICKHOUSE_CLIENT --query "INSERT INTO table_with_unsuccessful_commits SELECT rand(), toString(rand()) FROM numbers(10)" + +$CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_unsuccessful_commits DELETE WHERE value = 'hello' SETTINGS mutations_sync=2" + +$CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name" + +$CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_unsuccessful_commits MODIFY SETTING fault_probability_before_part_commit=1" + +$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE table_with_unsuccessful_commits FINAL SETTINGS alter_sync=0" + +i=0 retries=300 + +while [[ $i -lt $retries ]]; do + result=$($CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.replication_queue WHERE table = 'table_with_unsuccessful_commits' and database=currentDatabase()") + + if [[ $result ]]; then + break + fi + + ((++i)) +done + +$CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name" + +$CLICKHOUSE_CLIENT --query "DETACH TABLE table_with_unsuccessful_commits" + +$CLICKHOUSE_CLIENT --query "ATTACH TABLE table_with_unsuccessful_commits" + +$CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name" + +$CLICKHOUSE_CLIENT --query "DROP TABLE table_with_unsuccessful_commits" From 2eb1a058990dc2133e3c3d731c4b46ffff245e45 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Oct 2023 13:07:41 +0200 Subject: [PATCH 313/634] Fix --- .../0_stateless/02899_restore_parts_replicated_merge_tree.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh index c40996851f5..c496eed3b53 100755 --- a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh +++ b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh @@ -7,7 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_with_unsuccessful_commits" -$CLICKHOUSE_CLIENT --query "CREATE TABLE table_with_unsuccessful_commits (key UInt64, value String) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/unsuccessful', '1') ORDER BY tuple() SETTINGS cleanup_delay_period=1000, max_cleanup_delay_period=1000, old_parts_lifetime = 1697460529, remove_rolled_back_parts_immediately=0, replicated_max_ratio_of_wrong_parts=1, max_suspicious_broken_parts=1000000, max_suspicious_broken_parts_bytes=10000000000" +# will be flaky in 2031 +$CLICKHOUSE_CLIENT --query "CREATE TABLE table_with_unsuccessful_commits (key UInt64, value String) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/unsuccessful', '1') ORDER BY tuple() SETTINGS cleanup_delay_period=1000, max_cleanup_delay_period=1000, old_parts_lifetime = 1949748529, remove_rolled_back_parts_immediately=0, replicated_max_ratio_of_wrong_parts=1, max_suspicious_broken_parts=1000000, max_suspicious_broken_parts_bytes=10000000000" $CLICKHOUSE_CLIENT --query "INSERT INTO table_with_unsuccessful_commits SELECT rand(), toString(rand()) FROM numbers(10)" From dc0f8ed79311691974427781cf67f08321116365 Mon Sep 17 00:00:00 2001 From: helifu Date: Mon, 16 Oct 2023 19:10:35 +0800 Subject: [PATCH 314/634] Add a new column 'is_hot_reloadable' to indicate whether hot loading is supported --- src/Storages/System/StorageSystemServerSettings.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index 7a8b45de736..3a3acabc5a3 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -28,6 +28,7 @@ NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() {"description", std::make_shared()}, {"type", std::make_shared()}, {"is_obsolete", std::make_shared()}, + {"is_hot_reloadable", std::make_shared()} }; } @@ -72,6 +73,7 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context res_columns[4]->insert(setting.getDescription()); res_columns[5]->insert(setting.getTypeName()); res_columns[6]->insert(setting.isObsolete()); + res_columns[7]->insert((it != updated.end()) ? true : false); } } From 9a925a0802f86e8bebfd0681d234eb72c07b1b9e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Oct 2023 13:35:20 +0200 Subject: [PATCH 315/634] Fix typo --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f9935671d9c..e5f7903948c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3984,7 +3984,7 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT if (parts_for_replacement.size() > 0) { std::vector> holes_list; - /// Most part of the code bellow is just to write pretty message + /// Most part of the code below is just to write pretty message auto part_infos = parts_for_replacement.getPartInfos(); int64_t current_right_block = part_infos[0].min_block; for (const auto & top_level_part_to_replace : part_infos) From 51837eff33c15cc75f45547718fc5c583d9173d1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 16 Oct 2023 11:27:43 +0000 Subject: [PATCH 316/634] Enable test_query_is_lock_free[detach table] for the analyzer --- tests/analyzer_integration_broken_tests.txt | 1 - tests/integration/test_drop_is_lock_free/test.py | 5 ++++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 421ae81e9fe..e2da429ab2f 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -4,7 +4,6 @@ test_concurrent_backups_s3/test.py::test_concurrent_backups test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary test_distributed_backward_compatability/test.py::test_distributed_in_tuple test_distributed_type_object/test.py::test_distributed_type_object -test_drop_is_lock_free/test.py::test_query_is_lock_free[detach table] test_executable_table_function/test.py::test_executable_function_input_python test_mask_sensitive_info/test.py::test_encryption_functions test_merge_table_over_distributed/test.py::test_global_in diff --git a/tests/integration/test_drop_is_lock_free/test.py b/tests/integration/test_drop_is_lock_free/test.py index 61d52a1d9b1..1bb8767a9a0 100644 --- a/tests/integration/test_drop_is_lock_free/test.py +++ b/tests/integration/test_drop_is_lock_free/test.py @@ -125,7 +125,10 @@ def test_query_is_lock_free(lock_free_query, exclusive_table): SELECT count() FROM {exclusive_table}; """ ) - assert f"Table default.{exclusive_table} does not exist" in result + assert ( + f"Table default.{exclusive_table} does not exist" in result + or f"Unknown table expression identifier '{exclusive_table}'" in result + ) else: assert 0 == int( node.query( From 96f2a46a66aa0f63998d8a6a3d6e38d57bf8fbbf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 9 Oct 2023 16:08:22 +0200 Subject: [PATCH 317/634] Fix filtering by virtual columns with OR filter in query The problem with the initial implementation #52653 was: - OR can have multiple arguments - It simply not correct to assume that if there are two arguments this is OK. Consider the following example: "WHERE (column_not_from_partition_by = 1) OR false OR false" Will be converted to: "WHERE false OR false" And it will simply read nothing. Yes, we could apply some optimization for bool, but this will not always work, since to optimize things like "0 = 1" we need to execute it. And the only way to make handle this correctly (with ability to ignore some commands during filtering) is to make is_constant() function return has it use something from the input block, so that we can be sure, that we have some sensible, and not just "false". Plus we cannot simply ignore the difference of the input and output arguments of handling OR, we need to add always-true (1/true) if the size is different, since otherwise it could break invariants (see comment in the code). This includes (but not limited to): - _part* filtering for MergeTree - _path/_file for various File/HDFS/... engines - _table for Merge - ... P.S. analyzer does not have this bug, since it execute expression as whole, and this is what filterBlockWithQuery() should do actually instead, but this will be a more complex patch. Signed-off-by: Azat Khuzhin (cherry picked from commit b107712e0cf8e2671da7c1641b268df57e54bd7a) --- src/Storages/VirtualColumnUtils.cpp | 55 +++++++++++++++---- .../02840_merge__table_or_filter.sql.j2 | 5 ++ .../0_stateless/02896_multiple_OR.reference | 14 +++++ .../queries/0_stateless/02896_multiple_OR.sql | 28 ++++++++++ 4 files changed, 92 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02896_multiple_OR.reference create mode 100644 tests/queries/0_stateless/02896_multiple_OR.sql diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index d0d6233728e..219043f25c6 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -81,14 +82,33 @@ bool extractFunctions(const ASTPtr & expression, const std::functionname == "or") { - bool ret = true; + bool ret = false; ASTs or_args; for (const auto & child : function->arguments->children) - ret &= extractFunctions(child, is_constant, or_args); - /// We can keep condition only if it still OR condition (i.e. we - /// have dependent conditions for columns at both sides) - if (or_args.size() == 2) + ret |= extractFunctions(child, is_constant, or_args); + + if (!or_args.empty()) + { + /// In case of there are less number of arguments for which + /// is_constant() == true, we need to add always-true + /// implicitly to avoid breaking AND invariant. + /// + /// Consider the following: + /// + /// ((value = 10) OR (_table = 'v2')) AND ((_table = 'v1') OR (value = 20)) + /// + /// Without implicit always-true: + /// + /// (_table = 'v2') AND (_table = 'v1') + /// + /// With: + /// + /// (_table = 'v2' OR 1) AND (_table = 'v1' OR 1) -> (_table = 'v2') OR (_table = 'v1') + /// + if (or_args.size() != function->arguments->children.size()) + or_args.push_back(std::make_shared(Field(1))); result.push_back(makeASTForLogicalOr(std::move(or_args))); + } return ret; } } @@ -165,8 +185,10 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block if (!select.where() && !select.prewhere()) return unmodified; - // Provide input columns as constant columns to check if an expression is constant. - std::function is_constant = [&block, &context](const ASTPtr & node) + // Provide input columns as constant columns to check if an expression is + // constant and depends on the columns from provided block (the last is + // required to allow skipping some conditions for handling OR). + std::function is_constant = [&block, &context](const ASTPtr & expr) { auto actions = std::make_shared(block.getColumnsWithTypeAndName()); PreparedSetsPtr prepared_sets = std::make_shared(); @@ -178,13 +200,26 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block context, SizeLimits{}, 1, source_columns, std::move(actions), prepared_sets, true, true, true, { aggregation_keys, grouping_set_keys, GroupByKind::NONE }); - ActionsVisitor(visitor_data).visit(node); + ActionsVisitor(visitor_data).visit(expr); actions = visitor_data.getActions(); + auto expr_column_name = expr->getColumnName(); + + const auto * expr_const_node = actions->tryFindInOutputs(expr_column_name); + if (!expr_const_node) + return false; + auto filter_actions = ActionsDAG::buildFilterActionsDAG({expr_const_node}, {}, context); + const auto & nodes = filter_actions->getNodes(); + bool has_dependent_columns = std::any_of(nodes.begin(), nodes.end(), [&](const auto & node) + { + return block.has(node.result_name); + }); + if (!has_dependent_columns) + return false; + auto expression_actions = std::make_shared(actions); auto block_with_constants = block; expression_actions->execute(block_with_constants); - auto column_name = node->getColumnName(); - return block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column); + return block_with_constants.has(expr_column_name) && isColumnConst(*block_with_constants.getByName(expr_column_name).column); }; /// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. diff --git a/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 b/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 index a87ef7302c6..286e4545ef7 100644 --- a/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 +++ b/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 @@ -18,6 +18,11 @@ create view v2 as select * from d2; create table m as v1 engine=Merge(currentDatabase(), '^(v1|v2)$'); +{# -- FIXME: +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') or 0 or 0 settings {{ settings }}; +select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') or 0 or 0 settings {{ settings }}; +#} + -- avoid reorder set max_threads=1; -- { echoOn } diff --git a/tests/queries/0_stateless/02896_multiple_OR.reference b/tests/queries/0_stateless/02896_multiple_OR.reference new file mode 100644 index 00000000000..96480a75d11 --- /dev/null +++ b/tests/queries/0_stateless/02896_multiple_OR.reference @@ -0,0 +1,14 @@ +-- { echoOn } +SELECT * FROM or_bug WHERE (key = 1) OR false OR false; +1 +SELECT * FROM or_bug WHERE (key = 1) OR false; +1 +SELECT * FROM or_bug WHERE (key = 1); +1 +-- { echoOn } +select * from forms where text_field like '%this%' or 0 = 1 or 0 = 1; +5840ead423829c1eab29fa97 this is a test +select * from forms where text_field like '%this%' or 0 = 1; +5840ead423829c1eab29fa97 this is a test +select * from forms where text_field like '%this%'; +5840ead423829c1eab29fa97 this is a test diff --git a/tests/queries/0_stateless/02896_multiple_OR.sql b/tests/queries/0_stateless/02896_multiple_OR.sql new file mode 100644 index 00000000000..653ddebca1f --- /dev/null +++ b/tests/queries/0_stateless/02896_multiple_OR.sql @@ -0,0 +1,28 @@ +-- https://github.com/ClickHouse/ClickHouse/pull/52653 +DROP TABLE IF EXISTS or_bug; +CREATE TABLE or_bug (key UInt8) ENGINE=MergeTree ORDER BY key; +INSERT INTO TABLE or_bug VALUES (0), (1); + +-- { echoOn } +SELECT * FROM or_bug WHERE (key = 1) OR false OR false; +SELECT * FROM or_bug WHERE (key = 1) OR false; +SELECT * FROM or_bug WHERE (key = 1); +-- { echoOff } + +-- https://github.com/ClickHouse/ClickHouse/issues/55288 +DROP TABLE IF EXISTS forms; +CREATE TABLE forms +( + `form_id` FixedString(24), + `text_field` String +) +ENGINE = MergeTree +PRIMARY KEY form_id +ORDER BY form_id; +insert into forms values ('5840ead423829c1eab29fa97','this is a test'); + +-- { echoOn } +select * from forms where text_field like '%this%' or 0 = 1 or 0 = 1; +select * from forms where text_field like '%this%' or 0 = 1; +select * from forms where text_field like '%this%'; +-- { echoOff } From b0fbd798089033f03208822a0b750dc1406b039b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 16 Oct 2023 15:57:34 +0200 Subject: [PATCH 318/634] Fix test_system_merges after optimization for "x OR 1" conditions After the previous patch "x OR 1" will not execute "x", and because of this test_system_merges::test_mutation_simple started to fail since "sleep" function did not executed. Signed-off-by: Azat Khuzhin --- tests/integration/test_system_merges/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index d0fa7a1d426..6dbe6c891f2 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -188,7 +188,7 @@ def test_mutation_simple(started_cluster, replicated): # ALTER will sleep for 9s def alter(): node1.query( - f"ALTER TABLE {name} UPDATE a = 42 WHERE sleep(9) OR 1", + f"ALTER TABLE {name} UPDATE a = 42 WHERE sleep(9) = 0", settings=settings, ) From 0b2003cf2716064f04ccf2f6ce7a30566a46731d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 16 Oct 2023 16:11:55 +0200 Subject: [PATCH 319/634] Suppress data-race in rd_kafka_broker_set_nodename Refs: https://github.com/ClickHouse/ClickHouse/issues/55629 Signed-off-by: Azat Khuzhin --- tests/tsan_suppressions.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/tsan_suppressions.txt b/tests/tsan_suppressions.txt index 6a55155e330..67c7eae08f3 100644 --- a/tests/tsan_suppressions.txt +++ b/tests/tsan_suppressions.txt @@ -1 +1,2 @@ -# We have no suppressions! +# https://github.com/ClickHouse/ClickHouse/issues/55629 +race:rd_kafka_broker_set_nodename From 4d1edc27804b4defefbe16c7166249074436d92e Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 16 Oct 2023 15:00:13 +0000 Subject: [PATCH 320/634] return back unidiff module --- docker/test/style/Dockerfile | 2 +- tests/ci/worker/prepare-ci-ami.sh | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index de2c1c64938..a4feae27c67 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -19,7 +19,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ shellcheck \ yamllint \ locales \ - && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub pylint==2.6.2 \ + && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub unidiff pylint==2.6.2 \ && apt-get clean \ && rm -rf /root/.cache/pip diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index dc180207be3..f3e11b6d00a 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -9,7 +9,7 @@ set -xeuo pipefail echo "Running prepare script" export DEBIAN_FRONTEND=noninteractive -export RUNNER_VERSION=2.310.1 +export RUNNER_VERSION=2.304.0 export RUNNER_HOME=/home/ubuntu/actions-runner deb_arch() { @@ -90,7 +90,8 @@ systemctl restart docker sudo -u ubuntu docker buildx version sudo -u ubuntu docker buildx create --use --name default-builder -pip install boto3 pygithub requests urllib3 dohq-artifactory +# FIXME: remove unidiff as soon as no old PRs could use it, here and in Dockerfile +pip install boto3 pygithub requests urllib3 unidiff dohq-artifactory mkdir -p $RUNNER_HOME && cd $RUNNER_HOME From c9a2a4922da09d6f31f36fa94cba32d0107475d6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 16 Oct 2023 15:17:55 +0000 Subject: [PATCH 321/634] Fix StorageMaterializedView::isRemote --- src/Storages/StorageMaterializedView.cpp | 7 +++++++ src/Storages/StorageMaterializedView.h | 1 + 2 files changed, 8 insertions(+) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 78dc89857b7..c28fa90b118 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -478,6 +478,13 @@ ActionLock StorageMaterializedView::getActionLock(StorageActionBlockType type) return ActionLock{}; } +bool StorageMaterializedView::isRemote() const +{ + if (auto table = tryGetTargetTable()) + return table->isRemote(); + return false; +} + void registerStorageMaterializedView(StorageFactory & factory) { factory.registerStorage("MaterializedView", [](const StorageFactory::Arguments & args) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 95bedcd9ade..0f6a6fd3db7 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -22,6 +22,7 @@ public: std::string getName() const override { return "MaterializedView"; } bool isView() const override { return true; } + bool isRemote() const override; bool hasInnerTable() const { return has_inner_table; } From 8f3974d28959ad28a3d21ea49463c1d483ab1ae4 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 16 Oct 2023 17:30:02 +0200 Subject: [PATCH 322/634] 00002_log_and_exception_messages_formatting ... --- src/Storages/DataLakes/IcebergMetadataParser.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/DataLakes/IcebergMetadataParser.cpp b/src/Storages/DataLakes/IcebergMetadataParser.cpp index 250363d9210..c4194cd7814 100644 --- a/src/Storages/DataLakes/IcebergMetadataParser.cpp +++ b/src/Storages/DataLakes/IcebergMetadataParser.cpp @@ -302,7 +302,7 @@ struct IcebergMetadataParser::Impl if (status == 2) { - LOG_TEST(log, "Got delete file for {}", file_path); + LOG_TEST(log, "Processing delete file for path: {}", file_path); chassert(!keys.contains(file_path)); } else From b525f1c28adfd7c29f5284aebd313c9caa0e6545 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 16 Oct 2023 17:30:20 +0200 Subject: [PATCH 323/634] Update test.py --- tests/integration/test_storage_iceberg/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 3a5f1724793..89e711745b4 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -345,7 +345,7 @@ def test_delete_files(started_cluster, format_version): ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 0 - assert instance.contains_in_log("Got delete file for") + assert instance.contains_in_log("Processing delete file for path") write_iceberg_from_df( spark, From 4bc32c638768c07eceb97ee6823a9400b8e40b9a Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 16 Oct 2023 17:31:30 +0200 Subject: [PATCH 324/634] Fix uncaught exception in background tasj --- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index e90dcfcd8ad..dea05ce19d8 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -322,8 +322,19 @@ void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, Load void DatabasePostgreSQL::removeOutdatedTables() { std::lock_guard lock{mutex}; - auto connection_holder = pool->get(); - auto actual_tables = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema); + + std::set actual_tables; + try + { + auto connection_holder = pool->get(); + actual_tables = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + cleaner_task->scheduleAfter(cleaner_reschedule_ms); + return; + } if (cache_tables) { From 2da1ff4b0d33df0e061e2e627727046640236734 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 16 Oct 2023 19:07:11 +0300 Subject: [PATCH 325/634] Revert "Improve ColumnDecimal, ColumnVector getPermutation performance using pdqsort with RadixSort" --- base/base/sort.h | 26 --- contrib/pdqsort/pdqsort.h | 200 +----------------------- src/Columns/ColumnDecimal.cpp | 87 +---------- src/Columns/ColumnVector.cpp | 118 ++++++++------ src/Columns/RadixSortHelper.h | 25 --- tests/performance/merge_tree_insert.xml | 19 --- tests/performance/sort_patterns.xml | 28 ---- 7 files changed, 68 insertions(+), 435 deletions(-) delete mode 100644 src/Columns/RadixSortHelper.h delete mode 100644 tests/performance/sort_patterns.xml diff --git a/base/base/sort.h b/base/base/sort.h index 1a814587763..912545979dc 100644 --- a/base/base/sort.h +++ b/base/base/sort.h @@ -131,29 +131,3 @@ void sort(RandomIt first, RandomIt last) using comparator = std::less; ::sort(first, last, comparator()); } - -/** Try to fast sort elements for common sorting patterns: - * 1. If elements are already sorted. - * 2. If elements are already almost sorted. - * 3. If elements are already sorted in reverse order. - * - * Returns true if fast sort was performed or elements were already sorted, false otherwise. - */ -template -bool trySort(RandomIt first, RandomIt last, Compare compare) -{ -#ifndef NDEBUG - ::shuffle(first, last); -#endif - - ComparatorWrapper compare_wrapper = compare; - return ::pdqsort_try_sort(first, last, compare_wrapper); -} - -template -bool trySort(RandomIt first, RandomIt last) -{ - using value_type = typename std::iterator_traits::value_type; - using comparator = std::less; - return ::trySort(first, last, comparator()); -} diff --git a/contrib/pdqsort/pdqsort.h b/contrib/pdqsort/pdqsort.h index cbfc82a4f41..01e82b710ee 100644 --- a/contrib/pdqsort/pdqsort.h +++ b/contrib/pdqsort/pdqsort.h @@ -54,10 +54,8 @@ namespace pdqsort_detail { block_size = 64, // Cacheline size, assumes power of two. - cacheline_size = 64, + cacheline_size = 64 - /// Try sort allowed iterations - try_sort_iterations = 3, }; #if __cplusplus >= 201103L @@ -503,167 +501,6 @@ namespace pdqsort_detail { leftmost = false; } } - - template - inline bool pdqsort_try_sort_loop(Iter begin, - Iter end, - Compare comp, - size_t bad_allowed, - size_t iterations_allowed, - bool force_sort = false, - bool leftmost = true) { - typedef typename std::iterator_traits::difference_type diff_t; - - // Use a while loop for tail recursion elimination. - while (true) { - if (!force_sort && iterations_allowed == 0) { - return false; - } - - diff_t size = end - begin; - - // Insertion sort is faster for small arrays. - if (size < insertion_sort_threshold) { - if (leftmost) insertion_sort(begin, end, comp); - else unguarded_insertion_sort(begin, end, comp); - - return true; - } - - // Choose pivot as median of 3 or pseudomedian of 9. - diff_t s2 = size / 2; - if (size > ninther_threshold) { - sort3(begin, begin + s2, end - 1, comp); - sort3(begin + 1, begin + (s2 - 1), end - 2, comp); - sort3(begin + 2, begin + (s2 + 1), end - 3, comp); - sort3(begin + (s2 - 1), begin + s2, begin + (s2 + 1), comp); - std::iter_swap(begin, begin + s2); - } else sort3(begin + s2, begin, end - 1, comp); - - // If *(begin - 1) is the end of the right partition of a previous partition operation - // there is no element in [begin, end) that is smaller than *(begin - 1). Then if our - // pivot compares equal to *(begin - 1) we change strategy, putting equal elements in - // the left partition, greater elements in the right partition. We do not have to - // recurse on the left partition, since it's sorted (all equal). - if (!leftmost && !comp(*(begin - 1), *begin)) { - begin = partition_left(begin, end, comp) + 1; - continue; - } - - // Partition and get results. - std::pair part_result = - Branchless ? partition_right_branchless(begin, end, comp) - : partition_right(begin, end, comp); - Iter pivot_pos = part_result.first; - bool already_partitioned = part_result.second; - - // Check for a highly unbalanced partition. - diff_t l_size = pivot_pos - begin; - diff_t r_size = end - (pivot_pos + 1); - bool highly_unbalanced = l_size < size / 8 || r_size < size / 8; - - // If we got a highly unbalanced partition we shuffle elements to break many patterns. - if (highly_unbalanced) { - if (!force_sort) { - return false; - } - - // If we had too many bad partitions, switch to heapsort to guarantee O(n log n). - if (--bad_allowed == 0) { - std::make_heap(begin, end, comp); - std::sort_heap(begin, end, comp); - return true; - } - - if (l_size >= insertion_sort_threshold) { - std::iter_swap(begin, begin + l_size / 4); - std::iter_swap(pivot_pos - 1, pivot_pos - l_size / 4); - - if (l_size > ninther_threshold) { - std::iter_swap(begin + 1, begin + (l_size / 4 + 1)); - std::iter_swap(begin + 2, begin + (l_size / 4 + 2)); - std::iter_swap(pivot_pos - 2, pivot_pos - (l_size / 4 + 1)); - std::iter_swap(pivot_pos - 3, pivot_pos - (l_size / 4 + 2)); - } - } - - if (r_size >= insertion_sort_threshold) { - std::iter_swap(pivot_pos + 1, pivot_pos + (1 + r_size / 4)); - std::iter_swap(end - 1, end - r_size / 4); - - if (r_size > ninther_threshold) { - std::iter_swap(pivot_pos + 2, pivot_pos + (2 + r_size / 4)); - std::iter_swap(pivot_pos + 3, pivot_pos + (3 + r_size / 4)); - std::iter_swap(end - 2, end - (1 + r_size / 4)); - std::iter_swap(end - 3, end - (2 + r_size / 4)); - } - } - } else { - // If we were decently balanced and we tried to sort an already partitioned - // sequence try to use insertion sort. - if (already_partitioned && partial_insertion_sort(begin, pivot_pos, comp) - && partial_insertion_sort(pivot_pos + 1, end, comp)) { - return true; - } - } - - // Sort the left partition first using recursion and do tail recursion elimination for - // the right-hand partition. - if (pdqsort_try_sort_loop(begin, - pivot_pos, - comp, - bad_allowed, - iterations_allowed - 1, - force_sort, - leftmost)) { - force_sort = true; - } else { - return false; - } - - --iterations_allowed; - begin = pivot_pos + 1; - leftmost = false; - } - - return false; - } - - template - inline bool pdqsort_try_sort_impl(Iter begin, Iter end, Compare comp, size_t bad_allowed) - { - typedef typename std::iterator_traits::difference_type diff_t; - - static constexpr size_t iterations_allowed = pdqsort_detail::try_sort_iterations; - static constexpr size_t num_to_try = 16; - - diff_t size = end - begin; - - if (size > num_to_try * 10) - { - size_t out_of_order_elements = 0; - - for (size_t i = 1; i < num_to_try; ++i) - { - diff_t offset = size / num_to_try; - - diff_t prev_position = offset * (i - 1); - diff_t curr_position = offset * i; - diff_t next_position = offset * (i + 1) - 1; - - bool prev_less_than_curr = comp(*(begin + prev_position), *(begin + curr_position)); - bool curr_less_than_next = comp(*(begin + curr_position), *(begin + next_position)); - if ((prev_less_than_curr && curr_less_than_next) || (!prev_less_than_curr && !curr_less_than_next)) - continue; - - ++out_of_order_elements; - if (out_of_order_elements > iterations_allowed) - return false; - } - } - - return pdqsort_try_sort_loop(begin, end, comp, bad_allowed, iterations_allowed); - } } @@ -701,41 +538,6 @@ inline void pdqsort_branchless(Iter begin, Iter end) { pdqsort_branchless(begin, end, std::less()); } -template -inline bool pdqsort_try_sort(Iter begin, Iter end, Compare comp) { - if (begin == end) return true; - -#if __cplusplus >= 201103L - return pdqsort_detail::pdqsort_try_sort_impl::type>::value && - std::is_arithmetic::value_type>::value>( - begin, end, comp, pdqsort_detail::log2(end - begin)); -#else - return pdqsort_detail::pdqsort_try_sort_impl( - begin, end, comp, pdqsort_detail::log2(end - begin)); -#endif -} - -template -inline bool pdqsort_try_sort(Iter begin, Iter end) { - typedef typename std::iterator_traits::value_type T; - return pdqsort_try_sort(begin, end, std::less()); -} - -template -inline bool pdqsort_try_sort_branchless(Iter begin, Iter end, Compare comp) { - if (begin == end) return true; - - return pdqsort_detail::pdqsort_try_sort_impl( - begin, end, comp, pdqsort_detail::log2(end - begin)); -} - -template -inline bool pdqsort_try_sort_branchless(Iter begin, Iter end) { - typedef typename std::iterator_traits::value_type T; - return pdqsort_try_sort_branchless(begin, end, std::less()); -} - #undef PDQSORT_PREFER_MOVE diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 111c0e3cb1c..0d82818a431 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include @@ -16,7 +15,6 @@ #include #include #include -#include #include @@ -161,59 +159,6 @@ void ColumnDecimal::getPermutation(IColumn::PermutationSortDirection directio return data[lhs] > data[rhs]; }; - size_t data_size = data.size(); - res.resize(data_size); - - if (limit >= data_size) - limit = 0; - - for (size_t i = 0; i < data_size; ++i) - res[i] = i; - - if constexpr (is_arithmetic_v && !is_big_int_v) - { - if (!limit) - { - /// A case for radix sort - /// LSD RadixSort is stable - - bool reverse = direction == IColumn::PermutationSortDirection::Descending; - bool ascending = direction == IColumn::PermutationSortDirection::Ascending; - bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable; - - /// TODO: LSD RadixSort is currently not stable if direction is descending - bool use_radix_sort = (sort_is_stable && ascending) || !sort_is_stable; - - /// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters. - if (data_size >= 256 && data_size <= std::numeric_limits::max() && use_radix_sort) - { - for (size_t i = 0; i < data_size; ++i) - res[i] = i; - - bool try_sort = false; - - if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) - try_sort = trySort(res.begin(), res.end(), comparator_ascending); - else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) - try_sort = trySort(res.begin(), res.end(), comparator_ascending_stable); - else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) - try_sort = trySort(res.begin(), res.end(), comparator_descending); - else - try_sort = trySort(res.begin(), res.end(), comparator_descending_stable); - - if (try_sort) - return; - - PaddedPODArray> pairs(data_size); - for (UInt32 i = 0; i < static_cast(data_size); ++i) - pairs[i] = {data[i].value, i}; - - RadixSort>::executeLSD(pairs.data(), data_size, reverse, res.data()); - return; - } - } - } - if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) this->getPermutationImpl(limit, res, comparator_ascending, DefaultSort(), DefaultPartialSort()); else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) @@ -246,37 +191,7 @@ void ColumnDecimal::updatePermutation(IColumn::PermutationSortDirection direc return data[lhs] < data[rhs]; }; auto equals_comparator = [this](size_t lhs, size_t rhs) { return data[lhs] == data[rhs]; }; - auto sort = [&](auto begin, auto end, auto pred) - { - bool reverse = direction == IColumn::PermutationSortDirection::Descending; - bool ascending = direction == IColumn::PermutationSortDirection::Ascending; - bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable; - - /// TODO: LSD RadixSort is currently not stable if direction is descending - bool use_radix_sort = (sort_is_stable && ascending) || !sort_is_stable; - size_t size = end - begin; - - if (size >= 256 && size <= std::numeric_limits::max() && use_radix_sort) - { - bool try_sort = trySort(begin, end, pred); - if (try_sort) - return; - - PaddedPODArray> pairs(size); - size_t index = 0; - - for (auto * it = begin; it != end; ++it) - { - pairs[index] = {data[*it].value, static_cast(*it)}; - ++index; - } - - RadixSort>::executeLSD(pairs.data(), size, reverse, res.data()); - return; - } - - ::sort(begin, end, pred); - }; + auto sort = [](auto begin, auto end, auto pred) { ::sort(begin, end, pred); }; auto partial_sort = [](auto begin, auto mid, auto end, auto pred) { ::partial_sort(begin, mid, end, pred); }; if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 37e62c76596..e46384e4d03 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -193,6 +192,26 @@ struct ColumnVector::equals bool operator()(size_t lhs, size_t rhs) const { return CompareHelper::equals(parent.data[lhs], parent.data[rhs], nan_direction_hint); } }; +namespace +{ + template + struct ValueWithIndex + { + T value; + UInt32 index; + }; + + template + struct RadixSortTraits : RadixSortNumTraits + { + using Element = ValueWithIndex; + using Result = size_t; + + static T & extractKey(Element & elem) { return elem.value; } + static size_t extractResult(Element & elem) { return elem.index; } + }; +} + #if USE_EMBEDDED_COMPILER template @@ -235,25 +254,35 @@ template void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const { - size_t data_size = data.size(); - res.resize(data_size); + size_t s = data.size(); + res.resize(s); - if (data_size == 0) + if (s == 0) return; - if (limit >= data_size) + if (limit >= s) limit = 0; - for (size_t i = 0; i < data_size; ++i) - res[i] = i; - - if constexpr (is_arithmetic_v && !is_big_int_v) + if (limit) { - if (!limit) - { - /// A case for radix sort - /// LSD RadixSort is stable + for (size_t i = 0; i < s; ++i) + res[i] = i; + if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) + ::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint)); + else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) + ::partial_sort(res.begin(), res.begin() + limit, res.end(), less_stable(*this, nan_direction_hint)); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) + ::partial_sort(res.begin(), res.begin() + limit, res.end(), greater(*this, nan_direction_hint)); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable) + ::partial_sort(res.begin(), res.begin() + limit, res.end(), greater_stable(*this, nan_direction_hint)); + } + else + { + /// A case for radix sort + /// LSD RadixSort is stable + if constexpr (is_arithmetic_v && !is_big_int_v) + { bool reverse = direction == IColumn::PermutationSortDirection::Descending; bool ascending = direction == IColumn::PermutationSortDirection::Ascending; bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable; @@ -262,27 +291,13 @@ void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction bool use_radix_sort = (sort_is_stable && ascending && !std::is_floating_point_v) || !sort_is_stable; /// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters. - if (data_size >= 256 && data_size <= std::numeric_limits::max() && use_radix_sort) + if (s >= 256 && s <= std::numeric_limits::max() && use_radix_sort) { - bool try_sort = false; - - if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) - try_sort = trySort(res.begin(), res.end(), less(*this, nan_direction_hint)); - else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) - try_sort = trySort(res.begin(), res.end(), less_stable(*this, nan_direction_hint)); - else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) - try_sort = trySort(res.begin(), res.end(), greater(*this, nan_direction_hint)); - else - try_sort = trySort(res.begin(), res.end(), greater_stable(*this, nan_direction_hint)); - - if (try_sort) - return; - - PaddedPODArray> pairs(data_size); - for (UInt32 i = 0; i < static_cast(data_size); ++i) + PaddedPODArray> pairs(s); + for (UInt32 i = 0; i < static_cast(s); ++i) pairs[i] = {data[i], i}; - RadixSort>::executeLSD(pairs.data(), data_size, reverse, res.data()); + RadixSort>::executeLSD(pairs.data(), s, reverse, res.data()); /// Radix sort treats all NaNs to be greater than all numbers. /// If the user needs the opposite, we must move them accordingly. @@ -290,9 +305,9 @@ void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction { size_t nans_to_move = 0; - for (size_t i = 0; i < data_size; ++i) + for (size_t i = 0; i < s; ++i) { - if (isNaN(data[res[reverse ? i : data_size - 1 - i]])) + if (isNaN(data[res[reverse ? i : s - 1 - i]])) ++nans_to_move; else break; @@ -300,35 +315,38 @@ void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction if (nans_to_move) { - std::rotate(std::begin(res), std::begin(res) + (reverse ? nans_to_move : data_size - nans_to_move), std::end(res)); + std::rotate(std::begin(res), std::begin(res) + (reverse ? nans_to_move : s - nans_to_move), std::end(res)); } } - return; } } - } - if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) - this->getPermutationImpl(limit, res, less(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); - else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) - this->getPermutationImpl(limit, res, less_stable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); - else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) - this->getPermutationImpl(limit, res, greater(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); - else - this->getPermutationImpl(limit, res, greater_stable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); + /// Default sorting algorithm. + for (size_t i = 0; i < s; ++i) + res[i] = i; + + if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) + ::sort(res.begin(), res.end(), less(*this, nan_direction_hint)); + else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) + ::sort(res.begin(), res.end(), less_stable(*this, nan_direction_hint)); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) + ::sort(res.begin(), res.end(), greater(*this, nan_direction_hint)); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable) + ::sort(res.begin(), res.end(), greater_stable(*this, nan_direction_hint)); + } } template void ColumnVector::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const { + bool reverse = direction == IColumn::PermutationSortDirection::Descending; + bool ascending = direction == IColumn::PermutationSortDirection::Ascending; + bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable; + auto sort = [&](auto begin, auto end, auto pred) { - bool reverse = direction == IColumn::PermutationSortDirection::Descending; - bool ascending = direction == IColumn::PermutationSortDirection::Ascending; - bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable; - /// A case for radix sort if constexpr (is_arithmetic_v && !is_big_int_v) { @@ -339,10 +357,6 @@ void ColumnVector::updatePermutation(IColumn::PermutationSortDirection direct /// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters. if (size >= 256 && size <= std::numeric_limits::max() && use_radix_sort) { - bool try_sort = trySort(begin, end, pred); - if (try_sort) - return; - PaddedPODArray> pairs(size); size_t index = 0; diff --git a/src/Columns/RadixSortHelper.h b/src/Columns/RadixSortHelper.h deleted file mode 100644 index e7d8ea6e535..00000000000 --- a/src/Columns/RadixSortHelper.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -template -struct ValueWithIndex -{ - T value; - UInt32 index; -}; - -template -struct RadixSortTraits : RadixSortNumTraits -{ - using Element = ValueWithIndex; - using Result = size_t; - - static T & extractKey(Element & elem) { return elem.value; } - static size_t extractResult(Element & elem) { return elem.index; } -}; - -} diff --git a/tests/performance/merge_tree_insert.xml b/tests/performance/merge_tree_insert.xml index 3e1d2541480..1e987d27d50 100644 --- a/tests/performance/merge_tree_insert.xml +++ b/tests/performance/merge_tree_insert.xml @@ -18,29 +18,15 @@ merge_tree_insert_6 - - - decimal_primary_key_table_name - - merge_tree_insert_7 - merge_tree_insert_8 - merge_tree_insert_9 - - CREATE TABLE merge_tree_insert_1 (value_1 UInt64, value_2 UInt64, value_3 UInt64) ENGINE = MergeTree ORDER BY (value_1) CREATE TABLE merge_tree_insert_2 (value_1 UInt64, value_2 UInt64, value_3 UInt64) ENGINE = MergeTree ORDER BY (value_1, value_2) CREATE TABLE merge_tree_insert_3 (value_1 UInt64, value_2 UInt64, value_3 UInt64) ENGINE = MergeTree ORDER BY (value_1, value_2, value_3) - CREATE TABLE merge_tree_insert_4 (value_1 String, value_2 String, value_3 String) ENGINE = MergeTree ORDER BY (value_1) CREATE TABLE merge_tree_insert_5 (value_1 String, value_2 String, value_3 String) ENGINE = MergeTree ORDER BY (value_1, value_2) CREATE TABLE merge_tree_insert_6 (value_1 String, value_2 String, value_3 String) ENGINE = MergeTree ORDER BY (value_1, value_2, value_3) - CREATE TABLE merge_tree_insert_7 (value_1 Decimal64(8), value_2 Decimal64(8), value_3 Decimal64(8)) ENGINE = MergeTree ORDER BY (value_1) - CREATE TABLE merge_tree_insert_8 (value_1 Decimal64(8), value_2 Decimal64(8), value_3 Decimal64(8)) ENGINE = MergeTree ORDER BY (value_1, value_2) - CREATE TABLE merge_tree_insert_9 (value_1 Decimal64(8), value_2 Decimal64(8), value_3 Decimal64(8)) ENGINE = MergeTree ORDER BY (value_1, value_2, value_3) - INSERT INTO {integer_primary_key_table_name} SELECT rand64(0), rand64(1), rand64(2) FROM system.numbers LIMIT 500000 INSERT INTO {integer_primary_key_table_name} SELECT rand64(0), rand64(1), rand64(2) FROM system.numbers LIMIT 1000000 INSERT INTO {integer_primary_key_table_name} SELECT rand64(0), rand64(1), rand64(2) FROM system.numbers LIMIT 1500000 @@ -49,12 +35,7 @@ INSERT INTO {string_primary_key_table_name} SELECT toString(rand64(0)), toString(rand64(1)), toString(rand64(2)) FROM system.numbers LIMIT 1000000 INSERT INTO {string_primary_key_table_name} SELECT toString(rand64(0)), toString(rand64(1)), toString(rand64(2)) FROM system.numbers LIMIT 1500000 - INSERT INTO {decimal_primary_key_table_name} SELECT rand64(0) % 1000000, rand64(1) % 1500000, rand64(2) % 2000000 FROM system.numbers LIMIT 500000 - INSERT INTO {decimal_primary_key_table_name} SELECT rand64(0) % 1000000, rand64(1) % 1500000, rand64(2) % 2000000 FROM system.numbers LIMIT 1000000 - INSERT INTO {decimal_primary_key_table_name} SELECT rand64(0) % 1000000, rand64(1) % 1500000, rand64(2) % 2000000 FROM system.numbers LIMIT 1500000 - DROP TABLE IF EXISTS {integer_primary_key_table_name} DROP TABLE IF EXISTS {string_primary_key_table_name} - DROP TABLE IF EXISTS {decimal_primary_key_table_name} diff --git a/tests/performance/sort_patterns.xml b/tests/performance/sort_patterns.xml deleted file mode 100644 index fc49b20cc8c..00000000000 --- a/tests/performance/sort_patterns.xml +++ /dev/null @@ -1,28 +0,0 @@ - - - - integer_type - - UInt32 - UInt64 - - - - sort_expression - - key - key, value - key DESC - key DESC, value DESC - - - - - CREATE TABLE sequential_{integer_type} (key {integer_type}, value {integer_type}) Engine = Memory - - INSERT INTO sequential_{integer_type} SELECT number, number FROM numbers(500000000) - - SELECT key, value FROM sequential_{integer_type} ORDER BY {sort_expression} FORMAT Null - - DROP TABLE IF EXISTS sequential_{integer_type} - From d9ed5e2e944e6bf74c881fabf2c8d58ec1e0856a Mon Sep 17 00:00:00 2001 From: Vlad Seliverstov Date: Mon, 16 Oct 2023 10:11:01 -0600 Subject: [PATCH 326/634] Pretty op processors IDs in EXPLAIN PIPELINE graph = 1 output (#48852) * Prettify IDs op processors * bump * don't use static map --------- Co-authored-by: Vlad Seliverstov --- src/QueryPipeline/printPipeline.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/QueryPipeline/printPipeline.h b/src/QueryPipeline/printPipeline.h index e91909cb50b..2bdbd8f7a07 100644 --- a/src/QueryPipeline/printPipeline.h +++ b/src/QueryPipeline/printPipeline.h @@ -17,9 +17,11 @@ void printPipeline(const Processors & processors, const Statuses & statuses, Wri out << " rankdir=\"LR\";\n"; out << " { node [shape = rect]\n"; - auto get_proc_id = [](const IProcessor & proc) -> UInt64 + std::unordered_map pointer_to_id; + auto get_proc_id = [&](const IProcessor & proc) -> std::size_t { - return reinterpret_cast(&proc); + auto [it, inserted] = pointer_to_id.try_emplace(&proc, pointer_to_id.size()); + return it->second; }; auto statuses_iter = statuses.begin(); From 0a6a4b389496423f82b47aa2dd41398807d5551c Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 16 Oct 2023 18:15:11 +0200 Subject: [PATCH 327/634] Review fixes --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 15 +++++++++------ src/Storages/S3Queue/S3QueueFilesMetadata.h | 6 +++++- src/Storages/S3Queue/S3QueueSource.cpp | 7 ++++--- src/Storages/S3Queue/S3QueueSource.h | 2 +- src/Storages/S3Queue/StorageS3Queue.cpp | 11 ++++++----- src/Storages/S3Queue/StorageS3Queue.h | 2 +- 6 files changed, 26 insertions(+), 17 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 150174aabcb..963a64e257e 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -474,6 +474,7 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolder if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); + zk_client->tryRemove(node_name + ".retriable", -1); return; } @@ -534,7 +535,11 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt Coordination::Responses responses; if (holder->remove(&requests, &responses)) + { + LOG_TEST(log, "Moved file `{}` to processed", path); + zk_client->tryRemove(node_name + ".retriable", -1); return; + } /// Failed to update max processed node, retry. if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) @@ -814,14 +819,12 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() }; auto node_cmp = [](const Node & a, const Node & b) { - if (a.metadata.last_processed_timestamp == b.metadata.last_processed_timestamp) - return a.metadata.file_path < b.metadata.file_path; - else - return a.metadata.last_processed_timestamp < b.metadata.last_processed_timestamp; + return std::tie(a.metadata.last_processed_timestamp, a.metadata.file_path) + < std::tie(b.metadata.last_processed_timestamp, b.metadata.file_path); }; /// Ordered in ascending order of timestamps. - std::multiset sorted_nodes(node_cmp); + std::set sorted_nodes(node_cmp); LOG_TRACE(log, "Found {} nodes", nodes.size()); @@ -854,7 +857,7 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TEST(log, "Checking node limits (max size: {}, max age: {}) for {}", max_set_size, max_set_age_sec, get_nodes_str()); size_t nodes_to_remove = check_nodes_limit && nodes_limit_exceeded ? nodes.size() - max_set_size : 0; - for (const auto & node : sorted_nodes) + for (const auto & node : sorted_nodes) { if (nodes_to_remove) { diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index a2fd1fb6699..df9db87a621 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -149,7 +149,11 @@ class S3QueueFilesMetadata::ProcessingNodeHolder { friend class S3QueueFilesMetadata; public: - ProcessingNodeHolder(const std::string & processing_id_, const std::string & path_, const std::string & zk_node_path_, zkutil::ZooKeeperPtr zk_client_); + ProcessingNodeHolder( + const std::string & processing_id_, + const std::string & path_, + const std::string & zk_node_path_, + zkutil::ZooKeeperPtr zk_client_); ~ProcessingNodeHolder(); diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 1a44fe9cff8..6ea222df71f 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -167,11 +167,12 @@ Chunk StorageS3QueueSource::generate() return chunk; } } - catch (const Exception & e) + catch (...) { - LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", reader.getFile(), e.displayText()); + const auto message = getCurrentExceptionMessage(true); + LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", reader.getFile(), message); - files_metadata->setFileFailed(key_with_info->processing_holder, e.message()); + files_metadata->setFileFailed(key_with_info->processing_holder, message); appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); throw; diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index db3015f129e..7c8eb3eeb74 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -65,7 +65,7 @@ public: RemoveFileFunc remove_file_func_, const NamesAndTypesList & requested_virtual_columns_, ContextPtr context_, - const std::atomic & shutdown_called_, + const std::atomic & shutdown_called_, std::shared_ptr s3_queue_log_, const StorageID & storage_id_); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 37389eb1bd0..92f15aed62f 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -210,20 +210,21 @@ Pipe StorageS3Queue::read( Pipes pipes; const size_t adjusted_num_streams = std::min(num_streams, s3queue_settings->s3queue_processing_threads_num); + + auto file_iterator = createFileIterator(local_context, query_info.query); for (size_t i = 0; i < adjusted_num_streams; ++i) - pipes.emplace_back(createSource(column_names, storage_snapshot, query_info.query, max_block_size, local_context)); + pipes.emplace_back(createSource(file_iterator, column_names, storage_snapshot, max_block_size, local_context)); return Pipe::unitePipes(std::move(pipes)); } std::shared_ptr StorageS3Queue::createSource( + std::shared_ptr file_iterator, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - ASTPtr query, size_t max_block_size, ContextPtr local_context) { auto configuration_snapshot = updateConfigurationAndGetCopy(local_context); - auto file_iterator = createFileIterator(local_context, query); auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); auto internal_source = std::make_unique( @@ -350,12 +351,12 @@ bool StorageS3Queue::streamToViews() // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true); auto block_io = interpreter.execute(); + auto file_iterator = createFileIterator(s3queue_context, nullptr); Pipes pipes; for (size_t i = 0; i < s3queue_settings->s3queue_processing_threads_num; ++i) { - auto source = createSource( - block_io.pipeline.getHeader().getNames(), storage_snapshot, nullptr, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); + auto source = createSource(file_iterator, block_io.pipeline.getHeader().getNames(), storage_snapshot, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); pipes.emplace_back(std::move(source)); } auto pipe = Pipe::unitePipes(std::move(pipes)); diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 2a62078fcca..000015951ea 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -83,9 +83,9 @@ private: std::shared_ptr createFileIterator(ContextPtr local_context, ASTPtr query); std::shared_ptr createSource( + std::shared_ptr file_iterator, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - ASTPtr query, size_t max_block_size, ContextPtr local_context); From bfb5b8f441c93de24c835e12847db28efc0ad394 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 16 Oct 2023 18:24:05 +0200 Subject: [PATCH 328/634] Implement replica estimation on top of the analyzer --- src/Planner/Planner.cpp | 18 +++-- src/Planner/PlannerJoinTree.cpp | 129 +++++++++++++++++--------------- 2 files changed, 82 insertions(+), 65 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index b92a7fb0fea..7341ee4f1ba 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -40,10 +40,11 @@ #include #include -#include -#include -#include #include +#include +#include +#include +#include #include #include @@ -143,6 +144,7 @@ void checkStoragesSupportTransactions(const PlannerContextPtr & planner_context) * getQueryProcessingStage method will be called. * * StorageDistributed skip unused shards optimization relies on this. + * Parallel replicas estimation relies on this too. * * To collect filters that will be applied to specific table in case we have JOINs requires * to run query plan optimization pipeline. @@ -156,6 +158,11 @@ void checkStoragesSupportTransactions(const PlannerContextPtr & planner_context) void collectFiltersForAnalysis(const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context) { bool collect_filters = false; + const auto & query_context = planner_context->getQueryContext(); + const auto & settings = query_context->getSettingsRef(); + + bool parallel_replicas_estimation_enabled + = query_context->canUseParallelReplicasOnInitiator() && settings.parallel_replicas_min_number_of_rows_per_replica > 0; for (auto & [table_expression, table_expression_data] : planner_context->getTableExpressionNodeToData()) { @@ -165,7 +172,8 @@ void collectFiltersForAnalysis(const QueryTreeNodePtr & query_tree, const Planne continue; const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); - if (typeid_cast(storage.get())) + if (typeid_cast(storage.get()) + || (parallel_replicas_estimation_enabled && std::dynamic_pointer_cast(storage))) { collect_filters = true; break; @@ -187,8 +195,6 @@ void collectFiltersForAnalysis(const QueryTreeNodePtr & query_tree, const Planne dummy_storage_to_table_expression_data.emplace(dummy_storage, table_expression_data); } - const auto & query_context = planner_context->getQueryContext(); - SelectQueryOptions select_query_options; Planner planner(updated_query_tree, select_query_options); planner.buildQueryPlanIfNeeded(); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 16314e54122..1e4c51dda59 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -526,6 +526,41 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context); } +static UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info) +{ + auto const & main_query_node = select_query_info.query_tree->as(); + + /// Constness of limit and offset is validated during query analysis stage + size_t limit_length = 0; + if (main_query_node.hasLimit()) + limit_length = main_query_node.getLimit()->as().getValue().safeGet(); + + size_t limit_offset = 0; + if (main_query_node.hasOffset()) + limit_offset = main_query_node.getOffset()->as().getValue().safeGet(); + + /** If not specified DISTINCT, WHERE, GROUP BY, HAVING, ORDER BY, JOIN, LIMIT BY, LIMIT WITH TIES + * but LIMIT is specified, and limit + offset < max_block_size, + * then as the block size we will use limit + offset (not to read more from the table than requested), + * and also set the number of threads to 1. + */ + if (main_query_node.hasLimit() + && !main_query_node.isDistinct() + && !main_query_node.isLimitWithTies() + && !main_query_node.hasPrewhere() + && !main_query_node.hasWhere() + && select_query_info.filter_asts.empty() + && !main_query_node.hasGroupBy() + && !main_query_node.hasHaving() + && !main_query_node.hasOrderBy() + && !main_query_node.hasLimitBy() + && !select_query_info.need_aggregate + && !select_query_info.has_window + && limit_length <= std::numeric_limits::max() - limit_offset) + return limit_length + limit_offset; + return 0; +} + JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, const SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, @@ -580,54 +615,27 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } UInt64 max_block_size = settings.max_block_size; - - auto & main_query_node = select_query_info.query_tree->as(); - + UInt64 max_block_size_limited = 0; if (is_single_table_expression) { - size_t limit_length = 0; - if (main_query_node.hasLimit()) - { - /// Constness of limit is validated during query analysis stage - limit_length = main_query_node.getLimit()->as().getValue().safeGet(); - } - - size_t limit_offset = 0; - if (main_query_node.hasOffset()) - { - /// Constness of offset is validated during query analysis stage - limit_offset = main_query_node.getOffset()->as().getValue().safeGet(); - } - /** If not specified DISTINCT, WHERE, GROUP BY, HAVING, ORDER BY, JOIN, LIMIT BY, LIMIT WITH TIES * but LIMIT is specified, and limit + offset < max_block_size, * then as the block size we will use limit + offset (not to read more from the table than requested), * and also set the number of threads to 1. */ - if (main_query_node.hasLimit() && - !main_query_node.isDistinct() && - !main_query_node.isLimitWithTies() && - !main_query_node.hasPrewhere() && - !main_query_node.hasWhere() && - select_query_info.filter_asts.empty() && - !main_query_node.hasGroupBy() && - !main_query_node.hasHaving() && - !main_query_node.hasOrderBy() && - !main_query_node.hasLimitBy() && - !select_query_info.need_aggregate && - !select_query_info.has_window && - limit_length <= std::numeric_limits::max() - limit_offset) + max_block_size_limited = mainQueryNodeBlockSizeByLimit(select_query_info); + if (max_block_size_limited) { - if (limit_length + limit_offset < max_block_size) + if (max_block_size_limited < max_block_size) { - max_block_size = std::max(1, limit_length + limit_offset); + max_block_size = std::max(1, max_block_size_limited); max_streams = 1; max_threads_execute_query = 1; } - if (limit_length + limit_offset < select_query_info.local_storage_limits.local_limits.size_limits.max_rows) + if (max_block_size_limited < select_query_info.local_storage_limits.local_limits.size_limits.max_rows) { - table_expression_query_info.limit = limit_length + limit_offset; + table_expression_query_info.limit = max_block_size_limited; } } @@ -689,32 +697,35 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (storage_merge_tree && query_context->canUseParallelReplicasOnInitiator() && settings.parallel_replicas_min_number_of_rows_per_replica > 0) { - /// This is trash - /// It uses the old InterpreterSelectQuery to do the estimation of how many rows will be read - /// Ideally we should be able to use estimateNumberOfRowsToRead over the storage, but to do this - /// properly we need all the actions/filters, which aren't available yet - /// If we instead delay this check for later several things will happen: - /// * The header might be different (updatePrewhereOutputsIfNeeded) - /// * The storage will have been initiated (thus already preparing parallel replicas) - auto query_options = SelectQueryOptions( - QueryProcessingStage::WithMergeableState, - /* depth */ 1, - /* is_subquery_= */ true) - .ignoreProjections() - .ignoreAlias(); - InterpreterSelectQuery select( - table_expression_query_info.original_query, - query_context, - query_options, - table_expression_query_info.prepared_sets); - select.adjustParallelReplicasAfterAnalysis(); - planner_context->getMutableQueryContext()->setSetting( - "allow_experimental_parallel_reading_from_replicas", - select.getContext()->getSettingsRef().allow_experimental_parallel_reading_from_replicas.operator Field()); - planner_context->getMutableQueryContext()->setSetting( - "max_parallel_replicas", select.getContext()->getSettingsRef().max_parallel_replicas.operator Field()); - } + ActionDAGNodes filter_nodes; + if (table_expression_query_info.filter_actions_dag) + filter_nodes.nodes = table_expression_query_info.filter_actions_dag->getOutputs(); + UInt64 rows_to_read = storage_merge_tree->estimateNumberOfRowsToRead( + query_context, storage_snapshot, table_expression_query_info, filter_nodes); + if (max_block_size_limited && max_block_size_limited < rows_to_read) + rows_to_read = max_block_size_limited; + + size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; + LOG_TRACE( + &Poco::Logger::get("Planner"), + "Estimated {} rows to read. It is enough work for {} parallel replicas", + rows_to_read, + number_of_replicas_to_use); + + if (number_of_replicas_to_use <= 1) + { + planner_context->getMutableQueryContext()->setSetting( + "allow_experimental_parallel_reading_from_replicas", Field(0)); + planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", UInt64{0}); + LOG_DEBUG(&Poco::Logger::get("Planner"), "Disabling parallel replicas because there aren't enough rows to read"); + } + else if (number_of_replicas_to_use < settings.max_parallel_replicas) + { + planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", number_of_replicas_to_use); + LOG_DEBUG(&Poco::Logger::get("Planner"), "Reducing the number of replicas to use to {}", number_of_replicas_to_use); + } + } const auto & prewhere_actions = table_expression_data.getPrewhereFilterActions(); From f85e82931d009100bbd1dfe1998810c930aee765 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 16 Oct 2023 18:36:45 +0200 Subject: [PATCH 329/634] Make test less flaky --- ...9_restore_parts_replicated_merge_tree.reference | 7 +------ .../02899_restore_parts_replicated_merge_tree.sh | 14 +++++++++++--- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference index 3f4a9dbc3a9..7326d960397 100644 --- a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference +++ b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference @@ -1,6 +1 @@ -all_0_1_11_5 -all_4_4_0_5 -all_0_1_11_5 -all_4_4_0_5 -all_0_1_11_5 -all_4_4_0_5 +Ok diff --git a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh index c496eed3b53..c6165c1e983 100755 --- a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh +++ b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh @@ -26,7 +26,7 @@ $CLICKHOUSE_CLIENT --query "INSERT INTO table_with_unsuccessful_commits SELECT r $CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_unsuccessful_commits DELETE WHERE value = 'hello' SETTINGS mutations_sync=2" -$CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name" +original_parts=$($CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name") $CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_unsuccessful_commits MODIFY SETTING fault_probability_before_part_commit=1" @@ -44,12 +44,20 @@ while [[ $i -lt $retries ]]; do ((++i)) done -$CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name" +parts_after_mutation=$($CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name") $CLICKHOUSE_CLIENT --query "DETACH TABLE table_with_unsuccessful_commits" $CLICKHOUSE_CLIENT --query "ATTACH TABLE table_with_unsuccessful_commits" -$CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name" +parts_after_detach_attach=$($CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name") + +if [[ "$parts_after_detach_attach" == "$parts_after_mutation" && "$parts_after_mutation" == "$original_parts" ]]; then + echo "Ok" +else + echo "Original parts $original_parts" + echo "Parts after mutation $parts_after_mutation" + echo "Parts after detach attach $parts_after_detach_attach" +fi $CLICKHOUSE_CLIENT --query "DROP TABLE table_with_unsuccessful_commits" From df17cd467bc2092fa8a8dee14ef732c11c7c77a2 Mon Sep 17 00:00:00 2001 From: Jiebin Sun Date: Tue, 17 Oct 2023 00:41:38 +0800 Subject: [PATCH 330/634] Release more num_streams if data is small (#53867) * Release more num_streams if data is small Besides the sum_marks and min_marks_for_concurrent_read, we could also involve the system cores to get the num_streams if the data is small. Increasing the num_streams and decreasing the min_marks_for_concurrent_read would improve the parallel performance if the system has plentiful cores. Test the patch on 2x80 vCPUs system. Q39 of clickbench has got 3.3x performance improvement. Q36 has got 2.6x performance improvement. The overall geomean has got 9% gain. Signed-off-by: Jiebin Sun * Release more num_streams if data is small Change the min marks from 4 to 8 as the profit is small and 8 granules is the default block size. Signed-off-by: Jiebin Sun --------- Signed-off-by: Jiebin Sun --- .../QueryPlan/ReadFromMergeTree.cpp | 24 ++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 10f51563e9b..a4a5a629818 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -718,7 +718,29 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ { /// Reduce the number of num_streams if the data is small. if (info.sum_marks < num_streams * info.min_marks_for_concurrent_read && parts_with_ranges.size() < num_streams) - num_streams = std::max((info.sum_marks + info.min_marks_for_concurrent_read - 1) / info.min_marks_for_concurrent_read, parts_with_ranges.size()); + { + /* + If the data is fragmented, then allocate the size of parts to num_streams. If the data is not fragmented, besides the sum_marks and + min_marks_for_concurrent_read, involve the system cores to get the num_streams. Increase the num_streams and decrease the min_marks_for_concurrent_read + if the data is small but system has plentiful cores. It helps to improve the parallel performance of `MergeTreeRead` significantly. + Make sure the new num_streams `num_streams * increase_num_streams_ratio` will not exceed the previous calculated prev_num_streams. + The new info.min_marks_for_concurrent_read `info.min_marks_for_concurrent_read / increase_num_streams_ratio` should be larger than 8. + https://github.com/ClickHouse/ClickHouse/pull/53867 + */ + if ((info.sum_marks + info.min_marks_for_concurrent_read - 1) / info.min_marks_for_concurrent_read > parts_with_ranges.size()) + { + const size_t prev_num_streams = num_streams; + num_streams = (info.sum_marks + info.min_marks_for_concurrent_read - 1) / info.min_marks_for_concurrent_read; + const size_t increase_num_streams_ratio = std::min(prev_num_streams / num_streams, info.min_marks_for_concurrent_read / 8); + if (increase_num_streams_ratio > 1) + { + num_streams = num_streams * increase_num_streams_ratio; + info.min_marks_for_concurrent_read = (info.sum_marks + num_streams - 1) / num_streams; + } + } + else + num_streams = parts_with_ranges.size(); + } } auto read_type = is_parallel_reading_from_replicas ? ReadType::ParallelReplicas : ReadType::Default; From 31ad4a98c8b0c79fb7d071c5e078d4048efd6640 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 16 Oct 2023 18:38:29 +0000 Subject: [PATCH 331/634] Bump gRPC 1.34 --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index 3f975ecab37..e5276ec3693 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 3f975ecab377cd5f739af780566596128f17bb74 +Subproject commit e5276ec369342e0069f71c3354df9eac69ae3f07 From 67b9407530bb15f7e6d49cd1c2bde7b6c441389b Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Oct 2023 22:01:44 +0200 Subject: [PATCH 332/634] Revert "Integration check script fix ups" --- docs/en/development/continuous-integration.md | 58 ------------------- tests/README.md | 1 - tests/ci/build_download_helper.py | 42 +++++--------- tests/ci/pr_info.py | 36 +++++++----- tests/ci/worker/prepare-ci-ami.sh | 1 - 5 files changed, 36 insertions(+), 102 deletions(-) delete mode 100644 tests/README.md diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index 46a30f56f11..eec5ccbb9dc 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -67,48 +67,6 @@ This check means that the CI system started to process the pull request. When it Performs some simple regex-based checks of code style, using the [`utils/check-style/check-style`](https://github.com/ClickHouse/ClickHouse/blob/master/utils/check-style/check-style) binary (note that it can be run locally). If it fails, fix the style errors following the [code style guide](style.md). -#### Running style check locally: -```sh -mkdir -p /tmp/test_output -# running all checks -docker run --rm --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE clickhouse/style-test - -# run specified check script (e.g.: ./check-mypy) -docker run --rm --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE --entrypoint= -w/ClickHouse/utils/check-style clickhouse/style-test ./check-mypy - -# find all style check scripts under the directory: -cd ./utils/check-style - -# Check duplicate includes -./check-duplicate-includes.sh - -# Check c++ formatiing -./check-style - -# Check python formatting with black -./check-black - -# Check python type hinting with mypy -./check-mypy - -# Check code with codespell -./check-typos - -# Check docs spelling -./check-doc-aspell - -# Check whitespaces -./check-whitespaces - -# Check github actions workflows -./check-workflows - -# Check submodules -./check-submodules - -# Check shell scripts with shellcheck -./shellcheck-run.sh -``` ## Fast Test Normally this is the first check that is ran for a PR. It builds ClickHouse and @@ -117,15 +75,6 @@ some. If it fails, further checks are not started until it is fixed. Look at the report to see which tests fail, then reproduce the failure locally as described [here](tests.md#functional-test-locally). -#### Running Fast Test locally: -```sh -mkdir -p /tmp/test_output -mkdir -p /tmp/fasttest-workspace -cd ClickHouse -# this docker command performs minimal ClickHouse build and run FastTests against it -docker run --rm --cap-add=SYS_PTRACE -u $(id -u ${USER}):$(id -g ${USER}) --network=host -e FASTTEST_WORKSPACE=/fasttest-workspace -e FASTTEST_OUTPUT=/test_output -e FASTTEST_SOURCE=/ClickHouse --cap-add=SYS_PTRACE -e stage=clone_submodules --volume=/tmp/fasttest-workspace:/fasttest-workspace --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output clickhouse/fasttest -``` - #### Status Page Files - `runlog.out.log` is the general log that includes all other logs. @@ -173,13 +122,6 @@ Builds ClickHouse in various configurations for use in further steps. You have t ## Special Build Check Performs static analysis and code style checks using `clang-tidy`. The report is similar to the [build check](#build-check). Fix the errors found in the build log. -#### Running clang-tidy locally: -There is a convenience `packager` script that runs the clang-tidy build in docker -```sh -mkdir build_tidy -./docker/packager/packager --output-dir=./build_tidy --package-type=binary --compiler=clang-17 --debug-build --clang-tidy -``` - ## Functional Stateless Tests Runs [stateless functional tests](tests.md#functional-tests) for ClickHouse diff --git a/tests/README.md b/tests/README.md deleted file mode 100644 index a1fc0f530f2..00000000000 --- a/tests/README.md +++ /dev/null @@ -1 +0,0 @@ -Find CI documents and instructions on running CI checks localy [here](https://clickhouse.com/docs/en/development/continuous-integration). \ No newline at end of file diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 21012f6337d..b76c5433142 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -51,9 +51,9 @@ def get_gh_api( sleep: int = 3, **kwargs: Any, ) -> requests.Response: - """ - Request GH api w/o auth by default, and failover to the get_best_robot_token in case of receiving - "403 rate limit exceeded" or "404 not found" error + """It's a wrapper around get_with_retries that requests GH api w/o auth by + default, and falls back to the get_best_robot_token in case of receiving + "403 rate limit exceeded" error It sets auth automatically when ROBOT_TOKEN is already set by get_best_robot_token """ @@ -71,39 +71,27 @@ def get_gh_api( if grt.ROBOT_TOKEN is not None: set_auth_header() - token_is_set = "Authorization" in kwargs.get("headers", {}) - exc = Exception("A placeholder to satisfy typing and avoid nesting") - try_cnt = 0 - while try_cnt < retries: - try_cnt += 1 + need_retry = False + for _ in range(retries): try: - response = requests.get(url, **kwargs) + response = get_with_retries(url, 1, sleep, **kwargs) response.raise_for_status() return response - except requests.HTTPError as e: - exc = e - ratelimit_exceeded = ( - e.response.status_code == 403 + except requests.HTTPError as exc: + if ( + exc.response.status_code == 403 and b"rate limit exceeded" - in e.response._content # pylint:disable=protected-access - ) - try_auth = e.response.status_code == 404 - if (ratelimit_exceeded or try_auth) and not token_is_set: + in exc.response._content # pylint:disable=protected-access + ): logging.warning( "Received rate limit exception, setting the auth header and retry" ) set_auth_header() - token_is_set = True - try_cnt = 0 - continue - except Exception as e: - exc = e + need_retry = True + break - if try_cnt < retries: - logging.info("Exception '%s' while getting, retry %i", exc, try_cnt) - time.sleep(sleep) - - raise exc + if need_retry: + return get_with_retries(url, retries, sleep, **kwargs) def get_build_name_for_check(check_name: str) -> str: diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 7dbfe124760..830aa936bea 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -4,6 +4,8 @@ import logging import os from typing import Dict, List, Set, Union, Literal +from unidiff import PatchSet # type: ignore + from build_download_helper import get_gh_api from env_helper import ( GITHUB_REPOSITORY, @@ -169,11 +171,7 @@ class PRInfo: response_json = user_orgs_response.json() self.user_orgs = set(org["id"] for org in response_json) - self.diff_urls.append( - f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/master...{self.head_ref}" - ) - + self.diff_urls.append(github_event["pull_request"]["diff_url"]) elif "commits" in github_event: # `head_commit` always comes with `commits` commit_message = github_event["head_commit"]["message"] # type: str @@ -217,12 +215,12 @@ class PRInfo: # files changed in upstream AND master...{self.head_ref} # to get files, changed in current HEAD self.diff_urls.append( - f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/master...{self.head_ref}" + f"https://github.com/{GITHUB_REPOSITORY}/" + f"compare/master...{self.head_ref}.diff" ) self.diff_urls.append( - f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master" + f"https://github.com/{GITHUB_REPOSITORY}/" + f"compare/{self.head_ref}...master.diff" ) # Get release PR number. self.release_pr = get_pr_for_commit(self.base_ref, self.base_ref)[ @@ -234,8 +232,8 @@ class PRInfo: # For release PRs we must get not only files changed in the PR # itself, but as well files changed since we branched out self.diff_urls.append( - f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master" + f"https://github.com/{GITHUB_REPOSITORY}/" + f"compare/{self.head_ref}...master.diff" ) else: print("event.json does not match pull_request or push:") @@ -263,11 +261,19 @@ class PRInfo: raise TypeError("The event does not have diff URLs") for diff_url in self.diff_urls: - response = get_gh_api(diff_url, sleep=RETRY_SLEEP) + response = get_gh_api( + diff_url, + sleep=RETRY_SLEEP, + ) response.raise_for_status() - diff = response.json() - if "files" in diff: - self.changed_files = {f["filename"] for f in diff["files"]} + if "commits" in self.event and self.number == 0: + diff = response.json() + + if "files" in diff: + self.changed_files = {f["filename"] for f in diff["files"]} + else: + diff_object = PatchSet(response.text) + self.changed_files.update({f.path for f in diff_object}) print(f"Fetched info about {len(self.changed_files)} changed files") def get_dict(self): diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index f3e11b6d00a..20e7e3fd53e 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -90,7 +90,6 @@ systemctl restart docker sudo -u ubuntu docker buildx version sudo -u ubuntu docker buildx create --use --name default-builder -# FIXME: remove unidiff as soon as no old PRs could use it, here and in Dockerfile pip install boto3 pygithub requests urllib3 unidiff dohq-artifactory mkdir -p $RUNNER_HOME && cd $RUNNER_HOME From fc354d7fbf46581b131863b4b88cec557c2e2570 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 30 Sep 2023 18:43:20 +0200 Subject: [PATCH 333/634] Reset signals that is catched by clickhouse-client if pager is in use We need to reset signals that had been installed in the setupSignalHandler() (SIGINT/SIGQUIT), like other such tools dies (i.e. less and open file in editor) since terminal will send signals to both processes and so signals will be delivered to the clickhouse-client/local as well, which will be terminated when signal will be delivered second time. So a simple `clickhouse-client --pager less` will be terminated on a second C-C press. And it is really annoying when you are using this way to analyze logs. Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 20 +++++++++++++++++++- src/Client/ClientBase.h | 2 -- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8382f3f6993..02dba86f20a 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -564,7 +564,16 @@ try if (!pager.empty()) { if (SIG_ERR == signal(SIGPIPE, SIG_IGN)) - throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER); + throwFromErrno("Cannot set signal handler for SIGPIPE.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER); + /// We need to reset signals that had been installed in the + /// setupSignalHandler() since terminal will send signals to both + /// processes and so signals will be delivered to the + /// clickhouse-client/local as well, which will be terminated when + /// signal will be delivered second time. + if (SIG_ERR == signal(SIGINT, SIG_IGN)) + throwFromErrno("Cannot set signal handler for SIGINT.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER); + if (SIG_ERR == signal(SIGQUIT, SIG_IGN)) + throwFromErrno("Cannot set signal handler for SIGQUIT.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER); ShellCommand::Config config(pager); config.pipe_stdin_only = true; @@ -1300,6 +1309,15 @@ void ClientBase::resetOutput() { pager_cmd->in.close(); pager_cmd->wait(); + + if (SIG_ERR == signal(SIGPIPE, SIG_DFL)) + throwFromErrno("Cannot set signal handler for SIIGPIEP.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER); + if (SIG_ERR == signal(SIGINT, SIG_DFL)) + throwFromErrno("Cannot set signal handler for SIGINT.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER); + if (SIG_ERR == signal(SIGQUIT, SIG_DFL)) + throwFromErrno("Cannot set signal handler for SIGQUIT.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER); + + setupSignalHandler(); } pager_cmd = nullptr; diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 43f78b0bf78..b4c0bb40254 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -58,8 +58,6 @@ enum ProgressOption ProgressOption toProgressOption(std::string progress); std::istream& operator>> (std::istream & in, ProgressOption & progress); -void interruptSignalHandler(int signum); - class InternalTextLogs; class WriteBufferFromFileDescriptor; From 3257d5643dc260b936aefd485e21752e6edc99c0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 16 Oct 2023 22:24:23 +0200 Subject: [PATCH 334/634] Move adjusting input_format_values_allow_data_after_semicolon into the client base Signed-off-by: Azat Khuzhin --- programs/client/Client.cpp | 10 +--------- programs/local/LocalServer.cpp | 10 +--------- src/Client/ClientBase.cpp | 17 +++++++++++++++++ src/Client/ClientBase.h | 3 +++ 4 files changed, 22 insertions(+), 18 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 9f316e54f85..d73b8f42923 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -320,6 +320,7 @@ try registerAggregateFunctions(); processConfig(); + adjustSettings(); initTtyBuffer(toProgressOption(config().getString("progress", "default"))); { @@ -1264,15 +1265,6 @@ void Client::processConfig() global_context->setQueryKindInitial(); global_context->setQuotaClientKey(config().getString("quota_key", "")); global_context->setQueryKind(query_kind); - - if (is_multiquery && !global_context->getSettingsRef().input_format_values_allow_data_after_semicolon.changed) - { - Settings settings = global_context->getSettings(); - settings.input_format_values_allow_data_after_semicolon = true; - /// Do not send it to the server - settings.input_format_values_allow_data_after_semicolon.changed = false; - global_context->setSettings(settings); - } } diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e074cb638e8..d408d284ba6 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -494,6 +494,7 @@ try registerFormats(); processConfig(); + adjustSettings(); initTtyBuffer(toProgressOption(config().getString("progress", "default"))); applyCmdSettings(global_context); @@ -783,15 +784,6 @@ void LocalServer::processConfig() global_context->setQueryKindInitial(); global_context->setQueryKind(query_kind); - - if (is_multiquery && !global_context->getSettingsRef().input_format_values_allow_data_after_semicolon.changed) - { - Settings settings = global_context->getSettings(); - settings.input_format_values_allow_data_after_semicolon = true; - /// Do not send it to the server - settings.input_format_values_allow_data_after_semicolon.changed = false; - global_context->setSettings(settings); - } } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 02dba86f20a..e449bb7201e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -719,6 +719,23 @@ void ClientBase::initLogsOutputStream() } } +void ClientBase::adjustSettings() +{ + Settings settings = global_context->getSettings(); + + /// NOTE: Do not forget to set changed=false to avoid sending it to the server (to avoid breakage read only profiles) + + /// In case of multi-query we allow data after semicolon since it will be + /// parsed by the client and interpreted as new query + if (is_multiquery && !global_context->getSettingsRef().input_format_values_allow_data_after_semicolon.changed) + { + settings.input_format_values_allow_data_after_semicolon = true; + settings.input_format_values_allow_data_after_semicolon.changed = false; + } + + global_context->setSettings(settings); +} + void ClientBase::initTtyBuffer(ProgressOption progress) { if (tty_buf) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index b4c0bb40254..3ed5be0b831 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -182,6 +182,9 @@ protected: static bool isSyncInsertWithData(const ASTInsertQuery & insert_query, const ContextPtr & context); bool processMultiQueryFromFile(const String & file_name); + /// Adjust some settings after command line options and config had been processed. + void adjustSettings(); + void initTtyBuffer(ProgressOption progress); /// Should be one of the first, to be destroyed the last, From 4cb3de31101efa44db234413f69fe01efe42c8f3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 16 Oct 2023 22:26:49 +0200 Subject: [PATCH 335/634] Add pager property into ClientBase Signed-off-by: Azat Khuzhin --- programs/client/Client.cpp | 2 ++ programs/local/LocalServer.cpp | 2 ++ src/Client/ClientBase.cpp | 1 - src/Client/ClientBase.h | 2 ++ 4 files changed, 6 insertions(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index d73b8f42923..b3e687aa3e5 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1239,6 +1239,8 @@ void Client::processConfig() if (config().has("multiquery")) is_multiquery = true; + pager = config().getString("pager", ""); + is_default_format = !config().has("vertical") && !config().has("format"); if (config().has("vertical")) format = config().getString("format", "Vertical"); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index d408d284ba6..452af7a4ec0 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -578,6 +578,8 @@ void LocalServer::processConfig() if (config().has("multiquery")) is_multiquery = true; + pager = config().getString("pager", ""); + delayed_interactive = config().has("interactive") && (!queries.empty() || config().has("queries-file")); if (!is_interactive || delayed_interactive) { diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index e449bb7201e..f025e29a1ae 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -560,7 +560,6 @@ try } WriteBuffer * out_buf = nullptr; - String pager = config().getString("pager", ""); if (!pager.empty()) { if (SIG_ERR == signal(SIGPIPE, SIG_IGN)) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 3ed5be0b831..604c8cf4d5c 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -213,6 +213,8 @@ protected: bool stderr_is_a_tty = false; /// stderr is a terminal. uint64_t terminal_width = 0; + String pager; + String format; /// Query results output format. bool select_into_file = false; /// If writing result INTO OUTFILE. It affects progress rendering. bool select_into_file_and_stdout = false; /// If writing result INTO OUTFILE AND STDOUT. It affects progress rendering. From 2fc858bfe1604284f6fc1b3e3ffef82dc94e6676 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 16 Oct 2023 22:31:42 +0200 Subject: [PATCH 336/634] Adjust output_format_pretty_max_rows if --pager is specified --- src/Client/ClientBase.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f025e29a1ae..a5a2a6ab328 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -72,6 +72,7 @@ #include #include #include +#include #include #include #include @@ -732,6 +733,13 @@ void ClientBase::adjustSettings() settings.input_format_values_allow_data_after_semicolon.changed = false; } + /// If pager is specified then output_format_pretty_max_rows is ignored, this should be handled by pager. + if (!pager.empty() && !global_context->getSettingsRef().output_format_pretty_max_rows.changed) + { + settings.output_format_pretty_max_rows = std::numeric_limits::max(); + settings.output_format_pretty_max_rows.changed = false; + } + global_context->setSettings(settings); } From 4ca27bdb7b7cb2fb7a3c91a8a4232c785cbbd859 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 11 Oct 2023 18:33:00 +0200 Subject: [PATCH 337/634] Add URI filter for `/query` handler --- src/Server/HTTPHandlerFactory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index f660bafe23f..61c672db8ee 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -181,6 +181,7 @@ void addDefaultHandlersFactory( }; auto query_handler = std::make_shared>(std::move(dynamic_creator)); query_handler->allowPostAndGetParamsAndOptionsRequest(); + query_handler->attachNonStrictPath("/query"); factory.addHandler(query_handler); /// We check that prometheus handler will be served on current (default) port. From ab2d20a4b6ae6e1cb50a201a6ffca0d2e511aaf7 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 11 Oct 2023 21:09:39 +0200 Subject: [PATCH 338/634] Sync query parameter parsing logic between PredefinedQueryHandler and DynamicQueryHandler --- src/Server/HTTPHandler.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 7b88e8f3841..1aec67a7f81 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1193,6 +1193,16 @@ bool PredefinedQueryHandler::customizeQueryParam(ContextMutablePtr context, cons return true; } + if (startsWith(key, QUERY_PARAMETER_NAME_PREFIX)) + { + /// Save name and values of substitution in dictionary. + const String parameter_name = key.substr(strlen(QUERY_PARAMETER_NAME_PREFIX)); + + if (receive_params.contains(parameter_name)) + context->setQueryParameter(parameter_name, value); + return true; + } + return false; } From 3374568ad0c9386189af340e4aba463cf61cc815 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 11 Oct 2023 22:01:40 +0200 Subject: [PATCH 339/634] Register default HTTP handlers as the last ones due to aggressive handler matching for `/query` endpoint --- src/Server/HTTPHandlerFactory.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 61c672db8ee..8b0e8bbf674 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -44,11 +44,12 @@ static inline auto createHandlersFactoryFromConfig( Poco::Util::AbstractConfiguration::Keys keys; config.keys(prefix, keys); + bool enable_default_handlers = false; + for (const auto & key : keys) { - if (key == "defaults") - { - addDefaultHandlersFactory(*main_handler_factory, server, config, async_metrics); + if (key == "defaults") { + enable_default_handlers = true; } else if (startsWith(key, "rule")) { @@ -77,6 +78,10 @@ static inline auto createHandlersFactoryFromConfig( "{}.{}, must be 'rule' or 'defaults'", prefix, key); } + /// Adding default handlers as the last ones due to aggressive handler matching for `/query` + if (enable_default_handlers) + addDefaultHandlersFactory(*main_handler_factory, server, config, async_metrics); + return main_handler_factory; } @@ -181,7 +186,6 @@ void addDefaultHandlersFactory( }; auto query_handler = std::make_shared>(std::move(dynamic_creator)); query_handler->allowPostAndGetParamsAndOptionsRequest(); - query_handler->attachNonStrictPath("/query"); factory.addHandler(query_handler); /// We check that prometheus handler will be served on current (default) port. From 065db21c75c24a7e03f8a12c68198d9dc0544972 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 11 Oct 2023 22:23:17 +0200 Subject: [PATCH 340/634] Update URI filter for `query` HTTP handler --- src/Server/HTTPHandlerFactory.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 8b0e8bbf674..6ea1078182a 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -49,7 +49,7 @@ static inline auto createHandlersFactoryFromConfig( for (const auto & key : keys) { if (key == "defaults") { - enable_default_handlers = true; + addDefaultHandlersFactory(*main_handler_factory, server, config, async_metrics); } else if (startsWith(key, "rule")) { @@ -78,10 +78,6 @@ static inline auto createHandlersFactoryFromConfig( "{}.{}, must be 'rule' or 'defaults'", prefix, key); } - /// Adding default handlers as the last ones due to aggressive handler matching for `/query` - if (enable_default_handlers) - addDefaultHandlersFactory(*main_handler_factory, server, config, async_metrics); - return main_handler_factory; } @@ -186,6 +182,7 @@ void addDefaultHandlersFactory( }; auto query_handler = std::make_shared>(std::move(dynamic_creator)); query_handler->allowPostAndGetParamsAndOptionsRequest(); + query_handler->attachNonStrictPath("/?"); factory.addHandler(query_handler); /// We check that prometheus handler will be served on current (default) port. From cf1c2a667ee62efb80879a60687b253d0996e8e8 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 11 Oct 2023 22:23:49 +0200 Subject: [PATCH 341/634] Add a negative test case for `query` handler --- tests/integration/test_http_handlers_config/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index 1b347f6271f..039eeb7b6c7 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -441,6 +441,7 @@ def test_defaults_http_handlers(): == cluster.instance.http_request("?query=SELECT+1", method="GET").content ) + assert 404 == cluster.instance.http_request("/nonexistent?query=SELECT+1", method="GET").status_code def test_prometheus_handler(): with contextlib.closing( From d38e75e358b171736153c26b4c917b361f7d9514 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 11 Oct 2023 22:28:00 +0200 Subject: [PATCH 342/634] Add a test case for `param_`-prefixed parameters for predefined query handlers --- tests/integration/test_http_handlers_config/test.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index 039eeb7b6c7..56f7abee210 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -159,6 +159,13 @@ def test_predefined_query_handler(): assert cluster.instance.query("SELECT * FROM test_table") == "100\tTEST\n" cluster.instance.query("DROP TABLE test_table") + res4 = cluster.instance.http_request( + "test_predefined_handler_get?max_threads=1¶m_setting_name=max_threads", + method="GET", + headers={"XXX": "xxx"}, + ) + assert b"max_threads\t1\n" == res1.content + def test_fixed_static_handler(): with contextlib.closing( From 320eb24f14320dcce629fb32a2fa4b7374c82eb9 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 11 Oct 2023 22:29:38 +0200 Subject: [PATCH 343/634] Fix formatting --- src/Server/HTTPHandlerFactory.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 6ea1078182a..4c037a988d1 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -48,7 +48,8 @@ static inline auto createHandlersFactoryFromConfig( for (const auto & key : keys) { - if (key == "defaults") { + if (key == "defaults") + { addDefaultHandlersFactory(*main_handler_factory, server, config, async_metrics); } else if (startsWith(key, "rule")) From c68bf8058cf76e4c24094d8d2b29fe5e0d0fc39b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 11 Oct 2023 20:53:49 +0000 Subject: [PATCH 344/634] Automatic style fix --- tests/integration/test_http_handlers_config/test.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index 56f7abee210..fe61043b11f 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -448,7 +448,13 @@ def test_defaults_http_handlers(): == cluster.instance.http_request("?query=SELECT+1", method="GET").content ) - assert 404 == cluster.instance.http_request("/nonexistent?query=SELECT+1", method="GET").status_code + assert ( + 404 + == cluster.instance.http_request( + "/nonexistent?query=SELECT+1", method="GET" + ).status_code + ) + def test_prometheus_handler(): with contextlib.closing( From 72617c587e08a458ca3e0c81b5b13eb621248207 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 12 Oct 2023 15:17:54 +0300 Subject: [PATCH 345/634] Remove previous commits leftovers --- src/Server/HTTPHandlerFactory.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 4c037a988d1..1416112b40d 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -44,8 +44,6 @@ static inline auto createHandlersFactoryFromConfig( Poco::Util::AbstractConfiguration::Keys keys; config.keys(prefix, keys); - bool enable_default_handlers = false; - for (const auto & key : keys) { if (key == "defaults") From ce7eca0615966d4ca3b8fe2e8a6daee779ab2f69 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 16 Oct 2023 17:00:07 -0700 Subject: [PATCH 346/634] DWARF input format (#55450) * Add ReadBufferFromFileBase::isRegularLocalFile() * DWARF input format * Review comments * Changed things around ENABLE_EMBEDDED_COMPILER build setting * Added 'ranges' column * no-msan no-ubsan --- CMakeLists.txt | 1 - cmake/sanitize.cmake | 3 - cmake/target.cmake | 1 + contrib/llvm-project-cmake/CMakeLists.txt | 13 +- docker/test/fasttest/run.sh | 1 - docs/en/development/developer-instruction.md | 2 +- docs/en/interfaces/formats.md | 48 + src/Common/CurrentMetrics.cpp | 2 + src/Common/Elf.cpp | 20 +- src/Common/Elf.h | 22 +- src/Common/config.h.in | 1 + src/Formats/FormatFactory.cpp | 7 +- src/Formats/FormatFactory.h | 3 +- src/Formats/registerFormats.cpp | 4 + src/IO/AsynchronousReadBufferFromFile.h | 4 + src/IO/MMapReadBufferFromFile.cpp | 7 + src/IO/MMapReadBufferFromFile.h | 2 + src/IO/MMapReadBufferFromFileWithCache.h | 2 + src/IO/ReadBufferFromFile.h | 4 + src/IO/ReadBufferFromFileBase.h | 6 + .../Formats/Impl/DWARFBlockInputFormat.cpp | 958 ++++++++++++++++++ .../Formats/Impl/DWARFBlockInputFormat.h | 124 +++ src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageS3.cpp | 5 +- src/Storages/StorageURL.cpp | 5 +- src/configure_config.cmake | 3 +- tests/instructions/sanitizers.md | 3 +- tests/instructions/syntax.txt | 7 +- .../0_stateless/02896_dwarf_format.reference | 54 + .../queries/0_stateless/02896_dwarf_format.sh | 18 + tests/queries/0_stateless/data_dwarf/tiny.cpp | 15 + .../0_stateless/data_dwarf/tiny_dwarf4 | Bin 0 -> 64360 bytes .../0_stateless/data_dwarf/tiny_dwarf5 | Bin 0 -> 65344 bytes .../aspell-ignore/en/aspell-dict.txt | 2 + 34 files changed, 1313 insertions(+), 38 deletions(-) create mode 100644 src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/DWARFBlockInputFormat.h create mode 100644 tests/queries/0_stateless/02896_dwarf_format.reference create mode 100755 tests/queries/0_stateless/02896_dwarf_format.sh create mode 100644 tests/queries/0_stateless/data_dwarf/tiny.cpp create mode 100644 tests/queries/0_stateless/data_dwarf/tiny_dwarf4 create mode 100644 tests/queries/0_stateless/data_dwarf/tiny_dwarf5 diff --git a/CMakeLists.txt b/CMakeLists.txt index 4be0c63e3b8..a5b94efefc5 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -88,7 +88,6 @@ if (ENABLE_FUZZING) set (ENABLE_CLICKHOUSE_ODBC_BRIDGE OFF) set (ENABLE_LIBRARIES 0) set (ENABLE_SSL 1) - set (ENABLE_EMBEDDED_COMPILER 0) set (ENABLE_EXAMPLES 0) set (ENABLE_UTILS 0) set (ENABLE_THINLTO 0) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index b3c8f97a5e1..f17283774eb 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -54,9 +54,6 @@ if (SANITIZE) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${UBSAN_FLAGS}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${UBSAN_FLAGS}") - # llvm-tblgen, that is used during LLVM build, doesn't work with UBSan. - set (ENABLE_EMBEDDED_COMPILER 0 CACHE BOOL "") - else () message (FATAL_ERROR "Unknown sanitizer type: ${SANITIZE}") endif () diff --git a/cmake/target.cmake b/cmake/target.cmake index e4a2f060f1e..887f79bf24e 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -68,6 +68,7 @@ if (CMAKE_CROSSCOMPILING) set (ENABLE_ORC OFF CACHE INTERNAL "") set (ENABLE_GRPC OFF CACHE INTERNAL "") set (ENABLE_EMBEDDED_COMPILER OFF CACHE INTERNAL "") + set (ENABLE_DWARF_PARSER OFF CACHE INTERNAL "") else () message (FATAL_ERROR "Trying to cross-compile to unsupported system: ${CMAKE_SYSTEM_NAME}!") endif () diff --git a/contrib/llvm-project-cmake/CMakeLists.txt b/contrib/llvm-project-cmake/CMakeLists.txt index ce82a10d3eb..d6133f145bc 100644 --- a/contrib/llvm-project-cmake/CMakeLists.txt +++ b/contrib/llvm-project-cmake/CMakeLists.txt @@ -1,12 +1,17 @@ if (APPLE OR SANITIZE STREQUAL "undefined" OR SANITIZE STREQUAL "memory") - set (ENABLE_EMBEDDED_COMPILER_DEFAULT OFF) + # llvm-tblgen, that is used during LLVM build, doesn't work with UBSan. + set (ENABLE_EMBEDDED_COMPILER_DEFAULT OFF) + set (ENABLE_DWARF_PARSER_DEFAULT OFF) else() - set (ENABLE_EMBEDDED_COMPILER_DEFAULT ON) + set (ENABLE_EMBEDDED_COMPILER_DEFAULT ${ENABLE_LIBRARIES}) + set (ENABLE_DWARF_PARSER_DEFAULT ${ENABLE_LIBRARIES}) endif() -option (ENABLE_EMBEDDED_COMPILER "Enable support for JIT compilation during query execution" ${ENABLE_EMBEDDED_COMPILER_DEFAULT}) +option (ENABLE_EMBEDDED_COMPILER "Enable support for JIT compilation during query execution (uses LLVM library)" ${ENABLE_EMBEDDED_COMPILER_DEFAULT}) -if (NOT ENABLE_EMBEDDED_COMPILER) +option (ENABLE_DWARF_PARSER "Enable support for DWARF input format (uses LLVM library)" ${ENABLE_DWARF_PARSER_DEFAULT}) + +if (NOT ENABLE_EMBEDDED_COMPILER AND NOT ENABLE_DWARF_PARSER) message(STATUS "Not using LLVM") return() endif() diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 57789f003f9..1b72dab5e3c 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -175,7 +175,6 @@ function run_cmake "-DENABLE_LIBRARIES=0" "-DENABLE_TESTS=0" "-DENABLE_UTILS=0" - "-DENABLE_EMBEDDED_COMPILER=0" "-DENABLE_THINLTO=0" "-DENABLE_NURAFT=1" "-DENABLE_SIMDJSON=1" diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 8ec049dd004..f51542d538c 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -282,7 +282,7 @@ Also, you can browse sources on [GitHub](https://github.com/ClickHouse/ClickHous If you are not interested in functionality provided by third-party libraries, you can further speed up the build using `cmake` options ``` --DENABLE_LIBRARIES=0 -DENABLE_EMBEDDED_COMPILER=0 +-DENABLE_LIBRARIES=0 ``` In case of problems with any of the development options, you are on your own! diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 58c6280da35..df6acd89616 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -87,6 +87,7 @@ The supported formats are: | [RawBLOB](#rawblob) | ✔ | ✔ | | [MsgPack](#msgpack) | ✔ | ✔ | | [MySQLDump](#mysqldump) | ✔ | ✗ | +| [DWARF](#dwarf) | ✔ | ✗ | | [Markdown](#markdown) | ✗ | ✔ | @@ -2719,6 +2720,53 @@ FROM file(dump.sql, MySQLDump) └───┘ ``` +## DWARF {#dwarf} + +Parses DWARF debug symbols from an ELF file (executable, library, or object file). Similar to `dwarfdump`, but much faster (hundreds of MB/s) and with SQL. Produces one row for each Debug Information Entry (DIE) in the `.debug_info` section. Includes "null" entries that the DWARF encoding uses to terminate lists of children in the tree. + +Quick background: `.debug_info` consists of *units*, corresponding to compilation units. Each unit is a tree of *DIE*s, with a `compile_unit` DIE as its root. Each DIE has a *tag* and a list of *attributes*. Each attribute has a *name* and a *value* (and also a *form*, which specifies how the value is encoded). The DIEs represent things from the source code, and their *tag* tells what kind of thing it is. E.g. there are functions (tag = `subprogram`), classes/structs/enums (`class_type`/`structure_type`/`enumeration_type`), variables (`variable`), function arguments (`formal_parameter`). The tree structure mirrors the corresponding source code. E.g. a `class_type` DIE can contain `subprogram` DIEs representing methods of the class. + +Outputs the following columns: + - `offset` - position of the DIE in the `.debug_info` section + - `size` - number of bytes in the encoded DIE (including attributes) + - `tag` - type of the DIE; the conventional "DW_TAG_" prefix is omitted + - `unit_name` - name of the compilation unit containing this DIE + - `unit_offset` - position of the compilation unit containing this DIE in the `.debug_info` section + - `ancestor_tags` - array of tags of the ancestors of the current DIE in the tree, in order from innermost to outermost + - `ancestor_offsets` - offsets of ancestors, parallel to `ancestor_tags` + - a few common attributes duplicated from the attributes array for convenience: + - `name` + - `linkage_name` - mangled fully-qualified name; typically only functions have it (but not all functions) + - `decl_file` - name of the source code file where this entity was declared + - `decl_line` - line number in the source code where this entity was declared + - parallel arrays describing attributes: + - `attr_name` - name of the attribute; the conventional "DW_AT_" prefix is omitted + - `attr_form` - how the attribute is encoded and interpreted; the conventional DW_FORM_ prefix is omitted + - `attr_int` - integer value of the attribute; 0 if the attribute doesn't have a numeric value + - `attr_str` - string value of the attribute; empty if the attribute doesn't have a string value + +Example: find compilation units that have the most function definitions (including template instantiations and functions from included header files): +```sql +SELECT + unit_name, + count() AS c +FROM file('programs/clickhouse', DWARF) +WHERE tag = 'subprogram' AND NOT has(attr_name, 'declaration') +GROUP BY unit_name +ORDER BY c DESC +LIMIT 3 +``` +```text +┌─unit_name──────────────────────────────────────────────────┬─────c─┐ +│ ./src/Core/Settings.cpp │ 28939 │ +│ ./src/AggregateFunctions/AggregateFunctionSumMap.cpp │ 23327 │ +│ ./src/AggregateFunctions/AggregateFunctionUniqCombined.cpp │ 22649 │ +└────────────────────────────────────────────────────────────┴───────┘ + +3 rows in set. Elapsed: 1.487 sec. Processed 139.76 million rows, 1.12 GB (93.97 million rows/s., 752.77 MB/s.) +Peak memory usage: 271.92 MiB. +``` + ## Markdown {#markdown} You can export results using [Markdown](https://en.wikipedia.org/wiki/Markdown) format to generate output ready to be pasted into your `.md` files: diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 065719e477d..b6d61966973 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -153,6 +153,8 @@ M(ParquetDecoderThreadsActive, "Number of threads in the ParquetBlockInputFormat thread pool running a task.") \ M(ParquetEncoderThreads, "Number of threads in ParquetBlockOutputFormat thread pool.") \ M(ParquetEncoderThreadsActive, "Number of threads in ParquetBlockOutputFormat thread pool running a task.") \ + M(DWARFReaderThreads, "Number of threads in the DWARFBlockInputFormat thread pool.") \ + M(DWARFReaderThreadsActive, "Number of threads in the DWARFBlockInputFormat thread pool running a task.") \ M(OutdatedPartsLoadingThreads, "Number of threads in the threadpool for loading Outdated data parts.") \ M(OutdatedPartsLoadingThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \ M(DistributedBytesToInsert, "Number of pending bytes to process for asynchronous insertion into Distributed tables. Number of bytes for every shard is summed.") \ diff --git a/src/Common/Elf.cpp b/src/Common/Elf.cpp index b83718af2b1..5599df63948 100644 --- a/src/Common/Elf.cpp +++ b/src/Common/Elf.cpp @@ -16,15 +16,27 @@ namespace ErrorCodes } -Elf::Elf(const std::string & path) - : in(path, 0) +Elf::Elf(const std::string & path_) { + in.emplace(path_, 0); + init(in->buffer().begin(), in->buffer().size(), path_); +} + +Elf::Elf(const char * data, size_t size, const std::string & path_) +{ + init(data, size, path_); +} + +void Elf::init(const char * data, size_t size, const std::string & path_) +{ + path = path_; + mapped = data; + elf_size = size; + /// Check if it's an elf. - elf_size = in.buffer().size(); if (elf_size < sizeof(ElfEhdr)) throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "The size of supposedly ELF file '{}' is too small", path); - mapped = in.buffer().begin(); header = reinterpret_cast(mapped); if (memcmp(header->e_ident, "\x7F""ELF", 4) != 0) diff --git a/src/Common/Elf.h b/src/Common/Elf.h index 5a6bd9e302d..a1a63cf0038 100644 --- a/src/Common/Elf.h +++ b/src/Common/Elf.h @@ -9,16 +9,14 @@ #include #include -#include -using ElfAddr = ElfW(Addr); -using ElfEhdr = ElfW(Ehdr); -using ElfOff = ElfW(Off); -using ElfPhdr = ElfW(Phdr); -using ElfShdr = ElfW(Shdr); -using ElfNhdr = ElfW(Nhdr); -using ElfSym = ElfW(Sym); +using ElfEhdr = Elf64_Ehdr; +using ElfOff = Elf64_Off; +using ElfPhdr = Elf64_Phdr; +using ElfShdr = Elf64_Shdr; +using ElfNhdr = Elf64_Nhdr; +using ElfSym = Elf64_Sym; namespace DB @@ -44,7 +42,8 @@ public: const Elf & elf; }; - explicit Elf(const std::string & path); + explicit Elf(const std::string & path_); + Elf(const char * data, size_t size, const std::string & path_); bool iterateSections(std::function && pred) const; std::optional
findSection(std::function && pred) const; @@ -64,13 +63,16 @@ public: String getStoredBinaryHash() const; private: - MMapReadBufferFromFile in; + std::string path; // just for error messages + std::optional in; size_t elf_size; const char * mapped; const ElfEhdr * header; const ElfShdr * section_headers; const ElfPhdr * program_headers; const char * section_names = nullptr; + + void init(const char * data, size_t size, const std::string & path_); }; } diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 72932dda55d..dbd8572f7b9 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -43,6 +43,7 @@ #cmakedefine01 USE_AMQPCPP #cmakedefine01 USE_NATSIO #cmakedefine01 USE_EMBEDDED_COMPILER +#cmakedefine01 USE_DWARF_PARSER #cmakedefine01 USE_LDAP #cmakedefine01 USE_ROCKSDB #cmakedefine01 USE_LIBPQXX diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index b355d785715..9b391324455 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -257,7 +257,8 @@ InputFormatPtr FormatFactory::getInput( std::optional _max_parsing_threads, std::optional _max_download_threads, bool is_remote_fs, - CompressionMethod compression) const + CompressionMethod compression, + bool need_only_count) const { const auto& creators = getCreators(name); if (!creators.input_creator && !creators.random_access_input_creator) @@ -285,7 +286,9 @@ InputFormatPtr FormatFactory::getInput( // Decide whether to use ParallelParsingInputFormat. - bool parallel_parsing = max_parsing_threads > 1 && settings.input_format_parallel_parsing && creators.file_segmentation_engine && !creators.random_access_input_creator; + bool parallel_parsing = + max_parsing_threads > 1 && settings.input_format_parallel_parsing && creators.file_segmentation_engine && + !creators.random_access_input_creator && !need_only_count; if (settings.max_memory_usage && settings.min_chunk_bytes_for_parallel_parsing * max_parsing_threads * 2 > settings.max_memory_usage) parallel_parsing = false; diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 48a1869d563..d1e8c350391 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -167,7 +167,8 @@ public: bool is_remote_fs = false, // allows to do: buf -> parallel read -> decompression, // because parallel read after decompression is not possible - CompressionMethod compression = CompressionMethod::None) const; + CompressionMethod compression = CompressionMethod::None, + bool need_only_count = false) const; /// Checks all preconditions. Returns ordinary format if parallel formatting cannot be done. OutputFormatPtr getOutputFormatParallelIfPossible( diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index 580db61edde..ddeb1a34512 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -101,6 +101,7 @@ void registerInputFormatJSONAsObject(FormatFactory & factory); void registerInputFormatLineAsString(FormatFactory & factory); void registerInputFormatMySQLDump(FormatFactory & factory); void registerInputFormatParquetMetadata(FormatFactory & factory); +void registerInputFormatDWARF(FormatFactory & factory); void registerInputFormatOne(FormatFactory & factory); #if USE_HIVE @@ -143,6 +144,7 @@ void registerTemplateSchemaReader(FormatFactory & factory); void registerMySQLSchemaReader(FormatFactory & factory); void registerBSONEachRowSchemaReader(FormatFactory & factory); void registerParquetMetadataSchemaReader(FormatFactory & factory); +void registerDWARFSchemaReader(FormatFactory & factory); void registerOneSchemaReader(FormatFactory & factory); void registerFileExtensions(FormatFactory & factory); @@ -245,6 +247,7 @@ void registerFormats() registerInputFormatMySQLDump(factory); registerInputFormatParquetMetadata(factory); + registerInputFormatDWARF(factory); registerInputFormatOne(factory); registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(factory); @@ -282,6 +285,7 @@ void registerFormats() registerMySQLSchemaReader(factory); registerBSONEachRowSchemaReader(factory); registerParquetMetadataSchemaReader(factory); + registerDWARFSchemaReader(factory); registerOneSchemaReader(factory); } diff --git a/src/IO/AsynchronousReadBufferFromFile.h b/src/IO/AsynchronousReadBufferFromFile.h index d3b7ffbc7dc..5b39b707803 100644 --- a/src/IO/AsynchronousReadBufferFromFile.h +++ b/src/IO/AsynchronousReadBufferFromFile.h @@ -45,6 +45,8 @@ public: { return file_name; } + + bool isRegularLocalFile(size_t * /* out_view_offset */) override { return true; } }; /** Similar to AsynchronousReadBufferFromFile but also transparently shares open file descriptors. @@ -79,6 +81,8 @@ public: { return file_name; } + + bool isRegularLocalFile(size_t * /* out_view_offset */) override { return true; } }; } diff --git a/src/IO/MMapReadBufferFromFile.cpp b/src/IO/MMapReadBufferFromFile.cpp index 86e05d7ae44..0596eba565f 100644 --- a/src/IO/MMapReadBufferFromFile.cpp +++ b/src/IO/MMapReadBufferFromFile.cpp @@ -40,6 +40,13 @@ std::string MMapReadBufferFromFile::getFileName() const } +bool MMapReadBufferFromFile::isRegularLocalFile(size_t * out_view_offset) +{ + *out_view_offset = mapped.getOffset(); + return true; +} + + MMapReadBufferFromFile::MMapReadBufferFromFile(const std::string & file_name_, size_t offset, size_t length_) : file_name(file_name_) { diff --git a/src/IO/MMapReadBufferFromFile.h b/src/IO/MMapReadBufferFromFile.h index bc566a0489c..1043e0a661f 100644 --- a/src/IO/MMapReadBufferFromFile.h +++ b/src/IO/MMapReadBufferFromFile.h @@ -27,6 +27,8 @@ public: std::string getFileName() const override; + bool isRegularLocalFile(size_t * out_view_offset) override; + private: int fd = -1; std::string file_name; diff --git a/src/IO/MMapReadBufferFromFileWithCache.h b/src/IO/MMapReadBufferFromFileWithCache.h index ff84f81610a..cb87b03df8d 100644 --- a/src/IO/MMapReadBufferFromFileWithCache.h +++ b/src/IO/MMapReadBufferFromFileWithCache.h @@ -20,6 +20,8 @@ public: std::string getFileName() const override; off_t seek(off_t offset, int whence) override; + bool isRegularLocalFile(size_t * /* out_view_offset */) override { return true; } + private: MMappedFileCache::MappedPtr mapped; diff --git a/src/IO/ReadBufferFromFile.h b/src/IO/ReadBufferFromFile.h index 462453d974d..30325463cdf 100644 --- a/src/IO/ReadBufferFromFile.h +++ b/src/IO/ReadBufferFromFile.h @@ -53,6 +53,8 @@ public: } size_t getFileOffsetOfBufferEnd() const override { return file_offset_of_buffer_end; } + + bool isRegularLocalFile(size_t * /* out_view_offset */) override { return true; } }; @@ -103,6 +105,8 @@ public: { return file_name; } + + bool isRegularLocalFile(size_t * /* out_view_offset */) override { return true; } }; } diff --git a/src/IO/ReadBufferFromFileBase.h b/src/IO/ReadBufferFromFileBase.h index b77db29bc23..296edf9c689 100644 --- a/src/IO/ReadBufferFromFileBase.h +++ b/src/IO/ReadBufferFromFileBase.h @@ -54,6 +54,12 @@ public: void setProgressCallback(ContextPtr context); + /// Returns true if this file is on local filesystem, and getFileName() is its path. + /// I.e. it can be read using open() or mmap(). If this buffer is a "view" into a subrange of the + /// file, *out_view_offset is set to the start of that subrange, i.e. the difference between actual + /// file offset and what getPosition() returns. + virtual bool isRegularLocalFile(size_t * /* out_view_offset */ = nullptr) { return false; } + protected: std::optional file_size; ProfileCallback profile_callback; diff --git a/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp b/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp new file mode 100644 index 00000000000..c5f8059f93c --- /dev/null +++ b/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp @@ -0,0 +1,958 @@ +#include "DWARFBlockInputFormat.h" +#if USE_DWARF_PARSER && defined(__ELF__) && !defined(OS_FREEBSD) + +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace CurrentMetrics +{ + extern const Metric DWARFReaderThreads; + extern const Metric DWARFReaderThreadsActive; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int CANNOT_PARSE_ELF; + extern const int CANNOT_PARSE_DWARF; +} + +enum DwarfColumn +{ + COL_OFFSET, + COL_SIZE, + COL_TAG, + COL_UNIT_NAME, + COL_UNIT_OFFSET, + + COL_ANCESTOR_TAGS, + COL_ANCESTOR_OFFSETS, + + /// A few very common attributes get their own columns, just for convenience. + /// We put their values *both* in the dedicated columns and in the attr_str/attr_int arrays. + /// This duplication wastes considerable time and space (tens of percent), but I can't think of + /// an alternative that wouldn't be really inconvenient or confusing: + /// * omitting these attributes from the arrays would make collecting attribute stats inconvenient, + /// and would lose information about the form of the attribute, + /// * using empty value for the attribute would be confusing and error-prone, e.g. when collecting stats + /// about all attribute values the user would need to add these columns too, somehow, + /// * not having these dedicated columns would make it inconvenient to look up entry name/file/line. + /// (But maybe that's fine? I.e. maybe it's not very commonly used and maybe the array lookup is not that inconvenient? Idk.) + + COL_NAME, + COL_LINKAGE_NAME, + COL_DECL_FILE, + COL_DECL_LINE, + COL_RANGES, + + COL_ATTR_NAME, + COL_ATTR_FORM, + COL_ATTR_INT, + COL_ATTR_STR, + + COL_COUNT, +}; + +static NamesAndTypesList getHeaderForDWARF() +{ + std::vector cols(COL_COUNT); + cols[COL_OFFSET] = {"offset", std::make_shared()}; + cols[COL_SIZE] = {"size", std::make_shared()}; + cols[COL_TAG] = {"tag", std::make_shared(std::make_shared())}; + cols[COL_UNIT_NAME] = {"unit_name", std::make_shared(std::make_shared())}; + cols[COL_UNIT_OFFSET] = {"unit_offset", std::make_shared(std::make_shared())}; + cols[COL_ANCESTOR_TAGS] = {"ancestor_tags", std::make_shared(std::make_shared(std::make_shared()))}; + cols[COL_ANCESTOR_OFFSETS] = {"ancestor_offsets", std::make_shared(std::make_shared())}; + cols[COL_NAME] = {"name", std::make_shared()}; + cols[COL_LINKAGE_NAME] = {"linkage_name", std::make_shared()}; + cols[COL_DECL_FILE] = {"decl_file", std::make_shared(std::make_shared())}; + cols[COL_DECL_LINE] = {"decl_line", std::make_shared()}; + cols[COL_RANGES] = {"ranges", std::make_shared(std::make_shared(DataTypes{std::make_shared(), std::make_shared()}))}; + cols[COL_ATTR_NAME] = {"attr_name", std::make_shared(std::make_shared(std::make_shared()))}; + cols[COL_ATTR_FORM] = {"attr_form", std::make_shared(std::make_shared(std::make_shared()))}; + cols[COL_ATTR_INT] = {"attr_int", std::make_shared(std::make_shared())}; + cols[COL_ATTR_STR] = {"attr_str", std::make_shared(std::make_shared(std::make_shared()))}; + return NamesAndTypesList(cols.begin(), cols.end()); +} + +static const std::unordered_map & getColumnNameToIdx() +{ + static std::once_flag once; + static std::unordered_map name_to_idx; + std::call_once(once, [&] { + size_t i = 0; + for (const auto & c : getHeaderForDWARF()) + { + name_to_idx.emplace(c.name, i); + ++i; + } + }); + return name_to_idx; +} + +DWARFBlockInputFormat::UnitState::UnitState(llvm::DWARFUnit * u) + : dwarf_unit(u), end_offset(dwarf_unit->getNextUnitOffset()) + , offset(dwarf_unit->getOffset() + dwarf_unit->getHeaderSize()) +{ + /// This call is not thread safe, so we do it during initialization. + abbrevs = dwarf_unit->getAbbreviations(); + if (abbrevs == nullptr) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Couldn't find abbreviation set for unit at offset {}", dwarf_unit->getOffset()); + + /// This call initializes some data structures inside DWARFUnit that are needed for parsing attributes. + auto err = u->tryExtractDIEsIfNeeded(/*CUDieOnly*/ true); + if (err) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Failed to parse compilation unit entry: {}", llvm::toString(std::move(err))); +} + +static llvm::StringRef removePrefix(llvm::StringRef s, size_t prefix_len) +{ + if (s.size() >= prefix_len) + s = llvm::StringRef(s.data() + prefix_len, s.size() - prefix_len); + return s; +} + +template +static void append(C & col, llvm::StringRef s) +{ + col->insertData(s.data(), s.size()); +} + +DWARFBlockInputFormat::DWARFBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_, size_t num_threads_) + : IInputFormat(std::move(header_), &in_), format_settings(format_settings_), num_threads(num_threads_) +{ + auto tag_names = ColumnString::create(); + /// Note: TagString() returns empty string for tags that don't exist, and tag 0 doesn't exist. + constexpr std::string_view DW_TAG_ = "DW_TAG_"; + for (uint32_t tag = 0; tag <= UINT16_MAX; ++tag) + append(tag_names, removePrefix(llvm::dwarf::TagString(tag), DW_TAG_.size())); + tag_dict_column = ColumnUnique::create(std::move(tag_names), /*is_nullable*/ false); + + auto attr_names = ColumnString::create(); + constexpr std::string_view DW_AT_ = "DW_AT_"; + for (uint32_t attr = 0; attr <= UINT16_MAX; ++attr) + append(attr_names, removePrefix(llvm::dwarf::AttributeString(attr), DW_AT_.size())); + attr_name_dict_column = ColumnUnique::create(std::move(attr_names), /*is_nullable*/ false); + + auto attr_forms = ColumnString::create(); + constexpr std::string_view DW_FORM_ = "DW_FORM_"; + for (uint32_t form = 0; form <= UINT16_MAX; ++form) + append(attr_forms, removePrefix(llvm::dwarf::FormEncodingString(form), DW_FORM_.size())); + attr_form_dict_column = ColumnUnique::create(std::move(attr_forms), /*is_nullable*/ false); +} + +DWARFBlockInputFormat::~DWARFBlockInputFormat() +{ + stopThreads(); +} + +void DWARFBlockInputFormat::initELF() +{ + /// If it's a local file, mmap it. + if (ReadBufferFromFileBase * file_in = dynamic_cast(in)) + { + size_t offset = 0; + if (file_in->isRegularLocalFile(&offset) && offset == 0) + { + elf.emplace(file_in->getFileName()); + return; + } + } + + /// If can't mmap, read the entire file into memory. + /// We could read just the .debug_* sections, but typically they take up most of the binary anyway (60% for clickhouse debug build). + { + WriteBufferFromVector buf(file_contents); + copyData(*in, buf, is_stopped); + buf.finalize(); + } + elf.emplace(file_contents.data(), file_contents.size(), ""); +} + +void DWARFBlockInputFormat::initializeIfNeeded() +{ + if (elf.has_value()) + return; + + LOG_DEBUG(&Poco::Logger::get("DWARF"), "Opening ELF"); + initELF(); + if (is_stopped) + return; + + auto info_section = elf->findSectionByName(".debug_info"); + if (!info_section.has_value()) + throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "No .debug_info section"); + auto abbrev_section = elf->findSectionByName(".debug_abbrev"); + if (!abbrev_section.has_value()) + throw Exception(ErrorCodes::CANNOT_PARSE_ELF, "No .debug_abbrev section"); + LOG_DEBUG(&Poco::Logger::get("DWARF"), ".debug_abbrev is {:.3f} MiB, .debug_info is {:.3f} MiB", abbrev_section->size() * 1. / (1 << 20), info_section->size() * 1. / (1 << 20)); + + /// (The StringRef points into Elf's mmap of the whole file, or into file_contents.) + extractor.emplace(llvm::StringRef(info_section->begin(), info_section->size()), /*IsLittleEndian*/ true, /*AddressSize*/ 8); + + if (auto line_section = elf->findSectionByName(".debug_line")) + debug_line_extractor.emplace(llvm::StringRef(line_section->begin(), line_section->size()), /*IsLittleEndian*/ true, /*AddressSize*/ 8); + if (auto addr_section = elf->findSectionByName(".debug_addr")) + debug_addr_section.emplace(addr_section->begin(), addr_section->size()); + if (auto rnglists_section = elf->findSectionByName(".debug_rnglists")) + debug_rnglists_extractor.emplace(llvm::StringRef(rnglists_section->begin(), rnglists_section->size()), /*IsLittleEndian*/ true, /*AddressSize*/ 8); + if (auto ranges_section = elf->findSectionByName(".debug_ranges")) + debug_ranges_extractor.emplace(llvm::StringRef(ranges_section->begin(), ranges_section->size()), /*IsLittleEndian*/ true, /*AddressSize*/ 8); + + llvm::StringMap> sections; + elf->iterateSections([&](const Elf::Section & section, size_t /*idx*/) + { + std::string name = section.name(); + std::string name_without_dot = name.starts_with(".") ? name.substr(1) : name; + sections.try_emplace(name_without_dot, llvm::MemoryBuffer::getMemBuffer( + llvm::StringRef(section.begin(), section.size()), /*BufferName*/ name, /*RequiresNullTerminator*/ false)); + return false; + }); + dwarf_context = llvm::DWARFContext::create(sections, /*AddrSize*/ 8); + + for (std::unique_ptr & unit : dwarf_context->info_section_units()) + units_queue.emplace_back(unit.get()); + + LOG_DEBUG(&Poco::Logger::get("DWARF"), "{} units, reading in {} threads", units_queue.size(), num_threads); + + pool.emplace(CurrentMetrics::DWARFReaderThreads, CurrentMetrics::DWARFReaderThreadsActive, num_threads); + for (size_t i = 0; i < num_threads; ++i) + pool->scheduleOrThrowOnError( + [this, thread_group = CurrentThread::getGroup()]() + { + if (thread_group) + CurrentThread::attachToGroupIfDetached(thread_group); + SCOPE_EXIT_SAFE(if (thread_group) CurrentThread::detachFromGroupIfNotDetached();); + try + { + setThreadName("DWARFDecoder"); + + std::unique_lock lock(mutex); + while (!units_queue.empty() && !is_stopped) + { + if (delivery_queue.size() > num_threads) + { + wake_up_threads.wait(lock); + continue; + } + UnitState unit = std::move(units_queue.front()); + units_queue.pop_front(); + ++units_in_progress; + + lock.unlock(); + + size_t offset_before = unit.offset; + Chunk chunk = parseEntries(unit); + size_t offset_after = unit.offset; + + lock.lock(); + + --units_in_progress; + if (chunk) + { + delivery_queue.emplace_back(std::move(chunk), offset_after - offset_before); + deliver_chunk.notify_one(); + } + if (!unit.eof()) + units_queue.push_front(std::move(unit)); + } + } + catch (...) + { + std::lock_guard lock(mutex); + background_exception = std::current_exception(); + deliver_chunk.notify_all(); + } + }); +} + +void DWARFBlockInputFormat::stopThreads() +{ + { + std::unique_lock lock(mutex); // required even if is_stopped is atomic + is_stopped = true; + } + wake_up_threads.notify_all(); + if (pool) + pool->wait(); +} + +static inline void throwIfError(llvm::Error & e, const char * what) +{ + if (!e) + return; + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Failed to parse {}: {}", what, llvm::toString(std::move(e))); +} + +Chunk DWARFBlockInputFormat::parseEntries(UnitState & unit) +{ + const auto & header = getPort().getHeader(); + const auto & column_name_to_idx = getColumnNameToIdx(); + std::array need{}; + for (const std::string & name : header.getNames()) + need[column_name_to_idx.at(name)] = true; + auto form_params = unit.dwarf_unit->getFormParams(); + + /// For parallel arrays, we nominate one of them to be responsible for populating the offsets vector. + need[COL_ATTR_NAME] = need[COL_ATTR_NAME] || need[COL_ATTR_FORM] || need[COL_ATTR_INT] || need[COL_ATTR_STR]; + need[COL_ANCESTOR_TAGS] = need[COL_ANCESTOR_TAGS] || need[COL_ANCESTOR_OFFSETS]; + + auto col_offset = ColumnVector::create(); + auto col_size = ColumnVector::create(); + auto col_tag = ColumnVector::create(); + auto col_ancestor_tags = ColumnVector::create(); + auto col_ancestor_dwarf_offsets = ColumnVector::create(); + auto col_ancestor_array_offsets = ColumnVector::create(); + auto col_name = ColumnString::create(); + auto col_linkage_name = ColumnString::create(); + ColumnLowCardinality::Index col_decl_file; + auto col_decl_line = ColumnVector::create(); + auto col_ranges_start = ColumnVector::create(); + auto col_ranges_end = ColumnVector::create(); + auto col_ranges_offsets = ColumnVector::create(); + auto col_attr_name = ColumnVector::create(); + auto col_attr_form = ColumnVector::create(); + auto col_attr_int = ColumnVector::create(); + auto col_attr_str = ColumnLowCardinality::create(MutableColumnPtr(ColumnUnique::create(ColumnString::create()->cloneResized(1), /*is_nullable*/ false)), MutableColumnPtr(ColumnVector::create())); + auto col_attr_offsets = ColumnVector::create(); + size_t num_rows = 0; + auto err = llvm::Error::success(); + + while (num_rows < 65536) + { + ++num_rows; + uint64_t die_offset = unit.offset; + if (need[COL_OFFSET]) + col_offset->insertValue(die_offset); + if (need[COL_ANCESTOR_TAGS]) + { + for (size_t i = unit.stack.size() - 1; i != UINT64_MAX; --i) + { + col_ancestor_tags->insertValue(unit.stack[i].tag); + if (need[COL_ANCESTOR_OFFSETS]) + col_ancestor_dwarf_offsets->insertValue(unit.stack[i].offset); + } + col_ancestor_array_offsets->insertValue(col_ancestor_tags->size()); + } + + uint64_t abbrev_code = extractor->getULEB128(&unit.offset, &err); + throwIfError(err, "DIE header"); + + if (abbrev_code == 0) + { + if (need[COL_SIZE]) + col_size->insertValue(static_cast(unit.offset - die_offset)); + if (need[COL_TAG]) + col_tag->insertValue(0); // "null" + + if (need[COL_NAME]) col_name->insertDefault(); + if (need[COL_LINKAGE_NAME]) col_linkage_name->insertDefault(); + if (need[COL_DECL_FILE]) col_decl_file.insertPosition(0); + if (need[COL_DECL_LINE]) col_decl_line->insertDefault(); + if (need[COL_RANGES]) col_ranges_offsets->insertValue(col_ranges_start->size()); + if (need[COL_ATTR_NAME]) col_attr_offsets->insertValue(col_attr_name->size()); + + if (unit.stack.empty()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Stack underflow"); + unit.stack.pop_back(); + } + else + { + const llvm::DWARFAbbreviationDeclaration * abbrev = unit.abbrevs->getAbbreviationDeclaration(static_cast(abbrev_code)); + if (abbrev == nullptr || abbrev_code > UINT32_MAX) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Abbrev code in DIE header is out of bounds: {}, offset {}", abbrev_code, unit.offset); + + auto tag = abbrev->getTag(); + if (need[COL_TAG]) + col_tag->insertValue(tag); + + bool need_name = need[COL_NAME]; + bool need_linkage_name = need[COL_LINKAGE_NAME]; + bool need_decl_file = need[COL_DECL_FILE]; + bool need_decl_line = need[COL_DECL_LINE]; + + /// Address ranges can be encoded in the following ways: + /// * DW_AT_ranges - list of ranges; points to other sections, nontrivial to parse; + /// DW_AT_low_pc may also be present, used as base address during parsing + /// * [DW_AT_low_pc, DW_AT_high_pc) if DW_AT_high_pc has form "address" + /// * [DW_AT_low_pc, DW_AT_low_pc + DW_AT_high_pc) if DW_AT_high_pc has form "constant" + /// * [DW_AT_low_pc, DW_AT_low_pc + 1) if DW_AT_high_pc is absent + bool need_ranges = need[COL_RANGES]; + std::optional ranges; + bool ranges_rnglistx = false; // `ranges` has an extra indirection through offsets table + std::optional low_pc; + std::optional high_pc; + bool relative_high_pc = false; + + for (auto attr : abbrev->attributes()) + { + auto val = llvm::DWARFFormValue::createFromSValue(attr.Form, attr.isImplicitConst() ? attr.getImplicitConstValue() : 0); + /// This is relatively slow, maybe we should reimplement it. + if (!val.extractValue(*extractor, &unit.offset, form_params, unit.dwarf_unit)) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Failed to parse attribute {} of form {} at offset {}", + llvm::dwarf::AttributeString(attr.Attr), attr.Form, unit.offset); + + if (need[COL_ATTR_NAME]) + col_attr_name->insertValue(attr.Attr); + /// Note that in case of DW_FORM_implicit_const val.getForm() is different from attr.Form. + /// Not sure which one would be more useful in the attr_form column. Guessing attr.Form for now. + if (need[COL_ATTR_FORM]) + col_attr_form->insertValue(attr.Form); + + if (attr.Attr == llvm::dwarf::DW_AT_stmt_list && unit.filename_table == nullptr) + { + /// We expect that this attribute appears before any attributes that point into the filename table. + auto offset = val.getAsSectionOffset(); + if (offset.has_value()) + parseFilenameTable(unit, offset.value()); + } + + switch (val.getForm()) // (may be different from attr.Form because of DW_FORM_indirect) + { + /// A 64-bit value. + case llvm::dwarf::DW_FORM_data2: + case llvm::dwarf::DW_FORM_data4: + case llvm::dwarf::DW_FORM_data8: + case llvm::dwarf::DW_FORM_data1: + case llvm::dwarf::DW_FORM_sdata: + case llvm::dwarf::DW_FORM_udata: + case llvm::dwarf::DW_FORM_data16: + case llvm::dwarf::DW_FORM_flag: + case llvm::dwarf::DW_FORM_flag_present: + case llvm::dwarf::DW_FORM_loclistx: // points to .debug_loclists + case llvm::dwarf::DW_FORM_rnglistx: // points to .debug_rnglists + case llvm::dwarf::DW_FORM_sec_offset: // points to some other section, depending on attr.Attr + case llvm::dwarf::DW_FORM_implicit_const: + if (need[COL_ATTR_INT]) col_attr_int->insertValue(val.getRawUValue()); + + if (attr.Attr == llvm::dwarf::DW_AT_decl_line && std::exchange(need_decl_line, false)) + col_decl_line->insertValue(static_cast(val.getRawUValue())); + + /// Starting offset of this unit's data in .debug_addr section. + if (attr.Attr == llvm::dwarf::DW_AT_addr_base) + unit.addr_base = val.getRawUValue(); + /// Same for .debug_rnglists section. + if (attr.Attr == llvm::dwarf::DW_AT_rnglists_base) + unit.rnglists_base = val.getRawUValue(); + + if (attr.Attr == llvm::dwarf::DW_AT_high_pc) + { + high_pc = val.getRawUValue(); + relative_high_pc = true; + } + + if (attr.Attr == llvm::dwarf::DW_AT_ranges) + { + ranges = val.getRawUValue(); + ranges_rnglistx = val.getForm() == llvm::dwarf::DW_FORM_rnglistx; + } + + /// Some attribute values are indices into lookup tables that we can stringify usefully. + if ((attr.Attr == llvm::dwarf::DW_AT_decl_file || attr.Attr == llvm::dwarf::DW_AT_call_file) && + val.getRawUValue() < unit.filename_table_size) // filename + { + UInt64 idx = val.getRawUValue() + 1; + if (attr.Attr == llvm::dwarf::DW_AT_decl_file && std::exchange(need_decl_file, false)) + col_decl_file.insertPosition(idx); + + if (need[COL_ATTR_STR]) + { + auto data = unit.filename_table->getDataAt(idx); + col_attr_str->insertData(data.data, data.size); + } + } + else if (need[COL_ATTR_STR]) + { + static constexpr std::string_view DW_LANG_ = "DW_LANG_"; + static constexpr std::string_view DW_ATE_ = "DW_ATE_"; + if (attr.Attr == llvm::dwarf::DW_AT_language) // programming language + append(col_attr_str, removePrefix(llvm::dwarf::LanguageString(static_cast(val.getRawUValue())), + DW_LANG_.size())); + else if (attr.Attr == llvm::dwarf::DW_AT_encoding) // primitive type + append(col_attr_str, removePrefix(llvm::dwarf::AttributeEncodingString(static_cast(val.getRawUValue())), + DW_ATE_.size())); + else + col_attr_str->insertDefault(); + } + break; + + /// An address, i.e. just a 64-bit value. + /// May have indirection to .debug_addr section. + case llvm::dwarf::DW_FORM_addr: + case llvm::dwarf::DW_FORM_addrx: + case llvm::dwarf::DW_FORM_addrx1: + case llvm::dwarf::DW_FORM_addrx2: + case llvm::dwarf::DW_FORM_addrx3: + case llvm::dwarf::DW_FORM_addrx4: + case llvm::dwarf::DW_FORM_GNU_addr_index: + case llvm::dwarf::DW_FORM_LLVM_addrx_offset: + if (need[COL_ATTR_INT]) col_attr_int->insertValue(val.getAsAddress().value_or(0)); + if (need[COL_ATTR_STR]) col_attr_str->insertDefault(); + + if (need_ranges && (attr.Attr == llvm::dwarf::DW_AT_low_pc || attr.Attr == llvm::dwarf::DW_AT_high_pc)) + { + UInt64 addr; + if (val.getForm() == llvm::dwarf::DW_FORM_addr) + addr = val.getRawUValue(); + else if (val.getForm() == llvm::dwarf::DW_FORM_addrx || + (val.getForm() >= llvm::dwarf::DW_FORM_addrx1 && + val.getForm() <= llvm::dwarf::DW_FORM_addrx4)) + addr = fetchFromDebugAddr(unit.addr_base, val.getRawUValue()); + else + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Form {} for {} is not supported", llvm::dwarf::FormEncodingString(val.getForm()), llvm::dwarf::AttributeString(attr.Attr)); + + if (attr.Attr == llvm::dwarf::DW_AT_low_pc) + low_pc = addr; + else + high_pc = addr; + } + + break; + + /// A byte string. + case llvm::dwarf::DW_FORM_block2: + case llvm::dwarf::DW_FORM_block4: + case llvm::dwarf::DW_FORM_block: + case llvm::dwarf::DW_FORM_block1: + case llvm::dwarf::DW_FORM_exprloc: // DWARF expression + { + auto slice = val.getAsBlock().value_or(llvm::ArrayRef()); + if (need[COL_ATTR_STR]) col_attr_str->insertData(reinterpret_cast(slice.data()), slice.size()); + if (need[COL_ATTR_INT]) col_attr_int->insertDefault(); + break; + } + + /// A text string. + /// May have indirection to .debug_str or .debug_line_str. + case llvm::dwarf::DW_FORM_string: + case llvm::dwarf::DW_FORM_strp: + case llvm::dwarf::DW_FORM_strx: + case llvm::dwarf::DW_FORM_strp_sup: + case llvm::dwarf::DW_FORM_line_strp: + case llvm::dwarf::DW_FORM_strx1: + case llvm::dwarf::DW_FORM_strx2: + case llvm::dwarf::DW_FORM_strx3: + case llvm::dwarf::DW_FORM_strx4: + case llvm::dwarf::DW_FORM_GNU_str_index: + case llvm::dwarf::DW_FORM_GNU_strp_alt: + { + auto res = val.getAsCString(); + if (auto e = res.takeError()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, + "Error parsing string attribute: {}", llvm::toString(std::move(e))); + size_t len = strlen(*res); + + if (attr.Attr == llvm::dwarf::DW_AT_name) + { + if (std::exchange(need_name, false)) + col_name->insertData(*res, len); + if (tag == llvm::dwarf::DW_TAG_compile_unit) + unit.unit_name = *res; + } + if (attr.Attr == llvm::dwarf::DW_AT_linkage_name && std::exchange(need_linkage_name, false)) + col_linkage_name->insertData(*res, len); + + if (need[COL_ATTR_STR]) col_attr_str->insertData(*res, len); + if (need[COL_ATTR_INT]) col_attr_int->insertDefault(); + break; + } + + /// Offset of another entry in .debug_info. + case llvm::dwarf::DW_FORM_ref_addr: + case llvm::dwarf::DW_FORM_ref1: + case llvm::dwarf::DW_FORM_ref2: + case llvm::dwarf::DW_FORM_ref4: + case llvm::dwarf::DW_FORM_ref8: + case llvm::dwarf::DW_FORM_ref_udata: + case llvm::dwarf::DW_FORM_ref_sup4: + case llvm::dwarf::DW_FORM_ref_sig8: + case llvm::dwarf::DW_FORM_ref_sup8: + case llvm::dwarf::DW_FORM_GNU_ref_alt: + // If the offset is relative to the current unit, we convert it to be relative to the .debug_info + // section start. This seems more convenient for the user (e.g. for JOINs), but it's + // also confusing to see e.g. DW_FORM_ref4 (unit-relative reference) next to an absolute offset. + if (need[COL_ATTR_INT]) col_attr_int->insertValue(val.getAsReference().value_or(0)); + if (need[COL_ATTR_STR]) col_attr_str->insertDefault(); + break; + + default: + if (need[COL_ATTR_INT]) col_attr_int->insertDefault(); + if (need[COL_ATTR_STR]) col_attr_str->insertDefault(); + } + } + + if (need[COL_SIZE]) + col_size->insertValue(static_cast(unit.offset - die_offset)); + if (need[COL_ATTR_NAME]) + col_attr_offsets->insertValue(col_attr_name->size()); + + if (need_name) col_name->insertDefault(); + if (need_linkage_name) col_linkage_name->insertDefault(); + if (need_decl_file) col_decl_file.insertPosition(0); + if (need_decl_line) col_decl_line->insertDefault(); + + if (need_ranges) + { + if (ranges.has_value()) + parseRanges(*ranges, ranges_rnglistx, low_pc, unit, col_ranges_start, col_ranges_end); + else if (low_pc.has_value()) + { + UInt64 high; + if (!high_pc.has_value()) + high = *low_pc + 1; + else if (relative_high_pc) + high = *low_pc + *high_pc ; + else + high = *high_pc; + col_ranges_start->insertValue(*low_pc); + col_ranges_end->insertValue(high); + } + } + if (need[COL_RANGES]) col_ranges_offsets->insertValue(col_ranges_start->size()); + + if (abbrev->hasChildren()) + unit.stack.push_back(StackEntry{.offset = die_offset, .tag = tag}); + } + + if (unit.stack.empty()) + { + if (!unit.eof()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Unexpected end of DIE tree at offset {} != {}", unit.offset, unit.end_offset); + break; + } + } + + ColumnPtr immutable_attr_offsets = std::move(col_attr_offsets); + ColumnPtr immutable_ancestor_array_offsets = std::move(col_ancestor_array_offsets); + + Columns cols; + for (const std::string & name : header.getNames()) + { + switch (column_name_to_idx.at(name)) + { + case COL_OFFSET: + cols.push_back(std::exchange(col_offset, nullptr)); + break; + case COL_SIZE: + cols.push_back(std::exchange(col_size, nullptr)); + break; + case COL_TAG: + cols.push_back(ColumnLowCardinality::create(tag_dict_column, std::exchange(col_tag, nullptr), /*is_shared*/ true)); + break; + case COL_UNIT_NAME: + { + auto dict = ColumnString::create(); + dict->insertDefault(); + dict->insertData(unit.unit_name.data(), unit.unit_name.size()); + auto index = ColumnVector::create(); + index->insert(1); + auto indices = index->replicate({num_rows}); + cols.push_back(ColumnLowCardinality::create(ColumnUnique::create( + std::move(dict), /*is_nullable*/ false), indices)); + break; + } + case COL_UNIT_OFFSET: + { + auto dict = ColumnVector::create(); + dict->insertDefault(); + dict->insertValue(unit.dwarf_unit->getOffset()); + auto index = ColumnVector::create(); + index->insert(1); + auto indices = index->replicate({num_rows}); + cols.push_back(ColumnLowCardinality::create(ColumnUnique>::create( + std::move(dict), /*is_nullable*/ false), indices)); + break; + } + case COL_ANCESTOR_TAGS: + cols.push_back(ColumnArray::create(ColumnLowCardinality::create( + tag_dict_column, std::exchange(col_ancestor_tags, nullptr), /*is_shared*/ true), immutable_ancestor_array_offsets)); + break; + case COL_ANCESTOR_OFFSETS: + cols.push_back(ColumnArray::create(std::exchange(col_ancestor_dwarf_offsets, nullptr), immutable_ancestor_array_offsets)); + break; + case COL_NAME: + cols.push_back(std::exchange(col_name, nullptr)); + break; + case COL_LINKAGE_NAME: + cols.push_back(std::exchange(col_linkage_name, nullptr)); + break; + case COL_DECL_FILE: + cols.push_back(ColumnLowCardinality::create(unit.filename_table, col_decl_file.detachPositions(), /*is_shared*/ true)); + break; + case COL_DECL_LINE: + cols.push_back(std::exchange(col_decl_line, nullptr)); + break; + case COL_RANGES: + cols.push_back(ColumnArray::create(ColumnTuple::create(Columns{std::exchange(col_ranges_start, nullptr), std::exchange(col_ranges_end, nullptr)}), std::exchange(col_ranges_offsets, nullptr))); + break; + case COL_ATTR_NAME: + cols.push_back(ColumnArray::create(ColumnLowCardinality::create( + attr_name_dict_column, std::exchange(col_attr_name, nullptr), /*is_shared*/ true), immutable_attr_offsets)); + break; + case COL_ATTR_FORM: + cols.push_back(ColumnArray::create(ColumnLowCardinality::create( + attr_form_dict_column, std::exchange(col_attr_form, nullptr), /*is_shared*/ true), immutable_attr_offsets)); + break; + case COL_ATTR_INT: + cols.push_back(ColumnArray::create(std::exchange(col_attr_int, nullptr), immutable_attr_offsets)); + break; + case COL_ATTR_STR: + cols.push_back(ColumnArray::create(std::exchange(col_attr_str, nullptr), immutable_attr_offsets)); + break; + + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected column index"); + } + } + return Chunk(std::move(cols), num_rows); +} + +void DWARFBlockInputFormat::parseFilenameTable(UnitState & unit, uint64_t offset) +{ + if (!debug_line_extractor.has_value()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "There are DW_AT_stmt_list but no .debug_line section"); + + llvm::DWARFDebugLine::Prologue prologue; + auto error = prologue.parse(*debug_line_extractor, &offset, /*RecoverableErrorHandler*/ [&](auto e) + { + if (++seen_debug_line_warnings < 10) + LOG_INFO(&Poco::Logger::get("DWARF"), "{}", llvm::toString(std::move(e))); + }, *dwarf_context, unit.dwarf_unit); + + if (error) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Failed to parse .debug_line unit prologue: {}", llvm::toString(std::move(error))); + + auto col = ColumnString::create(); + col->insertDefault(); + /// DWARF v5 changed file indexes from 1-based to 0-based. + if (prologue.getVersion() <= 4) + col->insertDefault(); + for (const auto & entry : prologue.FileNames) + { + auto val = entry.Name.getAsCString(); + const char * c_str; + if (llvm::Error e = val.takeError()) + { + c_str = ""; + llvm::consumeError(std::move(e)); + } + else + c_str = *val; + col->insertData(c_str, strlen(c_str)); + } + unit.filename_table_size = col->size() - 1; + unit.filename_table = ColumnUnique::create(std::move(col), /*is_nullable*/ false); +} + +uint64_t DWARFBlockInputFormat::fetchFromDebugAddr(uint64_t addr_base, uint64_t idx) const +{ + if (!debug_addr_section.has_value()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Missing .debug_addr section."); + if (addr_base == UINT64_MAX) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Missing DW_AT_addr_base"); + uint64_t offset = addr_base + idx * 8; + if (offset + 8 > debug_addr_section->size()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, ".debug_addr offset out of bounds: {} vs {}.", offset, debug_addr_section->size()); + uint64_t res; + memcpy(&res, debug_addr_section->data() + offset, 8); + return res; +} + +void DWARFBlockInputFormat::parseRanges( + uint64_t offset, bool form_rnglistx, std::optional low_pc, const UnitState & unit, const ColumnVector::MutablePtr & col_ranges_start, + const ColumnVector::MutablePtr & col_ranges_end) const +{ + llvm::Optional base_addr; + if (low_pc.has_value()) + base_addr = llvm::object::SectionedAddress{.Address = *low_pc}; + + llvm::DWARFAddressRangesVector ranges; + + if (unit.dwarf_unit->getFormParams().Version < 5) + { + if (!debug_ranges_extractor.has_value()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Missing .debug_ranges section"); + + llvm::DWARFDebugRangeList list; + auto err = list.extract(*debug_ranges_extractor, &offset); + if (err) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Error parsing .debug_ranges list: {}", llvm::toString(std::move(err))); + + ranges = list.getAbsoluteRanges(base_addr); + } + else + { + if (!debug_rnglists_extractor.has_value()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Missing .debug_rnglists section"); + + if (form_rnglistx) + { + /// Fetch offset from a table. + if (unit.rnglists_base == UINT64_MAX) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Missing DW_AT_rnglists_base"); + uint64_t entry_size = unit.dwarf_unit->getFormParams().getDwarfOffsetByteSize(); + uint64_t lists_offset = unit.rnglists_base + offset * entry_size; + if (lists_offset + entry_size > debug_rnglists_extractor->size()) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "DW_FORM_rnglistx offset out of bounds: {} vs {}", lists_offset, debug_rnglists_extractor->size()); + + offset = 0; + memcpy(&offset, debug_rnglists_extractor->getData().data() + lists_offset, entry_size); + + offset += unit.rnglists_base; + } + + llvm::DWARFDebugRnglist list; + auto err = list.extract(*debug_rnglists_extractor, /*HeaderOffset, essentially unused*/ 0, &offset, + ".debug_rnglists", "rnglists"); + if (err) + throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Error parsing .debug_rnglists list: {}", llvm::toString(std::move(err))); + + auto lookup_addr = [&](uint32_t idx) -> llvm::Optional + { + uint64_t addr = fetchFromDebugAddr(unit.addr_base, idx); + return llvm::object::SectionedAddress{.Address = addr}; + }; + ranges = list.getAbsoluteRanges(base_addr, /*AddressByteSize*/ 8, lookup_addr); + } + + for (const llvm::DWARFAddressRange & r : ranges) + { + col_ranges_start->insertValue(r.LowPC); + col_ranges_end->insertValue(r.HighPC); + } +} + +Chunk DWARFBlockInputFormat::generate() +{ + initializeIfNeeded(); + + std::unique_lock lock(mutex); + bool ok = false; + SCOPE_EXIT({ + if (!ok) + { + is_stopped = true; + wake_up_threads.notify_all(); + } + }); + + while (true) + { + if (is_stopped) + return {}; + if (background_exception) + std::rethrow_exception(background_exception); + + if (!delivery_queue.empty()) + { + Chunk chunk = std::move(delivery_queue.front().first); + approx_bytes_read_for_chunk = delivery_queue.front().second; + delivery_queue.pop_front(); + wake_up_threads.notify_one(); + ok = true; + return chunk; + } + + if (units_queue.empty() && units_in_progress == 0) + return {}; + + deliver_chunk.wait(lock); + } +} + +void DWARFBlockInputFormat::resetParser() +{ + stopThreads(); + + pool.reset(); + background_exception = nullptr; + is_stopped = false; + units_queue.clear(); + delivery_queue.clear(); + units_in_progress = 0; + elf.reset(); + extractor.reset(); + + IInputFormat::resetParser(); +} + +DWARFSchemaReader::DWARFSchemaReader(ReadBuffer & in_) + : ISchemaReader(in_) +{ +} + +NamesAndTypesList DWARFSchemaReader::readSchema() +{ + return getHeaderForDWARF(); +} + +void registerDWARFSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader( + "DWARF", + [](ReadBuffer & buf, const FormatSettings &) + { + return std::make_shared(buf); + } + ); +} + +void registerInputFormatDWARF(FormatFactory & factory) +{ + factory.registerRandomAccessInputFormat( + "DWARF", + [](ReadBuffer & buf, + const Block & sample, + const FormatSettings & settings, + const ReadSettings &, + bool /* is_remote_fs */, + size_t /* max_download_threads */, + size_t max_parsing_threads) + { + return std::make_shared( + buf, + sample, + settings, + max_parsing_threads); + }); + factory.markFormatSupportsSubsetOfColumns("DWARF"); +} + +} + +#else + +namespace DB +{ +class FormatFactory; +void registerInputFormatDWARF(FormatFactory &) +{ +} + +void registerDWARFSchemaReader(FormatFactory &) {} +} + +#endif diff --git a/src/Processors/Formats/Impl/DWARFBlockInputFormat.h b/src/Processors/Formats/Impl/DWARFBlockInputFormat.h new file mode 100644 index 00000000000..07c00656e4d --- /dev/null +++ b/src/Processors/Formats/Impl/DWARFBlockInputFormat.h @@ -0,0 +1,124 @@ +#pragma once +#include "config.h" +#if USE_DWARF_PARSER && defined(__ELF__) && !defined(OS_FREEBSD) + +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class DWARFBlockInputFormat : public IInputFormat +{ +public: + DWARFBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_, size_t num_threads_); + ~DWARFBlockInputFormat() override; + + String getName() const override { return "DWARFBlockInputFormat"; } + + void resetParser() override; + + size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; } + +protected: + Chunk generate() override; + + void onCancel() override + { + is_stopped = 1; + } + +private: + struct StackEntry + { + uint64_t offset; + llvm::dwarf::Tag tag; + }; + + struct UnitState + { + const llvm::DWARFUnit * dwarf_unit; + const llvm::DWARFAbbreviationDeclarationSet * abbrevs; + uint64_t end_offset; + + std::string unit_name; + ColumnPtr filename_table; // from .debug_line + size_t filename_table_size = 0; + uint64_t addr_base = UINT64_MAX; + uint64_t rnglists_base = UINT64_MAX; + + uint64_t offset = 0; + std::vector stack; + + bool eof() const { return offset == end_offset; } + + explicit UnitState(llvm::DWARFUnit * u); + }; + + const FormatSettings format_settings; + size_t num_threads; + + /// Dictionary columns shared among all LowCardinality columns we produce. + ColumnPtr tag_dict_column; + ColumnPtr attr_name_dict_column; + ColumnPtr attr_form_dict_column; + + std::exception_ptr background_exception = nullptr; + std::atomic is_stopped{0}; + size_t approx_bytes_read_for_chunk = 0; + + std::optional pool; + std::mutex mutex; + std::condition_variable deliver_chunk; + std::condition_variable wake_up_threads; + std::deque units_queue; + std::deque> delivery_queue; + size_t units_in_progress = 0; + + std::optional elf; + PODArray file_contents; // if we couldn't mmap it + + std::unique_ptr dwarf_context; + std::optional extractor; // .debug_info + std::optional debug_line_extractor; // .debug_line + std::optional debug_addr_section; // .debug_addr + std::optional debug_rnglists_extractor; // .debug_rnglists + std::optional debug_ranges_extractor; // .debug_ranges + + std::atomic seen_debug_line_warnings {0}; + + void initializeIfNeeded(); + void initELF(); + void stopThreads(); + void parseFilenameTable(UnitState & unit, uint64_t offset); + Chunk parseEntries(UnitState & unit); + + /// Parse .debug_addr entry. + uint64_t fetchFromDebugAddr(uint64_t addr_base, uint64_t idx) const; + /// Parse .debug_ranges (DWARF4) or .debug_rnglists (DWARF5) entry. + void parseRanges( + uint64_t offset, bool form_rnglistx, std::optional low_pc, const UnitState & unit, + const ColumnVector::MutablePtr & col_ranges_start, + const ColumnVector::MutablePtr & col_ranges_end) const; +}; + +class DWARFSchemaReader : public ISchemaReader +{ +public: + DWARFSchemaReader(ReadBuffer & in_); + + NamesAndTypesList readSchema() override; +}; + +} + +#endif diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index cbafff50bf5..856c1f21d27 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1237,7 +1237,9 @@ public: chassert(file_num > 0); const auto max_parsing_threads = std::max(settings.max_threads / file_num, 1UL); - input_format = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings, need_only_count ? 1 : max_parsing_threads); + input_format = FormatFactory::instance().getInput( + storage->format_name, *read_buf, block_for_format, context, max_block_size, storage->format_settings, + max_parsing_threads, std::nullopt, /*is_remote_fs*/ false, CompressionMethod::None, need_only_count); input_format->setQueryInfo(query_info, context); if (need_only_count) input_format->needOnlyCount(); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index eb15ee36ced..f26d59cb559 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -614,10 +614,11 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() getContext(), max_block_size, format_settings, - need_only_count ? 1 : max_parsing_threads, + max_parsing_threads, /* max_download_threads= */ std::nullopt, /* is_remote_fs */ true, - compression_method); + compression_method, + need_only_count); if (query_info.has_value()) input_format->setQueryInfo(query_info.value(), getContext()); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 4d313b9e432..58f01312399 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -353,10 +353,11 @@ StorageURLSource::StorageURLSource( getContext(), max_block_size, format_settings, - need_only_count ? 1 : max_parsing_threads, + max_parsing_threads, /*max_download_threads*/ std::nullopt, /* is_remote_ fs */ true, - compression_method); + compression_method, + need_only_count); input_format->setQueryInfo(query_info, getContext()); if (need_only_count) diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 6e636e580c2..5b4615f0793 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -101,7 +101,8 @@ if (TARGET ch_contrib::ulid) set(USE_ULID 1) endif() if (TARGET ch_contrib::llvm) - set(USE_EMBEDDED_COMPILER 1) + set(USE_EMBEDDED_COMPILER ${ENABLE_EMBEDDED_COMPILER}) + set(USE_DWARF_PARSER ${ENABLE_DWARF_PARSER}) endif() if (TARGET ch_contrib::unixodbc) set(USE_ODBC 1) diff --git a/tests/instructions/sanitizers.md b/tests/instructions/sanitizers.md index 6de789bed34..3c50f6cbab7 100644 --- a/tests/instructions/sanitizers.md +++ b/tests/instructions/sanitizers.md @@ -67,5 +67,6 @@ sudo -u clickhouse UBSAN_OPTIONS='print_stacktrace=1' ./clickhouse-ubsan server # How to use Memory Sanitizer ``` -CC=clang-8 CXX=clang++-8 cmake -D ENABLE_HDFS=0 -D ENABLE_CAPNP=0 -D ENABLE_RDKAFKA=0 -D ENABLE_ICU=0 -D ENABLE_POCO_MONGODB=0 -D ENABLE_POCO_NETSSL=0 -D ENABLE_ODBC=0 -D ENABLE_MYSQL=0 -D ENABLE_EMBEDDED_COMPILER=0 -D USE_SIMDJSON=0 -D SANITIZE=memory .. +CC=clang CXX=clang++ cmake -D SANITIZE=memory .. +ninja ``` diff --git a/tests/instructions/syntax.txt b/tests/instructions/syntax.txt index 8ec0df48dcc..228b0eb6045 100644 --- a/tests/instructions/syntax.txt +++ b/tests/instructions/syntax.txt @@ -1,6 +1,5 @@ -# Quick syntax check (2 minutes on 16-core server) +# Relatively quick syntax check (20 minutes on 16-core server) mkdir build && cd build -CC=clang-8 CXX=clang++-8 cmake -D ENABLE_EMBEDDED_COMPILER=0 -D CMAKE_BUILD_TYPE=Debug .. -ninja re2_st -time jq --raw-output '.[] | .command' compile_commands.json | grep -P -- ' -o [^ ]+\.o' | grep -v -P -- '-c .+/contrib/' | grep -vP '\.s$' | sed -r -e 's/ -o [^ ]+\.o/ -fsyntax-only/' | sort -R | xargs -I{} -P$(nproc) sh -c '{}' +cmake -D CMAKE_BUILD_TYPE=Debug .. +time jq --raw-output '.[] | .command' compile_commands.json | grep -P -- ' -o [^ ]+\.o' | grep -v -P -- '-c .+/contrib/' | grep -vP '\.(s|asm)$' | sed -r -e 's/ -o [^ ]+\.o/ -fsyntax-only/' | sort -R | xargs -I{} -P$(nproc) sh -c '{}' diff --git a/tests/queries/0_stateless/02896_dwarf_format.reference b/tests/queries/0_stateless/02896_dwarf_format.reference new file mode 100644 index 00000000000..bb40c960a7b --- /dev/null +++ b/tests/queries/0_stateless/02896_dwarf_format.reference @@ -0,0 +1,54 @@ +4611 9 structure_type tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['namespace','compile_unit'] [42,11] _Head_base<0UL, std::thread::_State *, false> tuple 187 [] ['calling_convention','name','byte_size','decl_file','decl_line'] ['data1','strp','data1','data1','data1'] [5,0,8,12,187] ['','_Head_base<0UL, std::thread::_State *, false>','','tuple',''] +13857 5 formal_parameter tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['subprogram','compile_unit'] [13845,11] 0 [] ['type'] ['ref4'] [13382] [''] +10655 5 formal_parameter tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['subprogram','compile_unit'] [10644,11] 0 [] ['type'] ['ref4'] [9521] [''] +1031 5 formal_parameter tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['subprogram','structure_type','namespace','namespace','compile_unit'] [1015,744,83,42,11] 0 [] ['type','artificial'] ['ref4','flag_present'] [8729,1] ['',''] +10614 12 subprogram tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['compile_unit'] [11] wcstof wchar.h 383 [] ['name','decl_file','decl_line','type','declaration','external'] ['strp','data1','data2','ref4','flag_present','flag_present'] [0,22,383,10637,1,1] ['wcstof','wchar.h','','','',''] +4165 1 null tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['GNU_template_parameter_pack','structure_type','namespace','compile_unit'] [4155,4135,42,11] 0 [] [] [] [] [] +4453 7 subprogram tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['structure_type','namespace','compile_unit'] [4340,42,11] _Head_base tuple 94 [] ['name','decl_file','decl_line','declaration','external'] ['strp','data1','data1','flag_present','flag_present'] [0,12,94,1,1] ['_Head_base','tuple','','',''] +12314 11 subprogram tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['compile_unit'] [11] isxdigit ctype.h 118 [] ['name','decl_file','decl_line','type','declaration','external'] ['strp','data1','data1','ref4','flag_present','flag_present'] [0,35,118,8690,1,1] ['isxdigit','ctype.h','','','',''] +988 16 subprogram tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['structure_type','namespace','namespace','compile_unit'] [744,83,42,11] operator++ _ZNSt6chrono8durationIlSt5ratioILl1ELl1000000000EEEppEi chrono 558 [] ['linkage_name','name','decl_file','decl_line','type','declaration','external'] ['strp','strp','data1','data2','ref4','flag_present','flag_present'] [0,0,6,558,744,1,1] ['_ZNSt6chrono8durationIlSt5ratioILl1ELl1000000000EEEppEi','operator++','chrono','','','',''] +4338 1 null tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['subprogram','structure_type','namespace','compile_unit'] [4315,4135,42,11] 0 [] [] [] [] [] +16621068925129977565 +offset 12557109049662739353 4611 +size 8517422279110950867 30 +tag 17905751708343214984 template_type_parameter +unit_name 3809634300092035316 tests/queries/0_stateless/data_dwarf/tiny.cpp +unit_offset 14905628884015248742 0 +ancestor_tags 830906299496007322 ['structure_type','compile_unit'] +ancestor_offsets 16241290119094197593 [2261,2052,42,11] +name 4107899145070691821 tuple_element<0UL, std::tuple > > +linkage_name 17888821071277770853 _ZNSt6chrono15duration_valuesIlE3maxEv +decl_file 9405727367722015349 ctype.h +decl_line 16290890905382309833 540 +ranges 15709800572475971278 [(7312,7333)] +attr_name 3498641248020422385 ['linkage_name','name','decl_file','decl_line','type','declaration','external','accessibility','explicit'] +attr_form 13526257655047977794 ['strp','ref4','data1','data1','data1'] +attr_int 11015789207279219965 [4432,11,0,0,1] +attr_str 4424785029337272915 ['putwc','wchar.h','','','',''] +13857 1 null tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['subprogram','compile_unit'] [13827,12] 0 [] [] [] [] [] +2355 5 formal_parameter tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['subprogram','class_type','namespace','compile_unit'] [2349,2076,43,12] 0 [] ['type','artificial'] ['ref4','flag_present'] [7415,1] ['',''] +9267 7 imported_declaration tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['namespace','compile_unit'] [9241,12] cstdlib 200 [] ['decl_file','decl_line','import'] ['data1','data1','ref4'] [38,200,11139] ['cstdlib','',''] +8880 5 formal_parameter tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['subprogram','compile_unit'] [8860,12] 0 [] ['type'] ['ref4'] [8821] [''] +2941 1 null tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['subprogram','class_type','namespace','compile_unit'] [2915,2782,43,12] 0 [] [] [] [] [] +4165 5 subprogram tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['structure_type','namespace','compile_unit'] [4093,43,12] _Tuple_impl tuple 434 [] ['name','decl_file','decl_line','declaration','explicit'] ['strx1','data1','data2','flag_present','flag_present'] [0,11,434,1,1] ['_Tuple_impl','tuple','','',''] +3087 1 null tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['subprogram','structure_type','namespace','compile_unit'] [3067,2943,43,12] 0 [] [] [] [] [] +73 7 structure_type tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['namespace','namespace','compile_unit'] [62,43,12] duration > chrono 459 [] ['calling_convention','name','byte_size','decl_file','decl_line'] ['data1','strx1','data1','data1','data2'] [5,0,8,5,459] ['','duration >','','chrono',''] +9151 1 null tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['subprogram','compile_unit'] [9131,12] 0 [] [] [] [] [] +11619 5 formal_parameter tests/queries/0_stateless/data_dwarf/tiny.cpp 0 ['subprogram','compile_unit'] [11603,12] 0 [] ['type'] ['ref4'] [11491] [''] +17088623892243577903 +offset 16023471350623073201 13857 +size 5435489408010792888 18 +tag 17905751708343214984 template_type_parameter +unit_name 3809634300092035316 tests/queries/0_stateless/data_dwarf/tiny.cpp +unit_offset 14905628884015248742 0 +ancestor_tags 830906299496007322 ['structure_type','compile_unit'] +ancestor_offsets 18099428250664153593 [10047,12] +name 4107899145070691821 tuple_element<0UL, std::tuple > > +linkage_name 17888821071277770853 _ZNSt6chrono15duration_valuesIlE3maxEv +decl_file 13346398762032466121 ctype.h +decl_line 16290890905382309833 540 +ranges 15709800572475971278 [(7312,7333)] +attr_name 8508813696511238777 ['linkage_name','name','decl_file','decl_line','type','declaration','external','accessibility','explicit'] +attr_form 2204414726958745669 ['strx1','data1','data2','flag_present','flag_present'] +attr_int 15204072385076326204 [4432,11,0,0,1] +attr_str 7554613944797424512 ['putwc','wchar.h','','','',''] diff --git a/tests/queries/0_stateless/02896_dwarf_format.sh b/tests/queries/0_stateless/02896_dwarf_format.sh new file mode 100755 index 00000000000..50045509cbe --- /dev/null +++ b/tests/queries/0_stateless/02896_dwarf_format.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-msan, no-ubsan + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +for file in tiny_dwarf4 tiny_dwarf5 +do + $CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_dwarf/$file', DWARF) order by cityHash64(offset) limit 10" + $CLICKHOUSE_LOCAL -q "select sum(cityHash64(*)) from file('$CURDIR/data_dwarf/$file', DWARF)" + + # Select each column individually to make sure we didn't mess up any of the `if (need[COL_WHATEVER])` checks in the code. + for c in `$CLICKHOUSE_LOCAL -q "desc file('$CURDIR/data_dwarf/$file', DWARF)" | cut -f1` + do + $CLICKHOUSE_LOCAL -q "select '$c', sum(cityHash64($c) as h), argMin($c, h) as random_value_to_sanity_check_visually from file('$CURDIR/data_dwarf/$file', DWARF)" + done +done diff --git a/tests/queries/0_stateless/data_dwarf/tiny.cpp b/tests/queries/0_stateless/data_dwarf/tiny.cpp new file mode 100644 index 00000000000..ec591e177ef --- /dev/null +++ b/tests/queries/0_stateless/data_dwarf/tiny.cpp @@ -0,0 +1,15 @@ +#include +#include +#include +using namespace std; + +// Just an arbitrary program. We don't run it, just need its debug symbols. + +int main() { + thread t([]{ + this_thread::sleep_for(chrono::seconds(1)); + throw "hi"; + }); + this_thread::sleep_for(chrono::seconds(10)); + cout<<"unreachable\n"; +} diff --git a/tests/queries/0_stateless/data_dwarf/tiny_dwarf4 b/tests/queries/0_stateless/data_dwarf/tiny_dwarf4 new file mode 100644 index 0000000000000000000000000000000000000000..97fe4d008899bf351e35d2e490c224f46d016cac GIT binary patch literal 64360 zcmeIb3wTu3)job^CNqT0kVI}mPzcIZAmj!D1`U^B5)w#BL=?>=Boi2!OvYpqf{IYF z6{=RWUZ}N7Yqhp&ORcrm+KQi6@lva`7kl|y+G-77ZKZ0J*0%iKckR9BoEauafc~HF z|2*G$lC$<+YwfkyUVHDgFK3@Kn=5LTxLq#GTpsIei?ODY6sBAgro5vA0Oi&kYY6TW ztqGP7@eIK!dbt9is_Qj(GR)L?42wC>YI_=J z>`9yOe~y-y3iCnZl`nV8N=;|m{0wA$^O_k6ubEvsGtxRE5{`EF&gh*pd&cb2g07B&Vxdzn=D{{~ z?y~bN_ZDjpaVEW?Gfp}$R*lY1U>@S6t8#rDAG+HK!B_QzUxIwo>9edKe4rnFru2gk^;51h`qAgp{otSJ2mdGXfQAN^nKN6wml@PFt>f3~fB{HI=j=|>LGY5Ax3gMR_|boKi$2s$k_Q^teAwe20z zU{}1kGad|D!K#MZU~8x|v^Lxo4|O)wE{b$SLk-PqA|XXfPr0z6s%~kE6}+%v<+A$t z?D)FQP;=|-V12wf9;yJ^Mi$Kq*6Q38S4DlisIWU4zO*|OjKw>vmemJMLQqOuLv797 zkvMdUgyNy9`l4V(MctB~9!jl`&xMZd;V5$HQMt?scXS2UG`V!a26+%4u&7N3Cdcw4*B$3dJ-lO4b{O zm|&>0Gujc9va*aw*A;JVnKq4%bT*^yiO|{>h-9pY_!fmfchRC?NkO5tXw|9(3#)=f z1tr$GHB}1{1-jH3E+{Tm(O_{wiKZY}Trdlb*A3sBfx8#bD|es9WdNT64FA3G#)I(h z5?<8_7z@JpnEqHdX(LAbLDmM~c)1{PiKdq${_5A)V@U8?n>8*z^(G9p8P=zCyz}5~ z2zjhqap&cN9_^aO_`^5eis8U#-J@~cdEvp--;7UY9%07sa@~nn>3M>|*J-EKl!7;% zK9+*tq2s+N`10Y@%Cfek;F~mlTMB-M#_vqQ56>4d){b%ojh-fM@~0o8>Sp}jrR5N> zO(p^!Jx73yoo37=W{(4J@*sY%0}q2Hu9qG7tOVAwUUT4$E+p@B;72(4`yBX_9ryzd z{3r+hpaVbJf!|rKA}sGP2Y!^s4+4%fbLE$-5bA>DM>_cbFZh2J_&*C+*1I`l|53U1t<1`6y}y`gS(TsI6L%kaxpM2% znNNv%4$b;K(1#|zj(g6ya)cOPN80{AHPeh--Xc7tAd z?ffUfP^9X zImA6%@j5F6xIW-s?|T;2zaI#}^zBFEP_b+0S0H!T_7c=b-+1I}xDNJx6KRU}pkWqX z{8Z)kC%3;`xxMza?G-;^Mt!$}Sb6QH*ZO84O&hL+J+(e{mD{1u_U;4V0^GEx?_DN& zwxW+}Ji!sZ@5z6mdbdAYxg87@dn>Pn@>c?{RqNi1oc1BAt?tlDzflI_P=rroB#G`_3jKrwvXl<@U8InXmP&2g~957pCa{THoC|LFM({ z-BT%WQ03NV(T$wUfdWINu}*LMg3TI=xqa+C6-(5=*Z>(jXx zKh-}!Y}+@6!i7)TnYO=@r)+;f`Uey^&{yDKz8V==(tqHOYz%kp_M8gTDH|A3C)C$)Y`5 z|9)s&#S=N7$fh~Q0=acgM$XN9fcsR%6UjO}2{(tTK*vt;FA3j*@?2Ycpt5Cf<@QW_ zJaD$>zA?xrNo?PD4%$7&wAcKEeXeEUU=@_7T-gBa48>;qDm^s8mGj6R=#&qg?Dpf- z=d6kYD30(Y?62#i@>$5REN5|&+q>#t%k;ShxXiex3WAnRgV9QK&TGM zYEE>@`+>I2=gd#_pU){d(qL37n;YYfqdg9UzZ~{F0RN9>17nI!RfDO40bpRbdp0Qs z(Ly%8+*d02CxPBuITa12f=!AA+t>FTNvcfcPTSe`F!sGR$+kyN=lJQYL(vOEC)&Ur zY=dm=iE>b}eSKF8p8CF4Id^Z)9jeA*z(S{r%AXbOVcT^0+oZhEey>e#zkO%Ej~T#F z+!FpUU+hVL=T{0*JDR7j@2f;8^WA{lwo9XKonz%(eVWuUNZgO+q(IDF*f#@Fk!o7@ z0o{pIAEe`do>P$I`A4EX3^{KC#BF^N!&^H4;MCtSek6q1?Vt99?uq%#zP{6y#`{E5 z7`AVrAZq)Nl~T@bf6noT<5h~CGEssg`#brsrPz-c-?;$-<6?YwFgxngDKFjlXYAFV ze*!(#|A;;M8-JYnr5`U*A+TqE>9sz{)9J4i|7sQy9@=gY30JoVvzw&$CiR6T#&*B; zAex!XiZK(bymru`I$ce6+cE2jMEmYfzvC_6N2O1c-?3gytPoND|GWNwYXR;*cSrG@ z0M8Nd>?X@v7q;vN4n;-k(M&Mh9*gja%bZ|Uw5MZzs1wf~X2rW>kx*6jg{6E3P*E}= zSXdz!jo}*OIg8J5>4o;bedtgf!jB#}bf_0_FW?TqgMd>8<5~1y4;`ArI3QQ@+ui{k zu=3qQhxP$J3OF6Jz&^klz|8j!9f|PWx#`g2LMN5)|s0Dd4Pq0I|1te?*ohh z?giWd_!{5?fSH(=zX(_f_!i(gz)X~XCtx99=7%UR$~^@z6U)R^fKvc>0&W3)FJL9& zeSqr#N5M~R1DpeRAK)s$y@2Zg`vA8AW@5Q;AK(0UI%w-y;AO8U+CZ$cY;1~5T)KPe2PKaBrev?YcU^z-fXKhbpVpd{$)Z2EgZ=Og_* zC%xOI4@ddSK_Ba+UtrTqHC>)N82#&PdV{8);Y=U0>2c6^K>q1YdZSIh8T4I8pg#=y zZqS>Z>D?EJ0NUgypj+rW^61IvS7Xb6?<45L(WjWcAxZx7g#1#_%R!fVHS!nP@*6-e z1U+5-@YzdT)49}3)DME>r$3s2w1rOEdRsR8e<|o^I_aG@oqnOApS&rb{^GqO&~xDj zu14GJuWkxKzZrBc9}~K-wDYgi^eIkyi%pM#{w1Uz=cHd~(|3UWlOyE68}!#fPqJN= zo&IH=zCZmAe1!a5_%X`wFMZ)h(CdysXWhp%z0O&FgjqM&fi?95oD{`|@VnhsZQx4jBm zzZZ4>{mFmpBj^E)u}3Q3bWQKizBQolMEQ!H`nB8oML>V_NagS|=99Q3(2@EmiD880V*#{X@7*tr7CSPds@e51Pw zLKXyk7(vOyy^aIm`xf~YIr(~%_~M{XL0_CjzU2-+j%7-m$h!mdru}L>$nZP(u1w~8 z8hooT*Q0OE$Z_z|KRM*RMS0+BBi{$OCCb4;&@RVth&YIRMc!Qwz7%<7;LF9FlKmp% z+YY|f32{=6R`5;1obyQXklH$Q$Q7W&IHe?HEiaY$Nfyp0%=d1xng-wIvcj~m;4@3S zi>x_Q5&ql|SH>vs6m!H@;wKH6krYp*pY3f-qUU<^Q{oAElHvc|{?}N*oX0WeZ_Ifc zqo6tGQa)T^nlv=$W(?o(d=pX-O;j$PHR5IB=cUAXR>?`{*&<#61ri5q@PJLOLYZt@ zmN|n`r_bAnF9!!)hg^9Yn)5^)H+jii4DfINKGeZ@FUEOZaQSjgf$!s`fwfE1Wh?>y zajl>DKE%x#85xV1P+vM3CV&c|lUaPG!AqM_&bu(qEht|1X~A%@a_!dZ52N=%onE68 z`D1A$@F^ED&35+!c*g4>e#>sm0DY7#ecb@u=3fH_y5s@QB@@k=yZsbQOj zJsN&e!&^1HSHnj%d`82cY4}GC-`CKm UGca)ouf;a#;FMrB;Yr3QH?);*ng3^M* z8AaU^ExNL}xS+6fszzHln;Bi3--82`IBJ+*G`pY>2M0g95WZhTWlBwtw@k}gJYBfe;@?$0*z<*Q1aP(EKO=~ncO9ed9k}^gar*=$`<8nKd7BZ)^u6H9{PuXu zx($@UAvCzB5XgD~w@>5WcR3Cr`bggim3$w#{67LV14{XR?0O1lPk?v1SwjNETkh;% z0HtGY6$|`EP#kW3!!<+B2*fIh8R6~&S&}KnoE1Qi1cuBx$LG^mozVYM{SG(u!14?jE4uA*n;u$o&A}DXhmX z6`zjq#f)!|_(}IqMR>3~lSR7+;i1a7#azUnv=mEi5?QSdV(=^wtzkQ~Q*A)RweoUI zr=G{ZFT=CpG=OiTh4|qneSjqt^ApUO zz*vL4FTnF8wZY7%oNaI@%Y0ZHoD2m%stt~Ywssq2EHGA3rVR>1P6(oO8=L}iw#}IH z1(0+bq#D^Xm^|k*tpaWY6tZst|;a1KzgDr3iuVks(;2w~48+0T% zXs~k+OHc@7={A^1pvsvO0ZF&PR1&w#bQ`ou#ZtWAgQVMF3Muy#RF!Un)I0Zl#C02# z_>G9`HpqCIW+o-o&1|0@Gn=KEy;N#hWo%|QyV=5|nH_{Y-)G%ySJKRGcI#%g?yCzy zxkaK!)yzit_6I637Z{eSGv!{BYh^i_;0^fqO_a8sXwMU+0Vmq?q@!=o!?x%QPL~XR z1mPE8edqJLmZ4OXxSOi`UD8A7!og{MfCF8NUUmYe_dYfry9 z2-V8Z@C`__|J-2r+FW#MWD<6PdNrm3;v!!I~hk3sfvX z7R#Dg21*gQTvwcYr9M+`mKz|YcSzl)zyRxq&cJlqeJVER?6|*(@X~cd> z%p)2z>34`7B<4Fbk4`!15HP1^0s92Yu6d_S1E!AHr?)bfshtS#W&G!2hrlMJ3OvJp zJCRQZ0;ok-R@E7=Av}$@?2~1UF*0LaJ}GZ!{%V9}6(Bsm9+P!OvOch;&jKPJ@hqR~ zS7^f7YbHS8jCZlOp7rk7B~#H*-e4Aj5By{$ znOTF>S({y%6i|k6A;k?<%8R5T8ZzU3^oiW8=Q3|cglxscTI25kny9Q7GE2UM@Z0#$ zde!C5bdU}Xx|Vvr>Hqb;AWa&_HT)6(KyThCq&t#{9Ufkg6?GX&xA7++ObaMYa zP8R>&K)=nDUv)i{P{f+Za(xK2%l|KbCdLwB`MT@d!XkN|c^=R^@$dhpOSNRB%Dq5s z#_pH@F`W;MV9oh6&@+gB+SrAPT61Q>qU6mRgL{%V{}tdB!~E1~7@^I5 z6qV^8?Q%(td=rtO%KDNEeAqOa#eYts=H~qt4Z*)K`%`RsQ~!bp8?t|S_6R#-InXP%JvxZ;I-U3wvpP;^@N=zmHAL-&&|Y7Oe12-UqBSkZ(8(G^qr3k>>OrC0I&6`5Lw)+T=;i zOf}&FGCs#*27c-p%X%NjQjTMlS@VbH;O@#@c#>t^h5vv*gEct&35LrvT-oy=AaD+) zDBMDWt0K<-1Mtxxfp;^O0ipSSJ(RS&a2MLYowVW5#F}$H?o>2T;?-%G+?xFxxV^yW z43`Y_KS#uMw(3OG11p7$x3Tul47gn<(>JbRS4k~W@nkN~=X4t<*wSkje z*@cvHlElWSSU{w+IxWRDS>siPbr3BUn&zTjBETtghUT6{!E-*(qSI!#y4;FY3In6$ z0_zIwvW?rp@$dUym}62dLZ3z;_eourk?u6XvkQ97F~FqAl>f z?Ej$@Rroy;k#?vtPbtjbZJwtU#=};oQoW!s4s+C)ezae@(Gc>E+Gdb%Njn#jnDUNA z*K=h*kSS|`L5yV5gA73^vmSvF2>6Jhr_Fva(^_x~CO3n;`wmY>FC*zLHtC2l>3nx0 zvK7OA&MB@z-d`Z#%C7Y##tPM5e0Q4^%Y4?t7~Cl@44GK_y)p-r#JQL%UC4P_-Xpke zf=;=?McCE&)j|?DsY;%0LDOJbwa)Q_C~`0u))hGRdrvnN7dWVDEQHZnKE3 z6n6|)b`E1z5`!yaOqE$Nz?BI~(oEk2N!pcBw1cF$;mU-2F=bZ&1>6Z&CV0`O>R}Wv z@Ti)qOF@OE3R;%YhtmwMK(kwUYE`^~pr=Klfa09H*t#0%|uC+#3<)lY#YgYSbLorcM+rK_p%AKk98RS2jhige)yM9nxX zD(TAd?qX$Sf5YzRRwlJg?&$3C(%K)&M02lYv6GXqFv)$Ko!umHe?K3q?>)HHp42cF zGe^M1S%Ax40xoMj#;3Xf<5unP^0g3EJN)_|2)N;80Xu##;Ksq2A9gfX`no;Lgtp*!7@*FFr5e?mr6n z(!T`UyL>W}f2Bjf13LtKZMT5k&kA_x_X56|Ifa}Lj}`E(QUQ-t3;1@EfJb`-eCK`v zk3BBnyFVB3_&Wl=Hw25m+Tq{lW8B)|Pb?GgWRHNSZWi$L*9H9G1p&{zDd1W68BDNu zxPa#h1w3CP;DvPpUc6es4{sCj($@t1=xG5j|5Cs!-14a%{$t-XtOAGoU5^P2xLy-D z#Py-TF)r$}eE3)wjkA3Cco)sGeE0+x4YGXr6rFCWPIrc+t9u%aqjt>9Y10V`D+LsV z1QcH`p!7xov$)!=9W(pO0?vF$z})W%nD?rHv-S%p%%F#+ekAfWQs0+#<%K;2Y0*^hk+ zO?Sms+^pL1vnqh89Y4ETz?n@16Ut)(7JOX5LcV@lJ7Lk?0v10kpyCGtmi$b>xo--n zJSd=Qa3K>^=LuLkO+ZbVfb(k!CY3f4Ox_$5u;mH?SKTAv>IVgU@@WCvek15pd&10XOXs@R>UW-28xmTfQ&g)|Um`_NIW({$0TBxy2Orx#qWYlfB*w45p+G)F+&QUPl(60j~RAiPDu zB{vFKf46|ha{}7m5D@*RfR5p_C^I%qz@-ZXbgmQ7b+v%_E&<(-3+VZ80UO>H&^ux_ zlW&|X;K~vKn->V!(jegD>jhl3Nx;?{1YEsKz$YFN@X4nIT=Odd+x{Y8d(N2@{i(?U zu3aSH`eg!c2nyKIBjCoH1l;r$0pIz7fX99>;Ja?j;cBNoeyV`)olP)fMWcZFE&&ZP z9?V#IrNqv^LBIvO1g!psfQ$Z1z{Rf#XxuL#_zwY1eoO&tXEdK8V9g8xEoTd8T`nNh zB%rNFz}jmBtlKFd{1pKAXW{DVV<`T{}w_vt8a|xdaG`MeE zw~euCJ}yA4={m-ia?c;J${QK0Wj{ZBuN%({UOrj!urM|wlfNQ^!D#(i<777q4MYE!%g#sY+%nFYVTD*1gC- z8&K)71p=POf1ugRX<$N+W-ljTwjLsd6XL?lz-^&eB}! z<(7mkC3T;{8n!P<&gOG)DjomOhD#)eHe9CLauVyevD8Lw!$2oYB5j?@$}Q`D{6nXy zN6<+x^SSyna`q+3d3XVy89)GZ;wsjm6PM@-o$MvHvd|#!!|-vg>{~O{5=x%WWba}tsqMxZXTFK-uGcqCUjbs~pjG#i-#^%MQa-tz{2uF~k%KW~&0Njd zMBYod-3Nw@P#zyD-HV{7vauB)_y@#SONh5dMlXtHA!$bN-z0q+*t~2M@P-VmjJAp4W3EdP!AHGi_kkV6=(1My?7Qxz*A5 z3<%9ZCJ72R_tJvDA~!>gyfsOPyLG|$NMMZy|IH9_O+pZygt%Q9JOwRfwS+#w=rl$d zY9y|Sw?;OQ;9?ROYLckjRuQ+Kl6h<77IhPELAh}+wefPYFxtom7frkcd28eop(fsf zysf!(6hdN&2bn?$H^IvUA7@lTBs9K&TT5#lvg5TC|E`t3LYP?2WrVyg_uw{evbzUk zyAN$%ynOGQNaY^P`mh?g>i7wOw}U=(c*Z(!#?ajH z(}CEEni@LP8efd?cSwXIw=Q6hk=~PG<&mQT@e#wryRa4-d8&)Uu{v%Ot^r|WpIb2R z0c|z@9rwHp%vmo2>16MMdVp{X*}R^KKnIF=qfUX@2w9D1koOYAy&0E~;S$OTcQVEM zN-#MBB0ap3=zvF(hlv&$mqZb4?80p|dBa=CRpS;erqjy)iRgNTX1V|+tjB>Q;BO%9 z#=~ggTH^OR3K+SKn70KJ-ot7U>3F8DLG!!9AdhM4%A60P45Fi@qkuEuzA+B=6 zUt`j#Dyf(xC4Kl2GE8(yA(m5OV*~E1B{8>;My_xqz6>Ns=9g?{KFl`UMb-}vGvr>P z|7D}Ca24u#Y+9+|O42&hl|W^i zLr1!rOfRLBLeaymq_sO}qHPYJd-ss^LMkcDr`#h8R7Lz73J0XEN&~pWGX*7OPl|n# zT+Mdtr{S#F^~7Ex^~?4f;LVEBA&=aeioJu_^HU|6!{Co`&wJ!=6QZox_lXE4=O``Aig8pJ*`BPTCcQ+`Md?z7`IKSg zDu<~GrY5)962u!>ju@XOjNFq@Mp-8OCb5^fLQE#@FM^PD(e-et8JU+{%GAs_{1@_V zc3q#s_cx$3GGZT+`pXmzMN-%eNh2SW(=v7j+II{l?I9OiP%15U5^2}i_7>$BwZfxG z8m~Gr`-T-BM~p{d5|yT`7oJSoAXkcVO43Zy0*WRKl33B^ke2J9Nh?&ex7ZCYv>Pt` zT{+mf4L|Z0CpuO~)IGLr3bA766Z^c4WwurW zNNb0GsP>mK^gfYS_K9jI?^SE^m(-g4$7)Ufnp%wiK`q8#SF7+h)GGWL8$Y`U9T{eqpfWNIwLpC zqYon=bBW9ZRW+20z6P2%GkgbGo^+i&-6=q`+ymO6O!uPzmyVt{Vl*<$n+?X7fXEw_ zw*bKx5%f&Fs1!}c=gO-G=I11i&TB;QrzB3ki1PujJ1+*z??}wcW0(9biBt1F3GAPU z9V7G3{lrce(U&7x#yr7pW3Wtuw=y_if?s0r?7Y2@@HZsOm?p}-O9`cUzX3Lb4AVs` zH-etodG7%`oY)zn4yQVvIeA$#aEyuA0#WBQVpYa2G@^|2^QM7eA+V>6%A1W~ITm%BuusHHoA1>Jhwv#3>h50}OKKg@9=zF;C_vtt6h2cR8?^5<5ml-wtADi0I1@ z%s5xD*DzQm!CM%tmf+nCF3tNMBz%*>B2n%e2zqAay$bA8#1@NIPZE1(-hNORm>BQCxq6=JBBRzU-2C+4Q-7Z)LW9)b1zR6K3z!AWT zO(){pg4iu=-@<*h@D`JFHAM!56q`%LHV0x65v}ww0U7utBy$)UWJY@1LCVedkFNpb zDkwh}r-=|AI}_m%nU<7uH4vU3gnIzSbt$J<4!Y_njoc$y@EOozpT+I8%S2TfcR01N zI`1ZS(|ROm?ih|uDojy?qm4ewcgx`i^h*bm@C+c6<7(F@RCd`K7~vx-QfblIya|XY9^`1 zk20e5_#p@D!i$Irs-Z(Lv5yhcCT&wHCbp7@u;{7~tBHs(|9~LEO+=J=BsV@ZmnY|T zzJUeU^Rj%kxUVB;p{H>L!pejRekZu_*=2o#Gn;4ts z4p0Xxd=+CKcS{pfD!xz5$2_7mZ&vu9#58)i2_A5uw-bGet%MccLChCb)liro-)|-Q zLC4HY`irvjU8LQkk|eJOtReT3w!|ZvrrM}y35h+rP?2vj_KftT@B<7_R(d6>)f)0J z(R*Eit>Qr*ljKr&qRnH)DCs3n;ND1E#GWMm8&cW?T^TU8m!xmWlqiK1ewn0`Z39^0 zpNkw-c5xVvS<*hzzM{CL0%SUoSB-)DO_DywCR{5`m}Bn5<*ErwxGvM`d7DgEXcL70 z##n>L9`VB;60^pWG7*TmQ1KHl6+YEU2NCg{AW{kx%OU9{8O~Elqe!|!rke?p6&pwF zW|=_PSXmgKkG3>rP~MbMtY*M8ts$=j!6gWuk-eY{UDhLn=>XlHUq{k~FuADbyO6#kIl<(RijcDwi&TiC%AouOs_Jm7znkn~1(ap_RQv z_)enlcFW+>GD~+P)deK?OW3Z%cam#xiV6W?$hjA^uao}J;krCR{ByQBRHvp^R`_w! z{^g*(itMe%*Kl7gHhG#1FS(^>C|f>H%vY4Mxp+k4=&DLtI<9aRx!ks3EAk34nSvSe z3x+3nw3l2&Bep8Vwf(Lq!z7j5VZVJuPq)!l_%p=l8NO=XeZ=gPL<;dX5w|B0D$^nF z6Z@)LwZ5ExA?BkR-*T!kBtTT9ON=ZN04qF+}KFsIv)AU5cfnGs$ zcZz0uQ(zNmmpN$C7`5uFL2M<%Z`0?deUztv>-mUl~7;Yfbn~t=$ zfV?nPDx7OgFKmn=xS9(AnO$;eUoa{5Rf3?@t}H!c-y|aBK=9;2!A%at4~SUjK)gi6 zT?s^h59F*Ek17;I5?CzQiBNMSmDPSC)M%{`Jl0Uq>qzn;5o&&;I5MzaFSyFV!6Og_ z*EkR(iMY;z$OmHjnM@v#8=o3to=diS6Ie7HBEI54oW^ve_sqN{YR%ymP29hbMbXi3+~F?@jMvQ=0pD#Vg_&H>NW9-HS} z5q&`OAWz7-*oO`ARxqxn|LrwDU8Ib4HSmMse=kPFYgc3PY_;g?SK* z3l@U(XCR%GC$D+E%4fQ1p44t?(rM0=G?NB7(!AAAn%-SHjZ$<*8KwYG^yv&+(HfN2 zQL8D6HuAlV-d)_J$JLB~*UBzPxe`)(9L?`zkirr?44Cy2o1f!-5UY1jQgI)KfFD7? zqJXW&O0C9s0rCC;kDc4EAfQ2L0d1ne-ymQ#%t zcC-230dsV%{1UTzZZ|}H#-T0^2BmaXQ-t#+;e0L9~?E;+%usY$cw;-Z)+TJP&f9=JjdxNhFU)VP$kPare@k555s{;Q)a z#KHe@=p5tIMXPtWP6DN1L;l9fb|g3xG^5`}B%lqY?|e?BI-pajmNlu|vVM$I??5w? z>PnXfn%XaC)h{p99I7+5EI?C(Pcj+N+gO)3X&x+Iq0lwp z`M9GVZvzjjMV!%c@hz9{N>c4RNby>#c{j;PoP_$dFjqqT`{Y##sE?-1OR0YaERazD z3VHM3Dwp{v3zUX#)VJve`546h!Q0I7aEp9aEGG-($-R2U&qD=rGC+hhevk%5ck zxO%S3gA#a3El+mUuzCKo{O(avATr#*mtp+%;xiHw@Qt*phIz8V;>C9CFxPCqC)@8= z(af1bXU5f7UZTZ%wV zY8vb>7Ffw*FwbzNZb7o4n6+8M3Q*pmzH*eu(fwVP#!n z@=Z-Abl0Hq*uz`$;K;JjuoBf?YSS65vFWT+ zqG~hQbNuW%%Mw&+ir2Uxak-GCW7W+SRX;{wvz@dRL`bMUwWPFZL#gSaQquEMcu;AU zwK4?>h0dpes1oU`N?~?J+@tUy$>k#M!tubJ2~KQHGk(zosByO64-DR>pgTJo!NwdR z26F{qx<=M!>||MM&XRJpaFA>@`73i*4Wr^AwgPdAkke+@HqEt`-o?J zaU-8iUMjA8gcz&GXq6yrD6o`6^<>jG_Zkbd4vS`=3ma`z4ODsV%ltCFTy8UwyEF`gRL8yJKHSD$0R%~gEjaC%_Nqj$l(f+ ze%s? zR7Hzv(^@*BZQ->_bXdN4t~e7v*UF!uRj6RF9Y1Qy-*VOJrbJsd@F!X+7eA>QH%x+r z0F*u$TvAn2fiQ}SVxU+EV;(BDL=a0iWvm%U9|Z!|Yf%_V)wFI05x=*(-elB5OOlTe ziIE|G7q+_vngG!i3$*}=oLl%Ktr6i=A;sH5tLvbIb(UyMMp$dGc?~NSR;OB^kjE~Q z=qi;O47SDav%A_(RJ=n8+2G7ZMJPjsW*A$c-0^i`q?dXQ$`8(}e7oae{$eiEl!-mC zEXI}Lwg$V+{^3a(F7N0p&+}GwhTCiTJdbRCj2J&KnLFHWI3!^=YlM^B{saoMc3z{s zfvk7E(=YO#;(OiGb*I@1U&AE{}he z_tYPH^WX4}uLAzcJH10aH@F`2x-*A*2l@P&MLy48u59z%@F-qU>5USGA(f0xg*8-iYT#do-~O%k(j<9QEB z#${!m<@J&G0{jqeLuP?DJL?@E^s{^qAedR^^Z$&>XZ_4OBU&n+2o z?-bCz8OUp%|4z@&%i6pdo<46GsXyMdN{I$liT=VS$S%n8KSpDC9&lBA{aJrQV#&(0 z@d57`h#%s+$_q?JE@k}UN~EaH`o1?CISc|;^1U#+o< zRA$Ws-omW!dh>m8@8G?@8=2hm^DOsdmv^qs%8Z`rrO3y;B@j7X>GP9K4|t2Su0k&U zYSj*A!r10L6VEf2%j<*elb~<*UaxNl3i6z{^a2#1+B?^Gqjv!y2&emA^k&@PJ+;Q0 z@oDeiWr+XGJM}8>^fqttD$mz8@9?@kKeo!G&L_HktRNq zu-45Uvga05p+D1;`S6$LUZ2Z56j@QisA_NSD(~P#R#Gy~ zmT-CAM#+Z5zJ4E?e8!#rYR`_#+q{0yfs8WLowkGd+?VHCC^jZ$>=r( zwuQ|`dio~8AnP$WE@eMv zYW5R7ub|L$TkvD56_IMT4n4PnIw>nN?wyA&J@YE>oE_eCQ0da#0@U5)aeF4~=pFAc zA3Wb18|e8ym+2^^*&g)V4~Y}Ncq-hQEywH(dTvuTEAyU=497cJ%%-5{YiJIGRaOoO zW-rk5J!CK#turstBdMs_0`$BLW8`J|uTrKUX^-9=^!zo;<(-I>+P&J^?K2#|1Bb&` z+8k!*P`aJNb=n+eSc?bJF_j{@lZ1avl$YD>j zgXsAY$}>3Y1GI;$Joj(D6Anq*#9EazMVqZd&x6Q$3JfPuOFTUaU5c(z+UuM{;U!*;nrOM32qsA!vn~uz1@+~jDe_#lmp2q zR+Ca~)1=;xvQ+I=DXC`+C^hqrB&zU?fuwC+nUr?sK+=|87V7MPeF+9oCnfFp;jA7D z_W>np?q#CpUdwcX!eX|y3q4q^sjXl`%xdXqZ|{f( zxwzwkY2G~5tLM!-3W++O0i=Ky!C*_==~@cSMIUo?e#NBVuZ%ez%)nD89Lv$V6KB=~ z_AAcB;$V&?SMe{$U?_1f$CmD>`@FM&;$@B@zk&D|XMPE1b1ZopS92_Bm7_U^l-kW4 zLrUXhjv=XbG{=-uyv;}Wg+!IqoUJ-LW(>8#u-w&l4R++Z!fT_U)_nRZv!{nSM=;3U zJZ#({9K}XpPbj#qIocY*ej2=!b)`rfD`yQRCcGZbyvh$QLIwp}rcOA=Ew3sr!f$xdwbiGeHX<3)w`#wdT!5q^|g zm!y>H{nTJ9HqPUrs`{c}MMd3`o}LO;rBe>(bW8*zv^94};yQsg#$j^oUz0SMRkTDZ z>L5NbOi!7bB1?@)O6z&^RCX9A7Gv64%O+D13AKl!*dPy9wf0)UO0*tQ*0B}wrz#an zO^y?Ly}{-LrZGaz8qAc>)LNsJmZI^hBAiIEOGPdXB>I&Vorj6NjQGj5m)7Fb^RZJL(gQSp z3$%eh_^Eyil;x~W(`wQb^srKLz&;_;RN-u`wBe3jQRP@z6DL4N?R`|eGJ=^`!hi#- z2>MjfD7mQ>R@5DC@0vB3AUWC<)=$TpRu*4&<1Q;qG@j(HnSaLg{I=#uSAR{Vu%bGR zqG?)>_#CJ5vyAd7b$ZMam)4&^k@4nm1U;7$;g%4AKZ>ryf;k?7KT6kxjzMpg$qCb2 zHSrT@1l`Oj(b2q*)Krev{g_s7erR|gZK$Lu7>stHnRaZz_@u@u45sSQS6B`HAHiY8 z*;`dT9*YXLr59lVhNjU*Z%QbA1#7TqYZwncqs@^rQl{sNg33Ef-szk&``XbPFn>~# zRF*gz>gsM;mp@6nlw&N}a<=-zOCXw^Fx{cN2z07_JbIf6Ayhv>I&wY`4PVNq!Fu9x z9^8Bd{_EnM>8kJOD{p#!Pe-`*Y%7MzXgs8M-!P|3&JdG0Gp{w%tN#2d+T(JW4Txhp z4necw?NDG$NL7Gc1k>Q-(aj<9c(p4kgjQ+}t5wvI6QE|Q?BmsSR%fUy6jzgB3F=Yf zI4bANPE6miDU90Dq2rfn`VFTt$<6Qhnz$?#6a9xLb<*fc^wGC+&Z=c*ooMa0!jyD; z3abgRnHFDZucfr!$*Zel>zKzZV{sI#sFny<%~PV?kqElzR7(fA?ople*u=N8J(-R1 zGtdf}TU&#Xp5{n*DA*Zl3vq>tb$cj+o%-}9POK#8DUMBv7(6?QUBZivbBDz!UVH}s~Q*${kVJ$st95&2f)6s!B>v5RC;cbpbjMEPu zj~u6?I*wAQnH#*F-NDm0>Z6~C={*yUo9I32NLdi`Qx5D6&1-?ftgHnG7S*1-8v60j z4U;wOuynK;av3R!X^~S7`h$@x8)w+>hg|(Mz}}zJkM)#w?6n;>E($9yENySb&R5BV zU}5U+1a@n5#T<2-rQG7Nxz*}%N~F1cO>1+0b3A`Q4?^e7omW&k)p%Z|%HeB5M*))7 zzzOY-I*C$Vtn?9SRAU)P4tmAJBn|e4fUKB~&patp`Xp;8HQ{>g$#l~EUDhN?iv=}B zK2D8}egTrI?Q)J2M^@Ic+Bj}Ck7lirq@wXPsKygwDcu-UH#1w+*iY!`SRd-F!b5h+ zrK&nf32vog5Yt@RNwH0p9j`&?PLfW$LNU|`9KKpqCurQv!+bRnr;95pNs3ljhoypa zc|H#)vj{*O^HldW0Et7i`qZq|hS7CH3yQj2j?03DCrlN(3nxXfH;ze8TdbR00=lfW z4dGTSxP#%Yj+VJ|gKZnSTAHJ6+@2SHt9?yp1ScN4!k6J;17qM?jE-k7%s?_Rf+Ysx-It{{O@YZdZ)k6sr>cEc@w?(iM!5wI9>o>z5YlB_Qc+?>= z+8`JScVVMmh61fGgerx5x{UOM0zhfn;UrDj&;s?_*T6R5j0oeB38O74WmG2-!L)jpP8-J&5yavhZ59fua*KDYS2)Qm-q9_vx2=;o z#5wK zPn_~wJG!y$qj3q9BKGqW;gW|_N`gAe9>fHXXWF4lG(Jb3A0SXP+k{GrO=xzh2?eNpWe@CITQo-fCtbUA0f-f9iEspi(cURqsqmpZ4uwq2rf z0$oC=Sluprf;h=w#e0HLLd=&oqFF~GI0lZ|gKFv1R3sXjzn=NiZHgW8L%Z9?~DMphS%bNRk&+yXLD>FAi6Tn^0J~#Y1_Ab}gz*lQ09`;6*5Ieg*ISo%ci>D!sNHH_!%A(A$n#P4a)N~i zY`w^|IUc;U8>hbDUnPbPhLccX-XhkTu23_20E~h295Fb+HXP}&Xq685z|K&Jt}cY~ zwkuRSIzW{Ejef2R0r7X2b}@H&B}vxRiHENi%+4+?Aff=eFH9bXy3)}h1p#p|kw|N} zhm~AG9U~FE4uCswqNPp_mm|`!FjavzEnRlX=O#LB3DQ-d?*`A+FAJb_>a77Z9TYdz z*=e<*G2z%-TZrxkhz_9IBHdl<4QEa?uAP{#f8KW{JsDS|(K5ph+CdMmh~b8Q#i{9vUW=d{M`sXAUGv3MG?*gi=Gmu0wQm7&i&7R@pz zAk=)Nt142F?mR|1z7lvtv(<0}ar%r#Nug&;db@;z4IPWSG2ks)g8vnAq_e7P zc@!qYxlSDlCginXIFLq9onUzJq>kl5qs||K`^Sq1-n4J!u!${dR`A;lUxQA>jeVqOQvKEIpB@E+L zQZ8gv0n7tLwW*gYTX;$j)%>f@nH?Mf=fakzRE%52)tH)pEVXb$osNroY~$h>Nh2o!6h zDXmafY2Gx5z#&#ttf2)QNA{F(sgB7KmpuyT|)GG|4Feph< z-UAxVcs}!pg-j@?&m@Q?1~6BUXL<=C&Y~SbPdPN|Xo5upvaUTEq93zGINn657?8ow zJn;rkDEA+=d{QpQR0YX18@~N>m`q0nwTJufjj(u8k>dq}s_Jx|a~URciLP0ZXy^53 zVv1M4z;+okj=!%)IDb6sC8c9?9mlWDN4nQYvU~kVr<|xKw0?N!l;q0O9@ESY!m4H7 zUBR)*3{A)OtQ4{!3MG?+zv}tTu}RVcjXlWdb1!Nn8PE(t3do_sSwKCW9h1cl)9u)# z6q?my+C6{K8mphgBFU*eGPEaf@=!1NfdT7*Dao~C`s>QFt)KJKN#g<6GWOZntl zJrQe=)40^P7+Z@trlHQJq_*&C%w(b63h*U$95#?@Y&4*0$b`!&;{-Jfda9%&bH{kl zX;D>iiXhsSuWyL&cKF-lI8_GcY6~)6X&~ou;YDnJWscn>nVd_s0UZ;!9e#4%4d~Sb zhmfN^TikvY-QDVsHfSFzX23;kXUp9MSNmO&Xoi>C?Wl zRdLKs!+tkNO&X7LizmM>gT!xLj`f>n5{TYq1TTSgdQ`D~~)ghyU{1l1`IQNfCElsCRJ zhshZZs0Z^r`h=6E-&*Dri*mkd~)e}9XFp>dS1uPSCsyuu{3 zV@G%ExcRctt2%BzW3)hjSjc>ys9DF&_lQ2Ov#T^c^F;T{dm=X?w__0^>7?FtRe7kKX0ar4ccKk2ynw9Y8~@fq_4owIb@d@^U9 zj+^h}T&v^egE;r-xcLIkV>)g=d-E$DHy^qAP{%Fv#c2)mH7wB3e6glU#|^wm$9HM? zT@CkW_$v+fY51XriLY1dFU6QoRTt>EYK2F2#~1GK3BSJ{N@d1J0`c`bkF^RP$>WtT zcgwmCpY}_~&z|YVx3W#w;;(D)nu>oHz7qb0a>nHQoC4?IPJZXFZjjI{)M3{2X%a}> zx)372>57UEKeM3u%ToA%rtz^9{C?nF)<|pJd||NoeRW<1Fa+hBu6?>fTM^*#scEYr zMb2i8-;#pAN#l2<;P27$oxdMI$v@Zla_w4li?&Y1*Da|}lg1l8TQq)$#*d>ATwl|8 z=jZ&1|DDF~)BL7f{5^7(t0_g#iNK2;^cVa2^>be5YrONP0EmBB<9BF&qkjgz3rRV# z6nve=?@qzrr1ANVUjU%|Uu(Sc7X*kupz+RM3?Tk=v`?{{<7Wbh$G3xV4YPKol=l*i z-<^Wrq~(~Q!O!HlZULU{XNPV-+QVAE6nw5VJYU$X1qxW>@Ga4F{pAecspk$opbAuzjl=?v@|~in*~3@YSMa|MBB8S@)S8Y3;%GdE``58{l6*v zL#-((c4I#2%KIev`GYa5@(JzhCnD!$`85I?YgzvdJnfUOhYHgU-xGd}L*-6xe2pGo z3q6hfMK?g>nNI^grKpnd9N<$-EclR?V}=CXOs$<7Z-(AkI{W7}-neMf{`osP)W;0< z^E7`Y@VQoAqP{emU(*(TGZYv(cLASH|L^vL{|uP(A%9!Bk*we=;6DjDgD4lek6in7 zzGg_~PciU10|P+1e9r}5?0l-?D^|eTD0m%lfSdb~^Ia_`F_h^vgYm-?Lt(c%o%vRc z&lbExunrajFLu^blLn3dxW=2I*Ocprz@OyKvv!mx%lakp$ta2axgY#6I0WWvrdp@# zw7Y>H5BWRG6$0j!>qX6PrXo``eguBZVyHXEDoh!=#}Kc1oL~;{%-2i_&(OqcG~P^= zj6Uyae3R}+XKMcI@aqJWW2SaS&xe6em+ucXznL1rEOHG)$D|xH1%5{3TY*m}=ed6H zzt(ch)b1=T=QapoduY;?GfCqo;(O@K*V66K^p_35r_1-d{osG7<(Mg*kwg0p#V32& zWt`V+#F?*|avMAE0iNy3_+gXgMa^%fkVgJlC=cbBsd=+b@D+_WO9Im@p450VRc_P# zZWLCV#$+jQR6qC{;HAE_gEacw2t57Lw(=w~)}4YMW8Ig6zgx>OQ*YCr|D^d-?eGEc z_&e+xgNY;cF-xQwnq#`g@6zqo`1AF^r;~GSKlm?dIfYtIp_V@!^P&91w81Y_0H03& zxxiCTm`E<8&nAt}PqF7-jWkfn0?cd;E z(|EH4HF{=aK9sN1I>D*HGv7Ta{ph-W`0oWioqeo&yd6+ffREMT>)m1f-a)$4E8IkL zepb@)fDL=PNt^#k=UC2Nv`D`b0^vCBfMXqwq{sAJ4EZHJk4(*?*h<9|6(ZM$mDnpi zo~L>^8{|;^qtC>No{qEWnZDqFML?l84H=dI>*JbO>w@rd?#%6g)o zU>=Vyj+2E0vcGx)#3ZQ?2JnO-)r^U!p{cc0qFXFpLQ8x;DfRIY&Ch4<2_XY(C<8sA zhOT%bA9Av5pRlL7+-pC+M}w4bVjk9|dmwl;&B5{2q{RE2A65IwX;K?e>8!*0^(^%3 zes*x-s~lfCI3^!SU)4~5+ztJ14A>ML+YaKXZw3Vo_YyQ!q3Btg975$y=_4IzG|r~KTSs)dV!#RbI$ z^vuqhGGAU%EUQ)pOA1O0iuL)$!~92@XF2lCF8jISQGH|Ce)}Lzzd7L#ESw)YbbdfF z80zP<9r6&`ZimK1&^7H-D_AM%t1WN_a}zb!<1{QjoEYtpL#@d*f)_*aSSTFj2m16; z;bZ*H1Rc8LtP&3OHV4kdTsEf}pz;{*(a1I40%<=kJ z7e5<@pS%h7aEhCLs*}diq-m?_Ik$2?bkFdf_z9mD{l!lDHyH$|EgDUq~EJ^p6T8nOL%(KOXbDTM4hP7hz%flrgF3+fS$sd6Ne}A%>$`i zvj8|zAEQ0I*S1FLl6<7@56(t4z`+Qe!9MMjW(_frj^&si%10OLk=kU{v*-7RTw~qWeHYA}PeeIY=D|R{bxB;7p7Ta2(^WW01gC$8^_D*>+0P1N!^a^HHrI(M#+X`GRZtb0E@3xVE+(>i-vK z5AhxlULn?-49TsI%`3UPE>7*PRzcUsc5E>N;-eJ;*O?Gr<_~qotO9%qI8=bI!W6_h z`6?oQDN>=ZjGMtv?g(vNGd>_u(7G`Sc?#mAGD5-Ud%Ca#kQBl1Az?-yX=VZ)ibdj9 z0k5#MdX&1=?lhI&ja62^A}G#AWgg7{Fx zTI}~6_r>Q{F8sT-q){FfFwi{r&}88?VM0E)!=w8|`Z+qif%%|0FY|oV;ASJ>OutH} zH!$BYYG|If8}woX-amAx!|30u(;GNJGs^K4gj4fh1{|JFCiLH>(;K)=3&wE?DZ5E; z&a12ho=HspQ^<(Rz{d?ip%1fPBk+eD`oFF-Fp&1)W%M`V3|tS|za8oGaUOt|fjd-` zKgBD08vQlex(or@)X3Yd(;Ik@jRTj--=sI?zZx{AUq~ig_tBURSm#L3M{=p@KMfpN zwAt+E#CYDz%RE z>+}ZV7qycvlfOaUos!<1UobEwx!BsICH#s`Z|cv=Cl*(}+^zEzNZeUh&g&b9(0(TU zs(eK?@DxK(=)>}V0(fWo-qRGLfe#x)>GUa4>$#Nl=DebTtRLr9N z>huHfRC}BJ4SW-6adaV(-kkGl(&-H!SmZKs1K$OfvOMhQI-AjA+z4?RFAfg}WoBp}#^lnIGUipivaQX~<+pXg5R`E<6rR-^9)1G7#f^kY-AdW>C{l)TA z(*om}dIVyV-^8Zvi$3v`pGkU!D&8U`Fc1$^`7^PZ-nocJb~)8QvzX_r^tvjCy(&Eu zQ+d;0io2?GOlFF^Jjq>E)7X;KGHy)Ln9Ssc+T?T&C%q^Qs#CM(F3|Kf+CHM0@y&4Z zQ#k!m(WvAY_lKXtWxlR}OVfx(e{Z|qtd_3MpQ9nDM<+qe5U852U+A;&b;dua3psru zr!)SvF6en(&}Smu&h(kv1wE__eGc+XekV7xa%o?@a$My3qgiF65MU zLH}zP`cvJC1qA+m-h~{HJLMnM1^qS9J1gJ+LeS8_M9J@TF0HPuaW;5L>OD@U=FBUe z<1Ba8yOvfpcwF_xbEdm%Yh1-8rEZrD>pbNA;=IDyWt#K+;)QdIJYzhS^{$fgG0q}S ziN}=#vX7WH+Brw1Cb@EoJZY(oHC2~3x}0^M`n(mbaJ~quDqf& zXHHJx%%&!iTI3lI9jmKqkV=zGWn5KlgR``x!Iha;Q{|bFmIJ!8tfj<>#A=(Rf@!Hl zYAGqLYD!CoyP?Wi<}PVyaC(;4xtvus6}1q~kxF5P8fRIFr>qiD%iV=475@y*NcW~v z;&#`9-Q{$(l)36WRkbx_;FcC=ovXf~wx+~gV0bl_m8~ zPkl+1ry;K_Cucrr8OT>vnX|UR1M61jl@&X4ibgx7-D{ewJ2Sv^HUJEjSytQV(VPov znyYHco%3A{jnybn&SVAb=dM~oUUZISs&J}Hs%o^FlA78Ex64(hSdp`qDu{8q>g#K2 zot#(7BjPo9%F9NMq>gk9d8@syrDYIFzBHnjN&2km)14W~soL~Ki>6M?bEYL{XtN6P zrojtxrq!LCo-V!4^yCZ`0%v;iXjEPueQyw+2C%`;U_}c8JqQ#61HExKfM8zP2*~F| z-($AN!uaJF@w;iwpwXXz#F;8Q8~%46UWOsT&{iv&XT%K{YJ;?^mB0SjHn?osRy^rX zK#yt_M*hdI--_WNSi4)%8v9p;kpAZQWX>bZ@jF{J;zeqnVA2a!r?n)2ZZ`V50Qx57 zZwa7h_adz{ZA}1uiK4F$pl?$2?E&;&u`EX0lr6ETr|CD-r+$o7H^=W?N)FML$ymTs zQyB5sX^xph*=M1fX%PLOg${#y{|;H`pw zd5(|%3&_WZe~f43;B2_aUrAv{4j;$g@ZXZ3x}+nA*#8dsb@K0H|EuJuuIb2L_Wzpv zv^+Ypi~Y}#pSq$Wo7n#-`Kb#!vWETllb^boBX#V*gZ$LBA6dfw9ptAj{YWAEZy-N) zM53i#n3Z{kx}+mf?7t&f(<(N>&wmSYH++)2aZ(eQI`$$w zw|&xnFyw9=T?SI_#^~7;G7}*ey+HOM{*ITGcZRfY+CTF)ynS)*hQH*tetfjBI45mi z+AEOxHWD}RhRozVDWo@f-!7)laF6LS1;S6y@cd z4pQTJ8sR(k{};u(;pN;7V8}U`+YaSdfv!|*JcyJIBkEHf>ropaXCGLeO53;L*B$2} zrm5-Sj+-dB^~FNy*}n3iRo9Nm#7NTyE0yGSluMZ(bu0tRiTbAo=>Jj2ohm}^Mq}>? z64))b^<^|;Xg$uWwX_xtW2V=!A?M?cmj6&aUfI!c91gP9iRnf8rAI#8uk*SVmZ`zyuE-LxUfh08 zmR{%j#8Uaaobws=%cx&R(d<6!D1?tZY|rPZ3+-6>Z&~Lz8bN5GREgS0--qhN)%p+u z{p@%0{2xR9JGT$nvpwgS#h!&+zd%D%+t=|6Qj8kZh8)uLXh#zWyG7)cJS^~oY6r`I zjd(ZYbdU$U9Uc8K)`0L4a`sWjA7toZcse@jWC%Chj#lY8gg9uE-**)K>-g~v`_uNd z{_FVqoaZC2j39Fi0mP~_%)pk1BwyuS|n(aoVK(69sg7jNH!{It(Gnp+luapnKJAn_SOxjXmNq~=L$;}YWgd?ckx%Y}PO;F*SAVSf zOw2io>@c6#{;E83Uxo}#vu4M?zRUJS=^g7RgJpf`Q8(aA5B0g@PHB14RCfG#fFK(r z6|=WdJ^-@MeAe`2`?*=l;R+*T`M5FeSnA{H@MpuGN74VI+Q67>Q(0jOU;r2x-91$) za-xQ;Jk*iN^!*?o%pHLWlS5UCGIqG*6$+Ah%C_3hXAjf9?S8gB^>7+Lt!2o1Vdz91 z*hzJeYP&Zdq}bt(Hl~xlALWie7`aoHI1HF-RZ;e5Y5S;dTKZeRv`~NB{p)YX*zYg{ z=z)i~Ka6F2Qh(==3XyhXo{o!w8*QwNHC>$3*1A7mfQ%Z!}b{ zQ`gVizO3WHohYS%@qp%k7JX2~pz;P7wU1eM$*8U4w8*xhT*e5Z{b-sLu(?w^lHg^j zX3ajT8j);+)bYPcQxO08hqpe2FNlM<*8LdXI_n>-`dh{iuP|TzCwoG7?|kNP$1th! zVb&Ce?U=@dIUPt!D(9`#pEv;hL+VtltSTguZaue0&b zv{zUC6T$=gAGSwV2<+Kae5DW4wDwn${{qS)dT3vT@OE|kFuU=qZ(Lre zVpQ*2kD;3JtQa$~-1ctA3so`s>W;ad@Ye6H^gG-9{ZxE!{w?c8?+Ow5|KH#Lw-%uN z=f)bG6W|;HXE&i*WtHYTa7as&M>Ec<>N+=_xQuh=)il*EbJgSQVYH{Q&h5&}KR=Vs z0CFb5;wEx}#d@hxDfa$;5y(A;HRK}0w#Ey;9ms17kEAJA>d=cM}hld))^H9dBCZ_+kp## zcLUb}9|Yb6{1NaTU_0jJuLI`-e+FC$Y)AgL1E&Jpe?Web?*w2wmWhji6M(k^uR-`h z;9U4SfGdIfqMuq1JPvp_@FL)Yz?Hxq!0UnSST5`aP5?d#oD2LB@NP;EOIF&E$pu~y zTnW4zcs=kz;N8F-zz2ctum}B_d$eN3d`+}Oi2-5V?3*wQ6P@mS9Q^$F@pxjK(T%s? zJquV;ph3{zo)`!2gMs+=?nh`hjc_hkQ~q?H z{9=TsBD}Nmp|cl{3a6!>w|w9ve(I0n5I5Btw#+A++J7d($6CYdec{wE6nBv}$*2C} zo0Eh`p&w{N-R!DtQW3rx;k118>b}sIexV9au!fiU!s`%z7vc}LhM(^X--Ph@PLlp! zgnx{1KilQ`;vZ7+yVCFIPsoo#KSuJqil6!u;e{s&r?Rh8;f2=x!%by#4Z`|jjOr?F z>T^#PPJJf)1SyRKV-;3Z|gt;E|twE6%KI)$=@;)PZ;Hx0M@A2^Fg9brgK6*jKF{I1# zZnN+O$jbs>6y}uFE`okx;k&>qj`L9tz68uUPbLqswd2P{7u_jhpnxb1~p=e>`4G{uYe$ z^oK5={}Rym(Vs%vE)~vW3Fyx${kZRg-@GHkV-ZDEe<~U#fRdo1X>_JRf69z}--Z0N z1x0_mm0)zS{I^%FKTN%ksrZWQO&?1m&JQ9nE4IYgAFCL#mB@ZmFLYb%|5ypeQjPtl zT$TL)bI80`9;nzDRq!(vUZ`+|!c7XVQ+TVw_bB|N!Uq)oK;geB{GGzV>YkdJ{-$ix zB{41lPKZrdP}*4IX^c%vOU_JAO-gHIZ`!K#^yJjc5sIwgZf4EW*d|<{#8t!Cv@yx4 zxH$MC4nuLTk4r^pLzD}$%?hOH6h_TMx7mUw0*ie8cCA~GVc?EnXmB1Vv=07Di8gXyN?~O^;X%KRI>j z47-^9{f%BKD009+BU-Nnm04wYWXvF=pH2xyR1<&X;8mnWK#c=^8WEXl7UU0lE>Q_aGJf>pi6rf@oIPJ(dH=$lic2dAirDXEDOX z!t?bDbR)?22{?k^LE%g>LTu6c^)1_pN`{7~?k3$Z*wcd~&DMKRF&60T>o0wCbgi3l zuCJu+zmp|>F6v7_Nl)#%q&GplDrs_tZDBu`GzUdaqmsT2)CgaAqnDVHk1DW!vq`gd1-N@Ng(t*SRDrq0_ zK2~%Nm2?0xYA%&DB@{J}N}64TRMO-MC{Wf{6=+lm7w95=v2JYEsVuob=NmWaRE8+f z&AKYkN)+f8_MEgpdmBf>-oOCSD~xtV#KO7yJlz=1bwyQ?;npe=|I<~ZSHv)uREXj- z9AYh$B;9t9hji;f?Q@7OI}1B^7Fw@J8K+oZKy2+N#38!`#OZ?yAdCl9uhVy+o?+}2 zRCoKv=wpq?rE`!z+IWhcT0bKm{BNn`xq1(k)q9xUpV~&8E*KBj#ej441l@R-G_~p} zn>i$nfTWHdQmGyZk$5e@lr&IE(!7!qNz!M;+KHr?C73%D>BjMxRdAdK2U=K95fmy0 zQBK0eU_uchHgX1VPd$6k)=Db6UNy1n?G;0{C$qy}#oRf{UkdkeY!onifZ2t83 zwoW85T^k;BzCC1ee9=J6MndYebBx7y5pr&PaW46bw1mYW3q{1s&_WxrIApPOhEbrV zB|Zl7A`vn>;rx*0BI0#oO1L0oi*k-UKjdcV411e{&JVdo`F}_LQ5Oyj*>xB$D^t=% z14Hhx5ScUiVrR$$ii^Uuq$MRG50k}3#1Zl*mz9S+CKnc_>T zx+LUzQeK6pEOUpv_&Lc*t*H(9kh2gr0`sDf{Z1%|2g_bFT=J@5n&;}u(wUxv1tAU~IR_d*^ z-goT(b?6rH44fh&oDAf6@ukpP#2qlUA|wvMT=;$w`h(-Z%b|B$LmUTR3B6zR@us6C zl9I2A(EmDKdnNQ?F_1%8>eScY2z^Y*S|XEq^NG-BlrYMLHtns)LSHnqM9FK@4mm^L zFe#K9ZThpJ@0o;nBFuPuVCehSJTWyVO%LrS#7xpA6@|uDVv%UC9uYc4nsnBsq0=J1 zqC!utZwQ?i@jaET;Dy7Gpl9r%ha{VwEkBlZm(PixLa;XLsnB;LI;a5tNpW(Y9vJ%G zS8!Qq>GwzvS&)kJ7xvIKR-y4te?&;=D_X)mp*uvxt0V_bv{Z72akRuq()^o;825?L zR*rrz>$E#Bj1=T}66}gd{yMF!b*C;PNBxc`$U(Z@fe;>7fThAEY!y#2-1* z!+S#?O0k4LvN!Z$2m3iZ`O(KhAF-;I^!O8@PfG!;dh(Ob(EW;m64$i+pu6pk_iaO) zhmoek$dcOZ)wJXhQ8(>yID*Kk**80~gG9s~;0U{yqJc2YBs@X{$IgL{uY;b1Q?b=> z)xBe&<3}rXGf}7A<#c2iijSgb1=+fzUy$HYVTvy1$XZCr;WdMx?RJMQ=2F{l(fstB zxQHg*vk%fr)E2p^)e$D@5GQN}MU1?@)e$MqBmZjl-*BU&huGi^52nbQ9d;49Eri(J?0*e$52!^Ttm$g5f%4+u&>Y%=*F{~-~POHL^H zsC0&z`aP%o%v!i%pktpPDa`MUe?T$KAtBSQZg<3DRY4MEifD*e3ddY&cfr+pxolkV zD@aqv722%J9EB0V6dkT2c1acBT`0X#D)c<*?XSESOK%%WE^G@GU+&sg$K{;eOk&Ax zYjv#V+_C`Za=8f0IQR0fjEKP$4#E{8EPvv!DQ^6#`$0PG0 zuSu^ZKO0*eN4PRJ5=-v2t&Y#Q3b6o8JZ=}^d+c+mqTeFy)TVB=Ka1zCU~*3mf6%TK z-PX;x)5=8+?N<9s%tdbI3JyjuyDws(=w`ei!si6b&Px_lupMEO!{-KT#qmf8^VUke zN{>9BM&7tkJbn}DT!0nM?V!l;R);D;i7ZpgwcdoS?4KXCw_S}?gk3ZECEooBo! z!t+Ch7$1o6?-yn*njC%%QGV;2Oz~_fa9Sp(R@SN6AG|e8v(p&vtzqOr4O>h#%n3KS zir5uauDlDSw?=u-lin8Py;ypEC361nP{URtG;A%god0{&FcKBU0w#~w9p`iY$LcaB zPe9N}|Ke+;)ccTP+U)bm9JY)2mW`AtoT77_?s!U0f1o$hrQZ2L9l}wfdlmjKDJ+)U z?D$F-5xYo@yGbGl-F8}vKNU(k9rZ!B}*b z^*q>z(JSeUUY*0}wMs^>w=#NTE2B3bVD#2LMu*;K^!7g(y+fzduAUCD2d7D{o?+r3 z<8X0=@c=>kxOxr}WE@w|A%e`}>KP{vGfq(PMyPm+9Iqlzl=V;PJsfKGPfcc&Hj`2M zB1V~)FdE&!Xw2n|#$L&2{I!fG>|!+WQASxWGMY@=Z)N?be89e`e_}N4dq&eE<0)*$ z5JodI7|qIJl$*mSZ!V+R=P@d%Vl?M6MsqhZD!h%+xsMQv$$Ey-q&1Xm%;aBCxy59^ z&cvx7Fq-x$qv`l=3c_YYGs+pxXyz0~v*t0%buyZ_ic#U6KufO{mF+?+iyi$dBF2vS zHKVZ~5*n8MH%3#xV>B%|fr!(4Gnx_4C}#|#nKK#9TF5B3f>B-*qx^M@W^ZLwa1W#N zo+LCp^8lgv)gLli^A|>IgGUgtEu7K1VT?9pGP)**(Z&Ugu5~fGu94BEXBl1pHlrK< z&gjN(7;P4bBy&p-Mq7t5+BTliP5F#=EMau>WsGjw%4pZ4jP87%(Os`Hy8AGrd;ZDj z-tdtmdiObu?wiW!f%%N~R5Lo-%IK?m8GZ9Oqi^d`!^`5odx?F=e#huvgGUkZ-=DJY z2ZMe8$t09en#ZVYJ|kBpqly)bmR`-M@@7U=dl+5v9HV7#Fmivxs5&xdPZw+ zV$^y+qqb)lUHKNHb-!nH)t8Lc+fqp8hGC4Zp2(vu7_;YmhM zzs=~G&lx@INF`y<4P*4n>4ZkjU&5%Ufl)D!1fv#S&c5@mVYGNBqYL&hy6`zh7ro8s z;y*KTe#vNwNF$jg-5He*VN{mJsC+6T*SU-;Di|$oVpO?~QPozUs($()x9(VJ9 z((4l4JNwb96J&&Xy+M!6ssV3g7EKM2kGktXp&`DuPUq>=rL-%fxf5lKCcRmAlo&K+ zQc08;G?6irkdwgijk?xrw2ZM@U!yxNHRk!0y40XGqZz|*L8$c_BV)DdZIlj8z`g0v zs>_Ter~jiHb-C!$dS%IYSL*91wc;*Pqg5KM70g(ZWUQ<8c1n$wFW%H>y`@r9izmue zAMA3A@U3<^Key+h;IjC9Xma>AJ3<#)LuofOE0~@f8gnl;KWp{WranfEE*ymhi^u6K z@2+J9)68_TW4@uq90KE7X1v0mk@hmOf++`*@n2xvz>Mqt8Ewc|RxmB|n6d6EUAwe9 zWi~>;UDU_G)j%tXaLqFmsL>uQnnLW|F3ltshkm>6W3g$C4#V!Dku>gHJi6OlaoNC8 zdTujizoiTw54-5mOy0(LBdFyvqAvw0ZarKRgF`vm1btUCS)^qL1+$AD&E#!)h-h^B z5%(-y=h;)@sOB#0HlT4qH+wU+h30TPT0S5?axG^sJsL}O(MTW^Jl=0^O5Er{> zd#{<+OiH7|=ZiJM%_^jOX=+OcT4kOHVI zS}MpzwB;PYGb5&;@LnrT;T zIXznVK^sHOedt<3Da^tg_5h89?RK9yUH#1QvkV^vzH#tw96ZJ00H0j2#$Lb_?zz0YlLC%9v**` z6zd+T#eexX(7X2vxishevA&Yam<}WR?Rxuv_X6h$e#SC)NGOSH>6VW4NOg2M6^fqA^(ZdC6zk=j7ug23OqlSp+ z`Ie}6A;gk$tGJ(1p;X&MsEGbP(0XmcA)^25qk=6=L|-Jc&dP8$yp{cZ29IsQzdhL=ScZZ4f<04Z`gk zL@#zkX;+JNqPOTHqF?S*T$|`CqU$?TTPC+o^b`F>^qT>i%fL1^_KVxWi- z(XR#4+r%Idoh)tM6HAdWu8UmIt)d?Y&D1d3#b6O#;;Y(ZClF&r^d(%7R6oP$0pbu5 z-FgCXsE9uAgy_RW^mAMwzUXZtPDB^mnCf3(VfR8ARI~?4r$_Pd(8}o=FktBlbt>Q9@OUbs0WD&huYz<^f5kW!iB2`5HHb6NR(l1GF2k9dEQ9+G0FsNN*i0G?)HfFWR3oSB5^bpxNsp5k$T149ifG~3U<)qZ@{Z!kBEPtM1=hn%SL`+ zUGuP9?Ch0`ok!$i=TW)Xd5ji2D3oa;`U>%Die#-)(?w8Jo0uV@pAjt38zyVSMv;SI zu}#br(fdUZ_BVWyO@>*bo4qPmTr2ucxSdADn89Ex5dHebB)~CS=)*UPd=V^S#=<*K z^y?Rs4aZ!epR-ZS5r!T!58iV{zy2}Q$O?r%BBlhCBGGR^49y4U3w;!eDi*d0Oj#go zS?pLSY?Ih=p0G`h*@jq)g>57&ut?~cG53LTfv}C@q|X=nn3!ilxk%WOIMEA*J}%}B zP@KY+%!ytsbeY5wVLLCT13_hCK;M`z;V2b){6`={}}4amkK>GCJ~fN#DD=j{#6M*iA60FwpmPZ3tJvLs)a3|9W}x>J7x)D z)d^b~D^M%+(J}R))C*fWCw-~V$HugQ;t{qCPP9Sj<72jh(j;t|oM@xaC&ugsrA62( zSaP$_Wm?OH?fjS*5OkRs*p02XLg+iM6PJrYxwOsHE>?;`^SJM!?WQn#fVfHwDq{Z+ z;L)0?BW)L}#h?NvMq%-(HIuVltPz7AqfsD?x?T9M5QBcf{?W`d0Z)(QTq_2(k+?7p zX&0?x(0Z#M15G)1IXud+i)RMht@kt5i>SeltSbdA0|rNFS?k0O$Q~SR*Em~O3H_xu zu|W)KkbTk92$Yq!nbykJh-)wmY!pLQuyWkP(=pFV(1F1|7J0CrPaSv3|;H<;ioPhmw5jSO%i1`%G+eO?g zOpc@Crf1#8!$w?=mbFW|X4H#jqVCkcAt> z-9Yz?xKvy7{cznVYk-#@-QfcwZt#iP?h$bp+GrP&D~hZwo5h17u1=SpE#e^&H&PFy z^daS85qE{o^+zeUSHzubV{PdHjz>h?#Wvc|46{-n6>$T7se|n?5qF0y0}^B!?{N|L zpk-p?ofad)6C&rwH4no+VhpVZUlZqai@5-U{F@@NIHnwqH^hjDYsBkfi+D>U_OOLD zf~Gao?r*y|BoZIxM#984@wQ0pC5@v9?})_S7DBstS0w(;8sj}oQe`_-G2R!6$0Xse zNFwYArPeAauvL74XESX$w}}r#Vxg?^TtY08A|U8Pk*IS7x&EUJg8Gq2yjZqBmP=Gj z)7r#uMB;Fno>i^iip1+AmF=S>|4t;{sq+A^AxSk4S+_a7m6u3u;`bskBq04TW>_cw zApQu8b*9FjMB*zxJIGc_3PHoeBJsbLpl^}1*8FcgJ<=3^7Kv}_T;-Js{~{9alWL}7 z58KlEl$z85wTZvNOg^#NRpMijXlKF}@rg)`vza~J-$Y`$RM~5^zl+3p8P{sG4v{#@ zM@14xM53AitLpm?k+_{>DF3G-afjD0)7v6G6N&HYvYM?E{}i8##3%fc-y*&ciMfJp z8b+m$l#YtT0*hIFbwW+uzZ8iTGMs}{M%Ib1#MdJ67e4V^Wl%4!6W@r$=g6Mz;#-mU z2Df#3Ag%92;!a;SrJ`-(dy!ae(SWPXA32ty zhVg^WXP`t6J?S1V5&oWf(tTEcFMVVtvC3-PE~53Mn<+oikFT-^(O9(=3jrXpaH_tX z_71lR;|cb^jYEm)`Yqk?qNl+AwoPxTRxpv=@C6kd53%R_^@E%c%Ry>no`g#aDw)TfAr3qYNvUl{C-Lpo?X zM92gVu1D~%eSu{ZxJ@|dMJBQEdIaX{qkO@Y6nwJ?r!7^n@H!~6Lv-)Xo@c?kA1;;C zYMDw88Tb(bAJDyl^-@ePJIACKkz2k4hle`6ZG!f?Aw2?@U@z1Ubdg!QcCX%m=&9`# zy{SCK!v2P|FRrw|OD9`?r&c>fA9@d7EG+kNeRcxZHqlq6e=X9#NZ)4jYTXR2an>S@ z?U#Nt-Ot@7`jxQwml5$|{iv_hFGoZw^_y|YmrC7x(HC+0OT~Xc9H(B4J#b~eD^y}T z>C!6QQZ?_bBF=ycPA3JootNmj2(!ewNnp~Av~JNumkPSpiE)KC(Qg$4Www?gR>+JJ zJ=3QjZT>@S4DLz5I_tnts+anL*ZI`{+F$)!#UL8a#KPxD=qqJ8)b_!#4g<1vc}jU;-nd#M~wNt zOpxKwK!!^VeviP*RAAk;ItAWFy#nr<)4g)vU30COPA0aA5rwQq?=ff@`b>~i=68_T zonoX+vp=$=P5^iFWiAAbk{Ngzs-xbu+^IV*=L=}JiX^Fv8!|)RY0>}St8lkL>Mp7| zbV7sFlP7ZOo58tU?+%(u{VwoPWk$ukS+SXY;kWqF%OmvIvie7U|Q%$v|-GB_ieRi1_5mHI!zy}?h(;7rPx zSXhDJReFD4@GmKNrx+`R--}e0J;V5hErP~LdyWQD_B_H@EO&_UvVuMZna}y^J~dus zbprY*B=|l8uk;!E*9as7qJ_@mmbXJplu7>x#`XZ4>=ct_U?Z}t^xDlgYoXT^sh10R zQF>j@S6Oct*~kTwzD8fZ9_OtskI>_{c<^B)AA<79qdcH2LkfJF$;X`Bnej;+A7(Z4 zu@)bC(UBO}O+L2clRZ8LL_W6Uqf8B_55tG|G;G7``$Y8_Fuc21Fsz}E zrw`^HxkPdnlUh6gRdLlc5duw?;ZNtHWUb|aQF~(sV z?r=!2J%z*2_ejSeL<|i#MH=HA=`e#MS=0=d0`huh^_$XvQRJidXS0`vDu7TpK*2+rx&fQt%~D)XA1{%fx!c|i64k^832uCG7gzbG(8zQs2_#L zuo+2P#2dsR!v~_U21_TH;EDCwZc4u_CJnJz95IJpwN`M+;qD9>FwR>#2*m4I-qL|L z-Yd;94&HOj&~z$25E7K}9cH17@F$Q(6Qz7J%1D2Xs|8p_dCN-WJl(UM69*y=nKBEOjhD@aypz1b9R-d#eY5(- zpkyZd8X>MSLRD*!I6dxdzba9)Z=jQM8Op_YQ3e`yC&yTCsIO`Ah@l~sPOs^u{9Jgd%> zuR?EHTyRtRvAXA46(Y9zJvD2MF6yZ|rQ(354TaGKdmA=;7p8z2S3)czp!=fUpvwBc z*q0jahIj;Y+s%tdt-H9HpvbH7zFp6lyU^y%>%D6EnQ?5ZLfPAUTJMIGU?OtT|o zd$|L6kTp%K4j@9W8Zt9!M&Y=oNAl}1TEtVcx-=8TPl0-0I@b`y(=Y_Iv54u3hXYq1 z;!p&W9S%^M$mwuQhO;@616!gPEuY3{1!QZNP2_^SoMsm*O?_l5mp%F_s-E0yL(Xbt zXIP4?xCY1VzQ8MR%I@>6#VMY7P-{Xu47$L%2SHlCbn6WX7vRGBS@m ztC~zl|K>13=l-&+qKMO^- zUC1^jN9mhz!D0a>v^y+(#C5Y*4*7O^L&zHyn%;!k6icLAo&mE`WqhIzQDu&Js!G&01=LdJ`j(fc=C?I!X+ zUYpfI^jO+3z+d+uVe|{Ny^5CwlMOA{_T=hkh~fanzDd_%S!ORZ*>wkzAXG2xAtm%P z!a~0`MqOwO4F1^Ga4Wu{5T0*@1m9)|dtbO6v38$~{JU=vt)W$d~mX1;(!H ztt@vy6Gkv(cZa?a2aVuO$jB>3=3-1Ic7#=|X?D4XBJ{mSALIxFbs{LZ!aMt7|n=Y#`9JZrDSvo#RBki!p8>t>+=vPKWmJtHc2d$zDF*b0fZi223~fk0Zlq(tubzsF$IOr)y+Y=dl~IiMqe2{gVFPS(o8+y&2~#B zV|h;$$~*vxabS!=x8{?RZ3b_XHp?>lAi*J47P{fj*?9=nAw(u+k+8`O{v{F!LGA4C zZIPttUNiU*jL|>Tu~wRbg6QTn3;r@x7{d`$b+10{4x1c*28X4u^l@l(jDSBIwMEnnsIXa-9e@{vlr|Z-#;C|#T8Y7PF{mVZvLL9aSZ253j7}0Q9?776`e+&5u3H=`R zVXf_f)wkN-*SJnBl_`^GdKZ&LKZukQV7MWErr&17nOPMtr0>SAY6 z-uXGs;>CqI__5kD{HAM7t%jd7#SfIuZNx92&T!EWu2$7}s!Lj&9<3bS`V#yMYgSsp zsMxfE$(pkede)YMsb2nG>KG|xTzMli*5tYI1$1`i6}Z!K2y$}pgS5`2W#u_;w-q?8op%%5G9;lyh=E~HRfKZzpG#?Qc>iVmak3$8UD`lZ=Zk5ktGelxX^rxUYg zsU>F8X=Jjl&JuAXr+JDQM9iwwNT|Bnl2Fp=sEC-SlTckjK(USFay!LbRp$l7PCBjF zl(ySjgo&pUw|t>r+?3Oao4LYOUkm#ZI*l^Pz;7`l7M`aSsicJ>m9%JPqsqdMPA?r5 zi%wy}1i$thJ6UURmDSdiH=qT3s;XUR6}Ik zXn~(j_E@`?RP&Ge82-8IpKQ}+4El8iYX@`svArG3sdf$4q))qFu}0<&=1fxM{^bk| z#of!9#XIFTZ_Ob0GG~z9>GUzy^t_$TnWSlUHD?l6b~I-YQ*|?E5Yy~r&LFDlXwE1m z_clM#FL;ZjV6trNm@&+8R`FB*Y(0KmrJ-tRjjKGC`YNqXQmH#~F7>+c!}L{6E@x#) zO}QIg6?!Lam5MG^mmb-EovP#T(&iQUlQJ8cOX~1T@Jyv?gXA1{31&$2BQNMgO{brz zOroklzko62E)RYn20_w6fj+rombyI1GD4-pFKSwbU*5pv=lA1HPsMN9(>&1SMAu!` zSYPj|DO-*xosoTZ#VpQGOxkD^0cMx=9;eKBPLU@qwXvoOb5CcTr=EVt2K^{aKPe=$ z+*MK1==M0vF&p)`@`}=&IXQ(ho0@W{D6MiZr=v(Pf;ECN#tCwKzoKI%mQ&`=DTH|M zFr6?WK$aYnq}CHA$mB3i%)qpD4po^Px2xJ!H#nIrIUut{S}rZa>GXxBj&>GR)ht78E1ZpKB)L&rE#WFs zDl5RU@(QZcIx}V`Mkpx7OnEHQCaqCRGf_>Y=jRkVo8^*$Sih{a1(?|Li0@x|omzZU ztf#)wr3PsH22wHo(2@KNB+6%gC$*-NjGmB78nAyBX)<#@t(4);Tv6GvP)Ym@I;!4B zmMeKM^YR+-^a_IdRMyD9s-@->o@noaQcRF6^$P2!GmXoOFJI-(O7&JA|E3w6I4ZWH z#NE(URY}ds??ll~Y7h50R^>;V$_JF`8B3g9^fQX|lvKIVa!De(C3w&u)l_1^>~Wz# z>Z}T#f!;EapG|LB#eYU4sA^6UE!F#ERpnH>AG6k*Kd^Q>btogv>8zs2UEYPa<+30Of z2%-AVq$AA-FrlL9l$v-fKsTQQDD>2KR(z*kc%x#QYOBg8Yjv26dR%Jv4RbpG1Tl#- z=e6eas;hn#^>H502DoE73qfPh+abf4kV*$N5ln;6MmLMdv(+vw6oI-DyceaTTxT!SY&SHO@vnYoSH*d$=xsl={oRSK zL_YOePP1yBS$k`{PhpaDb_&Z0u{kYX_~M zL`P`KLF*pbNY6}sIn^g~V>}VHprpLq>24}309SOym9)Pr#oJ~MsN(rBUY>uM}MlmsoJ>*XHCS(!dCm9r97)>LY- zVQgt_E#|CeVFF8Ub2ehE{ovWiv368vkt;cKLvQD6;GI|MQ=f>bJrf!?(R!-*$Ag`i zpVGiyT(T52%*x7e3MlvF<i8!*doK6aA#Y9nxeH#M2VmdqX1Wf7utRdAz z*Xx^1`_12ZP2#s$kW=Kd)acX~Ac5M>qjBQo%34+%XRYR`tTp^pH2WG9!DRIY71x6+BJg=?D>P(O zrsk<#CTX0Qr?!!COGQ1U;HiaT)vjt5Oqg^5u7<7HK^FAp)|;aSoD!4Jy3%?UUa$Pv z*OB57j02^bMrT<|i%Pv*qZ6mt^4dmh`zV@MCAaVVgt_=|N=hhql{V7AY+@AeI8+lF zWj$$QOjky_=^B%1x}3BE^D#@WFN*a0B1%6;_a2f_eMVA^Dp4=Jp|ZA~bf5qwk(^2x zIi(ypoE1t5qOyl0czqnl>r>jZ-=e}BG=`Eg7jR=8mO;QRGNPweMnDr#0j2nf z#%0>2jkUNF;i}e3N~usw+R^Up9MpmgsA6V~#J+TNu)~Y3`8V#U^`Zwz58sOmm?$QQI9la7qYpBQJ zs|K@ElV;?W2+bEJ_ds24=#T;jcQ9^uc~uh?at`U}c9$TCX{@D+jxL9#VPPsA6-qkq zl#ll|+6>6bLEClCa#qyUVmgf6DMuOVWxcDuUaLSQ!pUBRi@F;SYC)=SH#SuAD1oda z+lb2;Pb?X0cu0@~11QLBIXyYV+sjnQ9u=LpYnet}ILpB0JU2!X+SDg=a>GRNQlc0a z*o!ixZVQXx64c>(7lth4${cp2RER5WM4nxqCc?f@8SJU9tEfWv1PmeQlqe~!sX~n^ zO(vFg&Jw>#Y3&u*Bn37#)TlU?5E+DGLm`uC;lgeK995W;s>%4lO~~Phc+6JOY&MH#v-&rC8-`K^pD0!W;E-I->8Ou%O0PMm)-P?)r&nGXh6CQDmea)3 zxWA>=n|?RTM)5rk{|zy#!O}5zKQU>#b8zyjqdBGtm8jX~E2wQddp&>l zTr4Z|ik-!&AW+YT;FDc{>r9K&GJtRCjH@mFmhc7fs@gP(T1t6$yu3Fb&&Vd;L~!jb zrzqV?*KB<>zHbYArOI7TpS>ibx{uqL<>ojmFl_lPH*hG=H#_7JKNXd^vdF_FUtZ

g(O6x*Tor*lxm9hQRvpfz?%L84H--ru3DQmc>JsnuUOD7UPtUV7qWqku zoSb>ZxZE@keVW>3OS6nnpQ(`pVLHr;)!?S%`Va#`=q@jTlOcf;KEK& z9uqX&lrC1fs`8kT&O=<0$91W%kkWA-t_YbjH|-~xcdDf!5ImiuVF*q$Up4SL(p7m& ztx#5-ylLP@hnSO7N*1t;D=OGQ(CucNoiR2WGapq*w1P#`fqh6{LnL$qc z(mt)xjP7TiG?QNC)SU#j#A(bG_)O0$#G18}=*gExEmbh>w5;nJ4N*U-Mp)iN$vG{9 zkNx=#?p5yps^$GsIin)*pV`pcKPSkv6i{_l*R2s2PfN4BV33#JS>v3G$(*-o=6LIQ z(O69Jil+LyjHI(~t8SVSa(~97Do1-ZTfd=a&v@q4%nNK0a#CvgwC6PCuZ*KO%^mDObkxaJB&U_?{L-z zu#RYW)20)h(davUWh>{5tA_90AUSC~o2qfj8)qmdzjC%#P`=2(rVJ?MPRy%{{DhH^ zGbKmDGp{tK+OkhRoMd)hVSHvkrCj1sReXntkju@x4PA*B52D911y~lmAi1{ zMMqHee25BGxNGP^@62Ixh70OWoUu8V1KgEN<0r+zB?Tf7i@dF-CN4yzchv|1D(K4* zbUIh6h2h&<^u@BgUl}jnVztiss}*D1V7;k^nUQaDz9gDP9$B?_BgxG}$F zV}7c}{1T1%$rx zJFTmamZUT`)TfkI)ubdRr?{(1QG2;~bNj3)Ey!P(Nr6Y1MCSMGQj_;xPUUu{b}CH1Z`t5`H69$fR^#Bm9LKE5mGX9~`B@t3 zs+>es?}l@^z)P=(C#kLxUPM+$xA6E*YT)5sUh}=5OE3KrnUA>ZOQHZlUPy|}R!3L8AWGe|!10QZfQ7%?-axh-H{e1IjvGL0^i<*6%yNx}@~WE2+Vg1Y)XUSC74hkCjlN+v6o6lF zd{A7a2W{40!bju=1oc`JR8pY{TaY#?D0L!2wU_juL;54OpnbMm?Y7_p!q#n|&7Ncn znP3YYWOGE@!V+!aqiqqxY>~rkQ6;wS-EBR(+j4^Sd4LVyHs6d-XW|Q@{6~H4bif6G27nV#38)54#1AuF4!jA_8=vFc2D}q+JK)jb znsyKHZongeCjc)1UIT2OrfF{h2jP>OVE_jp8juT^4;VCrzPPGsPXhJjG}TZoppv^pnZw0j>wU1~>-z zFTe&Hg#!ivh6B<78Xy=j15f~10%!m@@QL{iz;^&{$M@kL0Dd0uBH*`xKLd^djss#~ zkCA{vfD2F$I0*O%@F&3MfLMHob|hdPpa`%CPzqQIs0OS6TnX3$pilAd0lWct7w~65 zD(dn$z2E-5+(&1fYpGj0Pg~Z;Tw(_fHuHkz@Gr00#Yz>x)jg~*a&zF@Ebr6 zwB=%e6R>)?rrinr1>gsO9c8otupMv@;GY2cS^LWY`vGC7Q+)vOfRO;2eAEI~0$Kq( z0B-_pm^3T~Tmztq#OHt^uthy!7l3}YV-Da2KnEZ>7U=*g0lxz51H1_c9fEoYhy#oQ zqyxqSCIWH*yW&w+z(s%y0cC(o0QG?7fHikmi(i~-Qk<>mtB11FY#aug0e=hl0ieSl0_YCt2N(=E2ap072lx!SWdr8{ z3IXQPQcxOrvNVkGULz(0lx_-g8O~o4*`Dy{26cr@G0Oj zgslf`1>6Pr8nkZ#I^02k9)Mnep@2AmKpjg5&IU{aBO93kYR{^#F z?gTsncnq)~@B-i+zs2TTVP0Ez$? z0IC5^fU5zmfExif0qz7m1b7ngE5HH3n}81he+2vu@HyZ+z>k1#sEd(+K7j85M*xEX zaexj$0w5WX1(*&v7qAd;0f2s4wGz+-_%EOUuoAEq@EzbkfDM36fUSVr0QUf90-gZu z0Xz<%-$~sMI0!fd_z-Xy@OQu$fNufxL#E;!=nIGh^aTtCoC7!y^(YH?CSVF+9$*o` z2`C3F1Jnar0BZp20MioC&Vg?LYzN#5xCihs;7P!9fENL;0vZ5s1AYUTj(8h^4+Cxl zbO62pdwy z0N{DR-vJSc=+gkR0ZRZ&0m}fjfaQQzz)?WyNc7!+piyX(fV%+nvxeURmL@@0z*B&) z0a?jt(sU_Ia(!1aKwfSrI{fV%+q1NH(Q2RsXS9`F+25a2_=?*NAZ9|OAn69@{@ zzC0et5ds2!)I(cjlOeH+;TofJeoQ7bU5n7ntr%1!{zIjNoB`gg8%mGUI<>D|*^A_Eg`^elul>;$$s!-E~zbwCG7iqkLC2?-qT0bv_F7OddBvbRI?nvVg7!a zmTl$6OspGi7NFc!MZki9|AvAdkX5En0X@LPOm`_c=9Z?azS?$0H@AjIOYz#Pif(RM zn)NWnF7+|DnlX^@AN}}Yl-A!{UW%;EQ~bMPB*m0-8|am7eKJ&T^(Jj|FY+Le8^Fj=9Ba<~Jmi z(v8r}t^8IcD}w13!OEEqI_YC>!51s~6^d?dHJkZ*1N82Ce{EB?zpUR9-Jit1e|ACd zi4K9%HID|&{%|koLm+>iIb6ou&JR;y1VWVHW=DiH1q~m`5!K z6ulhu&g8t(1^xF*j(OxTQOVf`AyglhsKSX?^x=^*T`i!!;75@r|9O< zlvyqIE4q19RH68F*hiVhOp z*5^+Zf1n+{2Oa+>{0+dwk@PW-=CI)4zfp?5OVwMmKVJrVXL8!Rpx>e7m`8D`N`9|i z-mXwlBuWLnGx@VXCp}>z{xkJispzo*_B^QQ=Fy@#j{IKH3o(SzpILTMVBm5IpwChC zSXB|A0srkZ>8k#l^p6zXJn}U4jOg8Ye(@`doG$H{(VtmAujzvS9?(0pk5+`Y!PAoI zaG<2LO1)>^+1m<%KODM{ewV%c-dmV8eY*Mv0EFXuJFf9tqMp$`NTla?|1>a( zbS;pQmWou17vgTr+1%p9yC4?TfBGFL*3)v0G*GMbj1v@1&peSdS|)4WuCkb#-@3@B zdz-#HRF*5bXOq{T`^M_o=t3WXQ*V-;mi^_8KZ+7~5ezrV0?p{X{Tf(G8LGzOqa)tW zpY8T8-K{#itDv3V&a#x($#2tGE@yUj0rXU= zgXJr^-tYJPRO$Q239Lk!qfcnpqtULrxUqx2%JOBqGwLI$uPQD&>kbctLxZQWq9VBr zr`h;83BE-_A1|wR;uIPm48^EXUTePCR*qwJd_TLf1>XYN;B&fFRJ?HZ?Z*4zco zoLoiBoq_kAXDpsObxz)Ngz-HHybFx)K|pc*o(6rN+%)^VnKO%W@J`s&X$3jzEe6$1 z`Mo;e#7q4(wN5&Ar*Gb5&nn29Hr<(?oSsZQv$dqmFKtSeMT?vn$(hON>V35n`j1ZT z?(sL8efLOD>Gy1WZ=0j#1zu|Y*)OPBKWuOPV7}AU#b+A%(y^}|nkMS%mLja=_mxg` z2IIXY*YtmyyPD8QiYVOZX)m&P+JnNNvUssICQ2^Cm_%^UmB1!$PWDbZV7ve{H?WN8TkbfpR8%YBw zu2gP!2CyY8HO%H_sA=vVE?)p<_3@F>Uc0rsS{Ng(KZK2{;$(!*Fw|}-d55Udv4ZuX zGqQhiu-k|A3~0nU3w2Hmv5`Zu0K9p8pkyb{x+)gg$VDsdsZ0hlxAxG51#Yi%WiR3Y z94`S%KL#LXtt6rhaOkKcv53c*5G{ILkE+Wy)`wu zwcZgcW}^D}`W}|8*xj3#x4fW`6&2YFx{ehG=;-qRHph7LA^NiNzYh%cGu6lYLhy9+ zZ!k%~%CvRuXxpiz2aNl)&uzUwqL&N}w`AM$`A64B> zf$7%O4$@N!Bcd(@=34vM0VrJH`?pw;?=}TszWv=nlFGJ4lERyA!)MPgMLXzaz~P`q|LpZ9Qf`)G6o01Ns3V~NyY94 zc3Qv_>!oh&roXwl*=rr}3JNzZ`cZxK!mhvfOBA#bf8FbBi;-I7qKPBOEZp|>aAJyo zIoA-`h5HHx*Fi+2rtIJ2cAT+Y8##_<9t{g&)xJ&^P*qq zJNSk8n&C+Q3?svD!Jd8);m_K^BPq+vWOz;bQG~A7LQ=>rT<_DKF;fgj=zAJT}VU$CMK zA6+KjM|jFNlFIW5U0nX35GZH7)@{9<)~X$t$MwzpNq+|xq7EKUxvHQI&auP>|=QzP+h~xD+;DyKfJFaWUvEk$RQr6HtX!%=o zA4GiJbJRmT>3PFP(VR54@V;9th7d>YlVk8Vb~MH>(GA4-`WXDXJ;mQT!<>}H_!kWy z-~aj>8vl=_ay-+yG$#LNq+`RKwf3i1+E(ZnnvcoM?P2sF<;WY;Vsychrddn$4E{I6U#CEp0b;ydN4Mx=zWIX!E<;XC+ Date: Mon, 16 Oct 2023 17:52:39 -0700 Subject: [PATCH 347/634] Code refactoring according to code review --- .../mergetree-family/invertedindexes.md | 6 +++++ src/Interpreters/GinFilter.cpp | 13 ++++------- src/Interpreters/GinFilter.h | 9 ++++---- .../MergeTree/MergeTreeIndexInverted.cpp | 18 ++++++++------- .../02346_full_text_search.reference | 2 ++ .../0_stateless/02346_full_text_search.sql | 23 +++++++++++++++++++ 6 files changed, 51 insertions(+), 20 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index db3d6d0a479..7e5140b4c4f 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -58,6 +58,12 @@ where `N` specifies the tokenizer: - `inverted(0)` (or shorter: `inverted()`) set the tokenizer to "tokens", i.e. split strings along spaces, - `inverted(N)` with `N` between 2 and 8 sets the tokenizer to "ngrams(N)" +The maximum rows per postings list can be specified as the second parameter. This parameter can be used to control postings list sizes to avoid generating huge postings list files. The following variants exist: + +- `inverted(ngrams, max_rows_per_postings_list)`: Use given max_rows_per_postings_list (assuming it is not 0) +- `inverted(ngrams, 0)`: No limitation of maximum rows per postings list +- `inverted(ngrams)`: Use a default maximum rows which is 64K. + Being a type of skipping index, inverted indexes can be dropped or added to a column after table creation: ``` sql diff --git a/src/Interpreters/GinFilter.cpp b/src/Interpreters/GinFilter.cpp index e60d54026eb..5d823318313 100644 --- a/src/Interpreters/GinFilter.cpp +++ b/src/Interpreters/GinFilter.cpp @@ -21,18 +21,15 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -GinFilterParameters::GinFilterParameters(size_t ngrams_, UInt64 max_rows_) +GinFilterParameters::GinFilterParameters(size_t ngrams_, UInt64 max_rows_per_postings_list_) : ngrams(ngrams_) - , max_rows_in_postings_list(max_rows_) + , max_rows_per_postings_list(max_rows_per_postings_list_) { - /// 0 indicates no limitation of postings list's size - if (max_rows_in_postings_list == 0) - max_rows_in_postings_list = std::numeric_limits::max(); + if (max_rows_per_postings_list == UNLIMITED_ROWS_PER_POSTINGS_LIST) + max_rows_per_postings_list = std::numeric_limits::max(); if (ngrams > 8) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The size of inverted index filter cannot be greater than 8"); - if (max_rows_in_postings_list < MIN_ROWS_IN_POSTINGS_LIST) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The maximum rows in postings list must be no less than {}", MIN_ROWS_IN_POSTINGS_LIST); } GinFilter::GinFilter(const GinFilterParameters & params_) @@ -55,7 +52,7 @@ void GinFilter::add(const char * data, size_t len, UInt32 rowID, GinIndexStorePt } else { - auto builder = std::make_shared(params.max_rows_in_postings_list); + auto builder = std::make_shared(params.max_rows_per_postings_list); builder->add(rowID); store->setPostingsBuilder(term, builder); diff --git a/src/Interpreters/GinFilter.h b/src/Interpreters/GinFilter.h index 3e57a07ecd4..7924bd4ce3d 100644 --- a/src/Interpreters/GinFilter.h +++ b/src/Interpreters/GinFilter.h @@ -8,15 +8,16 @@ namespace DB { static inline constexpr auto INVERTED_INDEX_NAME = "inverted"; -static inline constexpr UInt64 MIN_ROWS_IN_POSTINGS_LIST = 8 * 1024; -static inline constexpr UInt64 DEFAULT_ROWS_IN_POSTINGS_LIST = 64 * 1024; +static inline constexpr UInt64 UNLIMITED_ROWS_PER_POSTINGS_LIST = 0; +static inline constexpr UInt64 MIN_ROWS_PER_POSTINGS_LIST = 8 * 1024; +static inline constexpr UInt64 DEFAULT_MAX_ROWS_PER_POSTINGS_LIST = 64 * 1024; struct GinFilterParameters { - GinFilterParameters(size_t ngrams_, UInt64 max_rows_); + GinFilterParameters(size_t ngrams_, UInt64 max_rows_per_postings_list_); size_t ngrams; - UInt64 max_rows_in_postings_list; + UInt64 max_rows_per_postings_list; }; struct GinSegmentWithRowIdRange diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index 169ae768b31..f04c20dc50e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -735,7 +735,7 @@ MergeTreeIndexPtr invertedIndexCreator( const IndexDescription & index) { size_t n = index.arguments.empty() ? 0 : index.arguments[0].get(); - UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_ROWS_IN_POSTINGS_LIST : index.arguments[1].get(); + UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_MAX_ROWS_PER_POSTINGS_LIST : index.arguments[1].get(); GinFilterParameters params(n, max_rows); /// Use SplitTokenExtractor when n is 0, otherwise use NgramTokenExtractor @@ -780,14 +780,16 @@ void invertedIndexValidator(const IndexDescription & index, bool /*attach*/) if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64) throw Exception(ErrorCodes::INCORRECT_QUERY, "The first Inverted index argument must be positive integer."); - if (index.arguments.size() == 2 && (index.arguments[1].getType() != Field::Types::UInt64 || - (index.arguments[1].get() != 0 && index.arguments[1].get() < MIN_ROWS_IN_POSTINGS_LIST))) - throw Exception(ErrorCodes::INCORRECT_QUERY, "The maximum rows in postings list must be no less than {}", MIN_ROWS_IN_POSTINGS_LIST); - + if (index.arguments.size() == 2) + { + if (index.arguments[1].getType() != Field::Types::UInt64) + throw Exception(ErrorCodes::INCORRECT_QUERY, "The second Inverted index argument must be UInt64"); + if (index.arguments[1].get() != UNLIMITED_ROWS_PER_POSTINGS_LIST && index.arguments[1].get() < MIN_ROWS_PER_POSTINGS_LIST) + throw Exception(ErrorCodes::INCORRECT_QUERY, "The maximum rows per postings list must be no less than {}", MIN_ROWS_PER_POSTINGS_LIST); + } /// Just validate size_t ngrams = index.arguments.empty() ? 0 : index.arguments[0].get(); - UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_ROWS_IN_POSTINGS_LIST : index.arguments[1].get(); - GinFilterParameters params(ngrams, max_rows); + UInt64 max_rows_per_postings_list = index.arguments.size() < 2 ? DEFAULT_MAX_ROWS_PER_POSTINGS_LIST : index.arguments[1].get(); + GinFilterParameters params(ngrams, max_rows_per_postings_list); } - } diff --git a/tests/queries/0_stateless/02346_full_text_search.reference b/tests/queries/0_stateless/02346_full_text_search.reference index 0cf74e14427..73fb0d04157 100644 --- a/tests/queries/0_stateless/02346_full_text_search.reference +++ b/tests/queries/0_stateless/02346_full_text_search.reference @@ -47,3 +47,5 @@ af inverted 102 clickhouse你好 1 AST Fuzzer crash, issue #54541 +Test max_rows_per_postings_list +1 diff --git a/tests/queries/0_stateless/02346_full_text_search.sql b/tests/queries/0_stateless/02346_full_text_search.sql index 7d3337b9407..be56f24d5da 100644 --- a/tests/queries/0_stateless/02346_full_text_search.sql +++ b/tests/queries/0_stateless/02346_full_text_search.sql @@ -277,3 +277,26 @@ SELECT * FROM tab WHERE str == 'b' AND 1.0; -- AND result_rows==1 -- LIMIT 1; -- +SELECT 'Test max_rows_per_postings_list'; +DROP TABLE IF EXISTS tab; +-- create table 'tab' with inverted index parameter (ngrams, max_rows_per_most_list) which is (0, 10240) +CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(0, 12040)) + Engine=MergeTree + ORDER BY (k) + AS + SELECT + number, + format('{},{},{},{}', hex(12345678), hex(87654321), hex(number/17 + 5), hex(13579012)) as s + FROM numbers(1024); +SELECT count(s) FROM tab WHERE hasToken(s, '4C4B4B4B4B4B5040'); +DROP TABLE IF EXISTS tab; +-- create table 'tab' with inverted index parameter (ngrams, max_rows_per_most_list) which is (0, 123) +-- it should throw exception since max_rows_per_most_list(123) is less than its minimum value(8196) +CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(3, 123)) + Engine=MergeTree + ORDER BY (k) + AS + SELECT + number, + format('{},{},{},{}', hex(12345678), hex(87654321), hex(number/17 + 5), hex(13579012)) as s + FROM numbers(1024); -- { serverError 80 } From 5b4594acf60af32e8c2ed84e5f7002faf5c4412c Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 16 Oct 2023 17:26:10 +0800 Subject: [PATCH 348/634] Fix incorrect decimal sorting --- ...cimal_sort_with_multiple_columns.reference | 600 ++++++++++++++++++ ...900_decimal_sort_with_multiple_columns.sql | 1 + 2 files changed, 601 insertions(+) create mode 100644 tests/queries/0_stateless/02900_decimal_sort_with_multiple_columns.reference create mode 100644 tests/queries/0_stateless/02900_decimal_sort_with_multiple_columns.sql diff --git a/tests/queries/0_stateless/02900_decimal_sort_with_multiple_columns.reference b/tests/queries/0_stateless/02900_decimal_sort_with_multiple_columns.reference new file mode 100644 index 00000000000..f99785d2b99 --- /dev/null +++ b/tests/queries/0_stateless/02900_decimal_sort_with_multiple_columns.reference @@ -0,0 +1,600 @@ +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 1 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 5 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 7 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 9 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 11 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 13 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 15 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 17 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +-1 19 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 2 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 4 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 6 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 8 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 10 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 12 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 14 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 16 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 +0 18 diff --git a/tests/queries/0_stateless/02900_decimal_sort_with_multiple_columns.sql b/tests/queries/0_stateless/02900_decimal_sort_with_multiple_columns.sql new file mode 100644 index 00000000000..bc74add253b --- /dev/null +++ b/tests/queries/0_stateless/02900_decimal_sort_with_multiple_columns.sql @@ -0,0 +1 @@ +select -number % 2 as i, toDecimal32(number % 20, 3) as j from numbers(600) order by i, j; From 9207dae99ba1b0b927c349e8fd94b6f94e7fe1ee Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 17 Oct 2023 06:09:20 +0200 Subject: [PATCH 349/634] Check if URL path to be empty in case of GET or HEAD query to `query` endpoint --- src/Server/HTTPHandlerFactory.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 1416112b40d..4f0962530d0 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -180,8 +180,15 @@ void addDefaultHandlersFactory( return std::make_unique(server, "query"); }; auto query_handler = std::make_shared>(std::move(dynamic_creator)); - query_handler->allowPostAndGetParamsAndOptionsRequest(); - query_handler->attachNonStrictPath("/?"); + query_handler->addFilter([](const auto & request) + { + return (startsWith(request.getURI(), "/?") + && (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD)) + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST; + } + ); factory.addHandler(query_handler); /// We check that prometheus handler will be served on current (default) port. From fc9a5a725c6cd661ea51bd7b4a9aa257ec60d0c8 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 17 Oct 2023 08:18:17 +0000 Subject: [PATCH 350/634] Revert "Revert "Integration check script fix ups"" This reverts commit 67b9407530bb15f7e6d49cd1c2bde7b6c441389b. --- docs/en/development/continuous-integration.md | 58 +++++++++++++++++++ tests/README.md | 1 + tests/ci/build_download_helper.py | 42 +++++++++----- tests/ci/pr_info.py | 36 +++++------- tests/ci/worker/prepare-ci-ami.sh | 1 + 5 files changed, 102 insertions(+), 36 deletions(-) create mode 100644 tests/README.md diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index eec5ccbb9dc..46a30f56f11 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -67,6 +67,48 @@ This check means that the CI system started to process the pull request. When it Performs some simple regex-based checks of code style, using the [`utils/check-style/check-style`](https://github.com/ClickHouse/ClickHouse/blob/master/utils/check-style/check-style) binary (note that it can be run locally). If it fails, fix the style errors following the [code style guide](style.md). +#### Running style check locally: +```sh +mkdir -p /tmp/test_output +# running all checks +docker run --rm --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE clickhouse/style-test + +# run specified check script (e.g.: ./check-mypy) +docker run --rm --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE --entrypoint= -w/ClickHouse/utils/check-style clickhouse/style-test ./check-mypy + +# find all style check scripts under the directory: +cd ./utils/check-style + +# Check duplicate includes +./check-duplicate-includes.sh + +# Check c++ formatiing +./check-style + +# Check python formatting with black +./check-black + +# Check python type hinting with mypy +./check-mypy + +# Check code with codespell +./check-typos + +# Check docs spelling +./check-doc-aspell + +# Check whitespaces +./check-whitespaces + +# Check github actions workflows +./check-workflows + +# Check submodules +./check-submodules + +# Check shell scripts with shellcheck +./shellcheck-run.sh +``` ## Fast Test Normally this is the first check that is ran for a PR. It builds ClickHouse and @@ -75,6 +117,15 @@ some. If it fails, further checks are not started until it is fixed. Look at the report to see which tests fail, then reproduce the failure locally as described [here](tests.md#functional-test-locally). +#### Running Fast Test locally: +```sh +mkdir -p /tmp/test_output +mkdir -p /tmp/fasttest-workspace +cd ClickHouse +# this docker command performs minimal ClickHouse build and run FastTests against it +docker run --rm --cap-add=SYS_PTRACE -u $(id -u ${USER}):$(id -g ${USER}) --network=host -e FASTTEST_WORKSPACE=/fasttest-workspace -e FASTTEST_OUTPUT=/test_output -e FASTTEST_SOURCE=/ClickHouse --cap-add=SYS_PTRACE -e stage=clone_submodules --volume=/tmp/fasttest-workspace:/fasttest-workspace --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output clickhouse/fasttest +``` + #### Status Page Files - `runlog.out.log` is the general log that includes all other logs. @@ -122,6 +173,13 @@ Builds ClickHouse in various configurations for use in further steps. You have t ## Special Build Check Performs static analysis and code style checks using `clang-tidy`. The report is similar to the [build check](#build-check). Fix the errors found in the build log. +#### Running clang-tidy locally: +There is a convenience `packager` script that runs the clang-tidy build in docker +```sh +mkdir build_tidy +./docker/packager/packager --output-dir=./build_tidy --package-type=binary --compiler=clang-17 --debug-build --clang-tidy +``` + ## Functional Stateless Tests Runs [stateless functional tests](tests.md#functional-tests) for ClickHouse diff --git a/tests/README.md b/tests/README.md new file mode 100644 index 00000000000..a1fc0f530f2 --- /dev/null +++ b/tests/README.md @@ -0,0 +1 @@ +Find CI documents and instructions on running CI checks localy [here](https://clickhouse.com/docs/en/development/continuous-integration). \ No newline at end of file diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index b76c5433142..21012f6337d 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -51,9 +51,9 @@ def get_gh_api( sleep: int = 3, **kwargs: Any, ) -> requests.Response: - """It's a wrapper around get_with_retries that requests GH api w/o auth by - default, and falls back to the get_best_robot_token in case of receiving - "403 rate limit exceeded" error + """ + Request GH api w/o auth by default, and failover to the get_best_robot_token in case of receiving + "403 rate limit exceeded" or "404 not found" error It sets auth automatically when ROBOT_TOKEN is already set by get_best_robot_token """ @@ -71,27 +71,39 @@ def get_gh_api( if grt.ROBOT_TOKEN is not None: set_auth_header() - need_retry = False - for _ in range(retries): + token_is_set = "Authorization" in kwargs.get("headers", {}) + exc = Exception("A placeholder to satisfy typing and avoid nesting") + try_cnt = 0 + while try_cnt < retries: + try_cnt += 1 try: - response = get_with_retries(url, 1, sleep, **kwargs) + response = requests.get(url, **kwargs) response.raise_for_status() return response - except requests.HTTPError as exc: - if ( - exc.response.status_code == 403 + except requests.HTTPError as e: + exc = e + ratelimit_exceeded = ( + e.response.status_code == 403 and b"rate limit exceeded" - in exc.response._content # pylint:disable=protected-access - ): + in e.response._content # pylint:disable=protected-access + ) + try_auth = e.response.status_code == 404 + if (ratelimit_exceeded or try_auth) and not token_is_set: logging.warning( "Received rate limit exception, setting the auth header and retry" ) set_auth_header() - need_retry = True - break + token_is_set = True + try_cnt = 0 + continue + except Exception as e: + exc = e - if need_retry: - return get_with_retries(url, retries, sleep, **kwargs) + if try_cnt < retries: + logging.info("Exception '%s' while getting, retry %i", exc, try_cnt) + time.sleep(sleep) + + raise exc def get_build_name_for_check(check_name: str) -> str: diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 830aa936bea..7dbfe124760 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -4,8 +4,6 @@ import logging import os from typing import Dict, List, Set, Union, Literal -from unidiff import PatchSet # type: ignore - from build_download_helper import get_gh_api from env_helper import ( GITHUB_REPOSITORY, @@ -171,7 +169,11 @@ class PRInfo: response_json = user_orgs_response.json() self.user_orgs = set(org["id"] for org in response_json) - self.diff_urls.append(github_event["pull_request"]["diff_url"]) + self.diff_urls.append( + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/master...{self.head_ref}" + ) + elif "commits" in github_event: # `head_commit` always comes with `commits` commit_message = github_event["head_commit"]["message"] # type: str @@ -215,12 +217,12 @@ class PRInfo: # files changed in upstream AND master...{self.head_ref} # to get files, changed in current HEAD self.diff_urls.append( - f"https://github.com/{GITHUB_REPOSITORY}/" - f"compare/master...{self.head_ref}.diff" + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/master...{self.head_ref}" ) self.diff_urls.append( - f"https://github.com/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master.diff" + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/{self.head_ref}...master" ) # Get release PR number. self.release_pr = get_pr_for_commit(self.base_ref, self.base_ref)[ @@ -232,8 +234,8 @@ class PRInfo: # For release PRs we must get not only files changed in the PR # itself, but as well files changed since we branched out self.diff_urls.append( - f"https://github.com/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master.diff" + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/{self.head_ref}...master" ) else: print("event.json does not match pull_request or push:") @@ -261,19 +263,11 @@ class PRInfo: raise TypeError("The event does not have diff URLs") for diff_url in self.diff_urls: - response = get_gh_api( - diff_url, - sleep=RETRY_SLEEP, - ) + response = get_gh_api(diff_url, sleep=RETRY_SLEEP) response.raise_for_status() - if "commits" in self.event and self.number == 0: - diff = response.json() - - if "files" in diff: - self.changed_files = {f["filename"] for f in diff["files"]} - else: - diff_object = PatchSet(response.text) - self.changed_files.update({f.path for f in diff_object}) + diff = response.json() + if "files" in diff: + self.changed_files = {f["filename"] for f in diff["files"]} print(f"Fetched info about {len(self.changed_files)} changed files") def get_dict(self): diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index 20e7e3fd53e..f3e11b6d00a 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -90,6 +90,7 @@ systemctl restart docker sudo -u ubuntu docker buildx version sudo -u ubuntu docker buildx create --use --name default-builder +# FIXME: remove unidiff as soon as no old PRs could use it, here and in Dockerfile pip install boto3 pygithub requests urllib3 unidiff dohq-artifactory mkdir -p $RUNNER_HOME && cd $RUNNER_HOME From ad38c433a1390d4402d7b53577a9693070d2e194 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 17 Oct 2023 08:48:25 +0000 Subject: [PATCH 351/634] fix commit labels for diff --- tests/ci/pr_info.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 7dbfe124760..c02c71d8761 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -171,7 +171,7 @@ class PRInfo: self.diff_urls.append( f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/master...{self.head_ref}" + f"compare/master...{github_event['pull_request']['head']['label']}" ) elif "commits" in github_event: @@ -218,11 +218,11 @@ class PRInfo: # to get files, changed in current HEAD self.diff_urls.append( f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/master...{self.head_ref}" + f"compare/master...{pull_request['head']['label']}" ) self.diff_urls.append( f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master" + f"compare/{pull_request['head']['label']}...master" ) # Get release PR number. self.release_pr = get_pr_for_commit(self.base_ref, self.base_ref)[ @@ -235,7 +235,7 @@ class PRInfo: # itself, but as well files changed since we branched out self.diff_urls.append( f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master" + f"compare/{pull_request['head']['label']}...master" ) else: print("event.json does not match pull_request or push:") From f56804e6e72b256d847417aecb0b2085d9a46438 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Tue, 17 Oct 2023 12:13:59 +0300 Subject: [PATCH 352/634] Use pool for proxified S3 http sessions --- src/IO/HTTPCommon.cpp | 8 ++++++++ src/IO/ReadBufferFromS3.cpp | 13 +++++++++++-- src/IO/S3/PocoHTTPClient.cpp | 2 +- 3 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index ef03acdde5d..718ae53bb7a 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -324,6 +324,14 @@ namespace auto retry_timeout = timeouts.connection_timeout.totalMilliseconds(); auto session = pool_ptr->second->get(retry_timeout); + const auto & session_data = session->sessionData(); + if (session_data.empty() || !Poco::AnyCast(&session_data)) + { + /// Reset session if it is not reusable. See comment for HTTPSessionReuseTag. + session->reset(); + } + session->attachSessionData({}); + setTimeouts(*session, timeouts); return session; diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index f346b6fb3f9..f19978ccb47 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -74,8 +74,17 @@ void resetSessionIfNeeded(bool read_all_range_successfully, std::optionalgetProxyHost().empty()) + { + /// Reset proxified sessions because proxy can change for every request. See ProxyConfigurationResolver. + resetSession(*read_result); + ProfileEvents::increment(ProfileEvents::ReadBufferFromS3ResetSessions); + } + else + { + DB::markSessionForReuse(session); + ProfileEvents::increment(ProfileEvents::ReadBufferFromS3PreservedSessions); + } } } } diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index d7c5b4539ae..15a9ce637e7 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -276,7 +276,7 @@ void PocoHTTPClient::makeRequestInternal( { /// Most sessions in pool are already connected and it is not possible to set proxy host/port to a connected session. const auto request_configuration = per_request_configuration(); - if (http_connection_pool_size && request_configuration.host.empty()) + if (http_connection_pool_size) makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); else makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); From 8744cd9c0dde99484bfc0356c768d70c44db2bac Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 11:15:53 +0200 Subject: [PATCH 353/634] Fix --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 963a64e257e..7d498dcb287 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -474,7 +474,7 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolder if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); - zk_client->tryRemove(node_name + ".retriable", -1); + zk_client->tryRemove(zookeeper_failed_path / node_name + ".retriable", -1); return; } @@ -537,7 +537,7 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); - zk_client->tryRemove(node_name + ".retriable", -1); + zk_client->tryRemove(zookeeper_failed_path / node_name + ".retriable", -1); return; } From c220cc65d6145f15c5b47b2d04f1e351c7f93d58 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 17 Oct 2023 11:35:15 +0200 Subject: [PATCH 354/634] Update test.py --- .../test_postgresql_replica_database_engine_2/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index e72e5f179f5..2a1c7634698 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -762,6 +762,7 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) def test_replica_consumer(started_cluster): table = "test_replica_consumer" + pg_manager_instance2.restart() for pm in [pg_manager, pg_manager_instance2]: pm.create_and_fill_postgres_table(table) From 42ed24995473ea4932c48fd89dd160be0a004546 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 12:03:49 +0200 Subject: [PATCH 355/634] Fix build --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 4 ++-- src/Storages/StorageS3.cpp | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 7d498dcb287..c64330e9df1 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -474,7 +474,7 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolder if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); - zk_client->tryRemove(zookeeper_failed_path / node_name + ".retriable", -1); + zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); return; } @@ -537,7 +537,7 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); - zk_client->tryRemove(zookeeper_failed_path / node_name + ".retriable", -1); + zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); return; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2d3ff786abc..4f54a938ada 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -210,11 +210,11 @@ private: /// If url doesn't contain globs, we didn't list s3 bucket and didn't get object info for the key. /// So we get object info lazily here on 'next()' request. - if (!answer.info) + if (!answer->info) { - answer.info = S3::getObjectInfo(*client, globbed_uri.bucket, answer.key, globbed_uri.version_id, request_settings); + answer->info = S3::getObjectInfo(*client, globbed_uri.bucket, answer->key, globbed_uri.version_id, request_settings); if (file_progress_callback) - file_progress_callback(FileProgress(0, answer.info->size)); + file_progress_callback(FileProgress(0, answer->info->size)); } return answer; From f20d055e1ecd24019ee1ea6fece98a5d634a338d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 17 Oct 2023 10:06:57 +0000 Subject: [PATCH 356/634] do not re-create serialization for each row Signed-off-by: Duc Canh Le --- src/Storages/RocksDB/EmbeddedRocksDBSink.cpp | 3 ++- src/Storages/RocksDB/EmbeddedRocksDBSink.h | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp index 47d036c943d..b79c7284843 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp @@ -26,6 +26,7 @@ EmbeddedRocksDBSink::EmbeddedRocksDBSink( break; ++primary_key_pos; } + serializations = getHeader().getSerializations(); } void EmbeddedRocksDBSink::consume(Chunk chunk) @@ -46,7 +47,7 @@ void EmbeddedRocksDBSink::consume(Chunk chunk) size_t idx = 0; for (const auto & elem : block) { - elem.type->getDefaultSerialization()->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value, {}); + serializations[idx]->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value, {}); ++idx; } status = batch.Put(wb_key.str(), wb_value.str()); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.h b/src/Storages/RocksDB/EmbeddedRocksDBSink.h index e9e98c7df50..011322df829 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.h @@ -24,6 +24,7 @@ private: StorageEmbeddedRocksDB & storage; StorageMetadataPtr metadata_snapshot; size_t primary_key_pos = 0; + Serializations serializations; }; } From 9a8734fa5ef2158ec7fa7b5a7a87d3ea1a4beffa Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 17 Oct 2023 10:22:30 +0000 Subject: [PATCH 357/634] Remove submodules --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index e5276ec3693..cb6b724ac49 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit e5276ec369342e0069f71c3354df9eac69ae3f07 +Subproject commit cb6b724ac49fe94098e30e3e0d9fd6901007ff38 From 685415d75a0ad1246680f16e7e9222bd76b7cc12 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 17 Oct 2023 12:40:41 +0200 Subject: [PATCH 358/634] Check if URL path is empty in case of POST or OPTIONS query to `query` endpoint --- src/Server/HTTPHandlerFactory.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 4f0962530d0..2dff0a0fbde 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -183,10 +183,12 @@ void addDefaultHandlersFactory( query_handler->addFilter([](const auto & request) { return (startsWith(request.getURI(), "/?") - && (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD)) - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST; + && (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD)) + || ((startsWith(request.getURI(), "/?") + || request.getURI() == "/") + && (request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST)); } ); factory.addHandler(query_handler); From f87938735addf32377825248a02423bd790bcf0f Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 16 Oct 2023 19:58:03 +0000 Subject: [PATCH 359/634] Fixes and improvements for Iceberg storage --- src/CMakeLists.txt | 1 + src/IO/ReadBuffer.h | 2 +- src/Storages/DataLakes/IStorageDataLake.h | 9 +- .../DataLakes/Iceberg/IcebergMetadata.cpp | 580 ++++++++++++++++++ .../DataLakes/Iceberg/IcebergMetadata.h | 95 +++ .../DataLakes/Iceberg/StorageIceberg.cpp | 69 +++ .../DataLakes/Iceberg/StorageIceberg.h | 80 +++ .../DataLakes/IcebergMetadataParser.h | 26 - src/Storages/DataLakes/StorageIceberg.h | 25 - src/Storages/DataLakes/registerDataLakes.cpp | 4 +- src/TableFunctions/ITableFunctionDataLake.h | 2 +- src/TableFunctions/TableFunctionIceberg.cpp | 4 +- 12 files changed, 838 insertions(+), 59 deletions(-) create mode 100644 src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp create mode 100644 src/Storages/DataLakes/Iceberg/IcebergMetadata.h create mode 100644 src/Storages/DataLakes/Iceberg/StorageIceberg.cpp create mode 100644 src/Storages/DataLakes/Iceberg/StorageIceberg.h delete mode 100644 src/Storages/DataLakes/IcebergMetadataParser.h delete mode 100644 src/Storages/DataLakes/StorageIceberg.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d13c9cbe9bc..d2985665db3 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -104,6 +104,7 @@ if (TARGET ch_contrib::nats_io) endif() add_headers_and_sources(dbms Storages/DataLakes) +add_headers_and_sources(dbms Storages/DataLakes/Iceberg) add_headers_and_sources(dbms Common/NamedCollections) if (TARGET ch_contrib::amqp_cpp) diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index b45bc8f3dbc..253fb07c9ab 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -204,7 +204,7 @@ public: * By default - the same as read. * Don't use for small reads. */ - [[nodiscard]] virtual size_t readBig(char * to, size_t n) { return read(to, n); } + [[nodiscard]] virtual size_t readBig(char * to, size_t n) { return read(to, n); } /** Do something to allow faster subsequent call to 'nextImpl' if possible. * It's used for asynchronous readers with double-buffering. diff --git a/src/Storages/DataLakes/IStorageDataLake.h b/src/Storages/DataLakes/IStorageDataLake.h index 7c481d196d2..77a22cd00fc 100644 --- a/src/Storages/DataLakes/IStorageDataLake.h +++ b/src/Storages/DataLakes/IStorageDataLake.h @@ -27,6 +27,12 @@ public: , base_configuration(configuration_) , log(&Poco::Logger::get(getName())) {} // NOLINT(clang-analyzer-optin.cplusplus.VirtualCall) + template + static StoragePtr create(const Configuration & configuration_, ContextPtr context_, Args && ...args) + { + return std::make_shared>(configuration_, context_, std::forward(args)...); + } + String getName() const override { return name; } static ColumnsDescription getTableStructureFromData( @@ -109,8 +115,7 @@ static StoragePtr createDataLakeStorage(const StorageFactory::Arguments & args) if (configuration.format == "auto") configuration.format = "Parquet"; - return std::make_shared( - configuration, args.getContext(), args.table_id, args.columns, args.constraints, + return DataLake::create(configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, getFormatSettings(args.getContext())); } diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp new file mode 100644 index 00000000000..c0bda44d28c --- /dev/null +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp @@ -0,0 +1,580 @@ +#include "config.h" + +#if USE_AWS_S3 && USE_AVRO + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FILE_DOESNT_EXIST; + extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; + extern const int UNSUPPORTED_METHOD; +} + +IcebergMetadata::IcebergMetadata( + const StorageS3::Configuration & configuration_, + DB::ContextPtr context_, + DB::Int32 metadata_version_, + DB::Int32 format_version_, + DB::String manifest_list_file_, + DB::Int32 current_schema_id_, + DB::NamesAndTypesList schema_) + : WithContext(context_) + , configuration(configuration_) + , metadata_version(metadata_version_) + , format_version(format_version_) + , manifest_list_file(std::move(manifest_list_file_)) + , current_schema_id(current_schema_id_) + , schema(std::move(schema_)) + , log(&Poco::Logger::get("IcebergMetadata")) +{ +} + +namespace +{ + +enum class ManifestEntryStatus +{ + EXISTING = 0, + ADDED = 1, + DELETED = 2, +}; + +enum class DataFileContent +{ + DATA = 0, + POSITION_DELETES = 1, + EQUALITY_DELETES = 2, +}; + +/** + * Iceberg supports the next data types (see https://iceberg.apache.org/spec/#schemas-and-data-types): + * - Primitive types: + * - boolean + * - int + * - long + * - float + * - double + * - decimal(P, S) + * - date + * - time (time of day in microseconds since midnight) + * - timestamp (in microseconds since 1970-01-01) + * - timestamptz (timestamp with timezone, stores values in UTC timezone) + * - string + * - uuid + * - fixed(L) (fixed-length byte array of length L) + * - binary + * - Complex types: + * - struct(field1: Type1, field2: Type2, ...) (tuple of typed values) + * - list(nested_type) + * - map(Key, Value) + * + * Example of table schema in metadata: + * { + * "type" : "struct", + * "schema-id" : 0, + * "fields" : [ + * { + * "id" : 1, + * "name" : "id", + * "required" : false, + * "type" : "long" + * }, + * { + * "id" : 2, + * "name" : "array", + * "required" : false, + * "type" : { + * "type" : "list", + * "element-id" : 5, + * "element" : "int", + * "element-required" : false + * }, + * { + * "id" : 3, + * "name" : "data", + * "required" : false, + * "type" : "binary" + * } + * } + */ + +DataTypePtr getSimpleTypeByName(const String & type_name) +{ + if (type_name == "boolean") + return DataTypeFactory::instance().get("Bool"); + if (type_name == "int") + return std::make_shared(); + if (type_name == "long") + return std::make_shared(); + if (type_name == "float") + return std::make_shared(); + if (type_name == "double") + return std::make_shared(); + if (type_name == "date") + return std::make_shared(); + /// Time type represents time of the day in microseconds since midnight. + /// We don't have similar type for it, let's use just Int64. + if (type_name == "time") + return std::make_shared(); + if (type_name == "timestamp") + return std::make_shared(6); + if (type_name == "timestamptz") + return std::make_shared(6, "UTC"); + if (type_name == "string" || type_name == "binary") + return std::make_shared(); + if (type_name == "uuid") + return std::make_shared(); + + if (type_name.starts_with("fixed[") && type_name.ends_with(']')) + { + ReadBufferFromString buf(std::string_view(type_name.begin() + 6, type_name.end() - 1)); + size_t n; + readIntText(n, buf); + return std::make_shared(n); + } + + if (type_name.starts_with("decimal(") && type_name.ends_with(')')) + { + ReadBufferFromString buf(std::string_view(type_name.begin() + 8, type_name.end() - 1)); + size_t precision; + size_t scale; + readIntText(precision, buf); + skipWhitespaceIfAny(buf); + assertChar(',', buf); + skipWhitespaceIfAny(buf); + tryReadIntText(scale, buf); + return createDecimal(precision, scale); + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown Iceberg type: {}", type_name); +} + +DataTypePtr getFieldType(const Poco::Dynamic::Var & type, bool required); + +DataTypePtr getComplexTypeFromObject(const Poco::JSON::Object::Ptr & type) +{ + String type_name = type->getValue("type"); + if (type_name == "list") + { + bool element_required = type->getValue("element-required"); + auto element_type = getFieldType(type->get("element"), element_required); + return std::make_shared(element_type); + } + + if (type_name == "map") + { + auto key_type = getFieldType(type->get("key"), true); + auto value_required = type->getValue("value-required"); + auto value_type = getFieldType(type->get("value"), value_required); + return std::make_shared(key_type, value_type); + } + + if (type_name == "struct") + { + DataTypes element_types; + Names element_names; + auto fields = type->get("fields").extract(); + element_types.reserve(fields->size()); + element_names.reserve(fields->size()); + for (size_t i = 0; i != fields->size(); ++i) + { + auto field = fields->getObject(static_cast(i)); + element_names.push_back(field->getValue("name")); + auto required = field->getValue("required"); + element_types.push_back(getFieldType(field->get("type"), required)); + } + + return std::make_shared(element_types, element_names); + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown Iceberg type: {}", type_name); +} + +DataTypePtr getFieldType(const Poco::Dynamic::Var & type, bool required) +{ + if (type.isString()) + { + const String & type_name = type.extract(); + auto data_type = getSimpleTypeByName(type_name); + return required ? data_type : makeNullable(data_type); + } + + if (type.isStruct()) + return getComplexTypeFromObject(type.extract()); + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected 'type' field: {}", type.toString()); + +} + +std::pair parseTableSchema(const Poco::JSON::Object::Ptr & metadata_object, int format_version) +{ + Poco::JSON::Object::Ptr schema; + Int32 current_schema_id; + + /// First check if schema was evolved, because we don't support it yet. + /// For version 2 we can check it by using field schemas, but in version 1 + /// this field is optional and we will check it later during parsing manifest files + /// (we will compare schema id from manifest file and currently used schema). + if (format_version == 2) + { + current_schema_id = metadata_object->getValue("current-schema-id"); + auto schemas = metadata_object->get("schemas").extract(); + if (schemas->size() > 1) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); + for (size_t i = 0; i != schemas->size(); ++i) + { + auto current_schema = schemas->getObject(static_cast(i)); + if (current_schema->getValue("schema-id") == current_schema_id) + { + schema = current_schema; + break; + } + } + } + else + { + schema = metadata_object->getObject("schema"); + current_schema_id = schema->getValue("schema-id"); + /// Field "schemas" is optional for version 1, but after version 2 was introduced, + /// in most cases this field is added for new tables in version 1 as well. + if (metadata_object->has("schemas") && metadata_object->get("schemas").extract()->size() > 1) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); + } + + NamesAndTypesList names_and_types; + auto fields = schema->get("fields").extract(); + for (size_t i = 0; i != fields->size(); ++i) + { + auto field = fields->getObject(static_cast(i)); + auto name = field->getValue("name"); + bool required = field->getValue("required"); + names_and_types.push_back({name, getFieldType(field->get("type"), required)}); + } + + return {std::move(names_and_types), current_schema_id}; +} + +MutableColumns parseAvro( + avro::DataFileReaderBase & file_reader, + const Block & header, + const FormatSettings & settings) +{ + auto deserializer = std::make_unique(header, file_reader.dataSchema(), true, true, settings); + MutableColumns columns = header.cloneEmptyColumns(); + + file_reader.init(); + RowReadExtension ext; + while (file_reader.hasMore()) + { + file_reader.decr(); + deserializer->deserializeRow(columns, file_reader.decoder(), ext); + } + return columns; +} + +/** + * Each version of table metadata is stored in a `metadata` directory and + * has format: v.metadata.json, where V - metadata version. + */ +std::pair getMetadataFileAndVersion(const StorageS3::Configuration & configuration) +{ + const auto metadata_files = S3DataLakeMetadataReadHelper::listFiles(configuration, "metadata", ".metadata.json"); + if (metadata_files.empty()) + { + throw Exception( + ErrorCodes::FILE_DOESNT_EXIST, + "The metadata file for Iceberg table with path {} doesn't exist", + configuration.url.key); + } + + std::vector> metadata_files_with_versions; + metadata_files_with_versions.reserve(metadata_files.size()); + for (const auto & file : metadata_files) + { + String version_str(file.begin() + 1, file.begin() + file.find_first_of('.')); + if (!std::all_of(version_str.begin(), version_str.end(), isdigit)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad metadata file name: {}. Expected vN.metadata.json where N is a number", file); + metadata_files_with_versions.emplace_back(std::stoi(version_str), file); + } + + /// Get the latest version of metadata file: v.metadata.json + return *std::max_element(metadata_files_with_versions.begin(), metadata_files_with_versions.end()); +} + +} + +std::unique_ptr parseIcebergMetadata(const StorageS3::Configuration & configuration, ContextPtr context_) +{ + const auto [metadata_version, metadata_file_path] = getMetadataFileAndVersion(configuration); + LOG_DEBUG(&Poco::Logger::get("IcebergMetadata"), "Parse metadata {}", metadata_file_path); + auto buf = S3DataLakeMetadataReadHelper::createReadBuffer(metadata_file_path, context_, configuration); + String json_str; + readJSONObjectPossiblyInvalid(json_str, *buf); + + Poco::JSON::Parser parser; /// For some reason base/base/JSON.h can not parse this json file + Poco::Dynamic::Var json = parser.parse(json_str); + Poco::JSON::Object::Ptr object = json.extract(); + + auto format_version = object->getValue("format-version"); + auto [schema, schema_id] = parseTableSchema(object, format_version); + + auto current_snapshot_id = object->getValue("current-snapshot-id"); + auto snapshots = object->get("snapshots").extract(); + + String manifest_list_file; + for (size_t i = 0; i < snapshots->size(); ++i) + { + const auto snapshot = snapshots->getObject(static_cast(i)); + if (snapshot->getValue("snapshot-id") == current_snapshot_id) + { + const auto path = snapshot->getValue("manifest-list"); + manifest_list_file = std::filesystem::path(configuration.url.key) / "metadata" / std::filesystem::path(path).filename(); + break; + } + } + + return std::make_unique(configuration, context_, metadata_version, format_version, manifest_list_file, schema_id, schema); +} + +/** + * Manifest file has the following format: '/iceberg_data/db/table_name/metadata/c87bfec7-d36c-4075-ad04-600b6b0f2020-m0.avro' + * + * `manifest file` is different in format version V1 and V2 and has the following contents: + * v1 v2 + * status req req + * snapshot_id req opt + * sequence_number opt + * file_sequence_number opt + * data_file req req + * Example format version V1: + * ┌─status─┬─────────snapshot_id─┬─data_file───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ + * │ 1 │ 2819310504515118887 │ ('/iceberg_data/db/table_name/data/00000-1-3edca534-15a0-4f74-8a28-4733e0bf1270-00001.parquet','PARQUET',(),100,1070,67108864,[(1,233),(2,210)],[(1,100),(2,100)],[(1,0),(2,0)],[],[(1,'\0'),(2,'0')],[(1,'c'),(2,'99')],NULL,[4],0) │ + * └────────┴─────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + * Example format version V2: + * ┌─status─┬─────────snapshot_id─┬─sequence_number─┬─file_sequence_number─┬─data_file───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ + * │ 1 │ 5887006101709926452 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ (0,'/iceberg_data/db/table_name/data/00000-1-c8045c90-8799-4eac-b957-79a0484e223c-00001.parquet','PARQUET',(),100,1070,[(1,233),(2,210)],[(1,100),(2,100)],[(1,0),(2,0)],[],[(1,'\0'),(2,'0')],[(1,'c'),(2,'99')],NULL,[4],[],0) │ + * └────────┴─────────────────────┴─────────────────┴──────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + * In case of partitioned data we'll have extra directory partition=value: + * ─status─┬─────────snapshot_id─┬─data_file──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ + * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=0/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00001.parquet','PARQUET',(0),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0\0'),(2,'1')],[(1,'\0\0\0\0\0\0\0\0'),(2,'1')],NULL,[4],0) │ + * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=1/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00002.parquet','PARQUET',(1),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0'),(2,'2')],[(1,'\0\0\0\0\0\0\0'),(2,'2')],NULL,[4],0) │ + * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=2/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00003.parquet','PARQUET',(2),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0'),(2,'3')],[(1,'\0\0\0\0\0\0\0'),(2,'3')],NULL,[4],0) │ + * └────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + */ +Strings IcebergMetadata::getDataFiles() +{ + if (!data_files.empty()) + return data_files; + + Strings manifest_files; + if (manifest_list_file.empty()) + return data_files; + + LOG_TEST(log, "Collect manifest files from manifest list {}", manifest_list_file); + + auto manifest_list_buf = S3DataLakeMetadataReadHelper::createReadBuffer(manifest_list_file, getContext(), configuration); + auto manifest_list_file_reader = std::make_unique(std::make_unique(*manifest_list_buf)); + + auto data_type = AvroSchemaReader::avroNodeToDataType(manifest_list_file_reader->dataSchema().root()->leafAt(0)); + Block header{{data_type->createColumn(), data_type, "manifest_path"}}; + auto columns = parseAvro(*manifest_list_file_reader, header, getFormatSettings(getContext())); + auto & col = columns.at(0); + + if (col->getDataType() != TypeIndex::String) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `manifest_path` field should be String type, got {}", + col->getFamilyName()); + } + + const auto * col_str = typeid_cast(col.get()); + for (size_t i = 0; i < col_str->size(); ++i) + { + const auto file_path = col_str->getDataAt(i).toView(); + const auto filename = std::filesystem::path(file_path).filename(); + manifest_files.emplace_back(std::filesystem::path(configuration.url.key) / "metadata_path" / filename); + } + + NameSet files; + LOG_TEST(log, "Collect data files"); + for (const auto & manifest_file : manifest_files) + { + LOG_TEST(log, "Process manifest file {}", manifest_file); + + auto buffer = S3DataLakeMetadataReadHelper::createReadBuffer(manifest_file, getContext(), configuration); + auto manifest_file_reader = std::make_unique(std::make_unique(*buffer)); + + /// Manifest file should always have table schema in avro file metadata. By now we don't support tables with evolved schema, + /// so we should check if all manifest files have the same schema as in table metadata. + auto avro_metadata = manifest_file_reader->metadata(); + std::vector schema_json = avro_metadata["schema"]; + String schema_json_string = String(reinterpret_cast(schema_json.data()), schema_json.size()); + Poco::JSON::Parser parser; + Poco::Dynamic::Var json = parser.parse(schema_json_string); + Poco::JSON::Object::Ptr schema_object = json.extract(); + if (schema_object->getValue("schema-id") != current_schema_id) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); + + avro::NodePtr root_node = manifest_file_reader->dataSchema().root(); + size_t leaves_num = root_node->leaves(); + size_t expected_min_num = format_version == 1 ? 3 : 2; + if (leaves_num < expected_min_num) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Unexpected number of columns {}. Expected at least {}", + root_node->leaves(), expected_min_num); + } + + avro::NodePtr status_node = root_node->leafAt(0); + if (status_node->type() != avro::Type::AVRO_INT) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `status` field should be Int type, got {}", + magic_enum::enum_name(status_node->type())); + } + + avro::NodePtr data_file_node = root_node->leafAt(static_cast(leaves_num) - 1); + if (data_file_node->type() != avro::Type::AVRO_RECORD) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `data_file` field should be Tuple type, got {}", + magic_enum::enum_name(data_file_node->type())); + } + + auto status_col_data_type = AvroSchemaReader::avroNodeToDataType(status_node); + auto data_col_data_type = AvroSchemaReader::avroNodeToDataType(data_file_node); + Block manifest_file_header{ + {status_col_data_type->createColumn(), status_col_data_type, "status"}, + {data_col_data_type->createColumn(), data_col_data_type, "data_file"}}; + + columns = parseAvro(*manifest_file_reader, manifest_file_header, getFormatSettings(getContext())); + if (columns.size() != 2) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Unexpected number of columns. Expected 2, got {}", columns.size()); + + if (columns.at(0)->getDataType() != TypeIndex::Int32) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `status` field should be Int32 type, got {}", + columns.at(0)->getFamilyName()); + } + if (columns.at(1)->getDataType() != TypeIndex::Tuple) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `file_path` field should be Tuple type, got {}", + columns.at(1)->getFamilyName()); + } + + const auto * status_int_column = assert_cast(columns.at(0).get()); + const auto & data_file_tuple_type = assert_cast(*data_type.get()); + const auto * data_file_tuple_column = assert_cast(columns.at(1).get()); + + if (status_int_column->size() != data_file_tuple_column->size()) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `file_path` and `status` have different rows number: {} and {}", + status_int_column->size(), + data_file_tuple_column->size()); + } + + ColumnPtr file_path_column = data_file_tuple_column->getColumnPtr(data_file_tuple_type.getPositionByName("file_path")); + + if (file_path_column->getDataType() != TypeIndex::String) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `file_path` field should be String type, got {}", + file_path_column->getFamilyName()); + } + + const auto * file_path_string_column = assert_cast(file_path_column.get()); + + ColumnPtr content_column; + const ColumnInt32 * content_int_column = nullptr; + if (format_version == 2) + { + content_column = data_file_tuple_column->getColumnPtr(data_file_tuple_type.getPositionByName("content")); + if (content_column->getDataType() != TypeIndex::Int32) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "The parsed column from Avro file of `content` field should be Int type, got {}", + content_column->getFamilyName()); + } + + content_int_column = assert_cast(content_column.get()); + } + + for (size_t i = 0; i < data_file_tuple_column->size(); ++i) + { + if (format_version == 2) + { + Int32 content_type = content_int_column->getElement(i); + if (DataFileContent(content_type) != DataFileContent::DATA) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: positional and equality deletes are not supported"); + } + + const auto status = status_int_column->getInt(i); + const auto data_path = std::string(file_path_string_column->getDataAt(i).toView()); + const auto pos = data_path.find(configuration.url.key); + const auto file_path = data_path.substr(pos); + if (pos == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected to find {} in data path: {}", configuration.url.key, data_path); + + if (ManifestEntryStatus(status) == ManifestEntryStatus::DELETED) + { + LOG_TEST(log, "Processing delete file for path: {}", file_path); + chassert(!files.contains(file_path)); + } + else + { + files.insert(file_path); + } + } + } + + return data_files; +} + +} + +#endif diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/DataLakes/Iceberg/IcebergMetadata.h new file mode 100644 index 00000000000..d42ad84f472 --- /dev/null +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.h @@ -0,0 +1,95 @@ +#pragma once + +#if USE_AWS_S3 && USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. + +#include +#include +#include + +namespace DB +{ + +/** + * Useful links: + * - https://iceberg.apache.org/spec/ + * + * Iceberg has two format versions, v1 and v2. The content of metadata files depends on the version. + * + * Unlike DeltaLake, Iceberg has several metadata layers: `table metadata`, `manifest list` and `manifest_files`. + * Metadata file - json file. + * Manifest list – an Avro file that lists manifest files; one per snapshot. + * Manifest file – an Avro file that lists data or delete files; a subset of a snapshot. + * All changes to table state create a new metadata file and replace the old metadata with an atomic swap. + * + * In order to find out which data files to read, we need to find the `manifest list` + * which corresponds to the latest snapshot. We find it by checking a list of snapshots + * in metadata's "snapshots" section. + * + * Example of metadata.json file. + * { + * "format-version" : 1, + * "table-uuid" : "ca2965ad-aae2-4813-8cf7-2c394e0c10f5", + * "location" : "/iceberg_data/db/table_name", + * "last-updated-ms" : 1680206743150, + * "last-column-id" : 2, + * "schema" : { "type" : "struct", "schema-id" : 0, "fields" : [ {}, {}, ... ] }, + * "current-schema-id" : 0, + * "schemas" : [ ], + * ... + * "current-snapshot-id" : 2819310504515118887, + * "refs" : { "main" : { "snapshot-id" : 2819310504515118887, "type" : "branch" } }, + * "snapshots" : [ { + * "snapshot-id" : 2819310504515118887, + * "timestamp-ms" : 1680206743150, + * "summary" : { + * "operation" : "append", "spark.app.id" : "local-1680206733239", + * "added-data-files" : "1", "added-records" : "100", + * "added-files-size" : "1070", "changed-partition-count" : "1", + * "total-records" : "100", "total-files-size" : "1070", "total-data-files" : "1", "total-delete-files" : "0", + * "total-position-deletes" : "0", "total-equality-deletes" : "0" + * }, + * "manifest-list" : "/iceberg_data/db/table_name/metadata/snap-2819310504515118887-1-c87bfec7-d36c-4075-ad04-600b6b0f2020.avro", + * "schema-id" : 0 + * } ], + * "statistics" : [ ], + * "snapshot-log" : [ ... ], + * "metadata-log" : [ ] + * } + */ +class IcebergMetadata : WithContext +{ +public: + IcebergMetadata(const StorageS3::Configuration & configuration_, + ContextPtr context_, + Int32 metadata_version_, + Int32 format_version_, + String manifest_list_file_, + Int32 current_schema_id_, + NamesAndTypesList schema_); + + /// Get data files. On first request it reads manifest_list file and iterates through manifest files to find all data files. + /// All subsequent calls will return saved list of files (because it cannot be changed without changing metadata file) + Strings getDataFiles(); + + /// Get table schema parsed from metadata. + NamesAndTypesList getTableSchema() const { return schema; } + + size_t getVersion() const { return metadata_version; } + +private: + const StorageS3::Configuration configuration; + Int32 metadata_version; + Int32 format_version; + String manifest_list_file; + Int32 current_schema_id; + NamesAndTypesList schema; + Strings data_files; + Poco::Logger * log; + +}; + +std::unique_ptr parseIcebergMetadata(const StorageS3::Configuration & configuration, ContextPtr context); + +} + +#endif diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp new file mode 100644 index 00000000000..f014cb246eb --- /dev/null +++ b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp @@ -0,0 +1,69 @@ +#include + +#if USE_AWS_S3 && USE_AVRO + +namespace DB +{ + +StoragePtr StorageIceberg::create( + const DB::StorageIceberg::Configuration & base_configuration, + DB::ContextPtr context_, + const DB::StorageID & table_id_, + const DB::ColumnsDescription & columns_, + const DB::ConstraintsDescription & constraints_, + const DB::String & comment, + std::optional format_settings_) +{ + auto configuration{base_configuration}; + configuration.update(context_); + auto metadata = parseIcebergMetadata(configuration, context_); + auto schema_from_metadata = metadata->getTableSchema(); + return std::make_shared(std::move(metadata), configuration, context_, table_id_, columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_, constraints_, comment, format_settings_); +} + +StorageIceberg::StorageIceberg( + std::unique_ptr metadata_, + const Configuration & configuration_, + ContextPtr context_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + std::optional format_settings_) + : StorageS3(configuration_, context_, table_id_, columns_, constraints_, comment, format_settings_) + , current_metadata(std::move(metadata_)) + , base_configuration(configuration_) +{ +} + +ColumnsDescription StorageIceberg::getTableStructureFromMetadata( + Configuration & base_configuration, + const std::optional &, + ContextPtr local_context) +{ + auto configuration{base_configuration}; + configuration.update(local_context); + auto metadata = parseIcebergMetadata(configuration, local_context); + return ColumnsDescription(metadata->getTableSchema()); +} + +void StorageIceberg::updateConfigurationImpl(ContextPtr local_context) +{ + const bool updated = base_configuration.update(local_context); + auto new_metadata = parseIcebergMetadata(base_configuration, local_context); + /// Check if nothing was changed. + if (!updated && new_metadata->getVersion() == current_metadata->getVersion()) + return; + + if (new_metadata->getVersion() != current_metadata->getVersion()) + current_metadata = std::move(new_metadata); + + auto updated_configuration{base_configuration}; + /// If metadata wasn't changed, we won't list data files again. + updated_configuration.keys = current_metadata->getDataFiles(); + StorageS3::useConfiguration(updated_configuration); +} + +} + +#endif diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.h b/src/Storages/DataLakes/Iceberg/StorageIceberg.h new file mode 100644 index 00000000000..487ef3bad23 --- /dev/null +++ b/src/Storages/DataLakes/Iceberg/StorageIceberg.h @@ -0,0 +1,80 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 && USE_AVRO + +# include +# include +# include +# include +# include +# include +# include + + +namespace DB +{ + + +class StorageIceberg : public StorageS3 +{ +public: + static constexpr auto name = "Iceberg"; + + using Configuration = StorageS3::Configuration; + + static StoragePtr create(const Configuration & base_configuration, + ContextPtr context_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + std::optional format_settings_); + + StorageIceberg( + std::unique_ptr metadata_, + const Configuration & configuration_, + ContextPtr context_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + std::optional format_settings_); + + String getName() const override { return name; } + + static ColumnsDescription getTableStructureFromMetadata( + Configuration & base_configuration, + const std::optional &, + ContextPtr local_context); + + static Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context) + { + return StorageS3::getConfiguration(engine_args, local_context, /* get_format_from_file */false); + } + + Configuration updateConfigurationAndGetCopy(ContextPtr local_context) override + { + std::lock_guard lock(configuration_update_mutex); + updateConfigurationImpl(local_context); + return StorageS3::getConfiguration(); + } + + void updateConfiguration(ContextPtr local_context) override + { + std::lock_guard lock(configuration_update_mutex); + updateConfigurationImpl(local_context); + } + +private: + void updateConfigurationImpl(ContextPtr local_context); + + std::unique_ptr current_metadata; + Configuration base_configuration; + std::mutex configuration_update_mutex; +}; + +} + +#endif diff --git a/src/Storages/DataLakes/IcebergMetadataParser.h b/src/Storages/DataLakes/IcebergMetadataParser.h deleted file mode 100644 index 226b1bd8b6c..00000000000 --- a/src/Storages/DataLakes/IcebergMetadataParser.h +++ /dev/null @@ -1,26 +0,0 @@ -#pragma once - -#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. - -#include -#include - -namespace DB -{ - -template -struct IcebergMetadataParser -{ -public: - IcebergMetadataParser(); - - Strings getFiles(const Configuration & configuration, ContextPtr context); - -private: - struct Impl; - std::shared_ptr impl; -}; - -} - -#endif diff --git a/src/Storages/DataLakes/StorageIceberg.h b/src/Storages/DataLakes/StorageIceberg.h deleted file mode 100644 index f1c9c485ef0..00000000000 --- a/src/Storages/DataLakes/StorageIceberg.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include -#include -#include -#include "config.h" - -#if USE_AWS_S3 && USE_AVRO -#include -#include -#endif - -namespace DB -{ - -struct StorageIcebergName -{ - static constexpr auto name = "Iceberg"; -}; - -#if USE_AWS_S3 && USE_AVRO -using StorageIcebergS3 = IStorageDataLake>; -#endif - -} diff --git a/src/Storages/DataLakes/registerDataLakes.cpp b/src/Storages/DataLakes/registerDataLakes.cpp index 1447a4777c5..118600f7212 100644 --- a/src/Storages/DataLakes/registerDataLakes.cpp +++ b/src/Storages/DataLakes/registerDataLakes.cpp @@ -4,7 +4,7 @@ #if USE_AWS_S3 #include -#include +#include #include @@ -35,7 +35,7 @@ void registerStorageDeltaLake(StorageFactory & factory) void registerStorageIceberg(StorageFactory & factory) { - REGISTER_DATA_LAKE_STORAGE(StorageIcebergS3, StorageIcebergName::name) + REGISTER_DATA_LAKE_STORAGE(StorageIceberg, StorageIceberg::name) } #endif diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index 6d50e9138ff..d8524963776 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -33,7 +33,7 @@ protected: if (TableFunction::configuration.structure != "auto") columns = parseColumnsListFromString(TableFunction::configuration.structure, context); - StoragePtr storage = std::make_shared( + StoragePtr storage = Storage::create( TableFunction::configuration, context, StorageID(TableFunction::getDatabaseName(), table_name), columns, ConstraintsDescription{}, String{}, std::nullopt); diff --git a/src/TableFunctions/TableFunctionIceberg.cpp b/src/TableFunctions/TableFunctionIceberg.cpp index 5407d647f0f..d37aace01c6 100644 --- a/src/TableFunctions/TableFunctionIceberg.cpp +++ b/src/TableFunctions/TableFunctionIceberg.cpp @@ -2,7 +2,7 @@ #if USE_AWS_S3 && USE_AVRO -#include +#include #include #include #include @@ -17,7 +17,7 @@ struct TableFunctionIcebergName static constexpr auto name = "iceberg"; }; -using TableFunctionIceberg = ITableFunctionDataLake; +using TableFunctionIceberg = ITableFunctionDataLake; void registerTableFunctionIceberg(TableFunctionFactory & factory) { From bb46aa4d576ef3ce29888ad981e77787617617fa Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 16 Oct 2023 20:26:43 +0000 Subject: [PATCH 360/634] Clean uo --- src/IO/ReadBuffer.h | 2 +- src/Storages/DataLakes/Iceberg/StorageIceberg.h | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 253fb07c9ab..b45bc8f3dbc 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -204,7 +204,7 @@ public: * By default - the same as read. * Don't use for small reads. */ - [[nodiscard]] virtual size_t readBig(char * to, size_t n) { return read(to, n); } + [[nodiscard]] virtual size_t readBig(char * to, size_t n) { return read(to, n); } /** Do something to allow faster subsequent call to 'nextImpl' if possible. * It's used for asynchronous readers with double-buffering. diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.h b/src/Storages/DataLakes/Iceberg/StorageIceberg.h index 487ef3bad23..e53bd241d7f 100644 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.h +++ b/src/Storages/DataLakes/Iceberg/StorageIceberg.h @@ -16,7 +16,11 @@ namespace DB { - +/// Storage for read-only integration with Apache Iceberg tables in Amazon S3 (see https://iceberg.apache.org/) +/// Right now it's implemented on top of StorageS3 and right now it doesn't support +/// many Iceberg features like schema evolution, partitioning, positional and equality deletes. +/// TODO: Implement Iceberg as a separate storage using IObjectStorage +/// (to support all object storages, not only S3) and add support for missing Iceberg features. class StorageIceberg : public StorageS3 { public: From 4892618aa9b922f5191796849851f001a84a1f6e Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 17 Oct 2023 11:42:19 +0000 Subject: [PATCH 361/634] Update with master, fix comments --- contrib/avro | 2 +- .../DataLakes/Iceberg/IcebergMetadata.cpp | 16 +- .../DataLakes/IcebergMetadataParser.cpp | 361 ------------------ 3 files changed, 7 insertions(+), 372 deletions(-) delete mode 100644 src/Storages/DataLakes/IcebergMetadataParser.cpp diff --git a/contrib/avro b/contrib/avro index 7832659ec98..2fb8a8a6ec0 160000 --- a/contrib/avro +++ b/contrib/avro @@ -1 +1 @@ -Subproject commit 7832659ec986075d560f930c288e973c64679552 +Subproject commit 2fb8a8a6ec0eab9109b68abf3b4857e8c476b918 diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp index c0bda44d28c..d10b3c2ceba 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp @@ -252,17 +252,13 @@ std::pair parseTableSchema(const Poco::JSON::Object::P { current_schema_id = metadata_object->getValue("current-schema-id"); auto schemas = metadata_object->get("schemas").extract(); - if (schemas->size() > 1) + if (schemas->size() != 1) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); - for (size_t i = 0; i != schemas->size(); ++i) - { - auto current_schema = schemas->getObject(static_cast(i)); - if (current_schema->getValue("schema-id") == current_schema_id) - { - schema = current_schema; - break; - } - } + + /// Now we sure that there is only one schema. + schema = schemas->getObject(0); + if (schema->getValue("schema-id") != current_schema_id) + throw Exception(ErrorCodes::BAD_ARGUMENTS, R"(Field "schema-id" of the schema doesn't match "current-schema-id" in metadata)"); } else { diff --git a/src/Storages/DataLakes/IcebergMetadataParser.cpp b/src/Storages/DataLakes/IcebergMetadataParser.cpp deleted file mode 100644 index c4194cd7814..00000000000 --- a/src/Storages/DataLakes/IcebergMetadataParser.cpp +++ /dev/null @@ -1,361 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 && USE_AVRO - -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int FILE_DOESNT_EXIST; - extern const int ILLEGAL_COLUMN; - extern const int BAD_ARGUMENTS; -} - -template -struct IcebergMetadataParser::Impl -{ - Poco::Logger * log = &Poco::Logger::get("IcebergMetadataParser"); - - /** - * Useful links: - * - https://iceberg.apache.org/spec/ - */ - - /** - * Iceberg has two format versions, currently we support only format V1. - * - * Unlike DeltaLake, Iceberg has several metadata layers: `table metadata`, `manifest list` and `manifest_files`. - * Metadata file - json file. - * Manifest list – a file that lists manifest files; one per snapshot. - * Manifest file – a file that lists data or delete files; a subset of a snapshot. - * All changes to table state create a new metadata file and replace the old metadata with an atomic swap. - */ - - static constexpr auto metadata_directory = "metadata"; - - /** - * Each version of table metadata is stored in a `metadata` directory and - * has format: v.metadata.json, where V - metadata version. - */ - String getMetadataFile(const Configuration & configuration) - { - static constexpr auto metadata_file_suffix = ".metadata.json"; - - const auto metadata_files = MetadataReadHelper::listFiles(configuration, metadata_directory, metadata_file_suffix); - if (metadata_files.empty()) - { - throw Exception( - ErrorCodes::FILE_DOESNT_EXIST, - "The metadata file for Iceberg table with path {} doesn't exist", - configuration.url.key); - } - - /// Get the latest version of metadata file: v.metadata.json - return *std::max_element(metadata_files.begin(), metadata_files.end()); - } - - /** - * In order to find out which data files to read, we need to find the `manifest list` - * which corresponds to the latest snapshot. We find it by checking a list of snapshots - * in metadata's "snapshots" section. - * - * Example of metadata.json file. - * - * { - * "format-version" : 1, - * "table-uuid" : "ca2965ad-aae2-4813-8cf7-2c394e0c10f5", - * "location" : "/iceberg_data/db/table_name", - * "last-updated-ms" : 1680206743150, - * "last-column-id" : 2, - * "schema" : { "type" : "struct", "schema-id" : 0, "fields" : [ {}, {}, ... ] }, - * "current-schema-id" : 0, - * "schemas" : [ ], - * ... - * "current-snapshot-id" : 2819310504515118887, - * "refs" : { "main" : { "snapshot-id" : 2819310504515118887, "type" : "branch" } }, - * "snapshots" : [ { - * "snapshot-id" : 2819310504515118887, - * "timestamp-ms" : 1680206743150, - * "summary" : { - * "operation" : "append", "spark.app.id" : "local-1680206733239", - * "added-data-files" : "1", "added-records" : "100", - * "added-files-size" : "1070", "changed-partition-count" : "1", - * "total-records" : "100", "total-files-size" : "1070", "total-data-files" : "1", "total-delete-files" : "0", - * "total-position-deletes" : "0", "total-equality-deletes" : "0" - * }, - * "manifest-list" : "/iceberg_data/db/table_name/metadata/snap-2819310504515118887-1-c87bfec7-d36c-4075-ad04-600b6b0f2020.avro", - * "schema-id" : 0 - * } ], - * "statistics" : [ ], - * "snapshot-log" : [ ... ], - * "metadata-log" : [ ] - * } - */ - struct Metadata - { - int format_version; - String manifest_list; - Strings manifest_files; - }; - Metadata processMetadataFile(const Configuration & configuration, ContextPtr context) - { - const auto metadata_file_path = getMetadataFile(configuration); - auto buf = MetadataReadHelper::createReadBuffer(metadata_file_path, context, configuration); - String json_str; - readJSONObjectPossiblyInvalid(json_str, *buf); - - Poco::JSON::Parser parser; /// For some reason base/base/JSON.h can not parse this json file - Poco::Dynamic::Var json = parser.parse(json_str); - Poco::JSON::Object::Ptr object = json.extract(); - - Metadata result; - result.format_version = object->getValue("format-version"); - - auto current_snapshot_id = object->getValue("current-snapshot-id"); - auto snapshots = object->get("snapshots").extract(); - - for (size_t i = 0; i < snapshots->size(); ++i) - { - const auto snapshot = snapshots->getObject(static_cast(i)); - if (snapshot->getValue("snapshot-id") == current_snapshot_id) - { - const auto path = snapshot->getValue("manifest-list"); - result.manifest_list = std::filesystem::path(configuration.url.key) / metadata_directory / std::filesystem::path(path).filename(); - break; - } - } - return result; - } - - /** - * Manifest list has Avro as default format (and currently we support only Avro). - * Manifest list file format of manifest list is: snap-2819310504515118887-1-c87bfec7-d36c-4075-ad04-600b6b0f2020.avro - * - * `manifest list` has the following contents: - * ┌─manifest_path────────────────────────────────────────────────────────────────────────────────────────┬─manifest_length─┬─partition_spec_id─┬───added_snapshot_id─┬─added_data_files_count─┬─existing_data_files_count─┬─deleted_data_files_count─┬─partitions─┬─added_rows_count─┬─existing_rows_count─┬─deleted_rows_count─┐ - * │ /iceberg_data/db/table_name/metadata/c87bfec7-d36c-4075-ad04-600b6b0f2020-m0.avro │ 5813 │ 0 │ 2819310504515118887 │ 1 │ 0 │ 0 │ [] │ 100 │ 0 │ 0 │ - * └──────────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────┴───────────────────┴─────────────────────┴────────────────────────┴───────────────────────────┴──────────────────────────┴────────────┴──────────────────┴─────────────────────┴────────────────────┘ - */ - void processManifestList(Metadata & metadata, const Configuration & configuration, ContextPtr context) - { - auto buf = MetadataReadHelper::createReadBuffer(metadata.manifest_list, context, configuration); - auto file_reader = std::make_unique(std::make_unique(*buf)); - - auto data_type = AvroSchemaReader::avroNodeToDataType(file_reader->dataSchema().root()->leafAt(0)); - Block header{{data_type->createColumn(), data_type, "manifest_path"}}; - auto columns = parseAvro(*file_reader, header, getFormatSettings(context)); - auto & col = columns.at(0); - - if (col->getDataType() != TypeIndex::String) - { - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `manifest_path` field should be String type, got {}", - col->getFamilyName()); - } - - const auto * col_str = typeid_cast(col.get()); - for (size_t i = 0; i < col_str->size(); ++i) - { - const auto file_path = col_str->getDataAt(i).toView(); - const auto filename = std::filesystem::path(file_path).filename(); - metadata.manifest_files.emplace_back(std::filesystem::path(configuration.url.key) / metadata_directory / filename); - } - } - - /** - * Manifest file has the following format: '/iceberg_data/db/table_name/metadata/c87bfec7-d36c-4075-ad04-600b6b0f2020-m0.avro' - * - * `manifest file` is different in format version V1 and V2 and has the following contents: - * v1 v2 - * status req req - * snapshot_id req opt - * sequence_number opt - * file_sequence_number opt - * data_file req req - * Example format version V1: - * ┌─status─┬─────────snapshot_id─┬─data_file───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ - * │ 1 │ 2819310504515118887 │ ('/iceberg_data/db/table_name/data/00000-1-3edca534-15a0-4f74-8a28-4733e0bf1270-00001.parquet','PARQUET',(),100,1070,67108864,[(1,233),(2,210)],[(1,100),(2,100)],[(1,0),(2,0)],[],[(1,'\0'),(2,'0')],[(1,'c'),(2,'99')],NULL,[4],0) │ - * └────────┴─────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ - * Example format version V2: - * ┌─status─┬─────────snapshot_id─┬─sequence_number─┬─file_sequence_number─┬─data_file───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ - * │ 1 │ 5887006101709926452 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ (0,'/iceberg_data/db/table_name/data/00000-1-c8045c90-8799-4eac-b957-79a0484e223c-00001.parquet','PARQUET',(),100,1070,[(1,233),(2,210)],[(1,100),(2,100)],[(1,0),(2,0)],[],[(1,'\0'),(2,'0')],[(1,'c'),(2,'99')],NULL,[4],[],0) │ - * └────────┴─────────────────────┴─────────────────┴──────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ - * In case of partitioned data we'll have extra directory partition=value: - * ─status─┬─────────snapshot_id─┬─data_file──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ - * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=0/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00001.parquet','PARQUET',(0),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0\0'),(2,'1')],[(1,'\0\0\0\0\0\0\0\0'),(2,'1')],NULL,[4],0) │ - * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=1/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00002.parquet','PARQUET',(1),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0'),(2,'2')],[(1,'\0\0\0\0\0\0\0'),(2,'2')],NULL,[4],0) │ - * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=2/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00003.parquet','PARQUET',(2),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0'),(2,'3')],[(1,'\0\0\0\0\0\0\0'),(2,'3')],NULL,[4],0) │ - * └────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ - */ - Strings getFilesForRead(const Metadata & metadata, const Configuration & configuration, ContextPtr context) - { - NameSet keys; - for (const auto & manifest_file : metadata.manifest_files) - { - auto buffer = MetadataReadHelper::createReadBuffer(manifest_file, context, configuration); - auto file_reader = std::make_unique(std::make_unique(*buffer)); - - avro::NodePtr root_node = file_reader->dataSchema().root(); - size_t leaves_num = root_node->leaves(); - size_t expected_min_num = metadata.format_version == 1 ? 3 : 2; - if (leaves_num < expected_min_num) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Unexpected number of columns {}. Expected at least {}", - root_node->leaves(), expected_min_num); - } - - avro::NodePtr status_node = root_node->leafAt(0); - if (status_node->type() != avro::Type::AVRO_INT) - { - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `status` field should be Int type, got {}", - magic_enum::enum_name(status_node->type())); - } - - avro::NodePtr data_file_node = root_node->leafAt(static_cast(leaves_num) - 1); - if (data_file_node->type() != avro::Type::AVRO_RECORD) - { - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `data_file` field should be Tuple type, got {}", - magic_enum::enum_name(data_file_node->type())); - } - - auto status_col_data_type = AvroSchemaReader::avroNodeToDataType(status_node); - auto data_col_data_type = AvroSchemaReader::avroNodeToDataType(data_file_node); - Block header{ - {status_col_data_type->createColumn(), status_col_data_type, "status"}, - {data_col_data_type->createColumn(), data_col_data_type, "data_file"}}; - - const auto columns = parseAvro(*file_reader, header, getFormatSettings(context)); - if (columns.size() != 2) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Unexpected number of columns. Expected 2, got {}", columns.size()); - } - - if (columns.at(0)->getDataType() != TypeIndex::Int32) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `status` field should be Int32 type, got {}", - columns.at(0)->getFamilyName()); - } - if (columns.at(1)->getDataType() != TypeIndex::Tuple) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `file_path` field should be Tuple type, got {}", - columns.at(1)->getFamilyName()); - } - - const auto status_int_column = assert_cast(columns.at(0).get()); - const auto data_file_tuple_column = assert_cast(columns.at(1).get()); - - if (status_int_column->size() != data_file_tuple_column->size()) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `file_path` and `status` have different rows number: {} and {}", - status_int_column->size(), data_file_tuple_column->size()); - } - - const auto * data_file_name_column = metadata.format_version == 1 - ? data_file_tuple_column->getColumnPtr(0).get() - : data_file_tuple_column->getColumnPtr(1).get(); - - if (data_file_name_column->getDataType() != TypeIndex::String) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `file_path` field should be String type, got {}", - data_file_name_column->getFamilyName()); - } - auto file_name_str_column = assert_cast(data_file_name_column); - - for (size_t i = 0; i < status_int_column->size(); ++i) - { - const auto status = status_int_column->getInt(i); - const auto data_path = std::string(file_name_str_column->getDataAt(i).toView()); - const auto pos = data_path.find(configuration.url.key); - const auto file_path = data_path.substr(pos); - if (pos == std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected to find {} in data path: {}", configuration.url.key, data_path); - - if (status == 2) - { - LOG_TEST(log, "Processing delete file for path: {}", file_path); - chassert(!keys.contains(file_path)); - } - else - keys.insert(file_path); - } - } - - return std::vector(keys.begin(), keys.end()); - } - - MutableColumns parseAvro( - avro::DataFileReaderBase & file_reader, - const Block & header, - const FormatSettings & settings) - { - auto deserializer = std::make_unique(header, file_reader.dataSchema(), true, true, settings); - MutableColumns columns = header.cloneEmptyColumns(); - - file_reader.init(); - RowReadExtension ext; - while (file_reader.hasMore()) - { - file_reader.decr(); - deserializer->deserializeRow(columns, file_reader.decoder(), ext); - } - return columns; - } - -}; - - -template -IcebergMetadataParser::IcebergMetadataParser() : impl(std::make_unique()) -{ -} - -template -Strings IcebergMetadataParser::getFiles(const Configuration & configuration, ContextPtr context) -{ - auto metadata = impl->processMetadataFile(configuration, context); - - /// When table first created and does not have any data - if (metadata.manifest_list.empty()) - return {}; - - impl->processManifestList(metadata, configuration, context); - return impl->getFilesForRead(metadata, configuration, context); -} - - -template IcebergMetadataParser::IcebergMetadataParser(); -template Strings IcebergMetadataParser::getFiles(const StorageS3::Configuration & configuration, ContextPtr); - -} - -#endif From 527e96db3f553dd63b1fcf4422a2ad23b704e743 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 17 Oct 2023 11:43:51 +0000 Subject: [PATCH 362/634] Fix --- src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp index d10b3c2ceba..2a213fc23b6 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp @@ -568,6 +568,7 @@ Strings IcebergMetadata::getDataFiles() } } + data_files = std::vector(files.begin(), files.end()); return data_files; } From 032d82e004d59346e1a9e8278bad068e568655a4 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 17 Oct 2023 12:12:10 +0000 Subject: [PATCH 363/634] Fixes --- .../DataLakes/Iceberg/IcebergMetadata.cpp | 113 ++++++++++-------- .../DataLakes/Iceberg/StorageIceberg.cpp | 3 +- 2 files changed, 62 insertions(+), 54 deletions(-) diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp index 2a213fc23b6..b6a1745089e 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp @@ -2,34 +2,34 @@ #if USE_AWS_S3 && USE_AVRO -#include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include -#include +# include +# include +# include namespace DB @@ -37,10 +37,10 @@ namespace DB namespace ErrorCodes { - extern const int FILE_DOESNT_EXIST; - extern const int ILLEGAL_COLUMN; - extern const int BAD_ARGUMENTS; - extern const int UNSUPPORTED_METHOD; +extern const int FILE_DOESNT_EXIST; +extern const int ILLEGAL_COLUMN; +extern const int BAD_ARGUMENTS; +extern const int UNSUPPORTED_METHOD; } IcebergMetadata::IcebergMetadata( @@ -236,7 +236,6 @@ DataTypePtr getFieldType(const Poco::Dynamic::Var & type, bool required) return getComplexTypeFromObject(type.extract()); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected 'type' field: {}", type.toString()); - } std::pair parseTableSchema(const Poco::JSON::Object::Ptr & metadata_object, int format_version) @@ -253,7 +252,10 @@ std::pair parseTableSchema(const Poco::JSON::Object::P current_schema_id = metadata_object->getValue("current-schema-id"); auto schemas = metadata_object->get("schemas").extract(); if (schemas->size() != 1) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); + throw Exception( + ErrorCodes::UNSUPPORTED_METHOD, + "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not " + "supported"); /// Now we sure that there is only one schema. schema = schemas->getObject(0); @@ -267,7 +269,10 @@ std::pair parseTableSchema(const Poco::JSON::Object::P /// Field "schemas" is optional for version 1, but after version 2 was introduced, /// in most cases this field is added for new tables in version 1 as well. if (metadata_object->has("schemas") && metadata_object->get("schemas").extract()->size() > 1) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); + throw Exception( + ErrorCodes::UNSUPPORTED_METHOD, + "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not " + "supported"); } NamesAndTypesList names_and_types; @@ -283,10 +288,7 @@ std::pair parseTableSchema(const Poco::JSON::Object::P return {std::move(names_and_types), current_schema_id}; } -MutableColumns parseAvro( - avro::DataFileReaderBase & file_reader, - const Block & header, - const FormatSettings & settings) +MutableColumns parseAvro(avro::DataFileReaderBase & file_reader, const Block & header, const FormatSettings & settings) { auto deserializer = std::make_unique(header, file_reader.dataSchema(), true, true, settings); MutableColumns columns = header.cloneEmptyColumns(); @@ -311,19 +313,19 @@ std::pair getMetadataFileAndVersion(const StorageS3::Configuratio if (metadata_files.empty()) { throw Exception( - ErrorCodes::FILE_DOESNT_EXIST, - "The metadata file for Iceberg table with path {} doesn't exist", - configuration.url.key); + ErrorCodes::FILE_DOESNT_EXIST, "The metadata file for Iceberg table with path {} doesn't exist", configuration.url.key); } std::vector> metadata_files_with_versions; metadata_files_with_versions.reserve(metadata_files.size()); - for (const auto & file : metadata_files) + for (const auto & path : metadata_files) { - String version_str(file.begin() + 1, file.begin() + file.find_first_of('.')); + String file_name(path.begin() + path.find_last_of('/') + 1, path.end()); + String version_str(file_name.begin() + 1, file_name.begin() + file_name.find_first_of('.')); if (!std::all_of(version_str.begin(), version_str.end(), isdigit)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad metadata file name: {}. Expected vN.metadata.json where N is a number", file); - metadata_files_with_versions.emplace_back(std::stoi(version_str), file); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Bad metadata file name: {}. Expected vN.metadata.json where N is a number", file_name); + metadata_files_with_versions.emplace_back(std::stoi(version_str), path); } /// Get the latest version of metadata file: v.metadata.json @@ -362,7 +364,8 @@ std::unique_ptr parseIcebergMetadata(const StorageS3::Configura } } - return std::make_unique(configuration, context_, metadata_version, format_version, manifest_list_file, schema_id, schema); + return std::make_unique( + configuration, context_, metadata_version, format_version, manifest_list_file, schema_id, schema); } /** @@ -402,7 +405,8 @@ Strings IcebergMetadata::getDataFiles() LOG_TEST(log, "Collect manifest files from manifest list {}", manifest_list_file); auto manifest_list_buf = S3DataLakeMetadataReadHelper::createReadBuffer(manifest_list_file, getContext(), configuration); - auto manifest_list_file_reader = std::make_unique(std::make_unique(*manifest_list_buf)); + auto manifest_list_file_reader + = std::make_unique(std::make_unique(*manifest_list_buf)); auto data_type = AvroSchemaReader::avroNodeToDataType(manifest_list_file_reader->dataSchema().root()->leafAt(0)); Block header{{data_type->createColumn(), data_type, "manifest_path"}}; @@ -422,7 +426,7 @@ Strings IcebergMetadata::getDataFiles() { const auto file_path = col_str->getDataAt(i).toView(); const auto filename = std::filesystem::path(file_path).filename(); - manifest_files.emplace_back(std::filesystem::path(configuration.url.key) / "metadata_path" / filename); + manifest_files.emplace_back(std::filesystem::path(configuration.url.key) / "metadata" / filename); } NameSet files; @@ -443,7 +447,10 @@ Strings IcebergMetadata::getDataFiles() Poco::Dynamic::Var json = parser.parse(schema_json_string); Poco::JSON::Object::Ptr schema_object = json.extract(); if (schema_object->getValue("schema-id") != current_schema_id) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); + throw Exception( + ErrorCodes::UNSUPPORTED_METHOD, + "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not " + "supported"); avro::NodePtr root_node = manifest_file_reader->dataSchema().root(); size_t leaves_num = root_node->leaves(); @@ -451,9 +458,7 @@ Strings IcebergMetadata::getDataFiles() if (leaves_num < expected_min_num) { throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Unexpected number of columns {}. Expected at least {}", - root_node->leaves(), expected_min_num); + ErrorCodes::BAD_ARGUMENTS, "Unexpected number of columns {}. Expected at least {}", root_node->leaves(), expected_min_num); } avro::NodePtr status_node = root_node->leafAt(0); @@ -500,7 +505,7 @@ Strings IcebergMetadata::getDataFiles() } const auto * status_int_column = assert_cast(columns.at(0).get()); - const auto & data_file_tuple_type = assert_cast(*data_type.get()); + const auto & data_file_tuple_type = assert_cast(*data_col_data_type.get()); const auto * data_file_tuple_column = assert_cast(columns.at(1).get()); if (status_int_column->size() != data_file_tuple_column->size()) @@ -546,7 +551,8 @@ Strings IcebergMetadata::getDataFiles() { Int32 content_type = content_int_column->getElement(i); if (DataFileContent(content_type) != DataFileContent::DATA) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: positional and equality deletes are not supported"); + throw Exception( + ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: positional and equality deletes are not supported"); } const auto status = status_int_column->getInt(i); @@ -563,6 +569,7 @@ Strings IcebergMetadata::getDataFiles() } else { + LOG_TEST(log, "Processing data file for path: {}", file_path); files.insert(file_path); } } diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp index f014cb246eb..afc99d62cca 100644 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp +++ b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp @@ -18,6 +18,7 @@ StoragePtr StorageIceberg::create( configuration.update(context_); auto metadata = parseIcebergMetadata(configuration, context_); auto schema_from_metadata = metadata->getTableSchema(); + configuration.keys = metadata->getDataFiles(); return std::make_shared(std::move(metadata), configuration, context_, table_id_, columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_, constraints_, comment, format_settings_); } @@ -52,7 +53,7 @@ void StorageIceberg::updateConfigurationImpl(ContextPtr local_context) const bool updated = base_configuration.update(local_context); auto new_metadata = parseIcebergMetadata(base_configuration, local_context); /// Check if nothing was changed. - if (!updated && new_metadata->getVersion() == current_metadata->getVersion()) + if (updated && new_metadata->getVersion() == current_metadata->getVersion()) return; if (new_metadata->getVersion() != current_metadata->getVersion()) From 4d2cf5206fcacc005d24c4b3617708dd604fb0a7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 14:12:15 +0200 Subject: [PATCH 364/634] Update documentation --- docs/en/engines/table-engines/integrations/s3queue.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 4769f573b54..60bc48f8bf9 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -241,6 +241,16 @@ For more information about virtual columns see [here](../../../engines/table-eng Constructions with `{}` are similar to the [remote](../../../sql-reference/table-functions/remote.md) table function. +## Limitations {#limitations} + +Duplicated rows can be as a result of: + +- an exception happens during parsing in the middle of file processing and retries are enabled via `s3queue_loading_retries`, duplicate rows are inevitabe; + +- `S3Queue` is configured on multiple servers pointing to the same path in zookeeper and keeper session expires before one server managed to commit processed file, which could lead to another server taking processing of the file, which could be partially or fully processed by the first server; + +- abnormal server termination (with SIGABRT, for example). + :::note If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: From e500dc22e4fc06f1eb3606a71dc688d318d9fa02 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 17 Oct 2023 14:14:58 +0200 Subject: [PATCH 365/634] Respect default format when using http_write_exception_in_output_format --- src/Formats/FormatFactory.cpp | 2 +- src/Interpreters/executeQuery.cpp | 49 +++++++++++++++++-- ...default_format_on_http_exception.reference | 25 ++++++++++ ...99_use_default_format_on_http_exception.sh | 33 +++++++++++++ 4 files changed, 105 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02899_use_default_format_on_http_exception.reference create mode 100755 tests/queries/0_stateless/02899_use_default_format_on_http_exception.sh diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 9b391324455..2713b7cb35f 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -410,7 +410,7 @@ std::unique_ptr FormatFactory::wrapReadBufferIfNeeded( static void addExistingProgressToOutputFormat(OutputFormatPtr format, ContextPtr context) { - auto element_id = context->getProcessListElement(); + auto element_id = context->getProcessListElementSafe(); if (element_id) { /// While preparing the query there might have been progress (for example in subscalar subqueries) so add it here diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index b96e05c1f9a..cda2d70714a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1332,7 +1332,46 @@ void executeQuery( BlockIO streams; OutputFormatPtr output_format; - std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr); + auto update_format_for_exception_if_needed = [&]() + { + if (!output_format) + { + try + { + String format_name = context->getDefaultFormat(); + output_format = FormatFactory::instance().getOutputFormat(format_name, ostr, {}, context, output_format_settings); + if (output_format) + { + /// Force an update of the headers before we start writing + result_details.content_type = output_format->getContentType(); + result_details.format = format_name; + set_result_details(result_details); + set_result_details = nullptr; + } + } + catch (const DB::Exception & e) + { + /// Ignore this exception and report the original one + LOG_WARNING(&Poco::Logger::get("executeQuery"), getExceptionMessageAndPattern(e, true)); + } + } + }; + + try + { + std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr); + } + catch (...) + { + if (handle_exception_in_output_format) + { + update_format_for_exception_if_needed(); + if (output_format) + handle_exception_in_output_format(*output_format); + } + throw; + } + auto & pipeline = streams.pipeline; std::unique_ptr compressed_buffer; @@ -1426,8 +1465,12 @@ void executeQuery( } catch (...) { - if (handle_exception_in_output_format && output_format) - handle_exception_in_output_format(*output_format); + if (handle_exception_in_output_format) + { + update_format_for_exception_if_needed(); + if (output_format) + handle_exception_in_output_format(*output_format); + } streams.onException(); throw; } diff --git a/tests/queries/0_stateless/02899_use_default_format_on_http_exception.reference b/tests/queries/0_stateless/02899_use_default_format_on_http_exception.reference new file mode 100644 index 00000000000..a943df06764 --- /dev/null +++ b/tests/queries/0_stateless/02899_use_default_format_on_http_exception.reference @@ -0,0 +1,25 @@ +INSERT WITH default_format=JSON +Content-Type:application/json;charset=UTF-8 +"exception":"Code:62. + +INSERT WITH default_format=XML +Content-Type:application/xml;charset=UTF-8 +Code:62.DB::Ex---tion: + +INSERT WITH default_format=BADFORMAT +Content-Type:text/plain;charset=UTF-8 +X-ClickHouse-Ex---tion-Code:62 +Code:62.DB::Ex---tion: + +INSERT WITH X-ClickHouse-Format: JSON +Content-Type:application/json;charset=UTF-8 +"exception":"Code:62. + +INSERT WITH X-ClickHouse-Format: XML +Content-Type:application/xml;charset=UTF-8 +Code:62.DB::Ex---tion: + +INSERT WITH X-ClickHouse-Format: BADFORMAT +Content-Type:text/plain;charset=UTF-8 +X-ClickHouse-Ex---tion-Code:62 +Code:62.DB::Ex---tion: diff --git a/tests/queries/0_stateless/02899_use_default_format_on_http_exception.sh b/tests/queries/0_stateless/02899_use_default_format_on_http_exception.sh new file mode 100755 index 00000000000..f92ab7db4fb --- /dev/null +++ b/tests/queries/0_stateless/02899_use_default_format_on_http_exception.sh @@ -0,0 +1,33 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_URL="$CLICKHOUSE_URL&http_write_exception_in_output_format=1" + +echo "INSERT WITH default_format=JSON" +echo "INSERT INTO system.numbers Select * from numbers(10);" \ + | ${CLICKHOUSE_CURL} -sS "${CH_URL}&default_format=JSON" -i | grep 'xception\|Content-Type' | sed 's/Exception/Ex---tion/' | awk '{ print $1 $2 $3 }' +echo "" +echo "INSERT WITH default_format=XML" +echo "INSERT INTO system.numbers Select * from numbers(10);" \ + | ${CLICKHOUSE_CURL} -sS "${CH_URL}&default_format=XML" -i | grep 'xception\|Content-Type' | sed 's/Exception/Ex---tion/' | awk '{ print $1 $2 $3 }' +echo "" +echo "INSERT WITH default_format=BADFORMAT" +echo "INSERT INTO system.numbers Select * from numbers(10);" \ + | ${CLICKHOUSE_CURL} -sS "${CH_URL}&default_format=BADFORMAT" -i | grep 'xception\|Content-Type' | sed 's/Exception/Ex---tion/' | awk '{ print $1 $2 $3 }' + + +echo "" +echo "INSERT WITH X-ClickHouse-Format: JSON" +echo "INSERT INTO system.numbers Select * from numbers(10);" \ + | ${CLICKHOUSE_CURL} -sS "${CH_URL}" -H 'X-ClickHouse-Format: JSON' -i | grep 'xception\|Content-Type' | sed 's/Exception/Ex---tion/' | awk '{ print $1 $2 $3 }' +echo "" +echo "INSERT WITH X-ClickHouse-Format: XML" +echo "INSERT INTO system.numbers Select * from numbers(10);" \ + | ${CLICKHOUSE_CURL} -sS "${CH_URL}" -H 'X-ClickHouse-Format: XML' -i | grep 'xception\|Content-Type' | sed 's/Exception/Ex---tion/' | awk '{ print $1 $2 $3 }' +echo "" +echo "INSERT WITH X-ClickHouse-Format: BADFORMAT" +echo "INSERT INTO system.numbers Select * from numbers(10);" \ + | ${CLICKHOUSE_CURL} -sS "${CH_URL}" -H 'X-ClickHouse-Format: BADFORMAT' -i | grep 'xception\|Content-Type' | sed 's/Exception/Ex---tion/' | awk '{ print $1 $2 $3 }' From c549083e2c8b009b447927ad669d27d882b688b0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 14:23:59 +0200 Subject: [PATCH 366/634] Fix --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index c64330e9df1..487ca7f7f4c 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -804,7 +804,7 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() /// Create a lock so that with distributed processing /// multiple nodes do not execute cleanup in parallel. - auto ephemeral_node = zkutil::EphemeralNodeHolder::create(zookeeper_cleanup_lock_path, *zk_client, toString(getCurrentTime())); + auto ephemeral_node = zkutil::EphemeralNodeHolder::tryCreate(zookeeper_cleanup_lock_path, *zk_client, toString(getCurrentTime())); if (!ephemeral_node) { LOG_TEST(log, "Cleanup is already being executed by another node"); From fd37e25fbd11909b620c13e87c41e7c3823474e4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 15:07:58 +0200 Subject: [PATCH 367/634] Fix doc --- .../en/engines/table-engines/integrations/s3queue.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 60bc48f8bf9..2a9b25e0c1a 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -243,17 +243,17 @@ Constructions with `{}` are similar to the [remote](../../../sql-reference/table ## Limitations {#limitations} -Duplicated rows can be as a result of: +1. Duplicated rows can be as a result of: -- an exception happens during parsing in the middle of file processing and retries are enabled via `s3queue_loading_retries`, duplicate rows are inevitabe; +- an exception happens during parsing in the middle of file processing and retries are enabled via `s3queue_loading_retries`; - `S3Queue` is configured on multiple servers pointing to the same path in zookeeper and keeper session expires before one server managed to commit processed file, which could lead to another server taking processing of the file, which could be partially or fully processed by the first server; -- abnormal server termination (with SIGABRT, for example). +- abnormal server termination. + +2. `S3Queue` is configured on multiple servers pointing to the same path in zookeeper and `Ordered` mode is used, then `s3queue_loading_retries` will not work. This will be fixed soon. + -:::note -If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. -::: ## Introspection {#introspection} For introspection use `system.s3queue` stateless table and `system.s3queue_log` persistent table. From b4a4c1e02e9c7417bc942ffe0400a41cfb52d78e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 17 Oct 2023 16:02:06 +0200 Subject: [PATCH 368/634] Only change headers if necessary --- src/Interpreters/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cda2d70714a..734e4e68776 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1340,7 +1340,7 @@ void executeQuery( { String format_name = context->getDefaultFormat(); output_format = FormatFactory::instance().getOutputFormat(format_name, ostr, {}, context, output_format_settings); - if (output_format) + if (output_format && output_format->supportsWritingException()) { /// Force an update of the headers before we start writing result_details.content_type = output_format->getContentType(); From 0c49f72433ebc693825587983f1707c3c09b6c40 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 17 Oct 2023 16:08:50 +0200 Subject: [PATCH 369/634] Tidy --- src/Planner/PlannerJoinTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 1e4c51dda59..53d61fd2eb8 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -526,7 +526,7 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context); } -static UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info) +UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info) { auto const & main_query_node = select_query_info.query_tree->as(); From ccfc9a98771faefc5c96570be02e3a531647ea1a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 17 Oct 2023 16:53:14 +0200 Subject: [PATCH 370/634] Fix path in 02403_big_http_chunk_size --- tests/queries/0_stateless/02403_big_http_chunk_size.python | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02403_big_http_chunk_size.python b/tests/queries/0_stateless/02403_big_http_chunk_size.python index 4d2f01db55b..3213b8cd387 100644 --- a/tests/queries/0_stateless/02403_big_http_chunk_size.python +++ b/tests/queries/0_stateless/02403_big_http_chunk_size.python @@ -14,7 +14,7 @@ def main(): sock = socket(AF_INET, SOCK_STREAM) sock.connect((host, port)) sock.settimeout(60) - s = "POST /play HTTP/1.1\r\n" + s = "POST / HTTP/1.1\r\n" s += "Host: %s\r\n" % host s += "Content-type: multipart/form-data\r\n" s += "Transfer-encoding: chunked\r\n" From b382705c2d1e5ff004f20a4efaa7de8d370b4caa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 16:29:20 +0200 Subject: [PATCH 371/634] Use diff object again, since JSON API limits the files --- tests/ci/pr_info.py | 13 +++++++++---- tests/ci/worker/prepare-ci-ami.sh | 1 - 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index c02c71d8761..faa6b539a6b 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -4,6 +4,8 @@ import logging import os from typing import Dict, List, Set, Union, Literal +from unidiff import PatchSet # type: ignore + from build_download_helper import get_gh_api from env_helper import ( GITHUB_REPOSITORY, @@ -263,11 +265,14 @@ class PRInfo: raise TypeError("The event does not have diff URLs") for diff_url in self.diff_urls: - response = get_gh_api(diff_url, sleep=RETRY_SLEEP) + response = get_gh_api( + diff_url, + sleep=RETRY_SLEEP, + headers={"Accept": "application/vnd.github.v3.diff"}, + ) response.raise_for_status() - diff = response.json() - if "files" in diff: - self.changed_files = {f["filename"] for f in diff["files"]} + diff_object = PatchSet(response.text) + self.changed_files.update({f.path for f in diff_object}) print(f"Fetched info about {len(self.changed_files)} changed files") def get_dict(self): diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index f3e11b6d00a..20e7e3fd53e 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -90,7 +90,6 @@ systemctl restart docker sudo -u ubuntu docker buildx version sudo -u ubuntu docker buildx create --use --name default-builder -# FIXME: remove unidiff as soon as no old PRs could use it, here and in Dockerfile pip install boto3 pygithub requests urllib3 unidiff dohq-artifactory mkdir -p $RUNNER_HOME && cd $RUNNER_HOME From a43ee4116283277b465d76963e28208692fbbe59 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 18 Sep 2023 20:02:46 +0200 Subject: [PATCH 372/634] increase s3_retry_attempts --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2c45108f9a5..1012e7a2d9a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -103,7 +103,7 @@ class IColumn; M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ - M(UInt64, s3_retry_attempts, 10, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ + M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ M(UInt64, s3_request_timeout_ms, 3000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ M(UInt64, s3_http_connection_pool_size, 1000, "How many reusable open connections to keep per S3 endpoint. Only applies to the S3 table engine and table function, not to S3 disks (for disks, use disk config instead). Global setting, can only be set in config, overriding it per session or per query has no effect.", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ From eb4519b9d6df7abee5360d435a477fa432a97b09 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 19:02:04 +0200 Subject: [PATCH 373/634] Fix --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 487ca7f7f4c..d952160491c 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -474,7 +474,8 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolder if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); - zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); + if (max_loading_retries) + zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); return; } @@ -537,7 +538,8 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); - zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); + if (max_loading_retries) + zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); return; } From 66337c23cce377ce8233dd1ce92655b383eab8f8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 20 Sep 2023 21:22:53 +0200 Subject: [PATCH 374/634] Use treeless checkout for image building --- .github/workflows/backport_branches.yml | 1 + .github/workflows/docs_check.yml | 1 + .github/workflows/master.yml | 1 + .github/workflows/nightly.yml | 1 + .github/workflows/pull_request.yml | 1 + .github/workflows/release_branches.yml | 1 + 6 files changed, 6 insertions(+) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index c20255579e2..0d55d094990 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -77,6 +77,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags + filter: tree:0 - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index 78724a4a924..203a149df65 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -73,6 +73,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags + filter: tree:0 - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 4771e584266..4bb9f785542 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -61,6 +61,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags + filter: tree:0 - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index ed8159b229e..8fabd86e2b3 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -54,6 +54,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags + filter: tree:0 - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 676fb993ca8..489203a8dcb 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -94,6 +94,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags + filter: tree:0 - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index fba56339d16..e3a75a08925 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -53,6 +53,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags + filter: tree:0 - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: From c19ca5c4d5a1749a8d1d7b0891ef560c1f51d660 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Tue, 17 Oct 2023 19:19:35 +0200 Subject: [PATCH 375/634] Address PR feedback --- docs/en/operations/settings/settings.md | 4 +- src/Core/Settings.h | 4 +- .../InterpreterShowColumnsQuery.cpp | 36 ++-- ...show_columns_mysql_compatibility.reference | 187 ++++++++++++------ ...02775_show_columns_mysql_compatibility.sql | 15 +- 5 files changed, 158 insertions(+), 88 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f22badb8c59..fe3f1298ff5 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3290,7 +3290,7 @@ Possible values: Default value: `0`. -## mysql_remap_string_as_text_in_show_columns {#mysql_remap_string_as_text_in_show_columns} +## mysql_map_string_to_text_in_show_columns {#mysql_map_string_to_text_in_show_columns} When enabled, [String](../../sql-reference/data-types/string.md) ClickHouse data type will be displayed as `TEXT` in [SHOW COLUMNS](../../sql-reference/statements/show.md#show_columns). @@ -3301,7 +3301,7 @@ Has effect only when [use_mysql_types_in_show_columns](#use_mysql_types_in_show_ Default value: `0`. -## mysql_remap_fixed_string_as_text_in_show_columns {#mysql_remap_fixed_string_as_text_in_show_columns} +## mysql_map_fixed_string_to_text_in_show_columns {#mysql_map_fixed_string_to_text_in_show_columns} When enabled, [FixedString](../../sql-reference/data-types/fixedstring.md) ClickHouse data type will be displayed as `TEXT` in [SHOW COLUMNS](../../sql-reference/statements/show.md#show_columns). diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 41a99ff1319..537ffc75323 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -206,8 +206,8 @@ class IColumn; \ M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \ M(Bool, use_mysql_types_in_show_columns, false, "Show native MySQL types in SHOW [FULL] COLUMNS", 0) \ - M(Bool, mysql_remap_string_as_text_in_show_columns, false, "If enabled, String type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Only works if use_mysql_types_in_show_columns is enabled too", 0) \ - M(Bool, mysql_remap_fixed_string_as_text_in_show_columns, false, "If enabled, FixedString type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Only works if use_mysql_types_in_show_columns is enabled too", 0) \ + M(Bool, mysql_map_string_to_text_in_show_columns, false, "If enabled, String type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Will only take effect if use_mysql_types_in_show_columns is enabled too", 0) \ + M(Bool, mysql_map_fixed_string_to_text_in_show_columns, false, "If enabled, FixedString type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Will only take effect if use_mysql_types_in_show_columns is enabled too", 0) \ \ M(UInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ", 0) \ \ diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index a213f27972a..55891f51063 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -26,8 +26,8 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() const auto & settings = getContext()->getSettingsRef(); const bool use_mysql_types = settings.use_mysql_types_in_show_columns; - const bool remap_string_as_text = settings.mysql_remap_string_as_text_in_show_columns; - const bool remap_fixed_string_as_text = settings.mysql_remap_fixed_string_as_text_in_show_columns; + const bool remap_string_as_text = settings.mysql_map_string_to_text_in_show_columns; + const bool remap_fixed_string_as_text = settings.mysql_map_fixed_string_to_text_in_show_columns; WriteBufferFromOwnString buf_database; String resolved_database = getContext()->resolveDatabase(query.database); @@ -40,11 +40,13 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() String rewritten_query; if (use_mysql_types) + { /// Cheapskate SQL-based mapping from native types to MySQL types, see https://dev.mysql.com/doc/refman/8.0/en/data-types.html /// Only used with setting 'use_mysql_types_in_show_columns = 1' /// Known issues: /// - Enums are translated to TEXT - rewritten_query += fmt::format(R"( + rewritten_query += fmt::format( + R"( WITH map( 'Int8', 'TINYINT', 'Int16', 'SMALLINT', @@ -56,8 +58,6 @@ WITH map( 'UInt64', 'BIGINT UNSIGNED', 'Float32', 'FLOAT', 'Float64', 'DOUBLE', - 'String', '{}', - 'FixedString', '{}', 'UUID', 'CHAR', 'Bool', 'TINYINT', 'Date', 'DATE', @@ -66,20 +66,26 @@ WITH map( 'DateTime64', 'DATETIME', 'Map', 'JSON', 'Tuple', 'JSON', - 'Object', 'JSON') AS native_to_mysql_mapping, - splitByRegexp('\(|\)', type_) AS split, - multiIf(startsWith(type_, 'LowCardinality(Nullable'), split[3], - startsWith(type_, 'LowCardinality'), split[2], - startsWith(type_, 'Nullable'), split[2], - split[1]) AS inner_type, - if (length(split) > 1, splitByString(', ', split[2]), []) AS decimal_scale_and_precision, - multiIf(inner_type = 'Decimal' AND toInt8(decimal_scale_and_precision[1]) <= 65 AND toInt8(decimal_scale_and_precision[2]) <= 30, concat('DECIMAL(', decimal_scale_and_precision[1], ', ', decimal_scale_and_precision[2], ')'), - mapContains(native_to_mysql_mapping, inner_type) = true, native_to_mysql_mapping[inner_type], - 'TEXT') AS mysql_type + 'Object', 'JSON', + 'String', '{}', + 'FixedString', '{}') AS native_to_mysql_mapping, )", remap_string_as_text ? "TEXT" : "BLOB", remap_fixed_string_as_text ? "TEXT" : "BLOB"); + rewritten_query += R"( + splitByRegexp('\(|\)', type_) AS split, + multiIf(startsWith(type_, 'LowCardinality(Nullable'), split[3], + startsWith(type_, 'LowCardinality'), split[2], + startsWith(type_, 'Nullable'), split[2], + split[1]) AS inner_type, + if (length(split) > 1, splitByString(', ', split[2]), []) AS decimal_scale_and_precision, + multiIf(inner_type = 'Decimal' AND toInt8(decimal_scale_and_precision[1]) <= 65 AND toInt8(decimal_scale_and_precision[2]) <= 30, concat('DECIMAL(', decimal_scale_and_precision[1], ', ', decimal_scale_and_precision[2], ')'), + mapContains(native_to_mysql_mapping, inner_type) = true, native_to_mysql_mapping[inner_type], + 'TEXT') AS mysql_type + )"; + } + rewritten_query += R"( SELECT name_ AS field, diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference index 29db3e1b73b..e038a3362df 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference @@ -120,7 +120,7 @@ ui32 INTEGER UNSIGNED NO \N ui64 BIGINT UNSIGNED NO \N ui8 TINYINT UNSIGNED NO \N uuid CHAR NO \N --- SHOW COLUMNS with mysql_remap_string_as_text_in_show_columns = 1 +-- SHOW COLUMNS with mysql_map_string_to_text_in_show_columns = 1 a TEXT NO \N agg TEXT NO \N b TINYINT NO \N @@ -181,7 +181,7 @@ ui32 INTEGER UNSIGNED NO \N ui64 BIGINT UNSIGNED NO \N ui8 TINYINT UNSIGNED NO \N uuid CHAR NO \N --- SHOW COLUMNS with mysql_remap_fixed_string_as_text_in_show_columns = 1 +-- SHOW COLUMNS with mysql_map_fixed_string_to_text_in_show_columns = 1 a TEXT NO \N agg TEXT NO \N b TINYINT NO \N @@ -242,64 +242,125 @@ ui32 INTEGER UNSIGNED NO \N ui64 BIGINT UNSIGNED NO \N ui8 TINYINT UNSIGNED NO \N uuid CHAR NO \N --- SHOW COLUMNS with both mysql_remap_string_as_text_in_show_columns = 1 and mysql_remap_fixed_string_as_text_in_show_columns = 1 -a TEXT NO \N -agg TEXT NO \N -b TINYINT NO \N -d DATE NO \N -d32 DATE NO \N -dec128 DECIMAL(38, 2) NO \N -dec128_native DECIMAL(35, 30) NO \N -dec128_text TEXT NO \N -dec256 TEXT NO \N -dec256_native DECIMAL(65, 2) NO \N -dec256_text TEXT NO \N -dec32 DECIMAL(9, 2) NO \N -dec64 DECIMAL(18, 2) NO \N -dt DATETIME NO \N -dt64 DATETIME NO \N -dt64_3_tz1 DATETIME NO \N -dt64_3_tz2 DATETIME NO \N -dt64_6 DATETIME NO \N -dt64_9 DATETIME NO \N -dt_tz1 DATETIME NO \N -dt_tz2 DATETIME NO \N -enm TEXT NO \N -f32 FLOAT NO \N -f64 DOUBLE NO \N -fs TEXT NO \N -i128 TEXT NO \N -i16 SMALLINT NO \N -i256 TEXT NO \N -i32 INTEGER NO \N -i64 BIGINT NO \N -i8 TINYINT NO \N -ip4 TEXT NO \N -ip6 TEXT NO \N -lfs TEXT NO \N -lnfs TEXT YES \N -lns TEXT YES \N -ls TEXT NO \N -m JSON NO \N -m_complex JSON NO \N -mpg TEXT NO \N -ndt64 DATETIME YES \N -ndt64_tz DATETIME YES \N -nested.col1 TEXT NO \N -nested.col2 TEXT NO \N -nfs TEXT YES \N -ns TEXT YES \N -o JSON NO \N -p TEXT NO \N -pg TEXT NO \N -r TEXT NO \N -s TEXT NO \N -sagg TEXT NO \N -t JSON NO \N -ui128 TEXT NO \N -ui16 SMALLINT UNSIGNED NO \N -ui256 TEXT NO \N -ui32 INTEGER UNSIGNED NO \N -ui64 BIGINT UNSIGNED NO \N -ui8 TINYINT UNSIGNED NO \N -uuid CHAR NO \N +-- SHOW COLUMNS with mysql_map_string_to_text_in_show_columns = 1 and without use_mysql_types_in_show_columns +a Array(String) NO \N +agg AggregateFunction(uniq, UInt64) NO \N +b Bool NO \N +d Date NO \N +d32 Date32 NO \N +dec128 Decimal(38, 2) NO \N +dec128_native Decimal(35, 30) NO \N +dec128_text Decimal(35, 31) NO \N +dec256 Decimal(76, 2) NO \N +dec256_native Decimal(65, 2) NO \N +dec256_text Decimal(66, 2) NO \N +dec32 Decimal(9, 2) NO \N +dec64 Decimal(18, 2) NO \N +dt DateTime NO \N +dt64 DateTime64(3) NO \N +dt64_3_tz1 DateTime64(3, \'UTC\') NO \N +dt64_3_tz2 DateTime64(3, \'Asia/Shanghai\') NO \N +dt64_6 DateTime64(6, \'UTC\') NO \N +dt64_9 DateTime64(9, \'UTC\') NO \N +dt_tz1 DateTime(\'UTC\') NO \N +dt_tz2 DateTime(\'Europe/Amsterdam\') NO \N +enm Enum8(\'hallo\' = 1, \'welt\' = 2) NO \N +f32 Float32 NO \N +f64 Float64 NO \N +fs FixedString(3) NO \N +i128 Int128 NO \N +i16 Int16 NO \N +i256 Int256 NO \N +i32 Int32 NO \N +i64 Int64 NO \N +i8 Int8 NO \N +ip4 IPv4 NO \N +ip6 IPv6 NO \N +lfs LowCardinality(FixedString(3)) NO \N +lnfs LowCardinality(Nullable(FixedString(3))) YES \N +lns LowCardinality(Nullable(String)) YES \N +ls LowCardinality(String) NO \N +m Map(Int32, String) NO \N +m_complex Map(Int32, Map(Int32, LowCardinality(Nullable(String)))) NO \N +mpg MultiPolygon NO \N +ndt64 Nullable(DateTime64(3)) YES \N +ndt64_tz Nullable(DateTime64(3, \'Asia/Shanghai\')) YES \N +nested.col1 Array(String) NO \N +nested.col2 Array(UInt32) NO \N +nfs Nullable(FixedString(3)) YES \N +ns Nullable(String) YES \N +o Object(\'json\') NO \N +p Point NO \N +pg Polygon NO \N +r Ring NO \N +s String NO \N +sagg SimpleAggregateFunction(sum, Float64) NO \N +t Tuple(Int32, String, Nullable(String), LowCardinality(String), LowCardinality(Nullable(String)), Tuple(Int32, String)) NO \N +ui128 UInt128 NO \N +ui16 UInt16 NO \N +ui256 UInt256 NO \N +ui32 UInt32 NO \N +ui64 UInt64 NO \N +ui8 UInt8 NO \N +uuid UUID NO \N +-- SHOW COLUMNS with mysql_map_fixed_string_to_text_in_show_columns = 1 and without use_mysql_types_in_show_columns +a Array(String) NO \N +agg AggregateFunction(uniq, UInt64) NO \N +b Bool NO \N +d Date NO \N +d32 Date32 NO \N +dec128 Decimal(38, 2) NO \N +dec128_native Decimal(35, 30) NO \N +dec128_text Decimal(35, 31) NO \N +dec256 Decimal(76, 2) NO \N +dec256_native Decimal(65, 2) NO \N +dec256_text Decimal(66, 2) NO \N +dec32 Decimal(9, 2) NO \N +dec64 Decimal(18, 2) NO \N +dt DateTime NO \N +dt64 DateTime64(3) NO \N +dt64_3_tz1 DateTime64(3, \'UTC\') NO \N +dt64_3_tz2 DateTime64(3, \'Asia/Shanghai\') NO \N +dt64_6 DateTime64(6, \'UTC\') NO \N +dt64_9 DateTime64(9, \'UTC\') NO \N +dt_tz1 DateTime(\'UTC\') NO \N +dt_tz2 DateTime(\'Europe/Amsterdam\') NO \N +enm Enum8(\'hallo\' = 1, \'welt\' = 2) NO \N +f32 Float32 NO \N +f64 Float64 NO \N +fs FixedString(3) NO \N +i128 Int128 NO \N +i16 Int16 NO \N +i256 Int256 NO \N +i32 Int32 NO \N +i64 Int64 NO \N +i8 Int8 NO \N +ip4 IPv4 NO \N +ip6 IPv6 NO \N +lfs LowCardinality(FixedString(3)) NO \N +lnfs LowCardinality(Nullable(FixedString(3))) YES \N +lns LowCardinality(Nullable(String)) YES \N +ls LowCardinality(String) NO \N +m Map(Int32, String) NO \N +m_complex Map(Int32, Map(Int32, LowCardinality(Nullable(String)))) NO \N +mpg MultiPolygon NO \N +ndt64 Nullable(DateTime64(3)) YES \N +ndt64_tz Nullable(DateTime64(3, \'Asia/Shanghai\')) YES \N +nested.col1 Array(String) NO \N +nested.col2 Array(UInt32) NO \N +nfs Nullable(FixedString(3)) YES \N +ns Nullable(String) YES \N +o Object(\'json\') NO \N +p Point NO \N +pg Polygon NO \N +r Ring NO \N +s String NO \N +sagg SimpleAggregateFunction(sum, Float64) NO \N +t Tuple(Int32, String, Nullable(String), LowCardinality(String), LowCardinality(Nullable(String)), Tuple(Int32, String)) NO \N +ui128 UInt128 NO \N +ui16 UInt16 NO \N +ui256 UInt256 NO \N +ui32 UInt32 NO \N +ui64 UInt64 NO \N +ui8 UInt8 NO \N +uuid UUID NO \N diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql index 5f7d1881702..e447dee47ed 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql @@ -78,13 +78,16 @@ SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 0; SELECT '-- SHOW COLUMNS with use_mysql_types_in_show_columns = 1'; SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 1; -SELECT '-- SHOW COLUMNS with mysql_remap_string_as_text_in_show_columns = 1'; -SHOW COLUMNS FROM tab SETTINGS mysql_remap_string_as_text_in_show_columns=1; +SELECT '-- SHOW COLUMNS with mysql_map_string_to_text_in_show_columns = 1'; +SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 1, mysql_map_string_to_text_in_show_columns=1; -SELECT '-- SHOW COLUMNS with mysql_remap_fixed_string_as_text_in_show_columns = 1'; -SHOW COLUMNS FROM tab SETTINGS mysql_remap_fixed_string_as_text_in_show_columns=1; +SELECT '-- SHOW COLUMNS with mysql_map_fixed_string_to_text_in_show_columns = 1'; +SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 1, mysql_map_fixed_string_to_text_in_show_columns=1; -SELECT '-- SHOW COLUMNS with both mysql_remap_string_as_text_in_show_columns = 1 and mysql_remap_fixed_string_as_text_in_show_columns = 1'; -SHOW COLUMNS FROM tab SETTINGS mysql_remap_string_as_text_in_show_columns=1, mysql_remap_fixed_string_as_text_in_show_columns=1; +SELECT '-- SHOW COLUMNS with mysql_map_string_to_text_in_show_columns = 1 and without use_mysql_types_in_show_columns'; +SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 0, mysql_map_string_to_text_in_show_columns=1; + +SELECT '-- SHOW COLUMNS with mysql_map_fixed_string_to_text_in_show_columns = 1 and without use_mysql_types_in_show_columns'; +SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 0, mysql_map_fixed_string_to_text_in_show_columns=1; DROP TABLE tab; From 9c9bc3fee243968b8743d66bbc844943cda2bb32 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 19:21:08 +0200 Subject: [PATCH 376/634] Add `filter:` for almost every checkout --- .github/workflows/backport_branches.yml | 5 +++++ .github/workflows/jepsen.yml | 2 ++ .github/workflows/master.yml | 13 +++++++++++++ .github/workflows/nightly.yml | 1 + .github/workflows/pull_request.yml | 3 +++ .github/workflows/release.yml | 1 + .github/workflows/release_branches.yml | 5 +++++ .github/workflows/tags_stable.yml | 1 + 8 files changed, 31 insertions(+) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 0d55d094990..eb4c29130c4 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -186,6 +186,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # For a proper version and performance artifacts + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -228,6 +229,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # For a proper version and performance artifacts + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -400,6 +402,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' @@ -449,6 +452,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' @@ -488,6 +492,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself + filter: tree:0 - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index e67df15c4d3..7f1fd16aa89 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -24,6 +24,7 @@ jobs: with: clear-repository: true fetch-depth: 0 + filter: tree:0 - name: Jepsen Test run: | sudo rm -fr "$TEMP_PATH" @@ -53,6 +54,7 @@ jobs: # with: # clear-repository: true # fetch-depth: 0 + # filter: tree:0 # - name: Jepsen Test # run: | # sudo rm -fr "$TEMP_PATH" diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 4bb9f785542..a9aa7717add 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -201,6 +201,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # For a proper version and performance artifacts + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -243,6 +244,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # For a proper version and performance artifacts + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -284,6 +286,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -582,6 +585,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' @@ -631,6 +635,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -673,6 +678,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -715,6 +721,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' @@ -764,6 +771,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -806,6 +814,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -848,6 +857,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -890,6 +900,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -932,6 +943,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -964,6 +976,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself + filter: tree:0 - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 8fabd86e2b3..6452b83fdd6 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -91,6 +91,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # Shallow clones should be disabled for a better relevancy of analysis + filter: tree:0 submodules: true - name: Set up JDK 11 uses: actions/setup-java@v1 diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 489203a8dcb..45ce125f1a8 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -267,6 +267,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # for performance artifact + filter: tree:0 submodules: true - name: Build run: | @@ -351,6 +352,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # for performance artifact + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -1022,6 +1024,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself + filter: tree:0 - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 0742ebfd449..051a3ea1eac 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -49,6 +49,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # otherwise we will have no version info + filter: tree:0 ref: ${{ env.GITHUB_TAG }} - name: Check docker clickhouse/clickhouse-server building run: | diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index e3a75a08925..5dd837c6456 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -162,6 +162,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -204,6 +205,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # For a proper version and performance artifacts + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -457,6 +459,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' @@ -506,6 +509,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' @@ -545,6 +549,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself + filter: tree:0 - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" diff --git a/.github/workflows/tags_stable.yml b/.github/workflows/tags_stable.yml index f5b42e9c882..0a3945829ca 100644 --- a/.github/workflows/tags_stable.yml +++ b/.github/workflows/tags_stable.yml @@ -38,6 +38,7 @@ jobs: with: ref: master fetch-depth: 0 + filter: tree:0 - name: Update versions, docker version, changelog, security env: GITHUB_TOKEN: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} From 92260b7aedf19d3c31b3a5f48b06804843cd47f5 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 19:27:58 +0200 Subject: [PATCH 377/634] Ignore docs_check.yml in pull_requests and wise versa --- .github/workflows/docs_check.yml | 1 + .github/workflows/pull_request.yml | 1 + 2 files changed, 2 insertions(+) diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index 203a149df65..dada9999a68 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -18,6 +18,7 @@ on: # yamllint disable-line rule:truthy - 'docs/**' - 'utils/check-style/aspell-ignore/**' - 'tests/ci/docs_check.py' + - '.github/workflows/docs_check.yml' jobs: CheckLabels: runs-on: [self-hosted, style-checker] diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 45ce125f1a8..5937f434135 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -18,6 +18,7 @@ on: # yamllint disable-line rule:truthy - 'docs/**' - 'utils/check-style/aspell-ignore/**' - 'tests/ci/docs_check.py' + - '.github/workflows/docs_check.yml' ########################################################################################## ##################################### SMALL CHECKS ####################################### ########################################################################################## From 2c82329ef74268421db55ccdef1bd207d6ad3d6e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 17 Oct 2023 17:33:23 +0000 Subject: [PATCH 378/634] Update submodule --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index cb6b724ac49..4b3a77e6008 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit cb6b724ac49fe94098e30e3e0d9fd6901007ff38 +Subproject commit 4b3a77e600859bb3e143247f91ef75e4286b7ec9 From 323486f9e877067b8c0e38a623d2adf76818da2a Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 17 Oct 2023 18:10:47 +0000 Subject: [PATCH 379/634] Add tests --- .../Formats/Impl/AvroRowInputFormat.cpp | 65 ++++++-- .../Impl/NativeORCBlockInputFormat.cpp | 5 +- .../DataLakes/Iceberg/IcebergMetadata.cpp | 128 ++++++++-------- .../DataLakes/Iceberg/StorageIceberg.cpp | 2 +- .../DataLakes/Iceberg/StorageIceberg.h | 2 +- .../integration/test_storage_iceberg/test.py | 144 +++++++++++++++++- 6 files changed, 261 insertions(+), 85 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 33f108f5ad7..aadb1e9e1d6 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -160,9 +160,12 @@ static void insertNumber(IColumn & column, WhichDataType type, T value) } template -static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::NodePtr & root_node, const DataTypePtr & target_type) +static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::NodePtr & root_node, const DataTypePtr & target_type, bool is_fixed) { auto logical_type = root_node->logicalType(); + size_t fixed_size = 0; + if (is_fixed) + fixed_size = root_node->fixedSize(); const auto & decimal_type = assert_cast(*target_type); if (decimal_type.getScale() != static_cast(logical_type.scale()) || decimal_type.getPrecision() != static_cast(logical_type.precision())) throw Exception( @@ -174,14 +177,18 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No decimal_type.getScale(), decimal_type.getPrecision()); - return [tmp = std::string(), target_type](IColumn & column, avro::Decoder & decoder) mutable + return [tmp = std::vector(), target_type, fixed_size](IColumn & column, avro::Decoder & decoder) mutable { static constexpr size_t field_type_size = sizeof(typename DecimalType::FieldType); - decoder.decodeString(tmp); - if (tmp.size() > field_type_size) + if (fixed_size) + tmp = decoder.decodeFixed(fixed_size); + else + tmp = decoder.decodeBytes(); + + if (tmp.size() > field_type_size || tmp.empty()) throw ParsingException( ErrorCodes::CANNOT_PARSE_UUID, - "Cannot parse type {}, expected binary data with size equal to or less than {}, got {}", + "Cannot parse type {}, expected non-empty binary data with size equal to or less than {}, got {}", target_type->getName(), field_type_size, tmp.size()); @@ -189,10 +196,12 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No { /// Extent value to required size by adding padding. /// Check if value is negative or positive. + std::vector padding; if (tmp[0] & 128) - tmp = std::string(field_type_size - tmp.size(), 0xff) + tmp; + padding = std::vector(field_type_size - tmp.size(), 0xff); else - tmp = std::string(field_type_size - tmp.size(), 0) + tmp; + padding = std::vector(field_type_size - tmp.size(), 0); + tmp.insert(tmp.begin(), padding.begin(), padding.end()); } typename DecimalType::FieldType field; @@ -282,15 +291,15 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro }; } if (target.isDecimal32()) - return createDecimalDeserializeFn(root_node, target_type); + return createDecimalDeserializeFn(root_node, target_type, false); if (target.isDecimal64()) - return createDecimalDeserializeFn(root_node, target_type); + return createDecimalDeserializeFn(root_node, target_type, false); if (target.isDecimal128()) - return createDecimalDeserializeFn(root_node, target_type); + return createDecimalDeserializeFn(root_node, target_type, false); if (target.isDecimal256()) - return createDecimalDeserializeFn(root_node, target_type); + return createDecimalDeserializeFn(root_node, target_type, false); if (target.isDateTime64()) - return createDecimalDeserializeFn(root_node, target_type); + return createDecimalDeserializeFn(root_node, target_type, false); break; case avro::AVRO_INT: if (target_type->isValueRepresentedByNumber()) @@ -515,6 +524,29 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro return true; }; } + if (target.isUUID()) + { + return [tmp = std::vector(), fixed_size](IColumn & column, avro::Decoder & decoder) mutable + { + decoder.decodeFixed(fixed_size, tmp); + if (tmp.size() != 36) + throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse UUID from type Fixed, because it's size ({}) is not equal to the size of UUID (36)", fixed_size); + + const UUID uuid = parseUUID({reinterpret_cast(tmp.data()), tmp.size()}); + assert_cast(column).insertValue(uuid); + return true; + }; + } + if (target.isDecimal32()) + return createDecimalDeserializeFn(root_node, target_type, true); + if (target.isDecimal64()) + return createDecimalDeserializeFn(root_node, target_type, true); + if (target.isDecimal128()) + return createDecimalDeserializeFn(root_node, target_type, true); + if (target.isDecimal256()) + return createDecimalDeserializeFn(root_node, target_type, true); + if (target.isDateTime64()) + return createDecimalDeserializeFn(root_node, target_type, true); break; } case avro::AVRO_SYMBOLIC: @@ -1210,7 +1242,16 @@ DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ClickHouse supports only 8 and 16-bit Enum."); } case avro::Type::AVRO_FIXED: + { + auto logical_type = node->logicalType(); + if (logical_type.type() == avro::LogicalType::UUID) + return std::make_shared(); + + if (logical_type.type() == avro::LogicalType::DECIMAL) + return createDecimal(logical_type.precision(), logical_type.scale()); + return std::make_shared(node->fixedSize()); + } case avro::Type::AVRO_ARRAY: return std::make_shared(avroNodeToDataType(node->leafAt(0))); case avro::Type::AVRO_NULL: diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 1b6cde11be7..7eb425b7d3a 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -122,6 +122,8 @@ static DataTypePtr parseORCType(const orc::Type * orc_type, bool skip_columns_wi return std::make_shared(); case orc::TypeKind::TIMESTAMP: return std::make_shared(9); + case orc::TypeKind::TIMESTAMP_INSTANT: + return std::make_shared(9, "UTC"); case orc::TypeKind::VARCHAR: case orc::TypeKind::BINARY: case orc::TypeKind::STRING: @@ -795,7 +797,8 @@ static ColumnWithTypeAndName readColumnFromORCColumn( return readColumnWithNumericData(orc_column, orc_type, column_name); case orc::DATE: return readColumnWithDateData(orc_column, orc_type, column_name, type_hint); - case orc::TIMESTAMP: + case orc::TIMESTAMP: [[fallthrough]]; + case orc::TIMESTAMP_INSTANT: return readColumnWithTimestampData(orc_column, orc_type, column_name); case orc::DECIMAL: { auto interal_type = parseORCType(orc_type, false, skipped); diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp index b6a1745089e..2b8b7082515 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp @@ -2,34 +2,34 @@ #if USE_AWS_S3 && USE_AVRO -# include +#include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include -# include -# include -# include +#include +#include +#include namespace DB @@ -37,10 +37,10 @@ namespace DB namespace ErrorCodes { -extern const int FILE_DOESNT_EXIST; -extern const int ILLEGAL_COLUMN; -extern const int BAD_ARGUMENTS; -extern const int UNSUPPORTED_METHOD; + extern const int FILE_DOESNT_EXIST; + extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; + extern const int UNSUPPORTED_METHOD; } IcebergMetadata::IcebergMetadata( @@ -182,7 +182,7 @@ DataTypePtr getSimpleTypeByName(const String & type_name) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown Iceberg type: {}", type_name); } -DataTypePtr getFieldType(const Poco::Dynamic::Var & type, bool required); +DataTypePtr getFieldType(const Poco::JSON::Object::Ptr & field, const String & type_key, bool required); DataTypePtr getComplexTypeFromObject(const Poco::JSON::Object::Ptr & type) { @@ -190,15 +190,15 @@ DataTypePtr getComplexTypeFromObject(const Poco::JSON::Object::Ptr & type) if (type_name == "list") { bool element_required = type->getValue("element-required"); - auto element_type = getFieldType(type->get("element"), element_required); + auto element_type = getFieldType(type, "element", element_required); return std::make_shared(element_type); } if (type_name == "map") { - auto key_type = getFieldType(type->get("key"), true); + auto key_type = getFieldType(type, "key", true); auto value_required = type->getValue("value-required"); - auto value_type = getFieldType(type->get("value"), value_required); + auto value_type = getFieldType(type, "value", value_required); return std::make_shared(key_type, value_type); } @@ -214,7 +214,7 @@ DataTypePtr getComplexTypeFromObject(const Poco::JSON::Object::Ptr & type) auto field = fields->getObject(static_cast(i)); element_names.push_back(field->getValue("name")); auto required = field->getValue("required"); - element_types.push_back(getFieldType(field->get("type"), required)); + element_types.push_back(getFieldType(field, "type", required)); } return std::make_shared(element_types, element_names); @@ -223,8 +223,12 @@ DataTypePtr getComplexTypeFromObject(const Poco::JSON::Object::Ptr & type) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown Iceberg type: {}", type_name); } -DataTypePtr getFieldType(const Poco::Dynamic::Var & type, bool required) +DataTypePtr getFieldType(const Poco::JSON::Object::Ptr & field, const String & type_key, bool required) { + if (field->isObject(type_key)) + return getComplexTypeFromObject(field->getObject(type_key)); + + auto type = field->get(type_key); if (type.isString()) { const String & type_name = type.extract(); @@ -232,10 +236,8 @@ DataTypePtr getFieldType(const Poco::Dynamic::Var & type, bool required) return required ? data_type : makeNullable(data_type); } - if (type.isStruct()) - return getComplexTypeFromObject(type.extract()); - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected 'type' field: {}", type.toString()); + } std::pair parseTableSchema(const Poco::JSON::Object::Ptr & metadata_object, int format_version) @@ -252,10 +254,7 @@ std::pair parseTableSchema(const Poco::JSON::Object::P current_schema_id = metadata_object->getValue("current-schema-id"); auto schemas = metadata_object->get("schemas").extract(); if (schemas->size() != 1) - throw Exception( - ErrorCodes::UNSUPPORTED_METHOD, - "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not " - "supported"); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); /// Now we sure that there is only one schema. schema = schemas->getObject(0); @@ -269,10 +268,7 @@ std::pair parseTableSchema(const Poco::JSON::Object::P /// Field "schemas" is optional for version 1, but after version 2 was introduced, /// in most cases this field is added for new tables in version 1 as well. if (metadata_object->has("schemas") && metadata_object->get("schemas").extract()->size() > 1) - throw Exception( - ErrorCodes::UNSUPPORTED_METHOD, - "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not " - "supported"); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); } NamesAndTypesList names_and_types; @@ -282,13 +278,16 @@ std::pair parseTableSchema(const Poco::JSON::Object::P auto field = fields->getObject(static_cast(i)); auto name = field->getValue("name"); bool required = field->getValue("required"); - names_and_types.push_back({name, getFieldType(field->get("type"), required)}); + names_and_types.push_back({name, getFieldType(field, "type", required)}); } return {std::move(names_and_types), current_schema_id}; } -MutableColumns parseAvro(avro::DataFileReaderBase & file_reader, const Block & header, const FormatSettings & settings) +MutableColumns parseAvro( + avro::DataFileReaderBase & file_reader, + const Block & header, + const FormatSettings & settings) { auto deserializer = std::make_unique(header, file_reader.dataSchema(), true, true, settings); MutableColumns columns = header.cloneEmptyColumns(); @@ -313,7 +312,9 @@ std::pair getMetadataFileAndVersion(const StorageS3::Configuratio if (metadata_files.empty()) { throw Exception( - ErrorCodes::FILE_DOESNT_EXIST, "The metadata file for Iceberg table with path {} doesn't exist", configuration.url.key); + ErrorCodes::FILE_DOESNT_EXIST, + "The metadata file for Iceberg table with path {} doesn't exist", + configuration.url.key); } std::vector> metadata_files_with_versions; @@ -323,8 +324,7 @@ std::pair getMetadataFileAndVersion(const StorageS3::Configuratio String file_name(path.begin() + path.find_last_of('/') + 1, path.end()); String version_str(file_name.begin() + 1, file_name.begin() + file_name.find_first_of('.')); if (!std::all_of(version_str.begin(), version_str.end(), isdigit)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Bad metadata file name: {}. Expected vN.metadata.json where N is a number", file_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad metadata file name: {}. Expected vN.metadata.json where N is a number", file_name); metadata_files_with_versions.emplace_back(std::stoi(version_str), path); } @@ -364,8 +364,7 @@ std::unique_ptr parseIcebergMetadata(const StorageS3::Configura } } - return std::make_unique( - configuration, context_, metadata_version, format_version, manifest_list_file, schema_id, schema); + return std::make_unique(configuration, context_, metadata_version, format_version, manifest_list_file, schema_id, schema); } /** @@ -405,8 +404,7 @@ Strings IcebergMetadata::getDataFiles() LOG_TEST(log, "Collect manifest files from manifest list {}", manifest_list_file); auto manifest_list_buf = S3DataLakeMetadataReadHelper::createReadBuffer(manifest_list_file, getContext(), configuration); - auto manifest_list_file_reader - = std::make_unique(std::make_unique(*manifest_list_buf)); + auto manifest_list_file_reader = std::make_unique(std::make_unique(*manifest_list_buf)); auto data_type = AvroSchemaReader::avroNodeToDataType(manifest_list_file_reader->dataSchema().root()->leafAt(0)); Block header{{data_type->createColumn(), data_type, "manifest_path"}}; @@ -447,10 +445,7 @@ Strings IcebergMetadata::getDataFiles() Poco::Dynamic::Var json = parser.parse(schema_json_string); Poco::JSON::Object::Ptr schema_object = json.extract(); if (schema_object->getValue("schema-id") != current_schema_id) - throw Exception( - ErrorCodes::UNSUPPORTED_METHOD, - "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not " - "supported"); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not supported"); avro::NodePtr root_node = manifest_file_reader->dataSchema().root(); size_t leaves_num = root_node->leaves(); @@ -458,7 +453,9 @@ Strings IcebergMetadata::getDataFiles() if (leaves_num < expected_min_num) { throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Unexpected number of columns {}. Expected at least {}", root_node->leaves(), expected_min_num); + ErrorCodes::BAD_ARGUMENTS, + "Unexpected number of columns {}. Expected at least {}", + root_node->leaves(), expected_min_num); } avro::NodePtr status_node = root_node->leafAt(0); @@ -481,9 +478,9 @@ Strings IcebergMetadata::getDataFiles() auto status_col_data_type = AvroSchemaReader::avroNodeToDataType(status_node); auto data_col_data_type = AvroSchemaReader::avroNodeToDataType(data_file_node); - Block manifest_file_header{ - {status_col_data_type->createColumn(), status_col_data_type, "status"}, - {data_col_data_type->createColumn(), data_col_data_type, "data_file"}}; + Block manifest_file_header + = {{status_col_data_type->createColumn(), status_col_data_type, "status"}, + {data_col_data_type->createColumn(), data_col_data_type, "data_file"}}; columns = parseAvro(*manifest_file_reader, manifest_file_header, getFormatSettings(getContext())); if (columns.size() != 2) @@ -551,8 +548,7 @@ Strings IcebergMetadata::getDataFiles() { Int32 content_type = content_int_column->getElement(i); if (DataFileContent(content_type) != DataFileContent::DATA) - throw Exception( - ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: positional and equality deletes are not supported"); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: positional and equality deletes are not supported"); } const auto status = status_int_column->getInt(i); diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp index afc99d62cca..165ecce5142 100644 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp +++ b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp @@ -37,7 +37,7 @@ StorageIceberg::StorageIceberg( { } -ColumnsDescription StorageIceberg::getTableStructureFromMetadata( +ColumnsDescription StorageIceberg::getTableStructureFromData( Configuration & base_configuration, const std::optional &, ContextPtr local_context) diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.h b/src/Storages/DataLakes/Iceberg/StorageIceberg.h index e53bd241d7f..a18865b5a54 100644 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.h +++ b/src/Storages/DataLakes/Iceberg/StorageIceberg.h @@ -48,7 +48,7 @@ public: String getName() const override { return name; } - static ColumnsDescription getTableStructureFromMetadata( + static ColumnsDescription getTableStructureFromData( Configuration & base_configuration, const std::optional &, ContextPtr local_context); diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 89e711745b4..c26445d65ca 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -41,6 +41,7 @@ def get_spark(): .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") .config("spark.sql.catalog.spark_catalog.type", "hadoop") .config("spark.sql.catalog.spark_catalog.warehouse", "/iceberg_data") + .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") .master("local") ) return builder.master("local").getOrCreate() @@ -129,12 +130,12 @@ def generate_data(spark, start, end): return df -def create_iceberg_table(node, table_name): +def create_iceberg_table(node, table_name, format="Parquet"): node.query( f""" DROP TABLE IF EXISTS {table_name}; CREATE TABLE {table_name} - ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/')""" + ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format})""" ) @@ -165,7 +166,7 @@ def test_single_iceberg_file(started_cluster, format_version): bucket = started_cluster.minio_bucket TABLE_NAME = "test_single_iceberg_file_" + format_version - inserted_data = "SELECT number, toString(number) FROM numbers(100)" + inserted_data = "SELECT number, toString(number) as string FROM numbers(100)" parquet_data_path = create_initial_data_file( started_cluster, instance, inserted_data, TABLE_NAME ) @@ -308,7 +309,7 @@ def test_types(started_cluster, format_version): [ ["a", "Nullable(Int32)"], ["b", "Nullable(String)"], - ["c", "Nullable(Date32)"], + ["c", "Nullable(Date)"], ["d", "Array(Nullable(String))"], ["e", "Nullable(Bool)"], ] @@ -367,3 +368,138 @@ def test_delete_files(started_cluster, format_version): ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 50 + + +@pytest.mark.parametrize("format_version", ["1", "2"]) +def test_evolved_schema(started_cluster, format_version): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + bucket = started_cluster.minio_bucket + TABLE_NAME = "test_evolved_schema_" + format_version + + write_iceberg_from_df( + spark, + generate_data(spark, 0, 100), + TABLE_NAME, + mode="overwrite", + format_version=format_version, + ) + + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + create_iceberg_table(instance, TABLE_NAME) + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + + spark.sql(f"ALTER TABLE {TABLE_NAME} ADD COLUMNS (x bigint)") + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") + assert "UNSUPPORTED_METHOD" in error + +def test_row_based_deletes(started_cluster): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + bucket = started_cluster.minio_bucket + TABLE_NAME = "test_row_based_deletes" + + spark.sql( + f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" + ) + spark.sql(f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(100)") + + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + create_iceberg_table(instance, TABLE_NAME) + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + + spark.sql(f"DELETE FROM {TABLE_NAME} WHERE id < 10") + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") + assert "UNSUPPORTED_METHOD" in error + +@pytest.mark.parametrize("format_version", ["1", "2"]) +def test_schema_inference(started_cluster, format_version): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + bucket = started_cluster.minio_bucket + for format in ["Parquet", "ORC", "Avro"]: + TABLE_NAME = "test_schema_inference_" + format + "_" + format_version + + # Types time, timestamptz, fixed are not supported in Spark. + spark.sql( + f"CREATE TABLE {TABLE_NAME} (intC int, longC long, floatC float, doubleC double, decimalC1 decimal(10, 3), decimalC2 decimal(20, 10), decimalC3 decimal(38, 30), dateC date, timestampC timestamp, stringC string, binaryC binary, arrayC1 array, mapC1 map, structC1 struct, complexC array>>, field2: struct>>) USING iceberg TBLPROPERTIES ('format-version' = '{format_version}', 'write.format.default' = '{format}')" + ) + + spark.sql( + f"insert into {TABLE_NAME} select 42, 4242, 42.42, 4242.4242, decimal(42.42), decimal(42.42), decimal(42.42), date('2020-01-01'), timestamp('2020-01-01 20:00:00'), 'hello', binary('hello'), array(1,2,3), map('key', 'value'), struct(42, 'hello'), array(struct(map('key', array(map('key', 42))), struct(42, 'hello')))" + ) + + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + create_iceberg_table(instance, TABLE_NAME, format) + + res = instance.query(f"DESC {TABLE_NAME} FORMAT TSVRaw") + expected = TSV( + [ + ["intC", "Nullable(Int32)"], + ["longC", "Nullable(Int64)"], + ["floatC", "Nullable(Float32)"], + ["doubleC", "Nullable(Float64)"], + ["decimalC1", "Nullable(Decimal(10, 3))"], + ["decimalC2", "Nullable(Decimal(20, 10))"], + ["decimalC3", "Nullable(Decimal(38, 30))"], + ["dateC", "Nullable(Date)"], + ["timestampC", "Nullable(DateTime64(6, \'UTC\'))"], + ["stringC", "Nullable(String)"], + ["binaryC", "Nullable(String)"], + ["arrayC1", "Array(Nullable(Int32))"], + ["mapC1", "Map(String, Nullable(String))"], + ["structC1", "Tuple(field1 Nullable(Int32), field2 Nullable(String))"], + ["complexC", "Array(Tuple(field1 Map(String, Array(Map(String, Nullable(Int32)))), field2 Tuple(field3 Nullable(Int32), field4 Nullable(String))))"], + ] + ) + + assert res == expected + + # Check that we can parse data + instance.query(f"SELECT * FROM {TABLE_NAME}") + + +@pytest.mark.parametrize("format_version", ["1", "2"]) +def test_metadata_file_selection(started_cluster, format_version): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + bucket = started_cluster.minio_bucket + TABLE_NAME = "test_metadata_selection_" + format_version + + spark.sql( + f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" + ) + + for i in range(50): + spark.sql(f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(10)") + + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + create_iceberg_table(instance, TABLE_NAME) + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 From 610506febc8eae443ea79b7cc2fc00b4757d1e1e Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Tue, 17 Oct 2023 21:23:24 +0200 Subject: [PATCH 380/634] fix max intersection endian write --- src/AggregateFunctions/AggregateFunctionMaxIntersections.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 5074e491f60..41197c5ec0b 100644 --- a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -129,7 +129,10 @@ public: { writePODBinary(value[i].first, buf); writePODBinary(zero_padding, buf); - writePODBinary(value[i].second, buf); + if constexpr (std::endian::native == std::endian::little) + writePODBinary(value[i].second, buf); + else + writePODBinary(std::byteswap(value[i].second), buf); } } From 585e27009f0edcd41d88b3c2e0fb3706948c2893 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Tue, 17 Oct 2023 21:26:12 +0200 Subject: [PATCH 381/634] fix spaces --- src/AggregateFunctions/AggregateFunctionMaxIntersections.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 41197c5ec0b..fb333da3b85 100644 --- a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -129,9 +129,9 @@ public: { writePODBinary(value[i].first, buf); writePODBinary(zero_padding, buf); - if constexpr (std::endian::native == std::endian::little) - writePODBinary(value[i].second, buf); - else + if constexpr (std::endian::native == std::endian::little) + writePODBinary(value[i].second, buf); + else writePODBinary(std::byteswap(value[i].second), buf); } } From 4848baf302a5741e0fe31285672cc5a5c58efccf Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 21:08:55 +0200 Subject: [PATCH 382/634] Adjust performance artifacts prepare to the filtered checkout --- docker/packager/binary/build.sh | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 6a27e7adceb..cc2613cbaf5 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -172,10 +172,15 @@ then # This is why we add this repository snapshot from CI to the performance test # package. mkdir "$PERF_OUTPUT"/ch - git -C "$PERF_OUTPUT"/ch init --bare - git -C "$PERF_OUTPUT"/ch remote add origin /build - git -C "$PERF_OUTPUT"/ch fetch --no-tags --depth 50 origin HEAD:pr - git -C "$PERF_OUTPUT"/ch fetch --no-tags --depth 50 origin master:master + # Copy .git only, but skip modules, using tar + tar c -C /build/ --exclude='.git/modules/**' .git | tar x -C "$PERF_OUTPUT"/ch + # Create branch pr and origin/master to have them for the following performance comparison + git -C "$PERF_OUTPUT"/ch branch pr + git -C "$PERF_OUTPUT"/ch fetch --no-tags --depth 50 origin master:origin/master + # Clean remote, to not have it stale + git -C "$PERF_OUTPUT"/ch remote | xargs -n1 git -C "$PERF_OUTPUT"/ch remote remove + # And clean all tags + git -C "$PERF_OUTPUT"/ch tag | xargs git -C "$PERF_OUTPUT"/ch tag -d git -C "$PERF_OUTPUT"/ch reset --soft pr git -C "$PERF_OUTPUT"/ch log -5 ( From fc24e599fddeefbe4a5a136b263efd255ac9a29e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 18 Oct 2023 07:26:32 +0000 Subject: [PATCH 383/634] Check if id is nullptr --- src/Parsers/ASTPartition.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTPartition.cpp b/src/Parsers/ASTPartition.cpp index 22eecd37565..d08f9ae1ca5 100644 --- a/src/Parsers/ASTPartition.cpp +++ b/src/Parsers/ASTPartition.cpp @@ -36,8 +36,9 @@ String ASTPartition::getID(char delim) const { if (value) return "Partition"; - else - return "Partition_ID" + (delim + id->getID()); + + std::string id_string = id ? id->getID() : ""; + return "Partition_ID" + (delim + id_string); } ASTPtr ASTPartition::clone() const From 70d366a8ebb8d2483ff9e8ff75d468809516e5ff Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 18 Oct 2023 07:59:49 +0000 Subject: [PATCH 384/634] Control feature flag randomization with env --- docker/test/stateless/stress_tests.lib | 12 +++++++----- tests/ci/stress_check.py | 16 +++++++++++++++- 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index 11945b68f70..e1642517187 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -61,11 +61,13 @@ function configure() sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-server/config.d/keeper_port.xml } - # Randomize all Keeper feature flags - randomize_config_boolean_value filtered_list - randomize_config_boolean_value multi_read - randomize_config_boolean_value check_not_exists - randomize_config_boolean_value create_if_not_exists + if [[ -n "$RANDOMIZE_KEEPER_FEATURE_FLAGS" ]] && [[ "$RANDOMIZE_KEEPER_FEATURE_FLAGS" -eq 1 ]]; then + # Randomize all Keeper feature flags + randomize_config_boolean_value filtered_list + randomize_config_boolean_value multi_read + randomize_config_boolean_value check_not_exists + randomize_config_boolean_value create_if_not_exists + fi sudo chown clickhouse /etc/clickhouse-server/config.d/keeper_port.xml sudo chgrp clickhouse /etc/clickhouse-server/config.d/keeper_port.xml diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index b147b12f8ac..2ae734d4345 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -32,14 +32,25 @@ from tee_popen import TeePopen from upload_result_helper import upload_results +def get_additional_envs() -> List[str]: + result = [] + result.append(f"RANDOMIZE_KEEPER_FEATURE_FLAGS=1") + + return result + + def get_run_command( build_path: Path, result_path: Path, repo_tests_path: Path, server_log_path: Path, + additional_envs: List[str], ci_logs_args: str, image: DockerImage, ) -> str: + envs = [f"-e {e}" for e in additional_envs] + env_str = " ".join(envs) + cmd = ( "docker run --cap-add=SYS_PTRACE " # For dmesg and sysctl @@ -50,7 +61,7 @@ def get_run_command( f"--volume={build_path}:/package_folder " f"--volume={result_path}:/test_output " f"--volume={repo_tests_path}:/usr/share/clickhouse-test " - f"--volume={server_log_path}:/var/log/clickhouse-server {image} " + f"--volume={server_log_path}:/var/log/clickhouse-server {env_str} {image} " ) return cmd @@ -148,11 +159,14 @@ def run_stress_test(docker_image_name: str) -> None: pr_info, stopwatch.start_time_str, check_name ) + additional_envs = get_additional_envs() + run_command = get_run_command( packages_path, result_path, repo_tests_path, server_log_path, + additional_envs, ci_logs_args, docker_image, ) From be7be17352c30ed37a760ea87089e05958a24f9b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 18 Oct 2023 10:22:18 +0200 Subject: [PATCH 385/634] add comment --- tests/ci/stress_check.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 2ae734d4345..75c425f6231 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -34,6 +34,9 @@ from upload_result_helper import upload_results def get_additional_envs() -> List[str]: result = [] + # some cloud-specific features require feature flags enabled + # so we need this ENV to be able to disable the randomization + # of feature flags result.append(f"RANDOMIZE_KEEPER_FEATURE_FLAGS=1") return result From 0b52d29cd81e35f266a725c0c2ad8580ff28115b Mon Sep 17 00:00:00 2001 From: huzhicheng <872237106@qq.com> Date: Wed, 18 Oct 2023 16:56:17 +0800 Subject: [PATCH 386/634] Query Cache: Only cache initial query --- src/Interpreters/executeQuery.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index b96e05c1f9a..07456a31eeb 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -988,7 +988,11 @@ static std::tuple executeQueryImpl( } QueryCachePtr query_cache = context->getQueryCache(); - const bool can_use_query_cache = query_cache != nullptr && settings.use_query_cache && !internal && (ast->as() || ast->as()); + const bool can_use_query_cache = query_cache != nullptr + && settings.use_query_cache + && !internal + && client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY + && (ast->as() || ast->as()); QueryCache::Usage query_cache_usage = QueryCache::Usage::None; if (!async_insert) From 71adb434e33e8c284959b53680a6a9bd89465a72 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 18 Oct 2023 09:19:09 +0000 Subject: [PATCH 387/634] Remove f-string --- tests/ci/stress_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 75c425f6231..56c96dc2338 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -37,7 +37,7 @@ def get_additional_envs() -> List[str]: # some cloud-specific features require feature flags enabled # so we need this ENV to be able to disable the randomization # of feature flags - result.append(f"RANDOMIZE_KEEPER_FEATURE_FLAGS=1") + result.append("RANDOMIZE_KEEPER_FEATURE_FLAGS=1") return result From fff6a8dcc80243b38c90b4d4fd8013d4a6278a01 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 18 Oct 2023 11:35:53 +0200 Subject: [PATCH 388/634] Temporarily disable flaky test --- .../test_postgresql_replica_database_engine_2/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 2a1c7634698..15625e6d8a4 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -759,7 +759,7 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) order_by="id", ) - +@pytest.mark.skip(reason="flaky test, will fix soon") def test_replica_consumer(started_cluster): table = "test_replica_consumer" pg_manager_instance2.restart() From 2d638d64706ce426bcdcce3043083619df708525 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 18 Oct 2023 03:47:25 +0000 Subject: [PATCH 389/634] rocksdb: not creating block during writing Signed-off-by: Duc Canh Le --- src/Storages/RocksDB/EmbeddedRocksDBSink.cpp | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp index b79c7284843..c451cfd1bf5 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp @@ -32,7 +32,7 @@ EmbeddedRocksDBSink::EmbeddedRocksDBSink( void EmbeddedRocksDBSink::consume(Chunk chunk) { auto rows = chunk.getNumRows(); - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + const auto & columns = chunk.getColumns(); WriteBufferFromOwnString wb_key; WriteBufferFromOwnString wb_value; @@ -44,12 +44,9 @@ void EmbeddedRocksDBSink::consume(Chunk chunk) wb_key.restart(); wb_value.restart(); - size_t idx = 0; - for (const auto & elem : block) - { - serializations[idx]->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value, {}); - ++idx; - } + for (size_t idx = 0; idx < columns.size(); ++idx) + serializations[idx]->serializeBinary(*columns[idx], i, idx == primary_key_pos ? wb_key : wb_value, {}); + status = batch.Put(wb_key.str(), wb_value.str()); if (!status.ok()) throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); From babbdd9202466f0007d85d780097ed74733fb01f Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 12:08:47 +0200 Subject: [PATCH 390/634] Fxi --- .../test_postgresql_replica_database_engine_2/test.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 15625e6d8a4..7ae8c9b8713 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -759,13 +759,16 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) order_by="id", ) -@pytest.mark.skip(reason="flaky test, will fix soon") def test_replica_consumer(started_cluster): table = "test_replica_consumer" pg_manager_instance2.restart() + pg_manager.create_postgres_table(table) + instance.query( + f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(0, 50)" + ) + for pm in [pg_manager, pg_manager_instance2]: - pm.create_and_fill_postgres_table(table) pm.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, From 33118442ddf248656bf2738405d6b82e85dd87c7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 13:01:35 +0200 Subject: [PATCH 391/634] Fix black check --- .../test_postgresql_replica_database_engine_2/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 7ae8c9b8713..e8053730c44 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -759,6 +759,7 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) order_by="id", ) + def test_replica_consumer(started_cluster): table = "test_replica_consumer" pg_manager_instance2.restart() From 663c2611eefabf34e2d1c7215eced5d02ca96610 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 13:05:29 +0200 Subject: [PATCH 392/634] Fix --- .../test_postgresql_replica_database_engine_2/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 15625e6d8a4..eee5c9624a7 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -759,6 +759,7 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) order_by="id", ) + @pytest.mark.skip(reason="flaky test, will fix soon") def test_replica_consumer(started_cluster): table = "test_replica_consumer" From be9e84c19407aa1c06cb3dd3a3aa4a6fbb8b7f7d Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 18 Oct 2023 11:09:39 +0000 Subject: [PATCH 393/634] Fix style --- .../integration/test_storage_iceberg/test.py | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index c26445d65ca..11198a7175b 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -41,7 +41,10 @@ def get_spark(): .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") .config("spark.sql.catalog.spark_catalog.type", "hadoop") .config("spark.sql.catalog.spark_catalog.warehouse", "/iceberg_data") - .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") + .config( + "spark.sql.extensions", + "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions", + ) .master("local") ) return builder.master("local").getOrCreate() @@ -402,6 +405,7 @@ def test_evolved_schema(started_cluster, format_version): error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") assert "UNSUPPORTED_METHOD" in error + def test_row_based_deletes(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -412,7 +416,9 @@ def test_row_based_deletes(started_cluster): spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" ) - spark.sql(f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(100)") + spark.sql( + f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(100)" + ) files = upload_directory( minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" @@ -430,6 +436,7 @@ def test_row_based_deletes(started_cluster): error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") assert "UNSUPPORTED_METHOD" in error + @pytest.mark.parametrize("format_version", ["1", "2"]) def test_schema_inference(started_cluster, format_version): instance = started_cluster.instances["node1"] @@ -465,13 +472,16 @@ def test_schema_inference(started_cluster, format_version): ["decimalC2", "Nullable(Decimal(20, 10))"], ["decimalC3", "Nullable(Decimal(38, 30))"], ["dateC", "Nullable(Date)"], - ["timestampC", "Nullable(DateTime64(6, \'UTC\'))"], + ["timestampC", "Nullable(DateTime64(6, 'UTC'))"], ["stringC", "Nullable(String)"], ["binaryC", "Nullable(String)"], ["arrayC1", "Array(Nullable(Int32))"], ["mapC1", "Map(String, Nullable(String))"], ["structC1", "Tuple(field1 Nullable(Int32), field2 Nullable(String))"], - ["complexC", "Array(Tuple(field1 Map(String, Array(Map(String, Nullable(Int32)))), field2 Tuple(field3 Nullable(Int32), field4 Nullable(String))))"], + [ + "complexC", + "Array(Tuple(field1 Map(String, Array(Map(String, Nullable(Int32)))), field2 Tuple(field3 Nullable(Int32), field4 Nullable(String))))", + ], ] ) @@ -494,7 +504,9 @@ def test_metadata_file_selection(started_cluster, format_version): ) for i in range(50): - spark.sql(f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(10)") + spark.sql( + f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(10)" + ) files = upload_directory( minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" From 2eabb260a7f95b7d0e99cfe999cbe6531ca3ed7a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 18 Oct 2023 12:09:22 +0000 Subject: [PATCH 394/634] Correctly grep fuzzer.log --- docker/test/fuzzer/run-fuzzer.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index e56fe6ca83a..326f2a751e6 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -337,8 +337,8 @@ quit # which is confusing. task_exit_code=$fuzzer_exit_code echo "failure" > status.txt - { rg --text -o "Found error:.*" fuzzer.log \ - || rg --text -ao "Exception:.*" fuzzer.log \ + { rg -ao "Found error:.*" fuzzer.log \ + || rg -ao "Exception:.*" fuzzer.log \ || echo "Fuzzer failed ($fuzzer_exit_code). See the logs." ; } \ | tail -1 > description.txt fi From 726c370d087ad47348f504a167dac32225b109f8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 11:30:24 +0200 Subject: [PATCH 395/634] Replace temporary REPORITORY url by a proper one --- docker/keeper/Dockerfile | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 0e10068d79a..4dd231ae305 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -2,8 +2,8 @@ # If the image is built from Dockerfile.alpine, then the `-alpine` suffix is added automatically, # so the only purpose of Dockerfile.ubuntu is to push `latest`, `head` and so on w/o suffixes FROM ubuntu:20.04 AS glibc-donor - ARG TARGETARCH + RUN arch=${TARGETARCH:-amd64} \ && case $arch in \ amd64) rarch=x86_64 ;; \ @@ -31,7 +31,9 @@ RUN arch=${TARGETARCH:-amd64} \ arm64) ln -sf /lib/ld-2.31.so /lib/ld-linux-aarch64.so.1 ;; \ esac -ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" +# lts / testing / prestable / etc +ARG REPO_CHANNEL="stable" +ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" ARG VERSION="23.9.1.1854" ARG PACKAGES="clickhouse-keeper" From 01fa15c33fe79614479fb8a9b188a9d45f4ab25f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 18 Oct 2023 12:20:14 +0000 Subject: [PATCH 396/634] Less copying while handling annoucement --- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- .../ParallelReplicasReadingCoordinator.cpp | 13 ++++--------- 2 files changed, 5 insertions(+), 10 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 8f4e90ea3d2..da40c197b47 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -574,7 +574,7 @@ void RemoteQueryExecutor::processMergeTreeInitialReadAnnouncement(InitialAllRang if (!extension || !extension->parallel_reading_coordinator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Coordinator for parallel reading from replicas is not initialized"); - extension->parallel_reading_coordinator->handleInitialAllRangesAnnouncement(announcement); + extension->parallel_reading_coordinator->handleInitialAllRangesAnnouncement(std::move(announcement)); } void RemoteQueryExecutor::finish() diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 57cd91cc995..bae3a36a779 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -385,7 +385,7 @@ void InOrderCoordinator::handleInitialAllRangesAnnouncement(InitialAllRang LOG_TRACE(log, "Received an announcement {}", announcement.describe()); /// To get rid of duplicates - for (const auto & part: announcement.description) + for (auto && part: announcement.description) { auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), [&part] (const Part & other) { return other.description.info == part.info; }); @@ -404,13 +404,8 @@ void InOrderCoordinator::handleInitialAllRangesAnnouncement(InitialAllRang if (covering_or_the_same_it != all_parts_to_read.end()) continue; - auto new_part = Part{ - .description = part, - .replicas = {announcement.replica_num} - }; - - auto insert_it = all_parts_to_read.insert(new_part); - auto & ranges = insert_it.first->description.ranges; + auto [inserted_it, _] = all_parts_to_read.emplace(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); + auto & ranges = inserted_it->description.ranges; std::sort(ranges.begin(), ranges.end()); } } @@ -517,7 +512,7 @@ void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(Init } - return pimpl->handleInitialAllRangesAnnouncement(announcement); + return pimpl->handleInitialAllRangesAnnouncement(std::move(announcement)); } ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelReadRequest request) From 8b49a9471357a0f5aa847350ecd532cb0cb7bffa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 11:34:12 +0200 Subject: [PATCH 397/634] Check sha512 sums for alpine server and keeper images --- docker/keeper/Dockerfile | 18 ++++++++---------- docker/server/Dockerfile.alpine | 19 ++++++++----------- 2 files changed, 16 insertions(+), 21 deletions(-) diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 4dd231ae305..77a8977145a 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -48,16 +48,14 @@ ARG PACKAGES="clickhouse-keeper" ARG TARGETARCH RUN arch=${TARGETARCH:-amd64} \ && for package in ${PACKAGES}; do \ - { \ - { echo "Get ${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" \ - && wget -c -q "${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" -O "/tmp/${package}-${VERSION}-${arch}.tgz" \ - && tar xvzf "/tmp/${package}-${VERSION}-${arch}.tgz" --strip-components=1 -C / ; \ - } || \ - { echo "Fallback to ${REPOSITORY}/${package}-${VERSION}.tgz" \ - && wget -c -q "${REPOSITORY}/${package}-${VERSION}.tgz" -O "/tmp/${package}-${VERSION}.tgz" \ - && tar xvzf "/tmp/${package}-${VERSION}.tgz" --strip-components=2 -C / ; \ - } ; \ - } || exit 1 \ + ( \ + cd /tmp \ + && echo "Get ${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" \ + && wget -c -q "${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" \ + && wget -c -q "${REPOSITORY}/${package}-${VERSION}-${arch}.tgz.sha512" \ + && sed 's:/output/:/tmp/:' < "${package}-${VERSION}-${arch}.tgz.sha512" | sha512sum -c \ + && tar xvzf "${package}-${VERSION}-${arch}.tgz" --strip-components=1 -C / \ + ) \ ; done \ && rm /tmp/*.tgz /install -r \ && addgroup -S -g 101 clickhouse \ diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index afe1be75bed..de553b748a0 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -23,7 +23,6 @@ COPY docker_related_config.xml /etc/clickhouse-server/config.d/ COPY entrypoint.sh /entrypoint.sh ARG TARGETARCH - RUN arch=${TARGETARCH:-amd64} \ && case $arch in \ amd64) mkdir -p /lib64 && ln -sf /lib/ld-2.31.so /lib64/ld-linux-x86-64.so.2 ;; \ @@ -45,16 +44,14 @@ ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" RUN arch=${TARGETARCH:-amd64} \ && for package in ${PACKAGES}; do \ - { \ - { echo "Get ${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" \ - && wget -c -q "${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" -O "/tmp/${package}-${VERSION}-${arch}.tgz" \ - && tar xvzf "/tmp/${package}-${VERSION}-${arch}.tgz" --strip-components=1 -C / ; \ - } || \ - { echo "Fallback to ${REPOSITORY}/${package}-${VERSION}.tgz" \ - && wget -c -q "${REPOSITORY}/${package}-${VERSION}.tgz" -O "/tmp/${package}-${VERSION}.tgz" \ - && tar xvzf "/tmp/${package}-${VERSION}.tgz" --strip-components=2 -C / ; \ - } ; \ - } || exit 1 \ + ( \ + cd /tmp \ + && echo "Get ${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" \ + && wget -c -q "${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" \ + && wget -c -q "${REPOSITORY}/${package}-${VERSION}-${arch}.tgz.sha512" \ + && sed 's:/output/:/tmp/:' < "${package}-${VERSION}-${arch}.tgz.sha512" | sha512sum -c \ + && tar xvzf "${package}-${VERSION}-${arch}.tgz" --strip-components=1 -C / \ + ) \ ; done \ && rm /tmp/*.tgz /install -r \ && addgroup -S -g 101 clickhouse \ From e4e1aac1548805cdfed9894b3078f690ddb7a143 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 11:34:57 +0200 Subject: [PATCH 398/634] Move comment about created user to a proper place --- docker/server/Dockerfile.ubuntu | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 402168c3e15..eef4fbf7c74 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -5,6 +5,13 @@ ARG DEBIAN_FRONTEND=noninteractive # ARG for quick switch to a given ubuntu mirror ARG apt_archive="http://archive.ubuntu.com" + +# user/group precreated explicitly with fixed uid/gid on purpose. +# It is especially important for rootless containers: in that case entrypoint +# can't do chown and owners of mounted volumes should be configured externally. +# We do that in advance at the begining of Dockerfile before any packages will be +# installed to prevent picking those uid / gid by some unrelated software. +# The same uid / gid (101) is used both for alpine and ubuntu. RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list \ && groupadd -r clickhouse --gid=101 \ && useradd -r -g clickhouse --uid=101 --home-dir=/var/lib/clickhouse --shell=/bin/bash clickhouse \ @@ -35,13 +42,6 @@ ARG deb_location_url="" # from a single binary url (useful for non-standard builds - with sanitizers, for arm64). ARG single_binary_location_url="" -# user/group precreated explicitly with fixed uid/gid on purpose. -# It is especially important for rootless containers: in that case entrypoint -# can't do chown and owners of mounted volumes should be configured externally. -# We do that in advance at the begining of Dockerfile before any packages will be -# installed to prevent picking those uid / gid by some unrelated software. -# The same uid / gid (101) is used both for alpine and ubuntu. - ARG TARGETARCH # install from a web location with deb packages From af31224ecd0c6c530ee2f38b0aed0e9b306e5135 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 14:55:57 +0200 Subject: [PATCH 399/634] Write only filename to sha512 files for tarballs --- packages/build | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/packages/build b/packages/build index cc089dd08ad..c2285b8ee7c 100755 --- a/packages/build +++ b/packages/build @@ -112,7 +112,8 @@ EOF tar -czf "$TARBALL" -C "$OUTPUT_DIR" "$PKG_DIR" fi - sha512sum "$TARBALL" > "$TARBALL".sha512 + # Cut the $OUTPUT_DIR/ from the sha512sum output to make it universal + sha512sum "$TARBALL" | sed "s|$OUTPUT_DIR/||" > "$TARBALL".sha512 rm -r "$PKG_PATH" } From c4bad255b711a4cd32eae643fa992747c6a4e1bf Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 14:37:33 +0200 Subject: [PATCH 400/634] Fix --- src/Storages/S3Queue/StorageS3Queue.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 92f15aed62f..72e74d3c2a0 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -354,16 +354,24 @@ bool StorageS3Queue::streamToViews() auto file_iterator = createFileIterator(s3queue_context, nullptr); Pipes pipes; + pipes.reserve(s3queue_settings->s3queue_processing_threads_num); for (size_t i = 0; i < s3queue_settings->s3queue_processing_threads_num; ++i) { - auto source = createSource(file_iterator, block_io.pipeline.getHeader().getNames(), storage_snapshot, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); + auto source = createSource( + file_iterator, block_io.pipeline.getHeader().getNames(), + storage_snapshot, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); + pipes.emplace_back(std::move(source)); } auto pipe = Pipe::unitePipes(std::move(pipes)); - std::atomic_size_t rows = 0; block_io.pipeline.complete(std::move(pipe)); + block_io.pipeline.setNumThreads(s3queue_settings->s3queue_processing_threads_num); + block_io.pipeline.setConcurrencyControl(s3queue_context->getSettingsRef().use_concurrency_control); + + std::atomic_size_t rows = 0; block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); + CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); From c7f78ac73a670b8e4421ae33af8d9a984930c89d Mon Sep 17 00:00:00 2001 From: Sinan Date: Wed, 18 Oct 2023 18:17:18 +0530 Subject: [PATCH 401/634] Add aggregation function for Largest-Triangle-Three-Buckets (#53145) * Added a simple lttb aggregate function * Added support for multiple datatypes * Added support for Date and Date32, updated LTTBData struct * Updated code to handle bucket size 0 and 1 * Added sort for LTTBData * Added tests and documentation * Added some code style fixes * Added function to new func ref file * Removed function from new func ref file * Apply suggestions from code review * Updated unit tests * updated LTTB data code * Minor style fixes * Updated code with std sort * updated tests * Renamed lttb to largestTriangleThreeBuckets * Added alias lttb --------- Co-authored-by: Alexey Milovidov --- .../aggregate-functions/reference/index.md | 1 + .../reference/largestTriangleThreeBuckets.md | 67 ++++ ...ateFunctionLargestTriangleThreeBuckets.cpp | 52 +++ ...egateFunctionLargestTriangleThreeBuckets.h | 327 ++++++++++++++++++ .../registerAggregateFunctions.cpp | 2 + ...eThreeBuckets_aggregate_function.reference | 7 + ...riangleThreeBuckets_aggregate_function.sql | 47 +++ .../aspell-ignore/en/aspell-dict.txt | 3 + 8 files changed, 506 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets.md create mode 100644 src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.h create mode 100644 tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.reference create mode 100644 tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.sql diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index 6c56aefd51d..b1f2c5bacbb 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -103,4 +103,5 @@ ClickHouse-specific aggregate functions: - [quantileInterpolatedWeighted](./quantileinterpolatedweighted.md) - [sparkBar](./sparkbar.md) - [sumCount](./sumcount.md) +- [largestTriangleThreeBuckets](./largestTriangleThreeBuckets.md) diff --git a/docs/en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets.md b/docs/en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets.md new file mode 100644 index 00000000000..ca033bb48c8 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets.md @@ -0,0 +1,67 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets +sidebar_position: 312 +sidebar_label: largestTriangleThreeBuckets +--- + +# largestTriangleThreeBuckets + +Applies the [Largest-Triangle-Three-Buckets](https://skemman.is/bitstream/1946/15343/3/SS_MSthesis.pdf) algorithm to the input data. +The algorithm is used for downsampling time series data for visualization. It is designed to operate on series sorted by x coordinate. +It works by dividing the sorted series into buckets and then finding the largest triangle in each bucket. The number of buckets is equal to the number of points in the resulting series. +the function will sort data by `x` and then apply the downsampling algorithm to the sorted data. + +**Syntax** + +``` sql +largestTriangleThreeBuckets(n)(x, y) +``` + +Alias: `lttb`. + +**Arguments** + +- `x` — x coordinate. [Integer](../../../sql-reference/data-types/int-uint.md) , [Float](../../../sql-reference/data-types/float.md) , [Decimal](../../../sql-reference/data-types/decimal.md) , [Date](../../../sql-reference/data-types/date.md), [Date32](../../../sql-reference/data-types/date32.md), [DateTime](../../../sql-reference/data-types/datetime.md), [DateTime64](../../../sql-reference/data-types/datetime64.md). +- `y` — y coordinate. [Integer](../../../sql-reference/data-types/int-uint.md) , [Float](../../../sql-reference/data-types/float.md) , [Decimal](../../../sql-reference/data-types/decimal.md) , [Date](../../../sql-reference/data-types/date.md), [Date32](../../../sql-reference/data-types/date32.md), [DateTime](../../../sql-reference/data-types/datetime.md), [DateTime64](../../../sql-reference/data-types/datetime64.md). + +**Parameters** + +- `n` — number of points in the resulting series. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Returned values** + +[Array](../../../sql-reference/data-types/array.md) of [Tuple](../../../sql-reference/data-types/tuple.md) with two elements: + +**Example** + +Input table: + +``` text +┌─────x───────┬───────y──────┐ +│ 1.000000000 │ 10.000000000 │ +│ 2.000000000 │ 20.000000000 │ +│ 3.000000000 │ 15.000000000 │ +│ 8.000000000 │ 60.000000000 │ +│ 9.000000000 │ 55.000000000 │ +│ 10.00000000 │ 70.000000000 │ +│ 4.000000000 │ 30.000000000 │ +│ 5.000000000 │ 40.000000000 │ +│ 6.000000000 │ 35.000000000 │ +│ 7.000000000 │ 50.000000000 │ +└─────────────┴──────────────┘ +``` + +Query: + +``` sql +SELECT largestTriangleThreeBuckets(4)(x, y) FROM largestTriangleThreeBuckets_test; +``` + +Result: + +``` text +┌────────largestTriangleThreeBuckets(3)(x, y)───────────┐ +│ [(1,10),(3,15),(5,40),(10,70)] │ +└───────────────────────────────────────────────────────┘ +``` + diff --git a/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp new file mode 100644 index 00000000000..9052f7a6661 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp @@ -0,0 +1,52 @@ +#include +#include +#include +#include + + +namespace ErrorCodes +{ +extern const int NOT_IMPLEMENTED; +} + +namespace DB +{ +struct Settings; + +namespace +{ + + AggregateFunctionPtr + createAggregateFunctionLargestTriangleThreeBuckets(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + { + assertBinary(name, argument_types); + + + if (!(isNumber(argument_types[0]) || isDateOrDate32(argument_types[0]) || isDateTime(argument_types[0]) + || isDateTime64(argument_types[0]))) + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Aggregate function {} only supports Date, Date32, DateTime, DateTime64 and Number as the first argument", + name); + + if (!(isNumber(argument_types[1]) || isDateOrDate32(argument_types[1]) || isDateTime(argument_types[1]) + || isDateTime64(argument_types[1]))) + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Aggregate function {} only supports Date, Date32, DateTime, DateTime64 and Number as the second argument", + name); + + return std::make_shared(argument_types, parameters); + } + +} + + +void registerAggregateFunctionLargestTriangleThreeBuckets(AggregateFunctionFactory & factory) +{ + factory.registerFunction(AggregateFunctionLargestTriangleThreeBuckets::name, createAggregateFunctionLargestTriangleThreeBuckets); + factory.registerAlias("lttb", AggregateFunctionLargestTriangleThreeBuckets::name); +} + + +} diff --git a/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.h b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.h new file mode 100644 index 00000000000..0f2e888e967 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.h @@ -0,0 +1,327 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +struct Settings; + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +struct LargestTriangleThreeBucketsData : public StatisticalSample +{ + void add(const Float64 xval, const Float64 yval, Arena * arena) + { + this->addX(xval, arena); + this->addY(yval, arena); + } + + void sort(Arena * arena) + { + // sort the this->x and this->y in ascending order of this->x using index + std::vector index(this->x.size()); + + std::iota(index.begin(), index.end(), 0); + ::sort(index.begin(), index.end(), [&](size_t i1, size_t i2) { return this->x[i1] < this->x[i2]; }); + + SampleX temp_x{}; + SampleY temp_y{}; + + for (size_t i = 0; i < this->x.size(); ++i) + { + temp_x.push_back(this->x[index[i]], arena); + temp_y.push_back(this->y[index[i]], arena); + } + + for (size_t i = 0; i < this->x.size(); ++i) + { + this->x[i] = temp_x[i]; + this->y[i] = temp_y[i]; + } + } + + PODArray> getResult(size_t total_buckets, Arena * arena) + { + // Sort the data + this->sort(arena); + + PODArray> result; + + // Handle special cases for small data list + if (this->x.size() <= total_buckets) + { + for (size_t i = 0; i < this->x.size(); ++i) + { + result.emplace_back(std::make_pair(this->x[i], this->y[i])); + } + return result; + } + + // Handle special cases for 0 or 1 or 2 buckets + if (total_buckets == 0) + return result; + if (total_buckets == 1) + { + result.emplace_back(std::make_pair(this->x.front(), this->y.front())); + return result; + } + if (total_buckets == 2) + { + result.emplace_back(std::make_pair(this->x.front(), this->y.front())); + result.emplace_back(std::make_pair(this->x.back(), this->y.back())); + return result; + } + + // Find the size of each bucket + size_t single_bucket_size = this->x.size() / total_buckets; + + // Include the first data point + result.emplace_back(std::make_pair(this->x[0], this->y[0])); + + for (size_t i = 1; i < total_buckets - 1; ++i) // Skip the first and last bucket + { + size_t start_index = i * single_bucket_size; + size_t end_index = (i + 1) * single_bucket_size; + + // Compute the average point in the next bucket + Float64 avg_x = 0; + Float64 avg_y = 0; + for (size_t j = end_index; j < (i + 2) * single_bucket_size; ++j) + { + avg_x += this->x[j]; + avg_y += this->y[j]; + } + avg_x /= single_bucket_size; + avg_y /= single_bucket_size; + + // Find the point in the current bucket that forms the largest triangle + size_t max_index = start_index; + Float64 max_area = 0.0; + for (size_t j = start_index; j < end_index; ++j) + { + Float64 area = std::abs( + 0.5 + * (result.back().first * this->y[j] + this->x[j] * avg_y + avg_x * result.back().second - result.back().first * avg_y + - this->x[j] * result.back().second - avg_x * this->y[j])); + if (area > max_area) + { + max_area = area; + max_index = j; + } + } + + // Include the selected point + result.emplace_back(std::make_pair(this->x[max_index], this->y[max_index])); + } + + // Include the last data point + result.emplace_back(std::make_pair(this->x.back(), this->y.back())); + + return result; + } +}; + +class AggregateFunctionLargestTriangleThreeBuckets final : public IAggregateFunctionDataHelper +{ +private: + UInt64 total_buckets{0}; + TypeIndex x_type; + TypeIndex y_type; + +public: + explicit AggregateFunctionLargestTriangleThreeBuckets(const DataTypes & arguments, const Array & params) + : IAggregateFunctionDataHelper({arguments}, {}, createResultType(arguments)) + { + if (params.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require one parameter", getName()); + + if (params[0].getType() != Field::Types::UInt64) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a UInt64", getName()); + + total_buckets = params[0].get(); + + this->x_type = WhichDataType(arguments[0]).idx; + this->y_type = WhichDataType(arguments[1]).idx; + } + + static constexpr auto name = "largestTriangleThreeBuckets"; + + String getName() const override { return name; } + + bool allocatesMemoryInArena() const override { return true; } + + static DataTypePtr createResultType(const DataTypes & arguments) + { + TypeIndex x_type = arguments[0]->getTypeId(); + TypeIndex y_type = arguments[1]->getTypeId(); + + UInt32 x_scale = 0; + UInt32 y_scale = 0; + + if (const auto * datetime64_type = typeid_cast(arguments[0].get())) + { + x_scale = datetime64_type->getScale(); + } + + if (const auto * datetime64_type = typeid_cast(arguments[1].get())) + { + y_scale = datetime64_type->getScale(); + } + + DataTypes types = {getDataTypeFromTypeIndex(x_type, x_scale), getDataTypeFromTypeIndex(y_type, y_scale)}; + + auto tuple = std::make_shared(std::move(types)); + + return std::make_shared(tuple); + } + + static DataTypePtr getDataTypeFromTypeIndex(TypeIndex type_index, UInt32 scale) + { + DataTypePtr data_type; + switch (type_index) + { + case TypeIndex::Date: + data_type = std::make_shared(); + break; + case TypeIndex::Date32: + data_type = std::make_shared(); + break; + case TypeIndex::DateTime: + data_type = std::make_shared(); + break; + case TypeIndex::DateTime64: + data_type = std::make_shared(scale); + break; + default: + data_type = std::make_shared>(); + } + return data_type; + } + + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + Float64 x = getFloat64DataFromColumn(columns[0], row_num, this->x_type); + Float64 y = getFloat64DataFromColumn(columns[1], row_num, this->y_type); + this->data(place).add(x, y, arena); + } + + Float64 getFloat64DataFromColumn(const IColumn * column, size_t row_num, TypeIndex type_index) const + { + switch (type_index) + { + case TypeIndex::Date: + return static_cast(*column).getData()[row_num]; + case TypeIndex::Date32: + return static_cast(*column).getData()[row_num]; + case TypeIndex::DateTime: + return static_cast(*column).getData()[row_num]; + case TypeIndex::DateTime64: + return static_cast(*column).getData()[row_num]; + default: + return column->getFloat64(row_num); + } + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override + { + auto & a = this->data(place); + const auto & b = this->data(rhs); + + a.merge(b, arena); + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + { + this->data(place).write(buf); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override + { + this->data(place).read(buf, arena); + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * arena) const override + { + auto res = this->data(place).getResult(total_buckets, arena); + + auto & col = assert_cast(to); + auto & col_offsets = assert_cast(col.getOffsetsColumn()); + + auto column_x_adder_func = getColumnAdderFunc(x_type); + auto column_y_adder_func = getColumnAdderFunc(y_type); + + for (size_t i = 0; i < res.size(); ++i) + { + auto & column_tuple = assert_cast(col.getData()); + column_x_adder_func(column_tuple.getColumn(0), res[i].first); + column_y_adder_func(column_tuple.getColumn(1), res[i].second); + } + + col_offsets.getData().push_back(col.getData().size()); + } + + std::function getColumnAdderFunc(TypeIndex type_index) const + { + switch (type_index) + { + case TypeIndex::Date: + return [](IColumn & column, Float64 value) + { + auto & col = assert_cast(column); + col.getData().push_back(static_cast(value)); + }; + case TypeIndex::Date32: + return [](IColumn & column, Float64 value) + { + auto & col = assert_cast(column); + col.getData().push_back(static_cast(value)); + }; + case TypeIndex::DateTime: + return [](IColumn & column, Float64 value) + { + auto & col = assert_cast(column); + col.getData().push_back(static_cast(value)); + }; + case TypeIndex::DateTime64: + return [](IColumn & column, Float64 value) + { + auto & col = assert_cast(column); + col.getData().push_back(static_cast(value)); + }; + default: + return [](IColumn & column, Float64 value) + { + auto & col = assert_cast(column); + col.getData().push_back(value); + }; + } + } +}; + +} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 21bd783e6bc..69823be07f6 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -82,6 +82,7 @@ void registerAggregateFunctionIntervalLengthSum(AggregateFunctionFactory &); void registerAggregateFunctionAnalysisOfVariance(AggregateFunctionFactory &); void registerAggregateFunctionFlameGraph(AggregateFunctionFactory &); void registerAggregateFunctionKolmogorovSmirnovTest(AggregateFunctionFactory & factory); +void registerAggregateFunctionLargestTriangleThreeBuckets(AggregateFunctionFactory & factory); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -176,6 +177,7 @@ void registerAggregateFunctions() registerAggregateFunctionAnalysisOfVariance(factory); registerAggregateFunctionFlameGraph(factory); registerAggregateFunctionKolmogorovSmirnovTest(factory); + registerAggregateFunctionLargestTriangleThreeBuckets(factory); registerWindowFunctions(factory); } diff --git a/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.reference b/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.reference new file mode 100644 index 00000000000..b4407dea13a --- /dev/null +++ b/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.reference @@ -0,0 +1,7 @@ +[] +[(1,10)] +[(1,10),(10,70)] +[(1,10),(3,15),(5,40),(10,70)] +[(0.02,0.16),(0.06,0.73),(0.08,0.88),(0.09,0),(0.11,0.31),(0.15,0.09),(0.19,0.27),(0.2,0.24),(0.21,0.46),(0.23,0.23),(0.25,0.34),(0.29,0.16),(0.36,0.99),(0.37,0.86),(0.38,0.75),(0.39,0.86),(0.41,0.63),(0.45,0.86),(0.47,0.06),(0.98,0.09)] +[('2023-01-14 00:00:00.000',35.96528042030847),('2023-03-15 00:00:00.000',98.77709508458238),('2023-06-05 00:00:00.000',8.107958052612418),('2023-09-29 00:00:00.000',80.05338096818797),('2023-12-31 00:00:00.000',98.52375935588333)] +[('2023-01-14 00:00:00.000',35.96528042030847),('2023-03-15 00:00:00.000',98.77709508458238),('2023-06-05 00:00:00.000',8.107958052612418),('2023-09-29 00:00:00.000',80.05338096818797),('2023-12-31 00:00:00.000',98.52375935588333)] diff --git a/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.sql b/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.sql new file mode 100644 index 00000000000..2eb8754cb64 --- /dev/null +++ b/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.sql @@ -0,0 +1,47 @@ +drop table if exists largestTriangleThreeBucketsTestFloat64Float64; + +CREATE TABLE largestTriangleThreeBucketsTestFloat64Float64 +( + x Float64, + y Float64 +) ENGINE = MergeTree order by (y,x); + +INSERT INTO largestTriangleThreeBucketsTestFloat64Float64 +VALUES (1.0, 10.0),(2.0, 20.0),(3.0, 15.0),(8.0, 60.0),(9.0, 55.0),(10.0, 70.0),(4.0, 30.0),(5.0, 40.0),(6.0, 35.0),(7.0, 50.0); + +select largestTriangleThreeBuckets(0)(x, y) FROM largestTriangleThreeBucketsTestFloat64Float64; + +select largestTriangleThreeBuckets(1)(x, y) FROM largestTriangleThreeBucketsTestFloat64Float64; + +select largestTriangleThreeBuckets(2)(x, y) FROM largestTriangleThreeBucketsTestFloat64Float64; + +SELECT largestTriangleThreeBuckets(4)(x, y) AS downsampled_data +FROM largestTriangleThreeBucketsTestFloat64Float64; + +drop table largestTriangleThreeBucketsTestFloat64Float64; + +drop table if exists largestTriangleThreeBucketsTestDecimal64Decimal64; + +CREATE TABLE largestTriangleThreeBucketsTestDecimal64Decimal64 +( + x Decimal64(2), + y Decimal64(2) +) ENGINE = MergeTree order by (y,x); + +INSERT INTO largestTriangleThreeBucketsTestDecimal64Decimal64(x, y) VALUES (0.63, 0.25), (0.02, 0.16), (0.29, 0.16), (0.2, 0.24), (0.41, 0.63), (0.06, 0.73), (0.36, 0.99), (0.57, 0.18), (0.98, 0.09), (0.73, 0.95), (0.45, 0.86), (0.37, 0.86), (0.6, 0.64), (0.11, 0.31), (0.7, 0.25), (0.85, 0.15), (0.68, 0.39), (0.9, 0.3), (0.25, 0.34), (0.09, 0.0), (0.91, 0.62), (0.47, 0.06), (0.08, 0.88), (0.48, 0.57), (0.55, 0.75), (0.19, 0.27), (0.87, 0.15), (0.15, 0.09), (0.77, 0.28), (0.5, 0.2), (0.39, 0.86), (0.52, 0.11), (0.38, 0.75), (0.71, 0.44), (0.21, 0.46), (0.88, 0.15), (0.83, 0.67), (0.23, 0.23); + +select largestTriangleThreeBuckets(20)(x, y) from largestTriangleThreeBucketsTestDecimal64Decimal64; + +drop table largestTriangleThreeBucketsTestDecimal64Decimal64; + +drop table if exists largestTriangleThreeBucketsTestDateTime64Float64; + +create table largestTriangleThreeBucketsTestDateTime64Float64 (x DateTime64(3), y Float64) engine = MergeTree order by (y,x); + +INSERT INTO largestTriangleThreeBucketsTestDateTime64Float64 (x, y) VALUES ('2023-09-06 00:00:00', 14.217481939467213), ('2023-09-11 00:00:00', 30.096113766096455), ('2023-01-31 00:00:00', 91.42364224984735), ('2023-12-14 00:00:00', 42.08543753438961), ('2023-10-31 00:00:00', 29.93227107709394), ('2023-12-31 00:00:00', 98.52375935588333), ('2023-07-07 00:00:00', 79.9367415060134), ('2023-08-02 00:00:00', 55.417182033825696), ('2023-03-15 00:00:00', 98.77709508458238), ('2023-09-05 00:00:00', 2.832505232031368), ('2023-06-05 00:00:00', 8.107958052612418), ('2023-02-08 00:00:00', 62.95788480328096), ('2023-02-17 00:00:00', 76.80522155552535), ('2023-11-13 00:00:00', 24.927527306242993), ('2023-02-03 00:00:00', 7.966981342350332), ('2023-05-31 00:00:00', 44.61922229800436), ('2023-09-21 00:00:00', 65.86974701469791), ('2023-01-14 00:00:00', 35.96528042030847), ('2023-02-19 00:00:00', 16.065599678978305), ('2023-05-24 00:00:00', 17.23630978966909), ('2023-11-15 00:00:00', 15.544172190379879), ('2023-12-03 00:00:00', 13.738382187690856), ('2023-10-09 00:00:00', 16.7137129521176), ('2023-11-19 00:00:00', 12.12866001303361), ('2023-06-10 00:00:00', 95.15764263905534), ('2023-07-06 00:00:00', 18.87765798627088), ('2023-03-13 00:00:00', 44.82941460384813), ('2023-01-29 00:00:00', 36.0214717111606), ('2023-12-19 00:00:00', 90.30173319497655), ('2023-07-15 00:00:00', 12.67101467231364), ('2023-07-06 00:00:00', 88.13662733228512), ('2023-05-10 00:00:00', 34.18711141027026), ('2023-11-12 00:00:00', 75.58716684321973), ('2023-10-28 00:00:00', 35.79179186729331), ('2023-11-14 00:00:00', 0.9318182359137728), ('2023-09-29 00:00:00', 80.05338096818797), ('2023-09-13 00:00:00', 16.130217942056866), ('2023-07-28 00:00:00', 11.186638594914744), ('2023-02-12 00:00:00', 69.43690757793445), ('2023-12-18 00:00:00', 12.832032764204616), ('2023-05-21 00:00:00', 74.25002458036471), ('2023-04-03 00:00:00', 51.5662427420719), ('2023-11-27 00:00:00', 96.44359131281784), ('2023-03-29 00:00:00', 33.018594418113324), ('2023-02-07 00:00:00', 84.58945099939815), ('2023-11-16 00:00:00', 40.61531555527268), ('2023-04-21 00:00:00', 60.0545791577218), ('2023-01-31 00:00:00', 87.23185155362057), ('2023-05-19 00:00:00', 77.4095289464808), ('2023-08-26 00:00:00', 18.700816570182067); + +select largestTriangleThreeBuckets(5)(x, y) from largestTriangleThreeBucketsTestDateTime64Float64; + +select lttb(5)(x, y) from largestTriangleThreeBucketsTestDateTime64Float64; + +drop table largestTriangleThreeBucketsTestDateTime64Float64; \ No newline at end of file diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 110ca6c10ae..3a5bd023f7d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1400,6 +1400,7 @@ dmesg domainWithoutWWW dont dotProduct +downsampling dplyr dragonbox dropoff @@ -1708,6 +1709,7 @@ kurtsamp laion lang laravel +largestTriangleThreeBuckets latencies ldap leftPad @@ -1757,6 +1759,7 @@ london lowcardinality lowerUTF lowercased +lttb lzma macOS mailrugo From 38895ad2da5fa6298040b90e503cb43f9e0c46d6 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 18 Oct 2023 12:59:12 +0000 Subject: [PATCH 402/634] Less copying in DefaultCoordinator --- .../ParallelReplicasReadingCoordinator.cpp | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index bae3a36a779..3834e175b78 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -134,7 +134,7 @@ public: void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) override; void markReplicaAsUnavailable(size_t replica_number) override; - void updateReadingState(const InitialAllRangesAnnouncement & announcement); + void updateReadingState(InitialAllRangesAnnouncement announcement); void finalizeReadingState(); size_t computeConsistentHash(const MergeTreePartInfo & info) const @@ -152,12 +152,12 @@ DefaultCoordinator::~DefaultCoordinator() LOG_DEBUG(log, "Coordination done: {}", toString(stats)); } -void DefaultCoordinator::updateReadingState(const InitialAllRangesAnnouncement & announcement) +void DefaultCoordinator::updateReadingState(InitialAllRangesAnnouncement announcement) { PartRefs parts_diff; /// To get rid of duplicates - for (const auto & part: announcement.description) + for (auto && part: announcement.description) { auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), [&part] (const Part & other) { return other.description.info.getPartNameV1() == part.info.getPartNameV1(); }); @@ -176,12 +176,7 @@ void DefaultCoordinator::updateReadingState(const InitialAllRangesAnnouncement & if (covering_or_the_same_it != all_parts_to_read.end()) continue; - auto new_part = Part{ - .description = part, - .replicas = {announcement.replica_num} - }; - - auto [insert_it, _] = all_parts_to_read.insert(new_part); + auto [insert_it, _] = all_parts_to_read.emplace(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); parts_diff.push_back(insert_it); } @@ -242,12 +237,14 @@ void DefaultCoordinator::finalizeReadingState() void DefaultCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { - updateReadingState(announcement); + const auto replica_num = announcement.replica_num; - if (announcement.replica_num >= stats.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Replica number ({}) is bigger than total replicas count ({})", announcement.replica_num, stats.size()); + updateReadingState(std::move(announcement)); - stats[announcement.replica_num].number_of_requests +=1; + if (replica_num >= stats.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Replica number ({}) is bigger than total replicas count ({})", replica_num, stats.size()); + + ++stats[replica_num].number_of_requests; ++sent_initial_requests; LOG_DEBUG(log, "Sent initial requests: {} Replicas count: {}", sent_initial_requests, replicas_count); From 3ef092f83f287cf5c6e97fa6603c5af79ad4c8af Mon Sep 17 00:00:00 2001 From: monchickey <75814968+monchickey@users.noreply.github.com> Date: Wed, 18 Oct 2023 21:04:27 +0800 Subject: [PATCH 403/634] Fix the compilation error that occurs when opening avx series instructions (#55049) * fix the compilation error that occurs when opening avx series instructions cmake command example: ```shell cmake .. -DCMAKE_BUILD_TYPE=Release -DENABLE_AVX=ON -DENABLE_AVX2=ON -DENABLE_AVX2_FOR_SPEC_OP=ON ``` * Update snappy CMakeLists.txt add SNAPPY_HAVE_X86_CRC32 parameter --- contrib/snappy-cmake/CMakeLists.txt | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/contrib/snappy-cmake/CMakeLists.txt b/contrib/snappy-cmake/CMakeLists.txt index f406de0e343..d923ddef6b6 100644 --- a/contrib/snappy-cmake/CMakeLists.txt +++ b/contrib/snappy-cmake/CMakeLists.txt @@ -24,6 +24,12 @@ else () set (SNAPPY_HAVE_SSSE3 0) endif () +if (ARCH_AMD64 AND ENABLE_SSE42) + set (SNAPPY_HAVE_X86_CRC32 1) +else () + set (SNAPPY_HAVE_X86_CRC32 0) +endif () + configure_file( "${SOURCE_DIR}/cmake/config.h.in" "${CMAKE_CURRENT_BINARY_DIR}/config.h") From f90e31eb3103547e509e02fcb74679bd9d9172e2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 15:23:16 +0200 Subject: [PATCH 404/634] Enable log by default --- src/Storages/S3Queue/S3QueueSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 351245e24be..66fe9b4ce31 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -21,7 +21,7 @@ class ASTStorage; M(String, keeper_path, "", "Zookeeper node path", 0) \ M(UInt32, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \ M(UInt32, s3queue_processing_threads_num, 1, "Number of processing threads", 0) \ - M(UInt32, s3queue_enable_logging_to_s3queue_log, 0, "Enable logging to system table system.s3queue_log", 0) \ + M(UInt32, s3queue_enable_logging_to_s3queue_log, 1, "Enable logging to system table system.s3queue_log", 0) \ M(UInt32, s3queue_tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \ M(UInt32, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt32, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ From dfdd2ee77cd9738d55a0c37a58ffe5e8b6f89a1d Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 18 Oct 2023 16:36:45 +0200 Subject: [PATCH 405/634] fix tests, to avoid timeouts do less retries --- .../test_checking_s3_blobs_paranoid/configs/setting.xml | 1 + .../integration/test_mask_sensitive_info/configs/users.xml | 5 +++++ .../test_s3_table_functions/configs/users.d/users.xml | 5 +++++ tests/integration/test_storage_s3/configs/defaultS3.xml | 5 +++++ tests/integration/test_storage_s3/configs/s3_retry.xml | 7 +++++++ tests/integration/test_storage_s3/test.py | 4 ++-- 6 files changed, 25 insertions(+), 2 deletions(-) create mode 100644 tests/integration/test_storage_s3/configs/s3_retry.xml diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml index f8749488f57..3bc0f322cb8 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml @@ -5,6 +5,7 @@ 1 1 + 5 diff --git a/tests/integration/test_mask_sensitive_info/configs/users.xml b/tests/integration/test_mask_sensitive_info/configs/users.xml index 4b6ba057ecb..f129a5bb3e3 100644 --- a/tests/integration/test_mask_sensitive_info/configs/users.xml +++ b/tests/integration/test_mask_sensitive_info/configs/users.xml @@ -1,4 +1,9 @@ + + + 5 + + diff --git a/tests/integration/test_s3_table_functions/configs/users.d/users.xml b/tests/integration/test_s3_table_functions/configs/users.d/users.xml index 4b6ba057ecb..f129a5bb3e3 100644 --- a/tests/integration/test_s3_table_functions/configs/users.d/users.xml +++ b/tests/integration/test_s3_table_functions/configs/users.d/users.xml @@ -1,4 +1,9 @@ + + + 5 + + diff --git a/tests/integration/test_storage_s3/configs/defaultS3.xml b/tests/integration/test_storage_s3/configs/defaultS3.xml index 7dac6d9fbb5..37454ef6781 100644 --- a/tests/integration/test_storage_s3/configs/defaultS3.xml +++ b/tests/integration/test_storage_s3/configs/defaultS3.xml @@ -1,4 +1,9 @@ + + + 5 + + http://resolver:8080 diff --git a/tests/integration/test_storage_s3/configs/s3_retry.xml b/tests/integration/test_storage_s3/configs/s3_retry.xml new file mode 100644 index 00000000000..727e23273cf --- /dev/null +++ b/tests/integration/test_storage_s3/configs/s3_retry.xml @@ -0,0 +1,7 @@ + + + + 5 + + + diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index ca2ddeb78b0..bbd2d2d4be0 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -55,7 +55,7 @@ def started_cluster(): "configs/named_collections.xml", "configs/schema_cache.xml", ], - user_configs=["configs/access.xml", "configs/users.xml"], + user_configs=["configs/access.xml", "configs/users.xml", "configs/s3_retry.xml"], ) cluster.add_instance( "dummy_without_named_collections", @@ -71,7 +71,7 @@ def started_cluster(): "s3_max_redirects", with_minio=True, main_configs=["configs/defaultS3.xml"], - user_configs=["configs/s3_max_redirects.xml"], + user_configs=["configs/s3_max_redirects.xml", "configs/s3_retry.xml"], ) cluster.add_instance( "s3_non_default", From d73afed7d5118ca644392e9a5d2153190b595707 Mon Sep 17 00:00:00 2001 From: zhongyuankai <872237106@qq.com> Date: Wed, 18 Oct 2023 23:09:38 +0800 Subject: [PATCH 406/634] fix style --- src/Interpreters/executeQuery.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 07456a31eeb..22230c53698 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -988,11 +988,11 @@ static std::tuple executeQueryImpl( } QueryCachePtr query_cache = context->getQueryCache(); - const bool can_use_query_cache = query_cache != nullptr - && settings.use_query_cache - && !internal - && client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY - && (ast->as() || ast->as()); + const bool can_use_query_cache = query_cache != nullptr + && settings.use_query_cache + && !internal + && client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY + && (ast->as() || ast->as()); QueryCache::Usage query_cache_usage = QueryCache::Usage::None; if (!async_insert) From fac205f08cebff653846bbd484e59ba80f673383 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 18 Oct 2023 17:12:11 +0200 Subject: [PATCH 407/634] Poke CI From 6f349038bb4389403e3740abc769e9d7b678b30e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 18 Oct 2023 17:24:41 +0200 Subject: [PATCH 408/634] Enable test_mutation_simple with the analyzer --- tests/analyzer_integration_broken_tests.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index e2da429ab2f..7ad0b513900 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -39,8 +39,6 @@ test_settings_profile/test.py::test_show_profiles test_shard_level_const_function/test.py::test_remote test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view -test_system_merges/test.py::test_mutation_simple[] -test_system_merges/test.py::test_mutation_simple[replicated] test_user_defined_object_persistence/test.py::test_persistence test_wrong_db_or_table_name/test.py::test_wrong_table_name test_zookeeper_config/test.py::test_chroot_with_same_root From 1308f26cccbb66c592cb6f7cb4790b2188fefb4c Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Wed, 18 Oct 2023 17:30:59 +0200 Subject: [PATCH 409/634] Update test.py --- .../configs/inf_s3_retries.xml | 9 +++++++++ .../test_checking_s3_blobs_paranoid/test.py | 14 ++++++++++++-- tests/integration/test_storage_s3/test.py | 6 +++++- 3 files changed, 26 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml new file mode 100644 index 00000000000..206eb4f2bad --- /dev/null +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml @@ -0,0 +1,9 @@ + + + + + + 1000000 + + + diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index eb791383fde..c5b78e139bd 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -22,6 +22,17 @@ def cluster(): ], with_minio=True, ) + cluster.add_instance( + "node_with_inf_s3_retries", + main_configs=[ + "configs/storage_conf.xml", + ], + user_configs=[ + "configs/setting.xml", + "configs/inf_s3_retries.xml", + ], + with_minio=True, + ) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -468,7 +479,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): def test_query_is_canceled_with_inf_retries(cluster, broken_s3): - node = cluster.instances["node"] + node = cluster.instances["node_with_inf_s3_retries"] broken_s3.setup_at_part_upload( count=10000000, @@ -490,7 +501,6 @@ def test_query_is_canceled_with_inf_retries(cluster, broken_s3): FROM system.numbers LIMIT 1000000 SETTINGS - s3_retry_attempts=1000000, s3_max_single_part_upload_size=100, s3_min_upload_part_size=10000, s3_check_objects_after_upload=0 diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index bbd2d2d4be0..01ade1acc4d 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -55,7 +55,11 @@ def started_cluster(): "configs/named_collections.xml", "configs/schema_cache.xml", ], - user_configs=["configs/access.xml", "configs/users.xml", "configs/s3_retry.xml"], + user_configs=[ + "configs/access.xml", + "configs/users.xml", + "configs/s3_retry.xml", + ], ) cluster.add_instance( "dummy_without_named_collections", From 70711b08988958598cab9b9522f299b24bf8f7e1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 18 Oct 2023 23:05:03 -0700 Subject: [PATCH 410/634] Remove temporary debug logging in MultiplexedConnections (#55764) --- src/Client/MultiplexedConnections.cpp | 36 +-------------------------- src/Client/MultiplexedConnections.h | 9 +------ 2 files changed, 2 insertions(+), 43 deletions(-) diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 1b1ca427e05..c7d7d0143c8 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include @@ -24,14 +23,6 @@ namespace ErrorCodes } -#define MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION \ - mutex_last_locked_by.store((getThreadId() << 32) | __LINE__); \ - memcpy(mutex_memory_dump.data(), &cancel_mutex, mutex_memory_dump.size()); \ - mutex_locked += 1; \ - SCOPE_EXIT({ mutex_locked -= 1; }); -/// When you remove this macro, please also remove the clang-tidy suppressions at the beginning + end of this file. - - MultiplexedConnections::MultiplexedConnections(Connection & connection, const Settings & settings_, const ThrottlerPtr & throttler) : settings(settings_) { @@ -86,7 +77,6 @@ MultiplexedConnections::MultiplexedConnections( void MultiplexedConnections::sendScalarsData(Scalars & data) { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (!sent_query) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send scalars data: query not yet sent."); @@ -102,7 +92,6 @@ void MultiplexedConnections::sendScalarsData(Scalars & data) void MultiplexedConnections::sendExternalTablesData(std::vector & data) { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (!sent_query) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send external tables data: query not yet sent."); @@ -131,7 +120,6 @@ void MultiplexedConnections::sendQuery( bool with_pending_data) { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (sent_query) throw Exception(ErrorCodes::LOGICAL_ERROR, "Query already sent."); @@ -189,7 +177,6 @@ void MultiplexedConnections::sendQuery( void MultiplexedConnections::sendIgnoredPartUUIDs(const std::vector & uuids) { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (sent_query) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send uuids after query is sent."); @@ -206,7 +193,6 @@ void MultiplexedConnections::sendIgnoredPartUUIDs(const std::vector & uuid void MultiplexedConnections::sendReadTaskResponse(const String & response) { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (cancelled) return; current_connection->sendReadTaskResponse(response); @@ -216,7 +202,6 @@ void MultiplexedConnections::sendReadTaskResponse(const String & response) void MultiplexedConnections::sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (cancelled) return; current_connection->sendMergeTreeReadTaskResponse(response); @@ -226,29 +211,13 @@ void MultiplexedConnections::sendMergeTreeReadTaskResponse(const ParallelReadRes Packet MultiplexedConnections::receivePacket() { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION Packet packet = receivePacketUnlocked({}); return packet; } void MultiplexedConnections::disconnect() { - /// We've seen this lock mysteriously get stuck forever, without any other thread seeming to - /// hold the mutex. This is temporary code to print some extra information next time it happens. - /// std::lock_guard lock(cancel_mutex); - if (!cancel_mutex.try_lock_for(std::chrono::hours(1))) - { - UInt64 last_locked = mutex_last_locked_by.load(); - std::array new_memory_dump; - memcpy(new_memory_dump.data(), &cancel_mutex, new_memory_dump.size()); - LOG_ERROR(&Poco::Logger::get("MultiplexedConnections"), "Deadlock in MultiplexedConnections::disconnect()! Mutex was last (instrumentedly) locked by thread {} on line {}, lock balance: {}, mutex memory when last locked: {}, mutex memory now: {}", last_locked >> 32, last_locked & 0xffffffff, mutex_locked.load(), hexString(mutex_memory_dump.data(), mutex_memory_dump.size()), hexString(new_memory_dump.data(), new_memory_dump.size())); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Deadlock in MultiplexedConnections::disconnect()"); - } -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wthread-safety-analysis" - std::lock_guard lock(cancel_mutex, std::adopt_lock); -#pragma clang diagnostic pop - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION + std::lock_guard lock(cancel_mutex); for (ReplicaState & state : replica_states) { @@ -264,7 +233,6 @@ void MultiplexedConnections::disconnect() void MultiplexedConnections::sendCancel() { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (!sent_query || cancelled) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot cancel. Either no query sent or already cancelled."); @@ -282,7 +250,6 @@ void MultiplexedConnections::sendCancel() Packet MultiplexedConnections::drain() { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (!cancelled) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot drain connections: cancel first."); @@ -323,7 +290,6 @@ Packet MultiplexedConnections::drain() std::string MultiplexedConnections::dumpAddresses() const { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION return dumpAddressesUnlocked(); } diff --git a/src/Client/MultiplexedConnections.h b/src/Client/MultiplexedConnections.h index 4c776e6e168..9f7b47e0562 100644 --- a/src/Client/MultiplexedConnections.h +++ b/src/Client/MultiplexedConnections.h @@ -106,14 +106,7 @@ private: std::optional replica_info; /// A mutex for the sendCancel function to execute safely in separate thread. - mutable std::timed_mutex cancel_mutex; - - /// Temporary instrumentation to debug a weird deadlock on cancel_mutex. - /// TODO: Once the investigation is done, get rid of these, and of INSTRUMENTED_LOCK_MUTEX, and - /// change cancel_mutex to std::mutex. - mutable std::atomic mutex_last_locked_by{0}; - mutable std::atomic mutex_locked{0}; - mutable std::array mutex_memory_dump; + mutable std::mutex cancel_mutex; friend struct RemoteQueryExecutorRoutine; }; From b708fa21f95dee582ef9e08a5a03ce15728186f3 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 19 Oct 2023 04:03:08 +0000 Subject: [PATCH 411/634] rocksdb: trivial count Signed-off-by: Duc Canh Le --- src/Core/Settings.h | 1 + src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 14 ++++++++++++++ src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 4 ++++ .../02892_rocksdb_trivial_count.reference | 2 ++ .../0_stateless/02892_rocksdb_trivial_count.sql | 4 ++++ 5 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/02892_rocksdb_trivial_count.reference create mode 100644 tests/queries/0_stateless/02892_rocksdb_trivial_count.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7f2747060b5..560245d4644 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -546,6 +546,7 @@ class IColumn; M(Bool, database_atomic_wait_for_drop_and_detach_synchronously, false, "When executing DROP or DETACH TABLE in Atomic database, wait for table data to be finally dropped or detached.", 0) \ M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ + M(Bool, rocksdb_enable_approximate_count, true, "If `optimize_trivial_count_query` is true, process trivial 'SELECT count() FROM rockdb_table' from metadata, the returned result is approximated.", 0) \ M(Bool, optimize_count_from_files, true, "Optimize counting rows from files in supported input formats", 0) \ M(Bool, use_cache_for_count_from_files, true, "Use cache to count the number of rows in files", 0) \ M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index be322a402ee..b08fd71d5ec 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -609,5 +609,19 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory) factory.registerStorage("EmbeddedRocksDB", create, features); } +std::optional StorageEmbeddedRocksDB::totalRows(const Settings & settings) const +{ + if (settings.rocksdb_enable_approximate_count) + { + std::shared_lock lock(rocksdb_ptr_mx); + if (!rocksdb_ptr) + return {}; + UInt64 estimated_rows; + if (!rocksdb_ptr->GetIntProperty("rocksdb.estimate-num-keys", &estimated_rows)) + return {}; + return estimated_rows; + } + return {}; +} } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index d0cf05f261c..336f6a8abe3 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -83,6 +83,10 @@ public: bool supportsDelete() const override { return true; } + bool supportsTrivialCountOptimization() const override { return true; } + + std::optional totalRows(const Settings & settings) const override; + private: const String primary_key; using RocksDBPtr = std::unique_ptr; diff --git a/tests/queries/0_stateless/02892_rocksdb_trivial_count.reference b/tests/queries/0_stateless/02892_rocksdb_trivial_count.reference new file mode 100644 index 00000000000..75d516255f1 --- /dev/null +++ b/tests/queries/0_stateless/02892_rocksdb_trivial_count.reference @@ -0,0 +1,2 @@ +121 +121 diff --git a/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql b/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql new file mode 100644 index 00000000000..cef18b2cc2a --- /dev/null +++ b/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql @@ -0,0 +1,4 @@ +CREATE TABLE dict (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY key; +INSERT INTO dict SELECT number, toString(number) FROM numbers(121); +SELECT count() FROM dict SETTINGS rocksdb_enable_approximate_count = 0; +SELECT count() FROM dict SETTINGS optimize_trivial_count_query = 1, rocksdb_enable_approximate_count = 1; From ab785b0815d3b267c8236616c8288c1a0fad0f64 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 19 Oct 2023 04:26:56 +0000 Subject: [PATCH 412/634] minor fix in test Signed-off-by: Duc Canh Le --- tests/queries/0_stateless/02892_rocksdb_trivial_count.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql b/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql index cef18b2cc2a..4fc1a681b26 100644 --- a/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql +++ b/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql @@ -1,4 +1,4 @@ CREATE TABLE dict (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY key; INSERT INTO dict SELECT number, toString(number) FROM numbers(121); SELECT count() FROM dict SETTINGS rocksdb_enable_approximate_count = 0; -SELECT count() FROM dict SETTINGS optimize_trivial_count_query = 1, rocksdb_enable_approximate_count = 1; +SELECT count() FROM dict SETTINGS optimize_trivial_count_query = 1, rocksdb_enable_approximate_count = 1, max_rows_to_read = 1; From d968dc1a15e3d7795f9a4568a92bd94c29627c9d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 19 Oct 2023 06:16:31 +0000 Subject: [PATCH 413/634] StorageJoin: supports trivial count() Signed-off-by: Duc Canh Le --- src/Storages/StorageJoin.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 4626d744a38..dc68c68a21b 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -85,6 +85,8 @@ public: const Names & getKeyNames() const { return key_names; } + bool supportsTrivialCountOptimization() const override { return true; } + private: Block sample_block; const Names key_names; From 9a155ba747dbc33427334d6e75e6114a15e08277 Mon Sep 17 00:00:00 2001 From: Aleksandr Musorin Date: Wed, 18 Oct 2023 10:48:03 +0200 Subject: [PATCH 414/634] Added test to check CapnProto cache Since current getting schema for CapnProto doesn't use cache, the integration test was for the future if someone add cache. Also, I was curious how schema cache affects performance and I compared reading binary files with Protobuf (I guess it works the same for CapnProto). ``` for i in {1..1000}; do clickhouse-client -q \ "select * from test.simple format Protobuf settings format_schema='/format_schemas/simple:KeyValuePair'" > simple-protobuf${i}.bin; done clickhouse-client --time -q "select * from file('simple-protobuf{1..999}.bin', 'Protobuf') format Null settings format_schema = 'simple:KeyValuePair'" ``` Protobuf with cache and without works with approximately the same time. --- .../test_format_schema_on_server/test.py | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/tests/integration/test_format_schema_on_server/test.py b/tests/integration/test_format_schema_on_server/test.py index 4d62dc976ce..3fa10ff1a1e 100644 --- a/tests/integration/test_format_schema_on_server/test.py +++ b/tests/integration/test_format_schema_on_server/test.py @@ -112,3 +112,66 @@ message MessageTmp { "SELECT * FROM test.simple FORMAT Protobuf SETTINGS format_schema='message_tmp:MessageTmp'" ) assert "NO_COLUMNS_SERIALIZED_TO_PROTOBUF_FIELDS)" in str(exc.value) + + +def test_drop_capn_proto_format(started_cluster): + create_simple_table() + instance.query("INSERT INTO test.simple VALUES (1, 'abc'), (2, 'def')") + capn_proto_schema = """ +@0x801f030c2b67bf19; + +struct MessageTmp { + key @0 :UInt64; + value @1 :Text; +} +""" + capn_schema_path_name = "message_tmp.capnp" + + database_path = os.path.abspath(os.path.join(instance.path, "database")) + format_schemas_path = os.path.join(database_path, "format_schemas") + with open(os.path.join(format_schemas_path, capn_schema_path_name), "w") as file: + file.write(capn_proto_schema) + + assert instance.http_query( + "SELECT * FROM test.simple FORMAT CapnProto SETTINGS format_schema='message_tmp:MessageTmp'" + ) == instance.query( + f"SELECT * FROM test.simple Format CapnProto SETTINGS format_schema='{format_schemas_path}/message_tmp:MessageTmp'" + ) + + new_schema = """ +@0x801f030c2b67bf19; + +struct MessageTmp { + key2 @0 :UInt64; + value2 @1 :Text; +} +""" + with open(os.path.join(format_schemas_path, capn_schema_path_name), "w") as file: + file.write(new_schema) + + instance.query("DROP TABLE IF EXISTS test.new_simple") + instance.query( + """ + CREATE TABLE test.new_simple (key2 UInt64, value2 String) + ENGINE = MergeTree ORDER BY tuple(); + """ + ) + instance.query("INSERT INTO test.new_simple VALUES (1, 'abc'), (2, 'def')") + + # instance.query("SYSTEM DROP FORMAT SCHEMA CACHE FOR CapnProto") + + # Tets works with new scheme + assert instance.http_query( + "SELECT * FROM test.new_simple FORMAT CapnProto SETTINGS format_schema='message_tmp:MessageTmp'" + ) == instance.query( + f"SELECT * FROM test.new_simple Format CapnProto SETTINGS format_schema='{format_schemas_path}/message_tmp:MessageTmp'" + ) + # Tests that stop working with old scheme + with pytest.raises(Exception) as exc: + instance.http_query( + "SELECT * FROM test.simple FORMAT CapnProto SETTINGS format_schema='message_tmp:MessageTmp'" + ) + assert ( + "Capnproto schema doesn't contain field with name key. (THERE_IS_NO_COLUMN)" + in str(exc.value) + ) From f18622b48b69705acd03be10aab0d9ecc656ee16 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 19 Oct 2023 09:12:14 +0000 Subject: [PATCH 415/634] Bump gRPC to 1v.36.4 --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index 4b3a77e6008..c52656e2bfc 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 4b3a77e600859bb3e143247f91ef75e4286b7ec9 +Subproject commit c52656e2bfcda3450bd6a7c247d2d9eeb8498524 From d957d60a5267aa4713826ebe25e2631450e09eba Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 18 Oct 2023 19:36:34 +0200 Subject: [PATCH 416/634] Use publicly available play.clickhouse.com to search built images --- tests/ci/docker_manifests_merge.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/docker_manifests_merge.py b/tests/ci/docker_manifests_merge.py index 9b61134fa75..0a81480c52a 100644 --- a/tests/ci/docker_manifests_merge.py +++ b/tests/ci/docker_manifests_merge.py @@ -209,7 +209,10 @@ def enrich_images(changed_images: Dict[str, str]) -> None: """ batch_count = 0 - ch_helper = ClickHouseHelper() + # We use always publicly available DB here intentionally + ch_helper = ClickHouseHelper( + "https://play.clickhouse.com", {"X-ClickHouse-User": "play"} + ) while ( batch_count <= MAX_COMMIT_BATCHES_TO_CHECK and len(images_to_find_tags_for) != 0 From d8b899d22bad4108a6eaac548e831032da1b3c6c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 18 Oct 2023 19:38:45 +0200 Subject: [PATCH 417/634] Use a proper tag from default.checks for enriched images --- tests/ci/docker_manifests_merge.py | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/tests/ci/docker_manifests_merge.py b/tests/ci/docker_manifests_merge.py index 0a81480c52a..1be2a1f2e7a 100644 --- a/tests/ci/docker_manifests_merge.py +++ b/tests/ci/docker_manifests_merge.py @@ -197,8 +197,8 @@ def enrich_images(changed_images: Dict[str, str]) -> None: WITH {commit_shas:Array(String)} AS commit_shas, {images:Array(String)} AS images SELECT - substring(test_name, 1, position(test_name, ':') -1) AS image_name, - argMax(commit_sha, check_start_time) AS commit_sha + splitByChar(':', test_name)[1] AS image_name, + argMax(splitByChar(':', test_name)[2], check_start_time) AS tag FROM checks WHERE check_name == 'Push multi-arch images to Dockerhub' @@ -232,15 +232,12 @@ def enrich_images(changed_images: Dict[str, str]) -> None: "Found images for commits %s..%s:\n %s", commit_shas[0], commit_shas[-1], - "\n ".join(f"{im['image_name']}:{im['commit_sha']}" for im in result), + "\n ".join(f"{im['image_name']}:{im['tag']}" for im in result), ) for row in result: image_name = row["image_name"] - commit_sha = row["commit_sha"] - # As we only get the SHAs of merge commits from master, the PR number will be always 0 - tag = f"0-{commit_sha}" - changed_images[image_name] = tag + changed_images[image_name] = row["tag"] images_to_find_tags_for.remove(image_name) batch_count += 1 From 297eda83fc153c2e83a9d8a65ffb9d6e12f52bbd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 19 Oct 2023 11:59:13 +0200 Subject: [PATCH 418/634] Attemp to fix test_dictionaries_redis flakiness --- .../integration/test_dictionaries_redis/test_long.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_dictionaries_redis/test_long.py b/tests/integration/test_dictionaries_redis/test_long.py index 094df789704..beb9513884a 100644 --- a/tests/integration/test_dictionaries_redis/test_long.py +++ b/tests/integration/test_dictionaries_redis/test_long.py @@ -6,6 +6,8 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance("node", with_redis=True) +POOL_SIZE = 16 + @pytest.fixture(scope="module") def start_cluster(): @@ -29,10 +31,10 @@ def start_cluster(): value UInt64 ) PRIMARY KEY date, id - SOURCE(REDIS(HOST '{}' PORT 6379 STORAGE_TYPE 'hash_map' DB_INDEX 0 PASSWORD 'clickhouse')) + SOURCE(REDIS(HOST '{}' PORT 6379 STORAGE_TYPE 'hash_map' DB_INDEX 0 PASSWORD 'clickhouse' POOL_SIZE '{}')) LAYOUT(COMPLEX_KEY_DIRECT()) """.format( - cluster.redis_host + cluster.redis_host, POOL_SIZE ) ) @@ -58,12 +60,14 @@ def start_cluster(): def test_redis_dict_long(start_cluster): assert ( - node.query("SELECT count(), uniqExact(date), uniqExact(id) FROM redis_dict") + node.query( + f"SELECT count(), uniqExact(date), uniqExact(id) FROM redis_dict SETTINGS max_threads={POOL_SIZE}" + ) == "1000\t1\t1000\n" ) assert ( node.query( - "SELECT count(DISTINCT dictGet('redis_dict', 'value', tuple(date, id % 1000))) FROM redis_dictionary_test" + f"SELECT count(DISTINCT dictGet('redis_dict', 'value', tuple(date, id % 1000))) FROM redis_dictionary_test SETTINGS max_threads={POOL_SIZE}" ) == "1000\n" ) From bde5c5a4a80dff093fe1a5faf0fb8af857c1d015 Mon Sep 17 00:00:00 2001 From: jetgm Date: Thu, 19 Oct 2023 18:17:28 +0800 Subject: [PATCH 419/634] add show merges --- docs/en/sql-reference/statements/show.md | 44 +++++++++++++++++++ .../InterpreterShowTablesQuery.cpp | 24 ++++++++++ src/Parsers/ASTShowTablesQuery.cpp | 6 +++ src/Parsers/ASTShowTablesQuery.h | 3 +- src/Parsers/ParserShowTablesQuery.cpp | 24 ++++++++++ src/Parsers/ParserShowTablesQuery.h | 2 +- .../0_stateless/00419_show_sql_queries.sh | 1 + 7 files changed, 102 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 21bd674bd24..0567a300738 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -651,3 +651,47 @@ If either `LIKE` or `ILIKE` clause is specified, the query returns a list of sys **See Also** - [system.functions](../../operations/system-tables/functions.md) table + +## SHOW MERGES + +Returns a list of merges. All merges are listed in the [system.merges](../../operations/system-tables/merges.md) table. + + +**Syntax** + +``` sql +SHOW MERGES [[NOT] LIKE|ILIKE ''] [LIMIT ] +``` + +**Examples** + +Query: + +``` sql +SHOW MERGES; +``` + +Result: + +```text +┌─table──────┬─database─┬─────elapsed─┬─merge_type─┐ +│ your_table │ default │ 0.039891507 │ Regular │ +└────────────┴──────────┴─────────────┴────────────┘ + +``` + +Query: + +``` sql +SHOW MERGES LIKE 'your_t%' LIMIT 1; +``` + +Result: + +```text +┌─table──────┬─database─┬─────elapsed─┬─merge_type─┐ +│ your_table │ default │ 0.049192524 │ Regular │ +└────────────┴──────────┴─────────────┴────────────┘ + +``` + diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 5fe0a862e05..94c513aa1e6 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -116,6 +116,30 @@ String InterpreterShowTablesQuery::getRewrittenQuery() return rewritten_query.str(); } + /// SHOW MERGES + if (query.merges) + { + WriteBufferFromOwnString rewritten_query; + rewritten_query << "SELECT table, database, elapsed, merge_type FROM system.merges"; + + if (!query.like.empty()) + { + rewritten_query + << " WHERE table " + << (query.not_like ? "NOT " : "") + << (query.case_insensitive_like ? "ILIKE " : "LIKE ") + << DB::quote << query.like; + } + + /// (*) + rewritten_query << " ORDER BY elapsed desc, database, table"; + + if (query.limit_length) + rewritten_query << " LIMIT " << query.limit_length; + + return rewritten_query.str(); + } + if (query.temporary && !query.getFrom().empty()) throw Exception(ErrorCodes::SYNTAX_ERROR, "The `FROM` and `TEMPORARY` cannot be used together in `SHOW TABLES`"); diff --git a/src/Parsers/ASTShowTablesQuery.cpp b/src/Parsers/ASTShowTablesQuery.cpp index 03ae96fa288..5470bde10c8 100644 --- a/src/Parsers/ASTShowTablesQuery.cpp +++ b/src/Parsers/ASTShowTablesQuery.cpp @@ -78,6 +78,12 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format (settings.hilite ? hilite_none : ""); formatLike(settings); } + else if (merges) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW MERGES" << (settings.hilite ? hilite_none : ""); + formatLike(settings); + formatLimit(settings, state, frame); + } else { settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW " << (temporary ? "TEMPORARY " : "") << diff --git a/src/Parsers/ASTShowTablesQuery.h b/src/Parsers/ASTShowTablesQuery.h index 2fd4c3dac96..f6fec2d1230 100644 --- a/src/Parsers/ASTShowTablesQuery.h +++ b/src/Parsers/ASTShowTablesQuery.h @@ -9,7 +9,7 @@ namespace DB { -/** Query SHOW TABLES or SHOW DATABASES or SHOW CLUSTERS or SHOW CACHES +/** Query SHOW TABLES or SHOW DATABASES or SHOW CLUSTERS or SHOW CACHES or SHOW MERGES */ class ASTShowTablesQuery : public ASTQueryWithOutput { @@ -19,6 +19,7 @@ public: bool cluster = false; bool dictionaries = false; bool m_settings = false; + bool merges = false; bool changed = false; bool temporary = false; bool caches = false; diff --git a/src/Parsers/ParserShowTablesQuery.cpp b/src/Parsers/ParserShowTablesQuery.cpp index e3728eb2cd6..2b220dd0a0c 100644 --- a/src/Parsers/ParserShowTablesQuery.cpp +++ b/src/Parsers/ParserShowTablesQuery.cpp @@ -27,6 +27,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserKeyword s_dictionaries("DICTIONARIES"); ParserKeyword s_caches("FILESYSTEM CACHES"); ParserKeyword s_settings("SETTINGS"); + ParserKeyword s_merges("MERGES"); ParserKeyword s_changed("CHANGED"); ParserKeyword s_from("FROM"); ParserKeyword s_in("IN"); @@ -98,6 +99,29 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec return false; } } + else if (s_merges.ignore(pos, expected)) + { + query->merges = true; + + if (s_not.ignore(pos, expected)) + query->not_like = true; + + if (bool insensitive = s_ilike.ignore(pos, expected); insensitive || s_like.ignore(pos, expected)) + { + if (insensitive) + query->case_insensitive_like = true; + + if (!like_p.parse(pos, like, expected)) + return false; + } + else if (query->not_like) + return false; + if (s_limit.ignore(pos, expected)) + { + if (!exp_elem.parse(pos, query->limit_length, expected)) + return false; + } + } else if (s_caches.ignore(pos, expected)) { query->caches = true; diff --git a/src/Parsers/ParserShowTablesQuery.h b/src/Parsers/ParserShowTablesQuery.h index 1b679c2e85a..e21401cf520 100644 --- a/src/Parsers/ParserShowTablesQuery.h +++ b/src/Parsers/ParserShowTablesQuery.h @@ -14,7 +14,7 @@ namespace DB class ParserShowTablesQuery : public IParserBase { protected: - const char * getName() const override { return "SHOW [FULL] [TEMPORARY] TABLES|DATABASES|CLUSTERS|CLUSTER 'name' [[NOT] [I]LIKE 'str'] [LIMIT expr]"; } + const char * getName() const override { return "SHOW [FULL] [TEMPORARY] TABLES|DATABASES|CLUSTERS|CLUSTER|MERGES 'name' [[NOT] [I]LIKE 'str'] [LIMIT expr]"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; diff --git a/tests/queries/0_stateless/00419_show_sql_queries.sh b/tests/queries/0_stateless/00419_show_sql_queries.sh index 99252eeb1ba..2b7f73932e2 100755 --- a/tests/queries/0_stateless/00419_show_sql_queries.sh +++ b/tests/queries/0_stateless/00419_show_sql_queries.sh @@ -9,3 +9,4 @@ $CLICKHOUSE_CLIENT -q "SHOW DATABASES" &>/dev/null $CLICKHOUSE_CLIENT -q "SHOW TABLES" &>/dev/null $CLICKHOUSE_CLIENT -q "SHOW ENGINES" &>/dev/null $CLICKHOUSE_CLIENT -q "SHOW FUNCTIONS" &>/dev/null +$CLICKHOUSE_CLIENT -q "SHOW MERGES" &>/dev/null From d5c960ceb6e3afecb9d0e8ee56da15a72ac361b5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 18 Oct 2023 15:08:44 +0200 Subject: [PATCH 420/634] Fix data race in CreatingSetsTransform The problem is that findOrPromiseToBuild() can also call shared_future::get(): (lldb) bt 10 * thread 194, name = 'MergeMutate', stop reason = step over * frame 0: 0x0000564126046bd2 clickhouse`std::rethrow_exception(p=(__ptr_ = 0x00007b5000130c80)) at exception_pointer_cxxabi.ipp:68:39 frame 1: 0x000056411c4c5ddd clickhouse`std::__1::__assoc_state>::copy(this=) at future:701:9 frame 2: 0x000056411c4c0e3f clickhouse`DB::PreparedSetsCache::findOrPromiseToBuild(std::__1::basic_string, std::__1::allocator> const&) [inlined] std::__1::shared_future>::get[abi:v15000](this=0x00007b0c00154eb8) const at future:2243:46 frame 3: 0x000056411c4c0e2e clickhouse`DB::PreparedSetsCache::findOrPromiseToBuild(this=0x00007b1c00077018, key="__set_10438532338982543705_15680061365857338647") at PreparedSets.cpp:349:116 Note, that my test did not find the problem because of #55261 had not been merged yet. Follow-up for: #55338 Fixes: #55279 v2: s/createDeprecated/createRuntime/ Signed-off-by: Azat Khuzhin --- .../Transforms/CreatingSetsTransform.cpp | 43 +++++++++++-------- 1 file changed, 26 insertions(+), 17 deletions(-) diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index e2b7e990685..b59c02be32e 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -7,8 +7,11 @@ #include #include +#include #include +#include + namespace DB { @@ -95,18 +98,18 @@ void CreatingSetsTransform::startSubquery() /// Retry if the set from cache fails to be built. while (true) { - auto from_cache = prepared_sets_cache->findOrPromiseToBuild(set_and_key->key); - if (from_cache.index() == 0) + try { - LOG_TRACE(log, "Building set, key: {}", set_and_key->key); - promise_to_build = std::move(std::get<0>(from_cache)); - } - else - { - LOG_TRACE(log, "Waiting for set to be built by another thread, key: {}", set_and_key->key); - SharedSet set_built_by_another_thread = std::move(std::get<1>(from_cache)); - try + auto from_cache = prepared_sets_cache->findOrPromiseToBuild(set_and_key->key); + if (from_cache.index() == 0) { + LOG_TRACE(log, "Building set, key: {}", set_and_key->key); + promise_to_build = std::move(std::get<0>(from_cache)); + } + else + { + LOG_TRACE(log, "Waiting for set to be built by another thread, key: {}", set_and_key->key); + SharedSet set_built_by_another_thread = std::move(std::get<1>(from_cache)); const SetPtr & ready_set = set_built_by_another_thread.get(); if (!ready_set) { @@ -118,14 +121,20 @@ void CreatingSetsTransform::startSubquery() done_with_set = true; set_from_cache = true; } - catch (const Exception & e) - { - /// Exception that is thrown by the future::get() is shared across all waiters and cannot be modified from multiple threads. - /// Re-create exception to allow later multiple modify (i.e. addMessage() during pipeline execution) - throw Exception(e); - } + break; + } + /// Exception that is thrown by the shared_future::get() is shared across all waiters and cannot be modified from multiple threads. + /// Re-create exception to allow later concurrent modify (i.e. addMessage() during pipeline execution) + /// + /// Note, that findOrPromiseToBuild() can also call shared_future::get() + catch (const Exception & e) + { + throw Exception(e); + } + catch (...) + { + throw Exception::createRuntime(ErrorCodes::UNKNOWN_EXCEPTION, getExceptionMessage(std::current_exception(), /* with_stacktrace= */ false)); } - break; } } From dd640cba72a3e98da129b1f27d7ab57011250f83 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 19 Oct 2023 13:21:57 +0200 Subject: [PATCH 421/634] Provide a better message for common MV pitfalls --- .../sql-reference/statements/create/view.md | 2 +- src/Parsers/ParserCreateQuery.cpp | 22 ++++++++++++++++++- .../02900_matview_create_to_errors.reference | 4 ++++ .../02900_matview_create_to_errors.sh | 11 ++++++++++ 4 files changed, 37 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02900_matview_create_to_errors.reference create mode 100755 tests/queries/0_stateless/02900_matview_create_to_errors.sh diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 11026340a0f..2a8d6788889 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -62,7 +62,7 @@ Materialized views store data transformed by the corresponding [SELECT](../../.. When creating a materialized view without `TO [db].[table]`, you must specify `ENGINE` – the table engine for storing data. -When creating a materialized view with `TO [db].[table]`, you must not use `POPULATE`. +When creating a materialized view with `TO [db].[table]`, you can't also use `POPULATE`. A materialized view is implemented as follows: when inserting data to the table specified in `SELECT`, part of the inserted data is converted by this `SELECT` query, and the result is inserted in the view. diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 44f375adb65..760ce73cf6f 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -29,6 +29,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int SYNTAX_ERROR; } namespace @@ -1342,6 +1343,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserKeyword s_view("VIEW"); ParserKeyword s_materialized("MATERIALIZED"); ParserKeyword s_populate("POPULATE"); + ParserKeyword s_empty("EMPTY"); ParserKeyword s_or_replace("OR REPLACE"); ParserToken s_dot(TokenType::Dot); ParserToken s_lparen(TokenType::OpeningRoundBracket); @@ -1437,8 +1439,26 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (s_populate.ignore(pos, expected)) is_populate = true; - else if (ParserKeyword{"EMPTY"}.ignore(pos, expected)) + else if (s_empty.ignore(pos, expected)) is_create_empty = true; + + if (ParserKeyword{"TO"}.ignore(pos, expected)) + throw Exception( + ErrorCodes::SYNTAX_ERROR, "When creating a materialized view you can't declare both 'ENGINE' and 'TO [db].[table]'"); + } + else + { + if (storage_p.ignore(pos, expected)) + throw Exception( + ErrorCodes::SYNTAX_ERROR, "When creating a materialized view you can't declare both 'TO [db].[table]' and 'ENGINE'"); + + if (s_populate.ignore(pos, expected)) + throw Exception( + ErrorCodes::SYNTAX_ERROR, "When creating a materialized view you can't declare both 'TO [db].[table]' and 'POPULATE'"); + + if (s_empty.ignore(pos, expected)) + throw Exception( + ErrorCodes::SYNTAX_ERROR, "When creating a materialized view you can't declare both 'TO [db].[table]' and 'EMPTY'"); } /// AS SELECT ... diff --git a/tests/queries/0_stateless/02900_matview_create_to_errors.reference b/tests/queries/0_stateless/02900_matview_create_to_errors.reference new file mode 100644 index 00000000000..d516bed81ab --- /dev/null +++ b/tests/queries/0_stateless/02900_matview_create_to_errors.reference @@ -0,0 +1,4 @@ +Code: 62. DB::Ex---tion: When creating a materialized view you can't declare both 'TO [db].[table]' and 'EMPTY'. (SYNTAX_ERROR) (version reference) +Code: 62. DB::Ex---tion: When creating a materialized view you can't declare both 'TO [db].[table]' and 'POPULATE'. (SYNTAX_ERROR) (version reference) +Code: 62. DB::Ex---tion: When creating a materialized view you can't declare both 'TO [db].[table]' and 'ENGINE'. (SYNTAX_ERROR) (version reference) +Code: 62. DB::Ex---tion: When creating a materialized view you can't declare both 'ENGINE' and 'TO [db].[table]'. (SYNTAX_ERROR) (version reference) diff --git a/tests/queries/0_stateless/02900_matview_create_to_errors.sh b/tests/queries/0_stateless/02900_matview_create_to_errors.sh new file mode 100755 index 00000000000..4604b37485d --- /dev/null +++ b/tests/queries/0_stateless/02900_matview_create_to_errors.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'create materialized view aaaa TO b EMPTY as Select * from a;' | sed -e 's/Exception/Ex---tion/ ; s/version .*/version reference)/g' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'create materialized view aaaa TO b POPULATE as Select * from a;' | sed -e 's/Exception/Ex---tion/ ; s/version .*/version reference)/g' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'create materialized view aaaa TO b ENGINE = ReplicatedMergeTree() as Select * from a;' | sed -e 's/Exception/Ex---tion/ ; s/version .*/version reference)/g' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'create materialized view aaaa ENGINE = ReplicatedMergeTree() TO b as Select * from a;' | sed -e 's/Exception/Ex---tion/ ; s/version .*/version reference)/g' From 062e1fb47ead0d6cb8bf3776ded1445a48493ca1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 19 Oct 2023 15:28:05 +0200 Subject: [PATCH 422/634] CI style --- tests/queries/0_stateless/02900_matview_create_to_errors.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02900_matview_create_to_errors.sh b/tests/queries/0_stateless/02900_matview_create_to_errors.sh index 4604b37485d..a709bd2f7a1 100755 --- a/tests/queries/0_stateless/02900_matview_create_to_errors.sh +++ b/tests/queries/0_stateless/02900_matview_create_to_errors.sh @@ -7,5 +7,5 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'create materialized view aaaa TO b EMPTY as Select * from a;' | sed -e 's/Exception/Ex---tion/ ; s/version .*/version reference)/g' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'create materialized view aaaa TO b POPULATE as Select * from a;' | sed -e 's/Exception/Ex---tion/ ; s/version .*/version reference)/g' -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'create materialized view aaaa TO b ENGINE = ReplicatedMergeTree() as Select * from a;' | sed -e 's/Exception/Ex---tion/ ; s/version .*/version reference)/g' -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'create materialized view aaaa ENGINE = ReplicatedMergeTree() TO b as Select * from a;' | sed -e 's/Exception/Ex---tion/ ; s/version .*/version reference)/g' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'create materialized view aaaa TO b ENGINE = MergeTree() as Select * from a;' | sed -e 's/Exception/Ex---tion/ ; s/version .*/version reference)/g' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'create materialized view aaaa ENGINE = MergeTree() TO b as Select * from a;' | sed -e 's/Exception/Ex---tion/ ; s/version .*/version reference)/g' From 0ece6e0263a934a81738b7bb21c28c5906b45a9a Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 19 Oct 2023 14:11:56 +0000 Subject: [PATCH 423/634] Use AsyncPipelineExecutor for HashedArrayDictionary --- src/Dictionaries/CacheDictionary.cpp | 2 + src/Dictionaries/HashedArrayDictionary.cpp | 84 +++++++++++++++++++--- src/Dictionaries/HashedArrayDictionary.h | 3 +- 3 files changed, 78 insertions(+), 11 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index d492128f250..e2a5384923f 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -14,6 +14,8 @@ #include #include +#include + #include namespace ProfileEvents diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 45525f1468b..792c5810aa2 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -6,12 +6,14 @@ #include #include #include +#include +#include #include +#include #include #include - namespace DB { @@ -22,6 +24,32 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } +namespace +{ + +class PipelineExecutorImpl +{ +public: + PipelineExecutorImpl(QueryPipeline & pipeline_, bool async) + : async_executor(async ? std::make_unique(pipeline_) : nullptr) + , executor(async ? nullptr : std::make_unique(pipeline_)) + {} + + bool pull(Block & block) + { + if (async_executor) + return async_executor->pull(block); + else + return executor->pull(block); + } + +private: + std::unique_ptr async_executor; + std::unique_ptr executor; +}; + +} + template HashedArrayDictionary::HashedArrayDictionary( const StorageID & dict_id_, @@ -409,7 +437,7 @@ void HashedArrayDictionary::updateData() if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { QueryPipeline pipeline(source_ptr->loadUpdatedAll()); - PullingPipelineExecutor executor(pipeline); + PipelineExecutorImpl executor(pipeline, configuration.use_async_executor); update_field_loaded_block.reset(); Block block; @@ -533,12 +561,12 @@ void HashedArrayDictionary::blockToAttributes(const Block & } template -void HashedArrayDictionary::resize(size_t added_rows) +void HashedArrayDictionary::resize(size_t total_rows) { - if (unlikely(!added_rows)) + if (unlikely(!total_rows)) return; - key_attribute.container.reserve(added_rows); + key_attribute.container.reserve(total_rows); } template @@ -727,14 +755,37 @@ void HashedArrayDictionary::loadData() { QueryPipeline pipeline; pipeline = QueryPipeline(source_ptr->loadAll()); + PipelineExecutorImpl executor(pipeline, configuration.use_async_executor); + + UInt64 pull_time_microseconds = 0; + UInt64 process_time_microseconds = 0; + + size_t total_rows = 0; + size_t total_blocks = 0; - PullingPipelineExecutor executor(pipeline); Block block; - while (executor.pull(block)) + while (true) { - resize(block.rows()); + Stopwatch watch_pull; + bool has_data = executor.pull(block); + pull_time_microseconds += watch_pull.elapsedMicroseconds(); + + if (!has_data) + break; + + ++total_blocks; + total_rows += block.rows(); + + Stopwatch watch_process; + resize(total_rows); blockToAttributes(block); + process_time_microseconds += watch_process.elapsedMicroseconds(); } + + LOG_DEBUG(&Poco::Logger::get("HashedArrayDictionary"), + "Finished {}reading {} blocks with {} rows from pipeline in {:.2f} sec and inserted into hashtable in {:.2f} sec", + configuration.use_async_executor ? "asynchronous " : "", + total_blocks, total_rows, pull_time_microseconds / 1000000.0, process_time_microseconds / 1000000.0); } else { @@ -843,6 +894,7 @@ void registerDictionaryArrayHashed(DictionaryFactory & factory) const DictionaryStructure & dict_struct, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, + ContextPtr global_context, DictionarySourcePtr source_ptr, DictionaryKeyType dictionary_key_type) -> DictionaryPtr { @@ -863,6 +915,12 @@ void registerDictionaryArrayHashed(DictionaryFactory & factory) HashedArrayDictionaryStorageConfiguration configuration{require_nonempty, dict_lifetime}; + ContextMutablePtr context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); + const auto & settings = context->getSettingsRef(); + + bool is_clickhouse_source = dynamic_cast(source_ptr.get()); + configuration.use_async_executor = is_clickhouse_source && settings.dictionary_use_async_executor; + if (dictionary_key_type == DictionaryKeyType::Simple) return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); else @@ -872,9 +930,15 @@ void registerDictionaryArrayHashed(DictionaryFactory & factory) using namespace std::placeholders; factory.registerLayout("hashed_array", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple); }, false); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr global_context, bool /*created_from_ddl*/) + { + return create_layout(a, b, c, d, global_context, std::move(e), DictionaryKeyType::Simple); + }, false); factory.registerLayout("complex_key_hashed_array", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex); }, true); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr global_context, bool /*created_from_ddl*/) + { + return create_layout(a, b, c, d, global_context, std::move(e), DictionaryKeyType::Complex); + }, true); } } diff --git a/src/Dictionaries/HashedArrayDictionary.h b/src/Dictionaries/HashedArrayDictionary.h index bcb3f85ef06..3b9446e4e8f 100644 --- a/src/Dictionaries/HashedArrayDictionary.h +++ b/src/Dictionaries/HashedArrayDictionary.h @@ -25,6 +25,7 @@ struct HashedArrayDictionaryStorageConfiguration { const bool require_nonempty; const DictionaryLifetime lifetime; + bool use_async_executor = false; }; template @@ -212,7 +213,7 @@ private: template void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func) const; - void resize(size_t added_rows); + void resize(size_t total_rows); const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; From b4851cf2ef5b297a7f2c4e248020688efeea0b34 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 19 Oct 2023 14:43:58 +0000 Subject: [PATCH 424/634] Use AsyncPipelineExecutor for reading clickhouse dictionary source --- docs/en/operations/settings/settings.md | 15 ++++++++ src/Core/Settings.h | 2 +- src/Dictionaries/CacheDictionary.cpp | 15 ++++---- src/Dictionaries/CacheDictionary.h | 20 ++++++----- src/Dictionaries/ClickHouseDictionarySource.h | 2 ++ src/Dictionaries/DictionarySourceHelpers.cpp | 19 ++++++++++ src/Dictionaries/DictionarySourceHelpers.h | 17 +++++++++ src/Dictionaries/DirectDictionary.cpp | 4 +-- src/Dictionaries/HashedArrayDictionary.cpp | 35 +++---------------- src/Dictionaries/HashedDictionary.cpp | 22 ++++++++---- src/Dictionaries/HashedDictionary.h | 1 + .../registerCacheDictionaries.cpp | 16 +++++++-- .../01765_hashed_dictionary_simple_key.sql | 3 +- ...098_hashed_array_dictionary_simple_key.sql | 3 +- 14 files changed, 114 insertions(+), 60 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index fc04906451a..48f7c5571aa 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4746,3 +4746,18 @@ a Tuple( l Nullable(String) ) ``` + +## dictionary_use_async_executor {#dictionary_use_async_executor} + +Execute a pipeline for reading dictionary source in several threads. It's supported only by dictionaries with local CLICKHOUSE source. + +You may specify it in `SETTINGS` section of dictionary definition: + +```sql +CREATE DICTIONARY t1_dict ( key String, attr UInt64 ) +PRIMARY KEY key +SOURCE(CLICKHOUSE(QUERY `SELECT key, attr FROM t1 GROUP BY key`)) +LIFETIME(MIN 0 MAX 3600) +LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) +SETTINGS(dictionary_use_async_executor=1, max_threads=8); +``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2c45108f9a5..d148dc42f0c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1069,7 +1069,7 @@ class IColumn; M(Bool, regexp_dict_flag_case_insensitive, false, "Use case-insensitive matching for a regexp_tree dictionary. Can be overridden in individual expressions with (?i) and (?-i).", 0) \ M(Bool, regexp_dict_flag_dotall, false, "Allow '.' to match newline characters for a regexp_tree dictionary.", 0) \ \ - M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ + M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading dictionary source in several threads. It's supported only by dictionaries with local CLICKHOUSE source.", 0) \ M(Bool, precise_float_parsing, false, "Prefer more precise (but slower) float parsing algorithm", 0) \ // End of FORMAT_FACTORY_SETTINGS diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index e2a5384923f..21da5a3734f 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -10,7 +10,8 @@ #include #include -#include +#include +#include #include #include @@ -52,8 +53,7 @@ CacheDictionary::CacheDictionary( DictionarySourcePtr source_ptr_, CacheDictionaryStoragePtr cache_storage_ptr_, CacheDictionaryUpdateQueueConfiguration update_queue_configuration_, - DictionaryLifetime dict_lifetime_, - bool allow_read_expired_keys_) + CacheDictionaryConfiguration configuration_) : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} @@ -65,9 +65,8 @@ CacheDictionary::CacheDictionary( { update(unit_to_update); }) - , dict_lifetime(dict_lifetime_) + , configuration(configuration_) , log(&Poco::Logger::get("ExternalDictionaries")) - , allow_read_expired_keys(allow_read_expired_keys_) , rnd_engine(randomSeed()) { if (!source_ptr->supportsSelectiveLoad()) @@ -211,7 +210,7 @@ Columns CacheDictionary::getColumns( HashMap requested_keys_to_fetched_columns_during_update_index; MutableColumns fetched_columns_during_update = request.makeAttributesResultColumns(); - if (not_found_keys_size == 0 && expired_keys_size > 0 && allow_read_expired_keys) + if (not_found_keys_size == 0 && expired_keys_size > 0 && configuration.allow_read_expired_keys) { /// Start async update only if allow read expired keys and all keys are found update_queue.tryPushToUpdateQueueOrThrow(update_unit); @@ -316,7 +315,7 @@ ColumnUInt8::Ptr CacheDictionary::hasKeys(const Columns & k allow_expired_keys_during_aggregation = true; } - else if (not_found_keys_size == 0 && expired_keys_size > 0 && allow_read_expired_keys) + else if (not_found_keys_size == 0 && expired_keys_size > 0 && configuration.allow_read_expired_keys) { /// Start async update only if allow read expired keys and all keys are found update_queue.tryPushToUpdateQueueOrThrow(update_unit); @@ -591,7 +590,7 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtrclone(), cache_storage_ptr, update_queue.getConfiguration(), - dict_lifetime, - allow_read_expired_keys); + configuration); } DictionarySourcePtr getSource() const override; - const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } + const DictionaryLifetime & getLifetime() const override { return configuration.lifetime; } const DictionaryStructure & getStructure() const override { return dict_struct; } @@ -194,12 +200,10 @@ private: CacheDictionaryStoragePtr cache_storage_ptr; mutable CacheDictionaryUpdateQueue update_queue; - const DictionaryLifetime dict_lifetime; + const CacheDictionaryConfiguration configuration; Poco::Logger * log; - const bool allow_read_expired_keys; - mutable pcg64 rnd_engine; /// This lock is used for the inner cache state update function lock it for diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index 124d4c8db3f..cfb6a0bcd37 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -59,6 +59,8 @@ public: bool hasUpdateField() const override; + bool isLocal() const { return configuration.is_local; } + DictionarySourcePtr clone() const override { return std::make_shared(*this); } std::string toString() const override; diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index fcad8398c0b..9f82843650d 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -9,6 +9,9 @@ #include #include +#include +#include + namespace DB { @@ -130,4 +133,20 @@ String TransformWithAdditionalColumns::getName() const { return "TransformWithAdditionalColumns"; } + +DictionaryPipelineExecutor::DictionaryPipelineExecutor(QueryPipeline & pipeline_, bool async) + : async_executor(async ? std::make_unique(pipeline_) : nullptr) + , executor(async ? nullptr : std::make_unique(pipeline_)) +{} + +bool DictionaryPipelineExecutor::pull(Block & block) +{ + if (async_executor) + return async_executor->pull(block); + else + return executor->pull(block); +} + +DictionaryPipelineExecutor::~DictionaryPipelineExecutor() = default; + } diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index 39c6e7b3c42..a545b5cdac7 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -16,6 +16,10 @@ namespace DB struct DictionaryStructure; class SettingsChanges; +class PullingPipelineExecutor; +class PullingAsyncPipelineExecutor; +class QueryPipeline; + /// For simple key Block blockForIds( @@ -51,4 +55,17 @@ private: size_t current_range_index = 0; }; +/// Wrapper for `Pulling(Async)PipelineExecutor` to dynamically dispatch calls to the right executor +class DictionaryPipelineExecutor +{ +public: + DictionaryPipelineExecutor(QueryPipeline & pipeline_, bool async); + bool pull(Block & block); + + ~DictionaryPipelineExecutor(); +private: + std::unique_ptr async_executor; + std::unique_ptr executor; +}; + } diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 36a0642abce..64c7eb14024 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -366,10 +366,10 @@ Pipe DirectDictionary::read(const Names & /* column_names * template void DirectDictionary::applySettings(const Settings & settings) { - if (dynamic_cast(source_ptr.get())) + if (const auto * clickhouse_source = dynamic_cast(source_ptr.get())) { /// Only applicable for CLICKHOUSE dictionary source. - use_async_executor = settings.dictionary_use_async_executor; + use_async_executor = settings.dictionary_use_async_executor && clickhouse_source->isLocal(); } } diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 792c5810aa2..33ee697d8c7 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include @@ -24,32 +23,6 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } -namespace -{ - -class PipelineExecutorImpl -{ -public: - PipelineExecutorImpl(QueryPipeline & pipeline_, bool async) - : async_executor(async ? std::make_unique(pipeline_) : nullptr) - , executor(async ? nullptr : std::make_unique(pipeline_)) - {} - - bool pull(Block & block) - { - if (async_executor) - return async_executor->pull(block); - else - return executor->pull(block); - } - -private: - std::unique_ptr async_executor; - std::unique_ptr executor; -}; - -} - template HashedArrayDictionary::HashedArrayDictionary( const StorageID & dict_id_, @@ -437,7 +410,7 @@ void HashedArrayDictionary::updateData() if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { QueryPipeline pipeline(source_ptr->loadUpdatedAll()); - PipelineExecutorImpl executor(pipeline, configuration.use_async_executor); + DictionaryPipelineExecutor executor(pipeline, configuration.use_async_executor); update_field_loaded_block.reset(); Block block; @@ -755,7 +728,7 @@ void HashedArrayDictionary::loadData() { QueryPipeline pipeline; pipeline = QueryPipeline(source_ptr->loadAll()); - PipelineExecutorImpl executor(pipeline, configuration.use_async_executor); + DictionaryPipelineExecutor executor(pipeline, configuration.use_async_executor); UInt64 pull_time_microseconds = 0; UInt64 process_time_microseconds = 0; @@ -918,8 +891,8 @@ void registerDictionaryArrayHashed(DictionaryFactory & factory) ContextMutablePtr context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); const auto & settings = context->getSettingsRef(); - bool is_clickhouse_source = dynamic_cast(source_ptr.get()); - configuration.use_async_executor = is_clickhouse_source && settings.dictionary_use_async_executor; + const auto * clickhouse_source = dynamic_cast(source_ptr.get()); + configuration.use_async_executor = clickhouse_source && clickhouse_source->isLocal() && settings.dictionary_use_async_executor; if (dictionary_key_type == DictionaryKeyType::Simple) return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index d6ee6e369c4..b88d52c2c52 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -20,7 +20,9 @@ #include #include +#include #include +#include #include #include #include @@ -709,7 +711,7 @@ void HashedDictionary::updateData() if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { QueryPipeline pipeline(source_ptr->loadUpdatedAll()); - PullingPipelineExecutor executor(pipeline); + DictionaryPipelineExecutor executor(pipeline, configuration.use_async_executor); update_field_loaded_block.reset(); Block block; @@ -938,7 +940,7 @@ void HashedDictionary::loadData() QueryPipeline pipeline = QueryPipeline(source_ptr->loadAll()); - PullingPipelineExecutor executor(pipeline); + DictionaryPipelineExecutor executor(pipeline, configuration.use_async_executor); Block block; DictionaryKeysArenaHolder arena_holder; @@ -1147,6 +1149,7 @@ void registerDictionaryHashed(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, + ContextPtr global_context, DictionaryKeyType dictionary_key_type, bool sparse) -> DictionaryPtr { @@ -1189,12 +1192,19 @@ void registerDictionaryHashed(DictionaryFactory & factory) if (max_load_factor < 0.5f || max_load_factor > 0.99f) throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}: max_load_factor parameter should be within [0.5, 0.99], got {}", full_name, max_load_factor); + ContextMutablePtr context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); + const auto & settings = context->getSettingsRef(); + + const auto * clickhouse_source = dynamic_cast(source_ptr.get()); + bool use_async_executor = clickhouse_source && clickhouse_source->isLocal() && settings.dictionary_use_async_executor; + HashedDictionaryConfiguration configuration{ static_cast(shards), static_cast(shard_load_queue_backlog), max_load_factor, require_nonempty, dict_lifetime, + use_async_executor, }; if (source_ptr->hasUpdateField() && shards > 1) @@ -1239,13 +1249,13 @@ void registerDictionaryHashed(DictionaryFactory & factory) using namespace std::placeholders; factory.registerLayout("hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ false); }, false); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr global_context, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), global_context, DictionaryKeyType::Simple, /* sparse = */ false); }, false); factory.registerLayout("sparse_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ true); }, false); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr global_context, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), global_context, DictionaryKeyType::Simple, /* sparse = */ true); }, false); factory.registerLayout("complex_key_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ false); }, true); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr global_context, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), global_context, DictionaryKeyType::Complex, /* sparse = */ false); }, true); factory.registerLayout("complex_key_sparse_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ true); }, true); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr global_context, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), global_context, DictionaryKeyType::Complex, /* sparse = */ true); }, true); } diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 30eecb9ab09..3302e667de4 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -28,6 +28,7 @@ struct HashedDictionaryConfiguration const float max_load_factor; const bool require_nonempty; const DictionaryLifetime lifetime; + bool use_async_executor = false; }; template diff --git a/src/Dictionaries/registerCacheDictionaries.cpp b/src/Dictionaries/registerCacheDictionaries.cpp index 0a68f5859d8..b79261955ff 100644 --- a/src/Dictionaries/registerCacheDictionaries.cpp +++ b/src/Dictionaries/registerCacheDictionaries.cpp @@ -2,7 +2,10 @@ #include "CacheDictionaryStorage.h" #include "SSDCacheDictionaryStorage.h" #include + +#include #include +#include #include namespace DB @@ -222,6 +225,16 @@ DictionaryPtr createCacheDictionaryLayout( storage = std::make_shared>(storage_configuration); } #endif + ContextMutablePtr context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); + const auto & settings = context->getSettingsRef(); + + const auto * clickhouse_source = dynamic_cast(source_ptr.get()); + bool use_async_executor = clickhouse_source && clickhouse_source->isLocal() && settings.dictionary_use_async_executor; + CacheDictionaryConfiguration configuration{ + allow_read_expired_keys, + dict_lifetime, + use_async_executor, + }; auto dictionary = std::make_unique>( dictionary_identifier, @@ -229,8 +242,7 @@ DictionaryPtr createCacheDictionaryLayout( std::move(source_ptr), std::move(storage), update_queue_configuration, - dict_lifetime, - allow_read_expired_keys); + configuration); return dictionary; } diff --git a/tests/queries/0_stateless/01765_hashed_dictionary_simple_key.sql b/tests/queries/0_stateless/01765_hashed_dictionary_simple_key.sql index db3431b1572..0b12b2fc8c9 100644 --- a/tests/queries/0_stateless/01765_hashed_dictionary_simple_key.sql +++ b/tests/queries/0_stateless/01765_hashed_dictionary_simple_key.sql @@ -24,7 +24,8 @@ CREATE DICTIONARY 01765_db.hashed_dictionary_simple_key_simple_attributes PRIMARY KEY id SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_simple_attributes_source_table')) LIFETIME(MIN 1 MAX 1000) -LAYOUT(HASHED()); +LAYOUT(HASHED()) +SETTINGS(dictionary_use_async_executor=1, max_threads=8); SELECT 'Dictionary hashed_dictionary_simple_key_simple_attributes'; SELECT 'dictGet existing value'; diff --git a/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.sql b/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.sql index 8d792836562..7d952223705 100644 --- a/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.sql +++ b/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.sql @@ -21,7 +21,8 @@ CREATE DICTIONARY hashed_array_dictionary_simple_key_simple_attributes PRIMARY KEY id SOURCE(CLICKHOUSE(TABLE 'simple_key_simple_attributes_source_table')) LAYOUT(HASHED_ARRAY()) -LIFETIME(MIN 1 MAX 1000); +LIFETIME(MIN 1 MAX 1000) +SETTINGS(dictionary_use_async_executor=1, max_threads=8); SELECT 'Dictionary hashed_array_dictionary_simple_key_simple_attributes'; SELECT 'dictGet existing value'; From a4f5e9e045ce8a1617f0a85cb5a5e26afc41022f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 19 Oct 2023 15:36:48 +0000 Subject: [PATCH 425/634] Improvements based on PR review --- src/Planner/PlannerJoinTree.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 53d61fd2eb8..6f83414fc20 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -53,7 +53,6 @@ #include #include #include -#include #include #include @@ -703,7 +702,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres UInt64 rows_to_read = storage_merge_tree->estimateNumberOfRowsToRead( query_context, storage_snapshot, table_expression_query_info, filter_nodes); - if (max_block_size_limited && max_block_size_limited < rows_to_read) + if (max_block_size_limited && (max_block_size_limited < rows_to_read)) rows_to_read = max_block_size_limited; size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; From 5a44efb64e948d3ae38789301a2a7e81e9f2c41b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 19 Oct 2023 15:44:23 +0000 Subject: [PATCH 426/634] Bump gRPC to v1.37.1 --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index c52656e2bfc..48d1d322358 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit c52656e2bfcda3450bd6a7c247d2d9eeb8498524 +Subproject commit 48d1d322358658d8ac6d9b284de9f4ad1ac2bc6b From 30ae14b67f731208cb5c92a410c116fee3425fff Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 19 Oct 2023 19:43:39 +0200 Subject: [PATCH 427/634] update the list of short messages --- .../00002_log_and_exception_messages_formatting.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index f4ec9b79a4c..63432f127aa 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -45,7 +45,8 @@ create temporary table known_short_messages (s String) as select * from (select 'Unknown identifier: ''{}''', 'User name is empty', 'Expected function, got: {}', 'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'Processed: {}%', 'Creating {}: {}', 'Table {}.{} doesn''t exist', 'Invalid cache key hex: {}', -'User has been dropped', 'Illegal type {} of argument of function {}. Should be DateTime or DateTime64' +'User has been dropped', 'Illegal type {} of argument of function {}. Should be DateTime or DateTime64', +'Bad SSH public key provided', 'Database {} does not exist', 'Substitution {} is not set', 'Invalid cache key hex: {}' ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. @@ -61,7 +62,8 @@ select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_st -- This table currently doesn't have enough information to do this reliably, so we just regex search for " (ERROR_NAME_IN_CAPS)" and hope that's good enough. -- For the "Code: 123. DB::Exception: " part, we just subtract 26 instead of searching for it. Because sometimes it's not at the start, e.g.: -- "Unexpected error, will try to restart main thread: Code: 341. DB::Exception: Unexpected error: Code: 57. DB::Exception:[...]" -select 'exceptions shorter than 30', max2(countDistinctOrDefault(message_format_string), 3) from logs where message ilike '%DB::Exception%' and if(length(regexpExtract(message, '(.*)\\([A-Z0-9_]+\\)')) as pref > 0, pref, length(message)) < 30 + 26 and message_format_string not in known_short_messages; +select 'exceptions shorter than 30', max2(countDistinctOrDefault(message_format_string), 3) from logs + where message ilike '%DB::Exception%' and if(length(regexpExtract(message, '(.*)\\([A-Z0-9_]+\\)')) as pref > 0, pref, length(message)) < 30 + 26 and message_format_string not in known_short_messages; -- Avoid too noisy messages: top 1 message frequency must be less than 30%. We should reduce the threshold From fb5bc095352178c3e29b29c6f6fa9c3fa577a17d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 19 Oct 2023 19:45:16 +0200 Subject: [PATCH 428/634] Fix caching objects in pygithub, and changelogs --- tests/ci/github_helper.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py index 9a8f3ff488d..15ee7dc9620 100644 --- a/tests/ci/github_helper.py +++ b/tests/ci/github_helper.py @@ -127,7 +127,7 @@ class GitHub(github.Github): label="release", ) - def sleep_on_rate_limit(self): + def sleep_on_rate_limit(self) -> None: for limit, data in self.get_rate_limit().raw_data.items(): if data["remaining"] == 0: sleep_time = data["reset"] - int(datetime.now().timestamp()) + 1 @@ -199,13 +199,13 @@ class GitHub(github.Github): # We don't want the cache_updated being always old, # for example in cases when the user is not updated for ages cache_updated = max( - datetime.fromtimestamp(cache_file.stat().st_mtime), cached_obj.updated_at + cache_file.stat().st_mtime, cached_obj.updated_at.timestamp() ) if obj_updated_at is None: # When we don't know about the object is updated or not, # we update it once per hour obj_updated_at = datetime.now() - timedelta(hours=1) - if obj_updated_at <= cache_updated: + if obj_updated_at.timestamp() <= cache_updated: return True, cached_obj return False, cached_obj From 48cff2fcec931470ccdac6ed3d2b763488bde9d5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 19 Oct 2023 13:25:58 +0000 Subject: [PATCH 429/634] Add diagnostic checks for issue 55041 --- src/Access/ContextAccess.cpp | 23 +++++++++++++++++++++++ src/Access/ContextAccess.h | 3 +-- 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index c3aeec15cdd..0c8d8db97cf 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -262,9 +262,14 @@ void ContextAccess::initialize() UserPtr changed_user = entity ? typeid_cast(entity) : nullptr; std::lock_guard lock2{ptr->mutex}; ptr->setUser(changed_user); + if (!ptr->user && !ptr->user_was_dropped) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041, a)"); }); setUser(access_control->read(*params.user_id)); + + if (!user && !user_was_dropped) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041, b)"); } @@ -378,12 +383,16 @@ UserPtr ContextAccess::tryGetUser() const String ContextAccess::getUserName() const { std::lock_guard lock{mutex}; + if (!user && !user_was_dropped) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); return user_name; } std::shared_ptr ContextAccess::getRolesInfo() const { std::lock_guard lock{mutex}; + if (!user && !user_was_dropped) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); if (roles_info) return roles_info; static const auto no_roles = std::make_shared(); @@ -394,6 +403,9 @@ RowPolicyFilterPtr ContextAccess::getRowPolicyFilter(const String & database, co { std::lock_guard lock{mutex}; + if (!user && !user_was_dropped) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); + RowPolicyFilterPtr filter; if (enabled_row_policies) filter = enabled_row_policies->getFilter(database, table_name, filter_type); @@ -414,6 +426,9 @@ std::shared_ptr ContextAccess::getQuota() const { std::lock_guard lock{mutex}; + if (!user && !user_was_dropped) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); + if (!enabled_quota) { if (roles_info) @@ -445,6 +460,8 @@ std::optional ContextAccess::getQuotaUsage() const SettingsChanges ContextAccess::getDefaultSettings() const { std::lock_guard lock{mutex}; + if (!user && !user_was_dropped) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); if (enabled_settings) { if (auto info = enabled_settings->getInfo()) @@ -457,6 +474,8 @@ SettingsChanges ContextAccess::getDefaultSettings() const std::shared_ptr ContextAccess::getDefaultProfileInfo() const { std::lock_guard lock{mutex}; + if (!user && !user_was_dropped) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); if (enabled_settings) return enabled_settings->getInfo(); static const auto everything_by_default = std::make_shared(*access_control); @@ -467,6 +486,8 @@ std::shared_ptr ContextAccess::getDefaultProfileInfo std::shared_ptr ContextAccess::getAccessRights() const { std::lock_guard lock{mutex}; + if (!user && !user_was_dropped) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); if (access) return access; static const auto nothing_granted = std::make_shared(); @@ -477,6 +498,8 @@ std::shared_ptr ContextAccess::getAccessRights() const std::shared_ptr ContextAccess::getAccessRightsWithImplicit() const { std::lock_guard lock{mutex}; + if (!user && !user_was_dropped) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); if (access_with_implicit) return access_with_implicit; static const auto nothing_granted = std::make_shared(); diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 4bd67f8881b..e555dec7f83 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -186,6 +186,7 @@ private: mutable std::atomic user_was_dropped = false; mutable std::atomic trace_log = nullptr; + mutable std::mutex mutex; mutable UserPtr user TSA_GUARDED_BY(mutex); mutable String user_name TSA_GUARDED_BY(mutex); mutable scope_guard subscription_for_user_change TSA_GUARDED_BY(mutex); @@ -198,8 +199,6 @@ private: mutable std::shared_ptr row_policies_of_initial_user TSA_GUARDED_BY(mutex); mutable std::shared_ptr enabled_quota TSA_GUARDED_BY(mutex); mutable std::shared_ptr enabled_settings TSA_GUARDED_BY(mutex); - - mutable std::mutex mutex; }; } From 84f2d46642c3c515451a96820f256d3925413d37 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 19 Oct 2023 18:18:34 +0000 Subject: [PATCH 430/634] Update version_date.tsv and changelogs after v23.3.14.78-lts --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.3.14.78-lts.md | 52 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 3 ++ 5 files changed, 58 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.3.14.78-lts.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 77a8977145a..2a332771fff 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.9.1.1854" +ARG VERSION="23.9.2.56" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index de553b748a0..7f81d10fc2e 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.9.1.1854" +ARG VERSION="23.9.2.56" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index eef4fbf7c74..80a5c869daa 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -30,7 +30,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.9.1.1854" +ARG VERSION="23.9.2.56" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.3.14.78-lts.md b/docs/changelogs/v23.3.14.78-lts.md new file mode 100644 index 00000000000..53ec0458e0d --- /dev/null +++ b/docs/changelogs/v23.3.14.78-lts.md @@ -0,0 +1,52 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.3.14.78-lts (c8f4ba52c65) FIXME as compared to v23.3.13.6-lts (25635e27551) + +#### Build/Testing/Packaging Improvement +* Backported in [#54312](https://github.com/ClickHouse/ClickHouse/issues/54312): Fix strange additional QEMU logging after [#47151](https://github.com/ClickHouse/ClickHouse/issues/47151), see https://s3.amazonaws.com/clickhouse-test-reports/50078/a4743996ee4f3583884d07bcd6501df0cfdaa346/stateless_tests__release__databasereplicated__[3_4].html. [#50442](https://github.com/ClickHouse/ClickHouse/pull/50442) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#55391](https://github.com/ClickHouse/ClickHouse/issues/55391): Resource with source code including submodules is built in Darwin special build task. It may be used to build ClickHouse without checkouting submodules. [#51435](https://github.com/ClickHouse/ClickHouse/pull/51435) ([Ilya Yatsishin](https://github.com/qoega)). +* Backported in [#54703](https://github.com/ClickHouse/ClickHouse/issues/54703): Enrich `changed_images.json` with the latest tag from master for images that are not changed in the pull request. [#54369](https://github.com/ClickHouse/ClickHouse/pull/54369) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#54679](https://github.com/ClickHouse/ClickHouse/issues/54679): We build and upload them for every push, which isn't worth it. [#54675](https://github.com/ClickHouse/ClickHouse/pull/54675) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix parameterized view with cte and multiple usage [#52328](https://github.com/ClickHouse/ClickHouse/pull/52328) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Check recursion depth in OptimizedRegularExpression [#52451](https://github.com/ClickHouse/ClickHouse/pull/52451) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix lightweight delete after drop of projection [#52517](https://github.com/ClickHouse/ClickHouse/pull/52517) ([Anton Popov](https://github.com/CurtizJ)). +* Fix sorting of sparse columns with large limit [#52827](https://github.com/ClickHouse/ClickHouse/pull/52827) ([Anton Popov](https://github.com/CurtizJ)). +* Fix adding sub-second intervals to DateTime [#53309](https://github.com/ClickHouse/ClickHouse/pull/53309) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix: allow IPv6 for bloom filter [#54200](https://github.com/ClickHouse/ClickHouse/pull/54200) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Check for overflow before addition in `analysisOfVariance` function [#54385](https://github.com/ClickHouse/ClickHouse/pull/54385) ([Antonio Andelic](https://github.com/antonio2368)). +* reproduce and fix the bug in removeSharedRecursive [#54430](https://github.com/ClickHouse/ClickHouse/pull/54430) ([Sema Checherinda](https://github.com/CheSema)). +* Fix aggregate projections with normalized states [#54480](https://github.com/ClickHouse/ClickHouse/pull/54480) ([Amos Bird](https://github.com/amosbird)). +* Fix possible parsing error in WithNames formats with disabled input_format_with_names_use_header [#54513](https://github.com/ClickHouse/ClickHouse/pull/54513) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix "Invalid number of rows in Chunk" in MaterializedPostgreSQL [#54844](https://github.com/ClickHouse/ClickHouse/pull/54844) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Prevent attaching parts from tables with different projections or indices [#55062](https://github.com/ClickHouse/ClickHouse/pull/55062) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix trash optimization (up to a certain extent) [#55353](https://github.com/ClickHouse/ClickHouse/pull/55353) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix parsing of arrays in cast operator [#55417](https://github.com/ClickHouse/ClickHouse/pull/55417) ([Anton Popov](https://github.com/CurtizJ)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Backport [#54430](https://github.com/ClickHouse/ClickHouse/issues/54430) to 23.3: reproduce and fix the bug in removeSharedRecursive"'. [#54731](https://github.com/ClickHouse/ClickHouse/pull/54731) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Test libunwind changes. [#51436](https://github.com/ClickHouse/ClickHouse/pull/51436) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Refactor CI_CONFIG [#52948](https://github.com/ClickHouse/ClickHouse/pull/52948) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Use pathlib.Path in S3Helper, rewrite build reports, improve small things [#54010](https://github.com/ClickHouse/ClickHouse/pull/54010) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fixed parameterized_view test after backporting a fix 23.3 [#54401](https://github.com/ClickHouse/ClickHouse/pull/54401) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Update automated commit status comment [#54441](https://github.com/ClickHouse/ClickHouse/pull/54441) ([vdimir](https://github.com/vdimir)). +* S3 artifacts [#54504](https://github.com/ClickHouse/ClickHouse/pull/54504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix CI skip build and skip tests checks [#54532](https://github.com/ClickHouse/ClickHouse/pull/54532) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Update WebObjectStorage.cpp [#54695](https://github.com/ClickHouse/ClickHouse/pull/54695) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Do not set PR status label [#54799](https://github.com/ClickHouse/ClickHouse/pull/54799) ([vdimir](https://github.com/vdimir)). +* Get rid of the most of `os.path` stuff [#55028](https://github.com/ClickHouse/ClickHouse/pull/55028) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* check if block is empty after async insert retries [#55143](https://github.com/ClickHouse/ClickHouse/pull/55143) ([Han Fei](https://github.com/hanfei1991)). +* MaterializedPostgreSQL: remove back check [#55297](https://github.com/ClickHouse/ClickHouse/pull/55297) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove existing moving/ dir if allow_remove_stale_moving_parts is off [#55480](https://github.com/ClickHouse/ClickHouse/pull/55480) ([Mike Kot](https://github.com/myrrc)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 77dd2690a60..df6aa706b4b 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,4 +1,6 @@ +v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.4.69-lts 2023-10-19 v23.8.3.48-lts 2023-09-27 v23.8.2.7-lts 2023-09-04 v23.8.1.2992-lts 2023-09-01 @@ -21,6 +23,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.14.78-lts 2023-10-18 v23.3.13.6-lts 2023-09-05 v23.3.12.11-lts 2023-09-04 v23.3.11.5-lts 2023-08-28 From 3bbaec3fd7d80922da7f04d01a53cc2226c3a284 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 19 Oct 2023 18:22:10 +0000 Subject: [PATCH 431/634] FunctionOpDate::getReturnTypeImpl(): Use validateFunctionArgumentTypes --- src/Functions/FunctionsOpDate.cpp | 20 ++++++-------------- 1 file changed, 6 insertions(+), 14 deletions(-) diff --git a/src/Functions/FunctionsOpDate.cpp b/src/Functions/FunctionsOpDate.cpp index dcbc1fc1256..997513109d6 100644 --- a/src/Functions/FunctionsOpDate.cpp +++ b/src/Functions/FunctionsOpDate.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -22,7 +23,6 @@ public: explicit FunctionOpDate(ContextPtr context_) : context(context_) {} - static FunctionPtr create(ContextPtr context) { return std::make_shared>(context); } String getName() const override { return name; } @@ -32,19 +32,11 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 1st argument of function {}. Should be a date or a date with time", - arguments[0].type->getName(), - getName()); - - if (!isInterval(arguments[1].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 2nd argument of function {}. Should be an interval", - arguments[1].type->getName(), - getName()); + FunctionArgumentDescriptors args{ + {"date", &isDateOrDate32OrDateTimeOrDateTime64, nullptr, "Date or date with time"}, + {"interval", &isInterval, nullptr, "Interval"} + }; + validateFunctionArgumentTypes(*this, arguments, args); auto op = FunctionFactory::instance().get(Op::internal_name, context); auto op_build = op->build(arguments); From 21d3c51cdeb9cc56e490e621a18c71502803e479 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 19 Oct 2023 18:38:32 +0000 Subject: [PATCH 432/634] Update version_date.tsv and changelogs after v23.9.2.56-stable --- docs/changelogs/v23.9.2.56-stable.md | 46 ++++++++++++++++++++++++++++ 1 file changed, 46 insertions(+) create mode 100644 docs/changelogs/v23.9.2.56-stable.md diff --git a/docs/changelogs/v23.9.2.56-stable.md b/docs/changelogs/v23.9.2.56-stable.md new file mode 100644 index 00000000000..3256c1b8906 --- /dev/null +++ b/docs/changelogs/v23.9.2.56-stable.md @@ -0,0 +1,46 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.9.2.56-stable (a1bf3f1de55) FIXME as compared to v23.9.1.1854-stable (8f9a227de1f) + +#### Build/Testing/Packaging Improvement +* Backported in [#55295](https://github.com/ClickHouse/ClickHouse/issues/55295): Resource with source code including submodules is built in Darwin special build task. It may be used to build ClickHouse without checkouting submodules. [#51435](https://github.com/ClickHouse/ClickHouse/pull/51435) ([Ilya Yatsishin](https://github.com/qoega)). +* Backported in [#55368](https://github.com/ClickHouse/ClickHouse/issues/55368): Solve issue with launching standalone clickhouse-keeper from clickhouse-server package. [#55226](https://github.com/ClickHouse/ClickHouse/pull/55226) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#55727](https://github.com/ClickHouse/ClickHouse/issues/55727): Fix integration check python script to use gh api url - Add Readme for CI tests. [#55716](https://github.com/ClickHouse/ClickHouse/pull/55716) ([Max K.](https://github.com/mkaynov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix deadlock in LDAP assigned role update [#55119](https://github.com/ClickHouse/ClickHouse/pull/55119) ([Julian Maicher](https://github.com/jmaicher)). +* Fix for background download in fs cache [#55252](https://github.com/ClickHouse/ClickHouse/pull/55252) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix functions execution over sparse columns [#55275](https://github.com/ClickHouse/ClickHouse/pull/55275) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect merging of Nested for SELECT FINAL FROM SummingMergeTree [#55276](https://github.com/ClickHouse/ClickHouse/pull/55276) ([Azat Khuzhin](https://github.com/azat)). +* Fix bug with inability to drop detached partition in replicated merge tree on top of S3 without zero copy [#55309](https://github.com/ClickHouse/ClickHouse/pull/55309) ([alesapin](https://github.com/alesapin)). +* Fix SIGSEGV in MergeSortingPartialResultTransform (due to zero chunks after remerge()) [#55335](https://github.com/ClickHouse/ClickHouse/pull/55335) ([Azat Khuzhin](https://github.com/azat)). +* Fix data-race in CreatingSetsTransform (on errors) due to throwing shared exception [#55338](https://github.com/ClickHouse/ClickHouse/pull/55338) ([Azat Khuzhin](https://github.com/azat)). +* Fix trash optimization (up to a certain extent) [#55353](https://github.com/ClickHouse/ClickHouse/pull/55353) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix parsing of arrays in cast operator [#55417](https://github.com/ClickHouse/ClickHouse/pull/55417) ([Anton Popov](https://github.com/CurtizJ)). +* Fix filtering by virtual columns with OR filter in query [#55418](https://github.com/ClickHouse/ClickHouse/pull/55418) ([Azat Khuzhin](https://github.com/azat)). +* Fix MongoDB connection issues [#55419](https://github.com/ClickHouse/ClickHouse/pull/55419) ([Nikolay Degterinsky](https://github.com/evillique)). +* Destroy fiber in case of exception in cancelBefore in AsyncTaskExecutor [#55516](https://github.com/ClickHouse/ClickHouse/pull/55516) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix crash in QueryNormalizer with cyclic aliases [#55602](https://github.com/ClickHouse/ClickHouse/pull/55602) ([vdimir](https://github.com/vdimir)). +* Fix filtering by virtual columns with OR filter in query (resubmit) [#55678](https://github.com/ClickHouse/ClickHouse/pull/55678) ([Azat Khuzhin](https://github.com/azat)). + +#### NO CL CATEGORY + +* Backported in [#55708](https://github.com/ClickHouse/ClickHouse/issues/55708):. [#55657](https://github.com/ClickHouse/ClickHouse/pull/55657) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#55691](https://github.com/ClickHouse/ClickHouse/issues/55691):. [#55682](https://github.com/ClickHouse/ClickHouse/pull/55682) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Add setting allow_experimental_partial_result [#54514](https://github.com/ClickHouse/ClickHouse/pull/54514) ([vdimir](https://github.com/vdimir)). +* Fix CI skip build and skip tests checks [#54532](https://github.com/ClickHouse/ClickHouse/pull/54532) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* check if block is empty after async insert retries [#55143](https://github.com/ClickHouse/ClickHouse/pull/55143) ([Han Fei](https://github.com/hanfei1991)). +* MaterializedPostgreSQL: remove back check [#55297](https://github.com/ClickHouse/ClickHouse/pull/55297) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Review [#51946](https://github.com/ClickHouse/ClickHouse/issues/51946) and partially revert it [#55336](https://github.com/ClickHouse/ClickHouse/pull/55336) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove existing moving/ dir if allow_remove_stale_moving_parts is off [#55480](https://github.com/ClickHouse/ClickHouse/pull/55480) ([Mike Kot](https://github.com/myrrc)). +* Bump curl to 8.4 [#55492](https://github.com/ClickHouse/ClickHouse/pull/55492) ([Robert Schulze](https://github.com/rschu1ze)). + From 66030152b02e941e317f9e6088f1090db9b6e585 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 19 Oct 2023 18:38:56 +0000 Subject: [PATCH 433/634] Update version_date.tsv and changelogs after v23.8.4.69-lts --- docs/changelogs/v23.8.4.69-lts.md | 51 +++++++++++++++++++++++++++++++ 1 file changed, 51 insertions(+) create mode 100644 docs/changelogs/v23.8.4.69-lts.md diff --git a/docs/changelogs/v23.8.4.69-lts.md b/docs/changelogs/v23.8.4.69-lts.md new file mode 100644 index 00000000000..065a57549be --- /dev/null +++ b/docs/changelogs/v23.8.4.69-lts.md @@ -0,0 +1,51 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.8.4.69-lts (d4d1e7b9ded) FIXME as compared to v23.8.3.48-lts (ebe4eb3d23e) + +#### Build/Testing/Packaging Improvement +* Backported in [#55673](https://github.com/ClickHouse/ClickHouse/issues/55673): If the database is already initialized, it doesn't need to be initialized again upon subsequent launches. This can potentially fix the issue of infinite container restarts when the database fails to load within 1000 attempts (relevant for very large databases and multi-node setups). [#50724](https://github.com/ClickHouse/ClickHouse/pull/50724) ([Alexander Nikolaev](https://github.com/AlexNik)). +* Backported in [#55293](https://github.com/ClickHouse/ClickHouse/issues/55293): Resource with source code including submodules is built in Darwin special build task. It may be used to build ClickHouse without checkouting submodules. [#51435](https://github.com/ClickHouse/ClickHouse/pull/51435) ([Ilya Yatsishin](https://github.com/qoega)). +* Backported in [#55366](https://github.com/ClickHouse/ClickHouse/issues/55366): Solve issue with launching standalone clickhouse-keeper from clickhouse-server package. [#55226](https://github.com/ClickHouse/ClickHouse/pull/55226) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#55725](https://github.com/ClickHouse/ClickHouse/issues/55725): Fix integration check python script to use gh api url - Add Readme for CI tests. [#55716](https://github.com/ClickHouse/ClickHouse/pull/55716) ([Max K.](https://github.com/mkaynov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix "Invalid number of rows in Chunk" in MaterializedPostgreSQL [#54844](https://github.com/ClickHouse/ClickHouse/pull/54844) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Move obsolete format settings to separate section [#54855](https://github.com/ClickHouse/ClickHouse/pull/54855) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix: insert quorum w/o keeper retries [#55026](https://github.com/ClickHouse/ClickHouse/pull/55026) ([Igor Nikonov](https://github.com/devcrafter)). +* Prevent attaching parts from tables with different projections or indices [#55062](https://github.com/ClickHouse/ClickHouse/pull/55062) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Proper cleanup in case of exception in ctor of ShellCommandSource [#55103](https://github.com/ClickHouse/ClickHouse/pull/55103) ([Alexander Gololobov](https://github.com/davenger)). +* Fix deadlock in LDAP assigned role update [#55119](https://github.com/ClickHouse/ClickHouse/pull/55119) ([Julian Maicher](https://github.com/jmaicher)). +* Fix for background download in fs cache [#55252](https://github.com/ClickHouse/ClickHouse/pull/55252) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix functions execution over sparse columns [#55275](https://github.com/ClickHouse/ClickHouse/pull/55275) ([Azat Khuzhin](https://github.com/azat)). +* Fix bug with inability to drop detached partition in replicated merge tree on top of S3 without zero copy [#55309](https://github.com/ClickHouse/ClickHouse/pull/55309) ([alesapin](https://github.com/alesapin)). +* Fix trash optimization (up to a certain extent) [#55353](https://github.com/ClickHouse/ClickHouse/pull/55353) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix parsing of arrays in cast operator [#55417](https://github.com/ClickHouse/ClickHouse/pull/55417) ([Anton Popov](https://github.com/CurtizJ)). +* Fix filtering by virtual columns with OR filter in query [#55418](https://github.com/ClickHouse/ClickHouse/pull/55418) ([Azat Khuzhin](https://github.com/azat)). +* Fix MongoDB connection issues [#55419](https://github.com/ClickHouse/ClickHouse/pull/55419) ([Nikolay Degterinsky](https://github.com/evillique)). +* Destroy fiber in case of exception in cancelBefore in AsyncTaskExecutor [#55516](https://github.com/ClickHouse/ClickHouse/pull/55516) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix crash in QueryNormalizer with cyclic aliases [#55602](https://github.com/ClickHouse/ClickHouse/pull/55602) ([vdimir](https://github.com/vdimir)). +* Fix filtering by virtual columns with OR filter in query (resubmit) [#55678](https://github.com/ClickHouse/ClickHouse/pull/55678) ([Azat Khuzhin](https://github.com/azat)). + +#### NO CL CATEGORY + +* Backported in [#55706](https://github.com/ClickHouse/ClickHouse/issues/55706):. [#55657](https://github.com/ClickHouse/ClickHouse/pull/55657) ([Antonio Andelic](https://github.com/antonio2368)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* S3 artifacts [#54504](https://github.com/ClickHouse/ClickHouse/pull/54504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix CI skip build and skip tests checks [#54532](https://github.com/ClickHouse/ClickHouse/pull/54532) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Update WebObjectStorage.cpp [#54695](https://github.com/ClickHouse/ClickHouse/pull/54695) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Do not set PR status label [#54799](https://github.com/ClickHouse/ClickHouse/pull/54799) ([vdimir](https://github.com/vdimir)). +* Get rid of the most of `os.path` stuff [#55028](https://github.com/ClickHouse/ClickHouse/pull/55028) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Clean data dir and always start an old server version in aggregate functions compatibility test. [#55105](https://github.com/ClickHouse/ClickHouse/pull/55105) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* check if block is empty after async insert retries [#55143](https://github.com/ClickHouse/ClickHouse/pull/55143) ([Han Fei](https://github.com/hanfei1991)). +* MaterializedPostgreSQL: remove back check [#55297](https://github.com/ClickHouse/ClickHouse/pull/55297) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove existing moving/ dir if allow_remove_stale_moving_parts is off [#55480](https://github.com/ClickHouse/ClickHouse/pull/55480) ([Mike Kot](https://github.com/myrrc)). +* Bump curl to 8.4 [#55492](https://github.com/ClickHouse/ClickHouse/pull/55492) ([Robert Schulze](https://github.com/rschu1ze)). + From 71faa8b1989f0bd87dfd4965c947ff64b8bbf562 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 19 Oct 2023 22:16:07 +0000 Subject: [PATCH 434/634] Improve checks --- src/Access/ContextAccess.cpp | 18 ++++++++++-------- src/Access/ContextAccess.h | 1 + 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 0c8d8db97cf..90fddd0085d 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -270,6 +270,8 @@ void ContextAccess::initialize() if (!user && !user_was_dropped) throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041, b)"); + + initialized = true; } @@ -383,7 +385,7 @@ UserPtr ContextAccess::tryGetUser() const String ContextAccess::getUserName() const { std::lock_guard lock{mutex}; - if (!user && !user_was_dropped) + if (initialized && !user && !user_was_dropped) throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); return user_name; } @@ -391,7 +393,7 @@ String ContextAccess::getUserName() const std::shared_ptr ContextAccess::getRolesInfo() const { std::lock_guard lock{mutex}; - if (!user && !user_was_dropped) + if (initialized && !user && !user_was_dropped) throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); if (roles_info) return roles_info; @@ -403,7 +405,7 @@ RowPolicyFilterPtr ContextAccess::getRowPolicyFilter(const String & database, co { std::lock_guard lock{mutex}; - if (!user && !user_was_dropped) + if (initialized && !user && !user_was_dropped) throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); RowPolicyFilterPtr filter; @@ -426,7 +428,7 @@ std::shared_ptr ContextAccess::getQuota() const { std::lock_guard lock{mutex}; - if (!user && !user_was_dropped) + if (initialized && !user && !user_was_dropped) throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); if (!enabled_quota) @@ -460,7 +462,7 @@ std::optional ContextAccess::getQuotaUsage() const SettingsChanges ContextAccess::getDefaultSettings() const { std::lock_guard lock{mutex}; - if (!user && !user_was_dropped) + if (initialized && !user && !user_was_dropped) throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); if (enabled_settings) { @@ -474,7 +476,7 @@ SettingsChanges ContextAccess::getDefaultSettings() const std::shared_ptr ContextAccess::getDefaultProfileInfo() const { std::lock_guard lock{mutex}; - if (!user && !user_was_dropped) + if (initialized && !user && !user_was_dropped) throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); if (enabled_settings) return enabled_settings->getInfo(); @@ -486,7 +488,7 @@ std::shared_ptr ContextAccess::getDefaultProfileInfo std::shared_ptr ContextAccess::getAccessRights() const { std::lock_guard lock{mutex}; - if (!user && !user_was_dropped) + if (initialized && !user && !user_was_dropped) throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); if (access) return access; @@ -498,7 +500,7 @@ std::shared_ptr ContextAccess::getAccessRights() const std::shared_ptr ContextAccess::getAccessRightsWithImplicit() const { std::lock_guard lock{mutex}; - if (!user && !user_was_dropped) + if (initialized && !user && !user_was_dropped) throw Exception(ErrorCodes::LOGICAL_ERROR, "ContextAccess is inconsistent (bug 55041)"); if (access_with_implicit) return access_with_implicit; diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index e555dec7f83..caf903b85bd 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -183,6 +183,7 @@ private: const AccessControl * access_control = nullptr; const Params params; + mutable std::atomic initialized = false; // can be removed after Bug 5504 is resolved mutable std::atomic user_was_dropped = false; mutable std::atomic trace_log = nullptr; From 194eca91f8fed9c9211442e2d0368205c75c72ea Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 19 Oct 2023 20:39:07 +0200 Subject: [PATCH 435/634] fix node setting in the test --- .../configs/s3_retries.xml | 9 +++++++++ .../test_checking_s3_blobs_paranoid/configs/setting.xml | 2 -- .../integration/test_checking_s3_blobs_paranoid/test.py | 1 + 3 files changed, 10 insertions(+), 2 deletions(-) create mode 100644 tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml new file mode 100644 index 00000000000..556bf60d385 --- /dev/null +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml @@ -0,0 +1,9 @@ + + + + + + 5 + + + diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml index 3bc0f322cb8..23ab57f9330 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml @@ -5,10 +5,8 @@ 1 1 - 5 - diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index c5b78e139bd..d6bcb3fb8f4 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -19,6 +19,7 @@ def cluster(): ], user_configs=[ "configs/setting.xml", + "configs/s3_retries.xml", ], with_minio=True, ) From c6e998290da1b0172919b2f1de2238bacf184865 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 20 Oct 2023 05:14:09 +0200 Subject: [PATCH 436/634] Allow querying inside a dropped database in clickhouse-local --- src/Client/LocalConnection.cpp | 8 +++++++- ...ickhouse_local_drop_current_database.reference | 10 ++++++++++ ...2900_clickhouse_local_drop_current_database.sh | 15 +++++++++++++++ 3 files changed, 32 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.reference create mode 100755 tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.sh diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index c05f039ffb6..857b9147487 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -95,12 +95,18 @@ void LocalConnection::sendQuery( else query_context = session.makeQueryContext(); query_context->setCurrentQueryId(query_id); + if (send_progress) { query_context->setProgressCallback([this] (const Progress & value) { this->updateProgress(value); }); query_context->setFileProgressCallback([this](const FileProgress & value) { this->updateProgress(Progress(value)); }); } - if (!current_database.empty()) + + /// Switch the database to the desired one (set by the USE query) + /// but don't attempt to do it if we are already in that database. + /// (there is a rare case when it matters - if we deleted the current database, + // we can still do some queries, but we cannot switch to the same database) + if (!current_database.empty() && current_database != query_context->getCurrentDatabase()) query_context->setCurrentDatabase(current_database); query_context->addQueryParameters(query_parameters); diff --git a/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.reference b/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.reference new file mode 100644 index 00000000000..4a321380536 --- /dev/null +++ b/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.reference @@ -0,0 +1,10 @@ +CREATE DATABASE foo; +USE foo; +SELECT 1; +1 +DROP DATABASE foo; +SELECT 2; +2 +USE _local; +SELECT 3; +3 diff --git a/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.sh b/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.sh new file mode 100755 index 00000000000..1af40f8778d --- /dev/null +++ b/tests/queries/0_stateless/02900_clickhouse_local_drop_current_database.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_LOCAL} --echo --multiquery " + CREATE DATABASE foo; + USE foo; + SELECT 1; + DROP DATABASE foo; + SELECT 2; + USE _local; + SELECT 3; +" From fe1df969a7a25b9dc9777e286676d588422344dd Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Fri, 20 Oct 2023 09:04:39 +0200 Subject: [PATCH 437/634] Add load_metadata_threads to the documentation --- docs/en/operations/storing-data.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index fc2933aa2cf..7e72f177385 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -198,6 +198,8 @@ These settings should be defined in the disk configuration section. - `max_elements` - a limit for a number of cache files. Default: `10000000`. +- `load_metadata_threads` - number of threads being used to load cache metadata on starting time. Default: `1`. + File Cache **query/profile settings**: Some of these settings will disable cache features per query/profile that are enabled by default or in disk configuration settings. For example, you can enable cache in disk configuration and disable it per query/profile setting `enable_filesystem_cache` to `false`. Also setting `cache_on_write_operations` to `true` in disk configuration means that "write-though" cache is enabled. But if you need to disable this general setting per specific queries then setting `enable_filesystem_cache_on_write_operations` to `false` means that write operations cache will be disabled for a specific query/profile. From 01093d4698c3381bed7bf69505d6bb9d62360e76 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Fri, 20 Oct 2023 09:11:46 +0200 Subject: [PATCH 438/634] Remove deprecated setting do_not_evict_index_and_mark_files --- .../server-configuration-parameters/settings.md | 1 - docs/en/operations/storing-data.md | 10 ++++------ .../configs/config.d/storage_conf.xml | 1 - .../configs/config.d/storage_configuration.xml | 1 - 4 files changed, 4 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index bd8e1da2f1e..c017fb7b9f5 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -569,7 +569,6 @@ Both the cache for `local_disk`, and temporary data will be stored in `/tiny_loc 10M 1M 1 - 0 diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 7e72f177385..796f65a9d30 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -114,7 +114,7 @@ Example of disk configuration: ## Using local cache {#using-local-cache} -It is possible to configure local cache over disks in storage configuration starting from version 22.3. +It is possible to configure local cache over disks in storage configuration starting from version 22.3. For versions 22.3 - 22.7 cache is supported only for `s3` disk type. For versions >= 22.8 cache is supported for any disk type: S3, Azure, Local, Encrypted, etc. For versions >= 23.5 cache is supported only for remote disk types: S3, Azure, HDFS. Cache uses `LRU` cache policy. @@ -192,8 +192,6 @@ These settings should be defined in the disk configuration section. - `enable_bypass_cache_with_threshold` - allows to skip cache completely in case the requested read range exceeds the threshold. Default: `false`. This threshold can be defined by `bypass_cache_threashold`. Default: `268435456` (`256Mi`). -- `do_not_evict_index_and_mark_files` - do not evict small frequently used files according to cache policy. Default: `false`. This setting was added in version 22.8. If you used filesystem cache before this version, then it will not work on versions starting from 22.8 if this setting is set to `true`. If you want to use this setting, clear old cache created before version 22.8 before upgrading. - - `max_file_segment_size` - a maximum size of a single cache file in bytes or in readable format (`ki, Mi, Gi, etc`, example `10Gi`). Default: `8388608` (`8Mi`). - `max_elements` - a limit for a number of cache files. Default: `10000000`. @@ -250,9 +248,9 @@ DESCRIBE FILESYSTEM CACHE 's3_cache' ``` ``` text -┌────max_size─┬─max_elements─┬─max_file_segment_size─┬─cache_on_write_operations─┬─enable_cache_hits_threshold─┬─current_size─┬─current_elements─┬─path────────┬─do_not_evict_index_and_mark_files─┐ -│ 10000000000 │ 1048576 │ 104857600 │ 1 │ 0 │ 3276 │ 54 │ /s3_cache/ │ 1 │ -└─────────────┴──────────────┴───────────────────────┴───────────────────────────┴─────────────────────────────┴──────────────┴──────────────────┴─────────────┴───────────────────────────────────┘ +┌────max_size─┬─max_elements─┬─max_file_segment_size─┬─boundary_alignment─┬─cache_on_write_operations─┬─cache_hits_threshold─┬─current_size─┬─current_elements─┬─path───────┬─background_download_threads─┬─enable_bypass_cache_with_threshold─┐ +│ 10000000000 │ 1048576 │ 104857600 │ 4194304 │ 1 │ 0 │ 3276 │ 54 │ /s3_cache/ │ 2 │ 0 │ +└─────────────┴──────────────┴───────────────────────┴────────────────────┴───────────────────────────┴──────────────────────┴──────────────┴──────────────────┴────────────┴─────────────────────────────┴────────────────────────────────────┘ ``` Cache current metrics: diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml index 9269776301d..c12bdf064ce 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml @@ -78,7 +78,6 @@ s3_r /s3_cache_r/ 1000000000 - 1 diff --git a/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml index b527c74e8de..b753123a5ef 100644 --- a/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml +++ b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml @@ -13,7 +13,6 @@ 10M 1M 1 - 0 From 6e3f35fc942b332c1506668edc2fc5be1242a88b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 20 Oct 2023 16:18:39 +0800 Subject: [PATCH 439/634] fix issue https://github.com/ClickHouse/ClickHouse/issues/55858 --- src/IO/readFloatText.h | 6 +++--- tests/queries/0_stateless/02900_issue_55858.reference | 4 ++++ tests/queries/0_stateless/02900_issue_55858.sql | 10 ++++++++++ 3 files changed, 17 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02900_issue_55858.reference create mode 100644 tests/queries/0_stateless/02900_issue_55858.sql diff --git a/src/IO/readFloatText.h b/src/IO/readFloatText.h index feab9589c2e..e3d549c8e17 100644 --- a/src/IO/readFloatText.h +++ b/src/IO/readFloatText.h @@ -148,11 +148,11 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf) static_assert('a' > '.' && 'A' > '.' && '\n' < '.' && '\t' < '.' && '\'' < '.' && '"' < '.', "Layout of char is not like ASCII"); static constexpr bool throw_exception = std::is_same_v; - - /// Fast path (avoid copying) if the buffer have at least MAX_LENGTH bytes. static constexpr int MAX_LENGTH = 316; + ReadBufferFromMemory * buf_from_memory = dynamic_cast(&buf); - if (likely(!buf.eof() && buf.position() + MAX_LENGTH <= buf.buffer().end())) + /// Fast path (avoid copying) if the buffer have at least MAX_LENGTH bytes or buf is ReadBufferFromMemory + if (likely(!buf.eof() && (buf_from_memory || buf.position() + MAX_LENGTH <= buf.buffer().end()))) { auto * initial_position = buf.position(); auto res = fast_float::from_chars(initial_position, buf.buffer().end(), x); diff --git a/tests/queries/0_stateless/02900_issue_55858.reference b/tests/queries/0_stateless/02900_issue_55858.reference new file mode 100644 index 00000000000..1d64c9ea17c --- /dev/null +++ b/tests/queries/0_stateless/02900_issue_55858.reference @@ -0,0 +1,4 @@ +0 +0 +\N +\N diff --git a/tests/queries/0_stateless/02900_issue_55858.sql b/tests/queries/0_stateless/02900_issue_55858.sql new file mode 100644 index 00000000000..b7b6704cdb5 --- /dev/null +++ b/tests/queries/0_stateless/02900_issue_55858.sql @@ -0,0 +1,10 @@ +set precise_float_parsing = 1; + +select cast('2023-01-01' as Float64); -- { serverError 6 } +select cast('2023-01-01' as Float32); -- { serverError 6 } +select toFloat32('2023-01-01'); -- { serverError 6 } +select toFloat64('2023-01-01'); -- { serverError 6 } +select toFloat32OrZero('2023-01-01'); +select toFloat64OrZero('2023-01-01'); +select toFloat32OrNull('2023-01-01'); +select toFloat64OrNull('2023-01-01'); From 57d60dc4f27e8f4550b115681a33000d76112b33 Mon Sep 17 00:00:00 2001 From: helifu Date: Fri, 20 Oct 2023 17:38:44 +0800 Subject: [PATCH 440/634] Add a new column xid for zookeeper_connection (#50702) * Refactor the code * Add a new column xid for zookeeper_connection * Support hostnames in the configuration * Fix a typo * Fix a typo * Fix a typo * Fix a bug about connect_time * Update test case * Update test case * Fix a special build check error * Resolve conflicts caused by rebase. * Update failed test case * Refactor the code according to comment * Fix two compilation errors --- .../system-tables/zookeeper_connection.md | 1 + src/Common/ZooKeeper/IKeeper.h | 9 +++ src/Common/ZooKeeper/TestKeeper.h | 3 + src/Common/ZooKeeper/ZooKeeper.cpp | 24 ++++-- src/Common/ZooKeeper/ZooKeeper.h | 13 +--- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 11 +-- src/Common/ZooKeeper/ZooKeeperImpl.h | 11 ++- .../examples/zkutil_test_commands_new_lib.cpp | 5 +- .../StorageSystemZooKeeperConnection.cpp | 77 +++++++++++-------- tests/config/config.d/zookeeper.xml | 2 +- .../test_zookeeper_fallback_session/test.py | 7 +- .../01158_zookeeper_log_long.reference | 10 +-- ...2735_system_zookeeper_connection.reference | 4 +- utils/keeper-bench/Runner.cpp | 8 +- utils/keeper-bench/Runner.h | 2 +- 15 files changed, 110 insertions(+), 77 deletions(-) diff --git a/docs/en/operations/system-tables/zookeeper_connection.md b/docs/en/operations/system-tables/zookeeper_connection.md index 2f0ed303ce3..2d257446dca 100644 --- a/docs/en/operations/system-tables/zookeeper_connection.md +++ b/docs/en/operations/system-tables/zookeeper_connection.md @@ -16,6 +16,7 @@ Columns: - `is_expired` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Is the current connection expired. - `keeper_api_version` ([String](../../sql-reference/data-types/string.md)) — Keeper API version. - `client_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Session id of the connection. +- `xid` ([Int32](../../sql-reference/data-types/int-uint.md)) — Xid of the current session. Example: diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 8ab0de4a21d..6d6f8afbaee 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -544,6 +544,15 @@ public: /// If expired, you can only destroy the object. All other methods will throw exception. virtual bool isExpired() const = 0; + /// Get the current connected node idx. + virtual Int8 getConnectedNodeIdx() const = 0; + + /// Get the current connected host and port. + virtual String getConnectedHostPort() const = 0; + + /// Get the xid of current connection. + virtual int32_t getConnectionXid() const = 0; + /// Useful to check owner of ephemeral node. virtual int64_t getSessionID() const = 0; diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 3a760270207..36db5accff1 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -40,6 +40,9 @@ public: bool isExpired() const override { return expired; } bool hasReachedDeadline() const override { return false; } + Int8 getConnectedNodeIdx() const override { return 0; } + String getConnectedHostPort() const override { return "TestKeeper:0000"; } + int32_t getConnectionXid() const override { return 0; } int64_t getSessionID() const override { return 0; } diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index cf1d94662c7..436a4e14f14 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -89,7 +89,7 @@ void ZooKeeper::init(ZooKeeperArgs args_) const Poco::Net::SocketAddress host_socket_addr{host_string}; LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, host_socket_addr.toString()); - nodes.emplace_back(Coordination::ZooKeeper::Node{host_socket_addr, secure}); + nodes.emplace_back(Coordination::ZooKeeper::Node{host_socket_addr, host.original_index, secure}); } catch (const Poco::Net::HostNotFoundException & e) { @@ -113,12 +113,7 @@ void ZooKeeper::init(ZooKeeperArgs args_) throw KeeperException::fromMessage(Coordination::Error::ZCONNECTIONLOSS, "Cannot use any of provided ZooKeeper nodes"); } - impl = std::make_unique(nodes, args, zk_log, [this](size_t node_idx, const Coordination::ZooKeeper::Node & node) - { - connected_zk_host = node.address.host().toString(); - connected_zk_port = node.address.port(); - connected_zk_index = node_idx; - }); + impl = std::make_unique(nodes, args, zk_log); if (args.chroot.empty()) LOG_TRACE(log, "Initialized, hosts: {}", fmt::join(args.hosts, ",")); @@ -179,6 +174,7 @@ std::vector ZooKeeper::shuffleHosts() const { ShuffleHost shuffle_host; shuffle_host.host = args.hosts[i]; + shuffle_host.original_index = static_cast(i); if (get_priority) shuffle_host.priority = get_priority(i); shuffle_host.randomize(); @@ -1312,6 +1308,20 @@ void ZooKeeper::setServerCompletelyStarted() zk->setServerCompletelyStarted(); } +Int8 ZooKeeper::getConnectedHostIdx() const +{ + return impl->getConnectedNodeIdx(); +} + +String ZooKeeper::getConnectedHostPort() const +{ + return impl->getConnectedHostPort(); +} + +int32_t ZooKeeper::getConnectionXid() const +{ + return impl->getConnectionXid(); +} size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination::Responses & responses) { diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index d61156b31a9..c41d1d8dbab 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -51,6 +51,7 @@ constexpr size_t MULTI_BATCH_SIZE = 100; struct ShuffleHost { String host; + UInt8 original_index = 0; Priority priority; UInt64 random = 0; @@ -551,10 +552,9 @@ public: void setServerCompletelyStarted(); - String getConnectedZooKeeperHost() const { return connected_zk_host; } - UInt16 getConnectedZooKeeperPort() const { return connected_zk_port; } - size_t getConnectedZooKeeperIndex() const { return connected_zk_index; } - UInt64 getConnectedTime() const { return connected_time; } + Int8 getConnectedHostIdx() const; + String getConnectedHostPort() const; + int32_t getConnectionXid() const; const DB::KeeperFeatureFlags * getKeeperFeatureFlags() const { return impl->getKeeperFeatureFlags(); } @@ -621,11 +621,6 @@ private: ZooKeeperArgs args; - String connected_zk_host; - UInt16 connected_zk_port; - size_t connected_zk_index; - UInt64 connected_time = timeInSeconds(std::chrono::system_clock::now()); - std::mutex mutex; Poco::Logger * log = nullptr; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 164ad6c0b65..709dabf1506 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -313,8 +313,8 @@ ZooKeeper::~ZooKeeper() ZooKeeper::ZooKeeper( const Nodes & nodes, const zkutil::ZooKeeperArgs & args_, - std::shared_ptr zk_log_, std::optional && connected_callback_) - : args(args_), connected_callback(std::move(connected_callback_)) + std::shared_ptr zk_log_) + : args(args_) { log = &Poco::Logger::get("ZooKeeperClient"); std::atomic_store(&zk_log, std::move(zk_log_)); @@ -445,9 +445,7 @@ void ZooKeeper::connect( throw; } connected = true; - - if (connected_callback.has_value()) - (*connected_callback)(i, node); + original_index = static_cast(node.original_index); if (i != 0) { @@ -912,6 +910,9 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea LOG_INFO(log, "Finalizing session {}. finalization_started: {}, queue_finished: {}, reason: '{}'", session_id, already_started, requests_queue.isFinished(), reason); + /// Reset the original index. + original_index = -1; + auto expire_session_if_not_expired = [&] { /// No new requests will appear in queue after finish() diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 56e199352e9..8b363398200 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -103,11 +103,11 @@ public: struct Node { Poco::Net::SocketAddress address; + UInt8 original_index; bool secure; }; using Nodes = std::vector; - using ConnectedCallback = std::function; /** Connection to nodes is performed in order. If you want, shuffle them manually. * Operation timeout couldn't be greater than session timeout. @@ -116,8 +116,7 @@ public: ZooKeeper( const Nodes & nodes, const zkutil::ZooKeeperArgs & args_, - std::shared_ptr zk_log_, - std::optional && connected_callback_ = {}); + std::shared_ptr zk_log_); ~ZooKeeper() override; @@ -125,6 +124,10 @@ public: /// If expired, you can only destroy the object. All other methods will throw exception. bool isExpired() const override { return requests_queue.isFinished(); } + Int8 getConnectedNodeIdx() const override { return original_index; } + String getConnectedHostPort() const override { return (original_index == -1) ? "" : args.hosts[original_index]; } + int32_t getConnectionXid() const override { return next_xid.load(); } + /// A ZooKeeper session can have an optional deadline set on it. /// After it has been reached, the session needs to be finalized. bool hasReachedDeadline() const override; @@ -219,7 +222,7 @@ private: ACLs default_acls; zkutil::ZooKeeperArgs args; - std::optional connected_callback = {}; + Int8 original_index = -1; /// Fault injection void maybeInjectSendFault(); diff --git a/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp b/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp index fe38b486ada..0c5805ea0fc 100644 --- a/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp +++ b/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp @@ -30,14 +30,15 @@ try splitInto<','>(hosts_strings, hosts_arg); ZooKeeper::Nodes nodes; nodes.reserve(hosts_strings.size()); - for (auto & host_string : hosts_strings) + for (size_t i = 0; i < hosts_strings.size(); ++i) { + std::string host_string = hosts_strings[i]; bool secure = bool(startsWith(host_string, "secure://")); if (secure) host_string.erase(0, strlen("secure://")); - nodes.emplace_back(ZooKeeper::Node{Poco::Net::SocketAddress{host_string},secure}); + nodes.emplace_back(ZooKeeper::Node{Poco::Net::SocketAddress{host_string}, static_cast(i) , secure}); } diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index 33268d58358..f3c581b3eb4 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -21,32 +21,23 @@ NamesAndTypesList StorageSystemZooKeeperConnection::getNamesAndTypes() auto feature_flags_enum = std::make_shared(std::move(feature_flags_enum_values)); return { - {"name", std::make_shared()}, - {"host", std::make_shared()}, - {"port", std::make_shared()}, - {"index", std::make_shared()}, - {"connected_time", std::make_shared()}, - {"session_uptime_elapsed_seconds", std::make_shared()}, - {"is_expired", std::make_shared()}, - {"keeper_api_version", std::make_shared()}, - {"client_id", std::make_shared()}, - {"enabled_feature_flags", std::make_shared(std::move(feature_flags_enum))} + /* 0 */ {"name", std::make_shared()}, + /* 1 */ {"host", std::make_shared()}, + /* 2 */ {"port", std::make_shared()}, + /* 3 */ {"index", std::make_shared()}, + /* 4 */ {"connected_time", std::make_shared()}, + /* 5 */ {"session_uptime_elapsed_seconds", std::make_shared()}, + /* 6 */ {"is_expired", std::make_shared()}, + /* 7 */ {"keeper_api_version", std::make_shared()}, + /* 8 */ {"client_id", std::make_shared()}, + /* 9 */ {"xid", std::make_shared()}, + /* 10*/ {"enabled_feature_flags", std::make_shared(std::move(feature_flags_enum))} }; } void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - res_columns[0]->insert("default"); - res_columns[1]->insert(context->getZooKeeper()->getConnectedZooKeeperHost()); - res_columns[2]->insert(context->getZooKeeper()->getConnectedZooKeeperPort()); - res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); - res_columns[4]->insert(context->getZooKeeper()->getConnectedTime()); - res_columns[5]->insert(context->getZooKeeperSessionUptime()); - res_columns[6]->insert(context->getZooKeeper()->expired()); - res_columns[7]->insert(0); - res_columns[8]->insert(context->getZooKeeper()->getClientID()); - const auto add_enabled_feature_flags = [&](const auto & zookeeper) { Array enabled_feature_flags; @@ -61,25 +52,45 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co } } } - res_columns[9]->insert(std::move(enabled_feature_flags)); + res_columns[10]->insert(std::move(enabled_feature_flags)); }; - add_enabled_feature_flags(context->getZooKeeper()); + /// For read-only snapshot type functionality, it's acceptable even though 'getZooKeeper' may cause data inconsistency. + auto fill_data = [&](const String & name, const zkutil::ZooKeeperPtr zookeeper, MutableColumns & columns) + { + Int8 index = zookeeper->getConnectedHostIdx(); + String host_port = zookeeper->getConnectedHostPort(); + if (index != -1 && !host_port.empty()) + { + size_t offset = host_port.find_last_of(':'); + String host = host_port.substr(0, offset); + UInt16 port = static_cast(Poco::NumberParser::parseUnsigned(host_port.substr(offset + 1))); + UInt32 uptime = zookeeper->getSessionUptime(); + time_t time = timeInSeconds(std::chrono::system_clock::now()) - uptime; + + columns[0]->insert(name); + columns[1]->insert(host); + columns[2]->insert(port); + columns[3]->insert(index); + columns[4]->insert(time); + columns[5]->insert(uptime); + columns[6]->insert(zookeeper->expired()); + columns[7]->insert(0); + columns[8]->insert(zookeeper->getClientID()); + columns[9]->insert(zookeeper->getConnectionXid()); + add_enabled_feature_flags(zookeeper); + } + }; + + /// default zookeeper. + fill_data("default", context->getZooKeeper(), res_columns); + + /// auxiliary zookeepers. for (const auto & elem : context->getAuxiliaryZooKeepers()) { - res_columns[0]->insert(elem.first); - res_columns[1]->insert(elem.second->getConnectedZooKeeperHost()); - res_columns[2]->insert(elem.second->getConnectedZooKeeperPort()); - res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); - res_columns[4]->insert(elem.second->getConnectedTime()); - res_columns[5]->insert(elem.second->getSessionUptime()); - res_columns[6]->insert(elem.second->expired()); - res_columns[7]->insert(0); - res_columns[8]->insert(elem.second->getClientID()); - add_enabled_feature_flags(elem.second); + fill_data(elem.first, elem.second, res_columns); } - } } diff --git a/tests/config/config.d/zookeeper.xml b/tests/config/config.d/zookeeper.xml index 9f984d4c544..07142b1a55e 100644 --- a/tests/config/config.d/zookeeper.xml +++ b/tests/config/config.d/zookeeper.xml @@ -3,7 +3,7 @@ random - localhost + 127.0.0.1 9181 diff --git a/tests/integration/test_zookeeper_fallback_session/test.py b/tests/integration/test_zookeeper_fallback_session/test.py index 570eca4f0a6..9afabfa3da3 100644 --- a/tests/integration/test_zookeeper_fallback_session/test.py +++ b/tests/integration/test_zookeeper_fallback_session/test.py @@ -45,14 +45,13 @@ def started_cluster(): def assert_uses_zk_node(node: ClickHouseInstance, zk_node): def check_callback(host): - return host.strip() == expected_zk_ip_addr - - expected_zk_ip_addr = node.cluster.get_instance_ip(zk_node) + return host.strip() == zk_node + # We don't convert the column 'host' of system.zookeeper_connection to ip address any more. host = node.query_with_retry( "select host from system.zookeeper_connection", check_callback=check_callback ) - assert host.strip() == expected_zk_ip_addr + assert host.strip() == zk_node def test_fallback_session(started_cluster: ClickHouseCluster): diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.reference b/tests/queries/0_stateless/01158_zookeeper_log_long.reference index 7ec52cb3366..547acb5b377 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.reference +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.reference @@ -1,9 +1,9 @@ log -::1 Response 0 Watch /test/01158/default/rmt/log 0 0 \N 0 0 ZOK CHILD CONNECTED 0 0 0 0 -::1 Request 0 Create /test/01158/default/rmt/log 0 0 \N 0 4 \N \N \N 0 0 0 0 -::1 Response 0 Create /test/01158/default/rmt/log 0 0 \N 0 4 ZOK \N \N /test/01158/default/rmt/log 0 0 0 0 -::1 Request 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 \N \N \N 0 0 0 0 -::1 Response 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 ZOK \N \N /test/01158/default/rmt/log/log-0000000000 0 0 0 0 +::ffff:127.0.0.1 Response 0 Watch /test/01158/default/rmt/log 0 0 \N 0 0 ZOK CHILD CONNECTED 0 0 0 0 +::ffff:127.0.0.1 Request 0 Create /test/01158/default/rmt/log 0 0 \N 0 4 \N \N \N 0 0 0 0 +::ffff:127.0.0.1 Response 0 Create /test/01158/default/rmt/log 0 0 \N 0 4 ZOK \N \N /test/01158/default/rmt/log 0 0 0 0 +::ffff:127.0.0.1 Request 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 \N \N \N 0 0 0 0 +::ffff:127.0.0.1 Response 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 ZOK \N \N /test/01158/default/rmt/log/log-0000000000 0 0 0 0 parts Request 0 Multi 0 0 \N 4 0 \N \N \N 0 0 0 0 Request 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 \N \N \N 0 0 0 0 diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference index c046c64acb2..a2e987666e4 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference @@ -1,2 +1,2 @@ -default ::1 9181 0 0 0 1 1 ['FILTERED_LIST','MULTI_READ','CHECK_NOT_EXISTS','CREATE_IF_NOT_EXISTS'] -zookeeper2 ::1 9181 0 0 0 1 +default 127.0.0.1 9181 0 0 0 1 1 ['FILTERED_LIST','MULTI_READ','CHECK_NOT_EXISTS','CREATE_IF_NOT_EXISTS'] +zookeeper2 localhost 9181 0 0 0 1 diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index 12f4c87c5ae..bc994502017 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -416,16 +416,16 @@ void Runner::createConnections() for (size_t session = 0; session < connection_info.sessions; ++session) { - connections.emplace_back(getConnection(connection_info)); + connections.emplace_back(getConnection(connection_info, connection_info_idx)); connections_to_info_map[connections.size() - 1] = connection_info_idx; } } std::cerr << "---- Done creating connections ----\n" << std::endl; } -std::shared_ptr Runner::getConnection(const ConnectionInfo & connection_info) +std::shared_ptr Runner::getConnection(const ConnectionInfo & connection_info, size_t connection_info_idx) { - Coordination::ZooKeeper::Node node{Poco::Net::SocketAddress{connection_info.host}, connection_info.secure}; + Coordination::ZooKeeper::Node node{Poco::Net::SocketAddress{connection_info.host}, static_cast(connection_info_idx), connection_info.secure}; std::vector nodes; nodes.push_back(node); zkutil::ZooKeeperArgs args; @@ -444,7 +444,7 @@ std::vector> Runner::refreshConnections if (connection->isExpired()) { const auto & connection_info = connection_infos[connections_to_info_map[connection_idx]]; - connection = getConnection(connection_info); + connection = getConnection(connection_info, connection_idx); } } return connections; diff --git a/utils/keeper-bench/Runner.h b/utils/keeper-bench/Runner.h index f899f1d538d..14d9b13938b 100644 --- a/utils/keeper-bench/Runner.h +++ b/utils/keeper-bench/Runner.h @@ -89,6 +89,6 @@ private: std::unordered_map connections_to_info_map; void createConnections(); - std::shared_ptr getConnection(const ConnectionInfo & connection_info); + std::shared_ptr getConnection(const ConnectionInfo & connection_info, size_t connection_info_idx); std::vector> refreshConnections(); }; From 1491e6d626dceee0e4ba5540150eb9228b2dcc2b Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 19 Oct 2023 10:49:53 +0200 Subject: [PATCH 441/634] Update URI filter for `query` endpoint once again --- src/Server/HTTPHandlerFactory.cpp | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 2dff0a0fbde..e230952a7a5 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -182,13 +182,15 @@ void addDefaultHandlersFactory( auto query_handler = std::make_shared>(std::move(dynamic_creator)); query_handler->addFilter([](const auto & request) { - return (startsWith(request.getURI(), "/?") - && (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD)) - || ((startsWith(request.getURI(), "/?") - || request.getURI() == "/") - && (request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST)); + bool pathMatchesGetOrHead = startsWith(request.getURI(), "/?") || startsWith(request.getURI(), "/query?"); + bool isGetOrHeadRequest = request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD; + + bool pathMatchesPostOrOptions = pathMatchesGetOrHead || request.getURI() == "/"; + bool isPostOrOptionsRequest = request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS; + + return (pathMatchesGetOrHead && isGetOrHeadRequest) || (pathMatchesPostOrOptions && isPostOrOptionsRequest); } ); factory.addHandler(query_handler); From b2d930652b577f738d412df953d503ad7cfb8a2e Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 19 Oct 2023 14:20:27 +0200 Subject: [PATCH 442/634] Add a test for configuration blocks order affecting HTTP handlers behavior --- .../test_http_handlers_config/test.py | 35 +++++++++++++++++++ .../defaults_first/config.xml | 17 +++++++++ .../defaults_last/config.xml | 17 +++++++++ 3 files changed, 69 insertions(+) create mode 100644 tests/integration/test_http_handlers_config/test_defaults_handlers_config_order/defaults_first/config.xml create mode 100644 tests/integration/test_http_handlers_config/test_defaults_handlers_config_order/defaults_last/config.xml diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index fe61043b11f..de0bdfbdcce 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -455,6 +455,41 @@ def test_defaults_http_handlers(): ).status_code ) +def test_defaults_http_handlers_config_order(): + def check_predefined_query_handler(): + assert ( + 200 + == cluster.instance.http_request( + "?query=SELECT+1", method="GET" + ).status_code + ) + assert ( + b"1\n" + == cluster.instance.http_request("?query=SELECT+1", method="GET").content + ) + response = cluster.instance.http_request( + "test_predefined_handler_get?max_threads=1&setting_name=max_threads", + method="GET", + headers={"XXX": "xxx"}, + ) + assert b"max_threads\t1\n" == response.content + assert ( + "text/tab-separated-values; charset=UTF-8" == response.headers["content-type"] + ) + + with contextlib.closing( + SimpleCluster( + ClickHouseCluster(__file__), "defaults_handlers_config_order_first", "test_defaults_handlers_config_order/defaults_first" + ) + ) as cluster: + check_predefined_query_handler() + + with contextlib.closing( + SimpleCluster( + ClickHouseCluster(__file__), "defaults_handlers_config_order_first", "test_defaults_handlers_config_order/defaults_last" + ) + ) as cluster: + check_predefined_query_handler() def test_prometheus_handler(): with contextlib.closing( diff --git a/tests/integration/test_http_handlers_config/test_defaults_handlers_config_order/defaults_first/config.xml b/tests/integration/test_http_handlers_config/test_defaults_handlers_config_order/defaults_first/config.xml new file mode 100644 index 00000000000..d07f18cce7b --- /dev/null +++ b/tests/integration/test_http_handlers_config/test_defaults_handlers_config_order/defaults_first/config.xml @@ -0,0 +1,17 @@ + + + Default server response + + + + + GET + xxx + /test_predefined_handler_get + + predefined_query_handler + SELECT name, value FROM system.settings WHERE name = {setting_name:String} + + + + diff --git a/tests/integration/test_http_handlers_config/test_defaults_handlers_config_order/defaults_last/config.xml b/tests/integration/test_http_handlers_config/test_defaults_handlers_config_order/defaults_last/config.xml new file mode 100644 index 00000000000..24340e4de25 --- /dev/null +++ b/tests/integration/test_http_handlers_config/test_defaults_handlers_config_order/defaults_last/config.xml @@ -0,0 +1,17 @@ + + + Default server response + + + + GET + xxx + /test_predefined_handler_get + + predefined_query_handler + SELECT name, value FROM system.settings WHERE name = {setting_name:String} + + + + + From dc31963748e9fe0fb27d5b4bd00b194ff2e29b2e Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Fri, 20 Oct 2023 13:00:06 +0200 Subject: [PATCH 443/634] Update URI filter for `query` endpoint once again --- src/Server/HTTPHandlerFactory.cpp | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index e230952a7a5..e1ee9586f83 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -182,15 +182,19 @@ void addDefaultHandlersFactory( auto query_handler = std::make_shared>(std::move(dynamic_creator)); query_handler->addFilter([](const auto & request) { - bool pathMatchesGetOrHead = startsWith(request.getURI(), "/?") || startsWith(request.getURI(), "/query?"); - bool isGetOrHeadRequest = request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD; + bool path_matches_get_or_head = startsWith(request.getURI(), "?") + || startsWith(request.getURI(), "/?") + || startsWith(request.getURI(), "/query?"); + bool is_get_or_head_request = request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD; - bool pathMatchesPostOrOptions = pathMatchesGetOrHead || request.getURI() == "/"; - bool isPostOrOptionsRequest = request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST + bool path_matches_post_or_options = path_matches_get_or_head + || request.getURI() == "/" + || request.getURI().empty(); + bool is_post_or_options_request = request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS; - return (pathMatchesGetOrHead && isGetOrHeadRequest) || (pathMatchesPostOrOptions && isPostOrOptionsRequest); + return (path_matches_get_or_head && is_get_or_head_request) || (path_matches_post_or_options && is_post_or_options_request); } ); factory.addHandler(query_handler); From c3c62d90c8401d39d22e5e0e2e0ea1c2671782bb Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 20 Oct 2023 13:48:23 +0200 Subject: [PATCH 444/634] Update MergeTreeData.cpp --- src/Storages/MergeTree/MergeTreeData.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a941b76ebb8..5f11f54f131 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -22,7 +22,6 @@ #include #include #include -#include #include #include #include @@ -80,7 +79,6 @@ #include #include #include -#include #include #include #include From 1edf5ff981626a70053d60e138f6f69532103f3c Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Fri, 20 Oct 2023 14:18:44 +0200 Subject: [PATCH 445/634] Fix `test_system_start_stop_listen` --- .../test_system_start_stop_listen/test.py | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_system_start_stop_listen/test.py b/tests/integration/test_system_start_stop_listen/test.py index 8a3081e0c15..440f5f83199 100644 --- a/tests/integration/test_system_start_stop_listen/test.py +++ b/tests/integration/test_system_start_stop_listen/test.py @@ -30,14 +30,10 @@ def started_cluster(): def http_works(port=8123): try: - response = requests.post(f"http://{main_node.ip_address}:{port}/ping") - if response.status_code == 400: - return True - except: - pass - - return False - + response = requests.get(f"http://{main_node.ip_address}:{port}/ping") + return response.status_code == 200 + except requests.exceptions.ConnectionError: + return False def assert_everything_works(): custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) From 1aa4a542bbe1affea09309a105e2b9064856bcbc Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 20 Oct 2023 12:30:21 +0000 Subject: [PATCH 446/634] Handle empty block in DictionaryPipelineExecutor --- src/Dictionaries/CacheDictionary.cpp | 3 --- src/Dictionaries/DictionarySourceHelpers.cpp | 15 +++++++++++++-- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 21da5a3734f..b40a60e0915 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -14,9 +14,6 @@ #include #include -#include -#include - #include namespace ProfileEvents diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 9f82843650d..d9a4d9ccbcf 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -17,6 +17,7 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; } @@ -142,9 +143,19 @@ DictionaryPipelineExecutor::DictionaryPipelineExecutor(QueryPipeline & pipeline_ bool DictionaryPipelineExecutor::pull(Block & block) { if (async_executor) - return async_executor->pull(block); - else + { + while (true) + { + bool has_data = async_executor->pull(block); + if (has_data && !block) + continue; + return has_data; + } + } + else if (executor) return executor->pull(block); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "DictionaryPipelineExecutor is not initialized"); } DictionaryPipelineExecutor::~DictionaryPipelineExecutor() = default; From 68b789d7a17dcf0b655ccfb444eb4a5d738394c5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 20 Oct 2023 12:40:28 +0000 Subject: [PATCH 447/634] Automatic style fix --- tests/integration/test_http_handlers_config/test.py | 13 ++++++++++--- .../test_system_start_stop_listen/test.py | 1 + 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index de0bdfbdcce..f6ac42a2db2 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -455,6 +455,7 @@ def test_defaults_http_handlers(): ).status_code ) + def test_defaults_http_handlers_config_order(): def check_predefined_query_handler(): assert ( @@ -474,23 +475,29 @@ def test_defaults_http_handlers_config_order(): ) assert b"max_threads\t1\n" == response.content assert ( - "text/tab-separated-values; charset=UTF-8" == response.headers["content-type"] + "text/tab-separated-values; charset=UTF-8" + == response.headers["content-type"] ) with contextlib.closing( SimpleCluster( - ClickHouseCluster(__file__), "defaults_handlers_config_order_first", "test_defaults_handlers_config_order/defaults_first" + ClickHouseCluster(__file__), + "defaults_handlers_config_order_first", + "test_defaults_handlers_config_order/defaults_first", ) ) as cluster: check_predefined_query_handler() with contextlib.closing( SimpleCluster( - ClickHouseCluster(__file__), "defaults_handlers_config_order_first", "test_defaults_handlers_config_order/defaults_last" + ClickHouseCluster(__file__), + "defaults_handlers_config_order_first", + "test_defaults_handlers_config_order/defaults_last", ) ) as cluster: check_predefined_query_handler() + def test_prometheus_handler(): with contextlib.closing( SimpleCluster( diff --git a/tests/integration/test_system_start_stop_listen/test.py b/tests/integration/test_system_start_stop_listen/test.py index 440f5f83199..84f75d66089 100644 --- a/tests/integration/test_system_start_stop_listen/test.py +++ b/tests/integration/test_system_start_stop_listen/test.py @@ -35,6 +35,7 @@ def http_works(port=8123): except requests.exceptions.ConnectionError: return False + def assert_everything_works(): custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) main_node.query(QUERY) From c690a3af8bedadcbf3b52475654b1221b53df3b4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 20 Oct 2023 15:04:42 +0200 Subject: [PATCH 448/634] Disable predicate pushdown if the CTE contains stateful functions --- src/Interpreters/PredicateRewriteVisitor.cpp | 2 +- ..._predicate_pushdown_cte_stateful.reference | 1 + .../02901_predicate_pushdown_cte_stateful.sql | 20 +++++++++++++++++++ 3 files changed, 22 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02901_predicate_pushdown_cte_stateful.reference create mode 100644 tests/queries/0_stateless/02901_predicate_pushdown_cte_stateful.sql diff --git a/src/Interpreters/PredicateRewriteVisitor.cpp b/src/Interpreters/PredicateRewriteVisitor.cpp index ea212991bd0..40de887da27 100644 --- a/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/src/Interpreters/PredicateRewriteVisitor.cpp @@ -166,7 +166,7 @@ static void getConjunctionHashesFrom(const ASTPtr & ast, std::set & bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, const Names & inner_columns) { if ((!optimize_final && subquery.final()) - || (!optimize_with && subquery.with()) + || (subquery.with() && (!optimize_with || hasNonRewritableFunction(subquery.with(), getContext()))) || subquery.withFill() || subquery.limitBy() || subquery.limitLength() || subquery.limitByLength() || subquery.limitByOffset() || hasNonRewritableFunction(subquery.select(), getContext()) diff --git a/tests/queries/0_stateless/02901_predicate_pushdown_cte_stateful.reference b/tests/queries/0_stateless/02901_predicate_pushdown_cte_stateful.reference new file mode 100644 index 00000000000..2e9c1e130da --- /dev/null +++ b/tests/queries/0_stateless/02901_predicate_pushdown_cte_stateful.reference @@ -0,0 +1 @@ +558 diff --git a/tests/queries/0_stateless/02901_predicate_pushdown_cte_stateful.sql b/tests/queries/0_stateless/02901_predicate_pushdown_cte_stateful.sql new file mode 100644 index 00000000000..f63e1d59471 --- /dev/null +++ b/tests/queries/0_stateless/02901_predicate_pushdown_cte_stateful.sql @@ -0,0 +1,20 @@ +CREATE TABLE t +( + `rDate` String, + `cpu_total` Int64 +) +ENGINE = Log; + +insert into t values ('2022-03-06', 22442 ), ('2022-03-05', 22382 ), ('2022-03-04', 22395 ), ('2022-03-03', 22306 ), ('2022-03-02', 22095 ), ('2022-03-01', 22065 ), ('2022-02-28', 21949 ), ('2022-02-27', 21884 ), ('2022-02-26', 21875 ), ('2022-02-25', 21858 ), ('2022-02-24', 21775 ), ('2022-02-23', 21639 ), ('2022-02-22', 21557 ), ('2022-02-21', 21381 ), ('2022-02-20', 21794 ), ('2022-02-19', 21808 ), ('2022-02-18', 21695 ), ('2022-02-17', 20874 ), ('2022-02-16', 20911 ), ('2022-02-15', 20898 ), ('2022-02-14', 20768 ), ('2022-02-13', 20588 ), ('2022-02-12', 20516 ), ('2022-02-11', 20501 ), ('2022-02-10', 20429 ), ('2022-02-09', 20208 ), ('2022-02-08', 20186 ), ('2022-02-07', 20192 ), ('2022-02-06', 20192 ), ('2022-02-05', 20175 ), ('2022-02-04', 20191 ), ('2022-02-03', 20214 ), ('2022-02-02', 20215 ), ('2022-02-01', 20220 ), ('2022-01-31', 20146 ), ('2022-01-30', 20137 ), ('2022-01-29', 20162 ), ('2022-01-28', 20164 ), ('2022-01-27', 20128 ), ('2022-01-26', 20139 ), ('2022-01-25', 20000 ), ('2022-01-24', 19778 ), ('2022-01-23', 19789 ), ('2022-01-22', 19628 ), ('2022-01-21', 19631 ), ('2022-01-20', 19386 ), ('2022-01-19', 19439 ), ('2022-01-18', 19477 ), ('2022-01-17', 19386 ), ('2022-01-16', 20013 ), ('2022-01-15', 19359 ), ('2022-01-14', 19356 ), ('2022-01-13', 19300 ), ('2022-01-12', 19237 ), ('2022-01-11', 19159 ), ('2022-01-10', 18970 ), ('2022-01-09', 18804 ), ('2022-01-08', 18816 ), ('2022-01-07', 18808 ), ('2022-01-06', 18693 ), ('2022-01-05', 18639 ), ('2022-01-04', 18579 ), ('2022-01-03', 18450 ), ('2022-01-02', 18458 ), ('2022-01-01', 18445 ), ('2021-12-31', 18443 ), ('2021-12-30', 18388 ), ('2021-12-29', 18348 ), ('2021-12-28', 18042 ), ('2021-12-26', 18049 ), ('2021-12-22', 17962 ); + +SELECT cpu_total_week +FROM + ( + WITH neighbor(cpu_total, 7) AS cpu_total_7 + SELECT + rDate, + floor(multiIf(cpu_total_7 = 0, 0, cpu_total - cpu_total_7), 2) AS cpu_total_week + FROM t + ) AS t_table_471873 +WHERE (rDate >= '2022-03-06') AND (rDate <= '2022-03-06') +SETTINGS enable_optimize_predicate_expression = 1; From 8bc384e7082fdfe365a042e4ebbb9eaeaba58e72 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 20 Oct 2023 13:14:15 +0000 Subject: [PATCH 449/634] More dictionaries support dictionary_use_async_executor --- src/Dictionaries/FlatDictionary.cpp | 6 ++--- src/Dictionaries/FlatDictionary.h | 1 + src/Dictionaries/IPAddressDictionary.cpp | 25 +++++++++++++------ src/Dictionaries/IPAddressDictionary.h | 17 ++++++++----- src/Dictionaries/PolygonDictionary.cpp | 3 ++- src/Dictionaries/PolygonDictionary.h | 2 ++ .../PolygonDictionaryImplementations.cpp | 11 ++++++-- src/Dictionaries/RangeHashedDictionary.h | 7 ++++-- src/Dictionaries/RegExpTreeDictionary.cpp | 13 +++++++--- src/Dictionaries/RegExpTreeDictionary.h | 1 + .../registerRangeHashedDictionary.cpp | 16 ++++++++---- .../0_stateless/01018_ip_dictionary_long.sql | 4 ++- .../01676_range_hashed_dictionary.sql | 4 ++- .../01681_cache_dictionary_simple_key.sql | 4 ++- .../01760_polygon_dictionaries.sql | 4 ++- 15 files changed, 83 insertions(+), 35 deletions(-) diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index b06137740da..539be048f6b 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -12,9 +12,9 @@ #include #include -#include #include +#include #include #include @@ -395,7 +395,7 @@ void FlatDictionary::updateData() if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { QueryPipeline pipeline(source_ptr->loadUpdatedAll()); - PullingPipelineExecutor executor(pipeline); + DictionaryPipelineExecutor executor(pipeline, configuration.use_async_executor); update_field_loaded_block.reset(); Block block; @@ -436,7 +436,7 @@ void FlatDictionary::loadData() if (!source_ptr->hasUpdateField()) { QueryPipeline pipeline(source_ptr->loadAll()); - PullingPipelineExecutor executor(pipeline); + DictionaryPipelineExecutor executor(pipeline, configuration.use_async_executor); Block block; while (executor.pull(block)) diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index d09b2c01b8e..a54916c5cd1 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -27,6 +27,7 @@ public: size_t max_array_size; bool require_nonempty; DictionaryLifetime dict_lifetime; + bool use_async_executor = false; }; FlatDictionary( diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 6bb06de7506..2e3c09c67c5 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -16,7 +16,9 @@ #include #include #include +#include #include +#include #include #include @@ -197,13 +199,11 @@ IPAddressDictionary::IPAddressDictionary( const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, - const DictionaryLifetime dict_lifetime_, - bool require_nonempty_) + IPAddressDictionary::Configuration configuration_) : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} - , dict_lifetime(dict_lifetime_) - , require_nonempty(require_nonempty_) + , configuration(configuration_) , access_to_key_from_attributes(dict_struct_.access_to_key_from_attributes) , logger(&Poco::Logger::get("IPAddressDictionary")) { @@ -369,7 +369,7 @@ void IPAddressDictionary::loadData() bool has_ipv6 = false; - PullingPipelineExecutor executor(pipeline); + DictionaryPipelineExecutor executor(pipeline, configuration.use_async_executor); Block block; while (executor.pull(block)) { @@ -525,7 +525,7 @@ void IPAddressDictionary::loadData() LOG_TRACE(logger, "{} ip records are read", ip_records.size()); - if (require_nonempty && 0 == element_count) + if (configuration.require_nonempty && 0 == element_count) throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, "{}: dictionary source is empty and 'require_nonempty' property is set.", getFullName()); } @@ -971,7 +971,7 @@ void registerDictionaryTrie(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* global_context */, + ContextPtr global_context, bool /*created_from_ddl*/) -> DictionaryPtr { if (!dict_struct.key || dict_struct.key->size() != 1) @@ -981,8 +981,17 @@ void registerDictionaryTrie(DictionaryFactory & factory) const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); + const auto * clickhouse_source = dynamic_cast(source_ptr.get()); + bool use_async_executor = clickhouse_source && clickhouse_source->isLocal() && context->getSettingsRef().dictionary_use_async_executor; + + IPAddressDictionary::Configuration configuration{ + .dict_lifetime = dict_lifetime, + .require_nonempty = require_nonempty, + .use_async_executor = use_async_executor, + }; // This is specialised dictionary for storing IPv4 and IPv6 prefixes. - return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), configuration); }; factory.registerLayout("ip_trie", create_layout, true); } diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index 40dc5dd6782..c5b9287c2b5 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -22,12 +22,18 @@ class Arena; class IPAddressDictionary final : public IDictionary { public: + struct Configuration + { + DictionaryLifetime dict_lifetime; + bool require_nonempty; + bool use_async_executor = false; + }; + IPAddressDictionary( const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, - const DictionaryLifetime dict_lifetime_, /// NOLINT - bool require_nonempty_); + Configuration configuration_); std::string getKeyDescription() const { return key_description; } @@ -53,12 +59,12 @@ public: std::shared_ptr clone() const override { - return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), configuration); } DictionarySourcePtr getSource() const override { return source_ptr; } - const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } + const DictionaryLifetime & getLifetime() const override { return configuration.dict_lifetime; } const DictionaryStructure & getStructure() const override { return dict_struct; } @@ -199,8 +205,7 @@ private: DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; - const DictionaryLifetime dict_lifetime; - const bool require_nonempty; + const Configuration configuration; const bool access_to_key_from_attributes; const std::string key_description{dict_struct.getKeyDescription()}; diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index ad36608343d..df3ae439b00 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -14,6 +14,7 @@ #include #include #include +#include namespace DB @@ -231,7 +232,7 @@ void IPolygonDictionary::loadData() { QueryPipeline pipeline(source_ptr->loadAll()); - PullingPipelineExecutor executor(pipeline); + DictionaryPipelineExecutor executor(pipeline, configuration.use_async_executor); Block block; while (executor.pull(block)) blockToAttributes(block); diff --git a/src/Dictionaries/PolygonDictionary.h b/src/Dictionaries/PolygonDictionary.h index d4864acdec4..a856d12b66c 100644 --- a/src/Dictionaries/PolygonDictionary.h +++ b/src/Dictionaries/PolygonDictionary.h @@ -56,6 +56,8 @@ public: /// Store polygon key column. That will allow to read columns from polygon dictionary. bool store_polygon_key_column = false; + + bool use_async_executor = false; }; IPolygonDictionary( diff --git a/src/Dictionaries/PolygonDictionaryImplementations.cpp b/src/Dictionaries/PolygonDictionaryImplementations.cpp index e33b3c13c19..3feca2ec410 100644 --- a/src/Dictionaries/PolygonDictionaryImplementations.cpp +++ b/src/Dictionaries/PolygonDictionaryImplementations.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include @@ -161,7 +163,7 @@ DictionaryPtr createLayout(const std::string & , const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* global_context */, + ContextPtr global_context, bool /*created_from_ddl*/) { const String database = config.getString(config_prefix + ".database", ""); @@ -219,11 +221,16 @@ DictionaryPtr createLayout(const std::string & , config.keys(layout_prefix, keys); const auto & dict_prefix = layout_prefix + "." + keys.front(); + ContextMutablePtr context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); + const auto * clickhouse_source = dynamic_cast(source_ptr.get()); + bool use_async_executor = clickhouse_source && clickhouse_source->isLocal() && context->getSettingsRef().dictionary_use_async_executor; + IPolygonDictionary::Configuration configuration { .input_type = input_type, .point_type = point_type, - .store_polygon_key_column = config.getBool(dict_prefix + ".store_polygon_key_column", false) + .store_polygon_key_column = config.getBool(dict_prefix + ".store_polygon_key_column", false), + .use_async_executor = use_async_executor, }; if (dict_struct.range_min || dict_struct.range_max) diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index 3f9bad941d5..624a57d65b5 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -29,7 +29,9 @@ #include #include +#include #include +#include namespace DB @@ -56,6 +58,7 @@ struct RangeHashedDictionaryConfiguration bool convert_null_range_bound_to_open; RangeHashedDictionaryLookupStrategy lookup_strategy; bool require_nonempty; + bool use_async_executor = false; }; template @@ -655,7 +658,7 @@ void RangeHashedDictionary::loadData() if (!source_ptr->hasUpdateField()) { QueryPipeline pipeline(source_ptr->loadAll()); - PullingPipelineExecutor executor(pipeline); + DictionaryPipelineExecutor executor(pipeline, configuration.use_async_executor); Block block; while (executor.pull(block)) @@ -919,7 +922,7 @@ void RangeHashedDictionary::updateData() if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { QueryPipeline pipeline(source_ptr->loadUpdatedAll()); - PullingPipelineExecutor executor(pipeline); + DictionaryPipelineExecutor executor(pipeline, configuration.use_async_executor); update_field_loaded_block.reset(); Block block; diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 7d6ad61c691..bbd101d55aa 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -310,7 +310,7 @@ void RegExpTreeDictionary::loadData() if (!source_ptr->hasUpdateField()) { QueryPipeline pipeline(source_ptr->loadAll()); - PullingPipelineExecutor executor(pipeline); + DictionaryPipelineExecutor executor(pipeline, configuration.use_async_executor); Block block; while (executor.pull(block)) @@ -867,12 +867,17 @@ void registerDictionaryRegExpTree(DictionaryFactory & factory) String dictionary_layout_prefix = config_prefix + ".layout" + ".regexp_tree"; const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; - RegExpTreeDictionary::Configuration configuration{ - .require_nonempty = config.getBool(config_prefix + ".require_nonempty", false), .lifetime = dict_lifetime}; - const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); + const auto * clickhouse_source = typeid_cast(source_ptr.get()); + bool use_async_executor = clickhouse_source && clickhouse_source->isLocal() && context->getSettingsRef().dictionary_use_async_executor; + + RegExpTreeDictionary::Configuration configuration{ + .require_nonempty = config.getBool(config_prefix + ".require_nonempty", false), + .lifetime = dict_lifetime, + .use_async_executor = use_async_executor, + }; return std::make_unique( dict_id, diff --git a/src/Dictionaries/RegExpTreeDictionary.h b/src/Dictionaries/RegExpTreeDictionary.h index 62008bb5aae..6597584ed45 100644 --- a/src/Dictionaries/RegExpTreeDictionary.h +++ b/src/Dictionaries/RegExpTreeDictionary.h @@ -40,6 +40,7 @@ public: { bool require_nonempty; DictionaryLifetime lifetime; + bool use_async_executor = false; }; const std::string name = "RegExpTree"; diff --git a/src/Dictionaries/registerRangeHashedDictionary.cpp b/src/Dictionaries/registerRangeHashedDictionary.cpp index 93784a0709e..4e20abfdb79 100644 --- a/src/Dictionaries/registerRangeHashedDictionary.cpp +++ b/src/Dictionaries/registerRangeHashedDictionary.cpp @@ -16,6 +16,7 @@ static DictionaryPtr createRangeHashedDictionary(const std::string & full_name, const DictionaryStructure & dict_struct, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, + ContextPtr global_context, DictionarySourcePtr source_ptr) { static constexpr auto layout_name = dictionary_key_type == DictionaryKeyType::Simple ? "range_hashed" : "complex_key_range_hashed"; @@ -52,11 +53,16 @@ static DictionaryPtr createRangeHashedDictionary(const std::string & full_name, else if (range_lookup_strategy == "max") lookup_strategy = RangeHashedDictionaryLookupStrategy::max; + auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); + const auto * clickhouse_source = dynamic_cast(source_ptr.get()); + bool use_async_executor = clickhouse_source && clickhouse_source->isLocal() && context->getSettingsRef().dictionary_use_async_executor; + RangeHashedDictionaryConfiguration configuration { .convert_null_range_bound_to_open = convert_null_range_bound_to_open, .lookup_strategy = lookup_strategy, - .require_nonempty = require_nonempty + .require_nonempty = require_nonempty, + .use_async_executor = use_async_executor, }; DictionaryPtr result = std::make_unique>( @@ -76,10 +82,10 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* global_context */, + ContextPtr global_context, bool /*created_from_ddl*/) -> DictionaryPtr { - return createRangeHashedDictionary(full_name, dict_struct, config, config_prefix, std::move(source_ptr)); + return createRangeHashedDictionary(full_name, dict_struct, config, config_prefix, global_context, std::move(source_ptr)); }; factory.registerLayout("range_hashed", create_layout_simple, false); @@ -89,10 +95,10 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* context */, + ContextPtr global_context, bool /*created_from_ddl*/) -> DictionaryPtr { - return createRangeHashedDictionary(full_name, dict_struct, config, config_prefix, std::move(source_ptr)); + return createRangeHashedDictionary(full_name, dict_struct, config, config_prefix, global_context, std::move(source_ptr)); }; factory.registerLayout("complex_key_range_hashed", create_layout_complex, true); diff --git a/tests/queries/0_stateless/01018_ip_dictionary_long.sql b/tests/queries/0_stateless/01018_ip_dictionary_long.sql index bb7f120163c..43025038f87 100644 --- a/tests/queries/0_stateless/01018_ip_dictionary_long.sql +++ b/tests/queries/0_stateless/01018_ip_dictionary_long.sql @@ -37,7 +37,9 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie PRIMARY KEY prefix SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ipv4_trie')) LAYOUT(IP_TRIE()) -LIFETIME(MIN 10 MAX 100); +LIFETIME(MIN 10 MAX 100) +SETTINGS(dictionary_use_async_executor=1, max_threads=8) +; -- fuzzer SELECT '127.0.0.0/24' = dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefixprefixprefixprefix', tuple(IPv4StringToNumOrDefault('127.0.0.0127.0.0.0'))); -- { serverError 36 } diff --git a/tests/queries/0_stateless/01676_range_hashed_dictionary.sql b/tests/queries/0_stateless/01676_range_hashed_dictionary.sql index 7d1fc60e90d..430f3a86dc1 100644 --- a/tests/queries/0_stateless/01676_range_hashed_dictionary.sql +++ b/tests/queries/0_stateless/01676_range_hashed_dictionary.sql @@ -29,7 +29,9 @@ PRIMARY KEY CountryID SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB 'database_for_range_dict')) LIFETIME(MIN 1 MAX 1000) LAYOUT(RANGE_HASHED()) -RANGE(MIN StartDate MAX EndDate); +RANGE(MIN StartDate MAX EndDate) +SETTINGS(dictionary_use_async_executor=1, max_threads=8) +; SELECT 'Dictionary not nullable'; SELECT 'dictGet'; diff --git a/tests/queries/0_stateless/01681_cache_dictionary_simple_key.sql b/tests/queries/0_stateless/01681_cache_dictionary_simple_key.sql index c6133e7a5ee..9ba8a6de796 100644 --- a/tests/queries/0_stateless/01681_cache_dictionary_simple_key.sql +++ b/tests/queries/0_stateless/01681_cache_dictionary_simple_key.sql @@ -24,7 +24,9 @@ CREATE DICTIONARY 01681_database_for_cache_dictionary.cache_dictionary_simple_ke PRIMARY KEY id SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_simple_attributes_source_table')) LIFETIME(MIN 1 MAX 1000) -LAYOUT(CACHE(SIZE_IN_CELLS 10)); +LAYOUT(CACHE(SIZE_IN_CELLS 10)) +SETTINGS(dictionary_use_async_executor=1, max_threads=8) +; SELECT 'Dictionary cache_dictionary_simple_key_simple_attributes'; SELECT 'dictGet existing value'; diff --git a/tests/queries/0_stateless/01760_polygon_dictionaries.sql b/tests/queries/0_stateless/01760_polygon_dictionaries.sql index 1589a091ee7..e74b3ce03b9 100644 --- a/tests/queries/0_stateless/01760_polygon_dictionaries.sql +++ b/tests/queries/0_stateless/01760_polygon_dictionaries.sql @@ -29,7 +29,9 @@ CREATE DICTIONARY 01760_db.dict_array PRIMARY KEY key SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons' DB '01760_db')) LIFETIME(0) -LAYOUT(POLYGON()); +LAYOUT(POLYGON()) +SETTINGS(dictionary_use_async_executor=1, max_threads=8) +; SELECT 'dictGet'; From 849e44138ab6692a11d456fced5de45399829130 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 20 Oct 2023 17:19:46 +0200 Subject: [PATCH 450/634] One final leftover in diff_urls of PRInfo --- tests/ci/pr_info.py | 38 +++++++++++++++++++++++++------------- 1 file changed, 25 insertions(+), 13 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index faa6b539a6b..8e7da53453f 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -171,10 +171,7 @@ class PRInfo: response_json = user_orgs_response.json() self.user_orgs = set(org["id"] for org in response_json) - self.diff_urls.append( - f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/master...{github_event['pull_request']['head']['label']}" - ) + self.diff_urls.append(self.compare_pr_url(github_event["pull_request"])) elif "commits" in github_event: # `head_commit` always comes with `commits` @@ -201,8 +198,7 @@ class PRInfo: self.head_ref = ref self.head_name = self.repo_full_name self.diff_urls.append( - f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/{github_event['before']}...{self.sha}" + self.compare_url(github_event["before"], self.sha) ) else: self.number = pull_request["number"] @@ -219,25 +215,31 @@ class PRInfo: # files changed in upstream AND master...{self.head_ref} # to get files, changed in current HEAD self.diff_urls.append( - f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/master...{pull_request['head']['label']}" + self.compare_url( + pull_request["base"]["repo"]["default_branch"], + pull_request["head"]["label"], + ) ) self.diff_urls.append( - f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/{pull_request['head']['label']}...master" + self.compare_url( + pull_request["head"]["label"], + pull_request["base"]["repo"]["default_branch"], + ) ) # Get release PR number. self.release_pr = get_pr_for_commit(self.base_ref, self.base_ref)[ "number" ] else: - self.diff_urls.append(pull_request["diff_url"]) + self.diff_urls.append(self.compare_pr_url(pull_request)) if "release" in self.labels: # For release PRs we must get not only files changed in the PR # itself, but as well files changed since we branched out self.diff_urls.append( - f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/{pull_request['head']['label']}...master" + self.compare_url( + pull_request["head"]["label"], + pull_request["base"]["repo"]["default_branch"], + ) ) else: print("event.json does not match pull_request or push:") @@ -260,6 +262,16 @@ class PRInfo: if need_changed_files: self.fetch_changed_files() + def compare_pr_url(self, pr_object: dict) -> str: + return self.compare_url(pr_object["base"]["label"], pr_object["head"]["label"]) + + @staticmethod + def compare_url(first: str, second: str) -> str: + return ( + "https://api.github.com/repos/" + f"{GITHUB_REPOSITORY}/compare/{first}...{second}" + ) + def fetch_changed_files(self): if not getattr(self, "diff_urls", False): raise TypeError("The event does not have diff URLs") From 3551d7f2d69accee769c084ac37906d31afa8f12 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 20 Oct 2023 15:46:17 +0000 Subject: [PATCH 451/634] Update docs --- .../functions/date-time-functions.md | 15 +++++++++++---- .../check-style/aspell-ignore/en/aspell-dict.txt | 6 ++++++ 2 files changed, 17 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 9b6eba9b5f0..8db971fb926 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1910,6 +1910,7 @@ Result: ``` **See Also** + - [subDate](#subDate) ## timestamp\_add @@ -2053,6 +2054,7 @@ Result: Alias: `ADDDATE` **See Also** + - [date_add](#date_add) ## subDate @@ -2095,6 +2097,7 @@ Result: Alias: `SUBDATE` **See Also** + - [date_sub](#date_sub) ## now {#now} @@ -2388,9 +2391,11 @@ Like function `YYYYMMDDhhmmssToDate()` but produces a [DateTime64](../../sql-ref Accepts an additional, optional `precision` parameter after the `timezone` parameter. -## addYears, addMonths, addWeeks, addDays, addHours, addMinutes, addSeconds, addQuarters +## addYears, addQuarters, addMonths, addWeeks, addDays, addHours, addMinutes, addSeconds, addMilliseconds, addMicroseconds, addNanoseconds -Function adds a Date/DateTime interval to a Date/DateTime and then return the Date/DateTime. For example: +These functions add units of the interval specified by the function name to a date or a date with time. A date or date with time is returned. + +Example: ``` sql WITH @@ -2407,9 +2412,11 @@ SELECT └─────────────────────┴──────────────────────────┘ ``` -## subtractYears, subtractMonths, subtractWeeks, subtractDays, subtractHours, subtractMinutes, subtractSeconds, subtractQuarters +## subtractYears, subtractQuarters, subtractMonths, subtractWeeks, subtractDays, subtractHours, subtractMinutes, subtractSeconds, subtractMilliseconds, subtractMicroseconds, subtractNanoseconds -Function subtract a Date/DateTime interval to a Date/DateTime and then return the Date/DateTime. For example: +These functions subtract units of the interval specified by the function name from a date or a date with time. A date or date with time is returned. + +Example: ``` sql WITH diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 3a5bd023f7d..7a31def0abf 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -992,8 +992,11 @@ activerecord addDate addDays addHours +addMicroseconds +addMilliseconds addMinutes addMonths +addNanoseconds addQuarters addSeconds addWeeks @@ -2313,8 +2316,11 @@ substrings subtitiles subtractDays subtractHours +subtractMicroseconds +subtractMilliseconds subtractMinutes subtractMonths +subtractNanoseconds subtractQuarters subtractSeconds subtractWeeks From 69a387b77882e0807ae86e0f5f762ad7225e69a5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 20 Oct 2023 15:46:25 +0000 Subject: [PATCH 452/634] Refactorings in FunctionDateOrDateTimeAddInterval --- .../FunctionDateOrDateTimeAddInterval.h | 229 ++++++------------ src/Functions/SubtractSubSeconds.cpp | 28 --- src/Functions/addMicroseconds.cpp | 16 ++ src/Functions/addMilliseconds.cpp | 16 ++ src/Functions/addNanoseconds.cpp | 16 ++ src/Functions/addSubSeconds.cpp | 28 --- src/Functions/subtractMicroseconds.cpp | 16 ++ src/Functions/subtractMilliseconds.cpp | 16 ++ src/Functions/subtractNanoseconds.cpp | 16 ++ 9 files changed, 175 insertions(+), 206 deletions(-) delete mode 100644 src/Functions/SubtractSubSeconds.cpp create mode 100644 src/Functions/addMicroseconds.cpp create mode 100644 src/Functions/addMilliseconds.cpp create mode 100644 src/Functions/addNanoseconds.cpp delete mode 100644 src/Functions/addSubSeconds.cpp create mode 100644 src/Functions/subtractMicroseconds.cpp create mode 100644 src/Functions/subtractMilliseconds.cpp create mode 100644 src/Functions/subtractNanoseconds.cpp diff --git a/src/Functions/FunctionDateOrDateTimeAddInterval.h b/src/Functions/FunctionDateOrDateTimeAddInterval.h index 4444feb6129..4828dd1cc72 100644 --- a/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -47,25 +47,21 @@ struct AddNanosecondsImpl { static constexpr auto name = "addNanoseconds"; - static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale) { Int64 multiplier = DecimalUtils::scaleMultiplier(9 - scale); return DateTime64(DecimalUtils::multiplyAdd(t.value, multiplier, delta)); } - - static inline NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16) { Int64 multiplier = DecimalUtils::scaleMultiplier(9); return DateTime64(DecimalUtils::multiplyAdd(static_cast(t), multiplier, delta)); } - - static inline NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, UInt16) { throw Exception(ErrorCodes::LOGICAL_ERROR, "addNanoseconds() cannot be used with Date"); } - - static inline NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, UInt16) { throw Exception(ErrorCodes::LOGICAL_ERROR, "addNanoseconds() cannot be used with Date32"); } @@ -75,27 +71,23 @@ struct AddMicrosecondsImpl { static constexpr auto name = "addMicroseconds"; - static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale) { Int64 multiplier = DecimalUtils::scaleMultiplier(std::abs(6 - scale)); return DateTime64(scale <= 6 ? DecimalUtils::multiplyAdd(t.value, multiplier, delta) : DecimalUtils::multiplyAdd(delta, multiplier, t.value)); } - - static inline NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16) { Int64 multiplier = DecimalUtils::scaleMultiplier(6); return DateTime64(DecimalUtils::multiplyAdd(static_cast(t), multiplier, delta)); } - - static inline NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, UInt16) { throw Exception(ErrorCodes::LOGICAL_ERROR, "addMicroseconds() cannot be used with Date"); } - - static inline NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, UInt16) { throw Exception(ErrorCodes::LOGICAL_ERROR, "addMicroseconds() cannot be used with Date32"); } @@ -105,27 +97,23 @@ struct AddMillisecondsImpl { static constexpr auto name = "addMilliseconds"; - static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale) { Int64 multiplier = DecimalUtils::scaleMultiplier(std::abs(3 - scale)); return DateTime64(scale <= 3 ? DecimalUtils::multiplyAdd(t.value, multiplier, delta) : DecimalUtils::multiplyAdd(delta, multiplier, t.value)); } - - static inline NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16) { Int64 multiplier = DecimalUtils::scaleMultiplier(3); return DateTime64(DecimalUtils::multiplyAdd(static_cast(t), multiplier, delta)); } - - static inline NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, UInt16) { throw Exception(ErrorCodes::LOGICAL_ERROR, "addMilliseconds() cannot be used with Date"); } - - static inline NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, UInt16) { throw Exception(ErrorCodes::LOGICAL_ERROR, "addMilliseconds() cannot be used with Date32"); } @@ -135,25 +123,21 @@ struct AddSecondsImpl { static constexpr auto name = "addSeconds"; - static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale) { return DateTime64(DecimalUtils::multiplyAdd(delta, DecimalUtils::scaleMultiplier(scale), t.value)); } - - static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16) { return static_cast(t + delta); } - - static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) { // use default datetime64 scale - static_assert(DataTypeDateTime64::default_scale == 3, ""); + static_assert(DataTypeDateTime64::default_scale == 3); return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta) * 1000; } - - static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return static_cast(time_zone.fromDayNum(DayNum(d)) + delta); } @@ -163,25 +147,21 @@ struct AddMinutesImpl { static constexpr auto name = "addMinutes"; - static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale) { return t + 60 * delta * DecimalUtils::scaleMultiplier(scale); } - - static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16) { return static_cast(t + delta * 60); } - - static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) { // use default datetime64 scale - static_assert(DataTypeDateTime64::default_scale == 3, ""); + static_assert(DataTypeDateTime64::default_scale == 3); return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 60) * 1000; } - - static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return static_cast(time_zone.fromDayNum(DayNum(d)) + delta * 60); } @@ -191,25 +171,21 @@ struct AddHoursImpl { static constexpr auto name = "addHours"; - static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale) { return t + 3600 * delta * DecimalUtils::scaleMultiplier(scale); } - - static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16) { return static_cast(t + delta * 3600); } - - static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) { // use default datetime64 scale - static_assert(DataTypeDateTime64::default_scale == 3, ""); + static_assert(DataTypeDateTime64::default_scale == 3); return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 3600) * 1000; } - - static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return static_cast(time_zone.fromDayNum(DayNum(d)) + delta * 3600); } @@ -219,25 +195,21 @@ struct AddDaysImpl { static constexpr auto name = "addDays"; - static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) { auto multiplier = DecimalUtils::scaleMultiplier(scale); auto d = std::div(t, multiplier); return time_zone.addDays(d.quot, delta) * multiplier + d.rem; } - - static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return static_cast(time_zone.addDays(t, delta)); } - - static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, UInt16) { return d + delta; } - - static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, UInt16) { return static_cast(d + delta); } @@ -247,25 +219,21 @@ struct AddWeeksImpl { static constexpr auto name = "addWeeks"; - static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) { auto multiplier = DecimalUtils::scaleMultiplier(scale); auto d = std::div(t, multiplier); return time_zone.addDays(d.quot, delta * 7) * multiplier + d.rem; } - - static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return static_cast(time_zone.addWeeks(t, delta)); } - - static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, UInt16) { return static_cast(d + delta * 7); } - - static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, UInt16) { return static_cast(d + delta * 7); } @@ -275,25 +243,21 @@ struct AddMonthsImpl { static constexpr auto name = "addMonths"; - static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) { auto multiplier = DecimalUtils::scaleMultiplier(scale); auto d = std::div(t, multiplier); return time_zone.addMonths(d.quot, delta) * multiplier + d.rem; } - - static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return static_cast(time_zone.addMonths(t, delta)); } - - static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return time_zone.addMonths(DayNum(d), delta); } - - static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return time_zone.addMonths(ExtendedDayNum(d), delta); } @@ -303,25 +267,21 @@ struct AddQuartersImpl { static constexpr auto name = "addQuarters"; - static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) { auto multiplier = DecimalUtils::scaleMultiplier(scale); auto d = std::div(t, multiplier); return time_zone.addQuarters(d.quot, delta) * multiplier + d.rem; } - - static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return static_cast(time_zone.addQuarters(t, delta)); } - - static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return time_zone.addQuarters(DayNum(d), delta); } - - static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return time_zone.addQuarters(ExtendedDayNum(d), delta); } @@ -331,25 +291,21 @@ struct AddYearsImpl { static constexpr auto name = "addYears"; - static inline NO_SANITIZE_UNDEFINED DateTime64 - execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale = 0) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) { auto multiplier = DecimalUtils::scaleMultiplier(scale); auto d = std::div(t, multiplier); return time_zone.addYears(d.quot, delta) * multiplier + d.rem; } - - static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return static_cast(time_zone.addYears(t, delta)); } - - static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return time_zone.addYears(DayNum(d), delta); } - - static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16 = 0) + static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) { return time_zone.addYears(ExtendedDayNum(d), delta); } @@ -361,7 +317,7 @@ struct SubtractIntervalImpl : public Transform using Transform::Transform; template - inline NO_SANITIZE_UNDEFINED auto execute(T t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) const + NO_SANITIZE_UNDEFINED auto execute(T t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) const { /// Signed integer overflow is Ok. return Transform::execute(t, -delta, time_zone, scale); @@ -382,11 +338,11 @@ struct SubtractYearsImpl : SubtractIntervalImpl { static constexpr template -struct Adder +struct Processor { const Transform transform; - explicit Adder(Transform transform_) + explicit Processor(Transform transform_) : transform(std::move(transform_)) {} @@ -427,7 +383,6 @@ struct Adder } private: - template static Int64 checkOverflow(Value val) { @@ -458,32 +413,32 @@ private: template struct DateTimeAddIntervalImpl { - static ColumnPtr execute(Transform transform, const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, UInt16 scale = 0) + static ColumnPtr execute(Transform transform, const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, UInt16 scale) { using FromValueType = typename FromDataType::FieldType; using FromColumnType = typename FromDataType::ColumnType; using ToColumnType = typename ToDataType::ColumnType; - auto op = Adder{std::move(transform)}; + const IColumn & source_column = *arguments[0].column; + const IColumn & delta_column = *arguments[1].column; const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); - const ColumnPtr source_col = arguments[0].column; - auto result_col = result_type->createColumn(); auto col_to = assert_cast(result_col.get()); - const IColumn & delta_column = *arguments[1].column; - if (const auto * sources = checkAndGetColumn(source_col.get())) + auto processor = Processor{std::move(transform)}; + + if (const auto * sources = checkAndGetColumn(&source_column)) { if (const auto * delta_const_column = typeid_cast(&delta_column)) - op.vectorConstant(sources->getData(), col_to->getData(), delta_const_column->getInt(0), time_zone, scale); + processor.vectorConstant(sources->getData(), col_to->getData(), delta_const_column->getInt(0), time_zone, scale); else - op.vectorVector(sources->getData(), col_to->getData(), delta_column, time_zone, scale); + processor.vectorVector(sources->getData(), col_to->getData(), delta_column, time_zone, scale); } - else if (const auto * sources_const = checkAndGetColumnConst(source_col.get())) + else if (const auto * sources_const = checkAndGetColumnConst(&source_column)) { - op.constantVector( + processor.constantVector( sources_const->template getValue(), col_to->getData(), delta_column, time_zone, scale); } @@ -506,7 +461,7 @@ template <> struct ResultDataTypeMap { using ResultDataType = DataTy template <> struct ResultDataTypeMap { using ResultDataType = DataTypeDate32; }; template <> struct ResultDataTypeMap { using ResultDataType = DataTypeDateTime64; }; template <> struct ResultDataTypeMap { using ResultDataType = DataTypeDateTime64; }; -template <> struct ResultDataTypeMap { using ResultDataType = DataTypeInt8; }; // error +template <> struct ResultDataTypeMap { using ResultDataType = DataTypeInt8; }; // error } template @@ -516,10 +471,7 @@ public: static constexpr auto name = Transform::name; static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override - { - return name; - } + String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } @@ -532,30 +484,28 @@ public: "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", getName(), arguments.size()); - if (!isNativeNumber(arguments[1].type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} (delta) must be a number", - getName()); - if (arguments.size() == 2) { - if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + if (!isDateOrDate32OrDateTimeOrDateTime64(arguments[0].type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. " "Should be a date or a date with time", arguments[0].type->getName(), getName()); } else { - if (!WhichDataType(arguments[0].type).isDateTime() - || !WhichDataType(arguments[2].type).isString()) - { - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {} supports 2 or 3 arguments. " - "The 1st argument must be of type Date or DateTime. " - "The 2nd argument must be a number. " - "The 3rd argument (optional) must be a constant string with timezone name. " + if (!WhichDataType(arguments[0].type).isDateTime()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. " + "Should be a DateTime", arguments[0].type->getName(), getName()); + + if (!WhichDataType(arguments[2].type).isString()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of third argument of function {}. " + "The 3rd argument must be a constant string with a timezone name. " "The timezone argument is allowed only when the 1st argument has the type DateTime", - getName()); - } + arguments[2].type->getName(), getName()); } + if (!isNativeNumber(arguments[1].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be a number", getName()); + switch (arguments[0].type->getTypeId()) { case TypeIndex::Date: @@ -567,10 +517,8 @@ public: case TypeIndex::DateTime64: return resolveReturnType(arguments); default: - { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid type of 1st argument of function {}: " "{}, expected: Date, DateTime or DateTime64.", getName(), arguments[0].type->getName()); - } } } @@ -591,17 +539,11 @@ public: using ResultDataType = TransformResultDataType; if constexpr (std::is_same_v) - { return std::make_shared(); - } else if constexpr (std::is_same_v) - { return std::make_shared(); - } else if constexpr (std::is_same_v) - { return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); - } else if constexpr (std::is_same_v) { static constexpr auto target_scale = std::invoke( @@ -627,9 +569,7 @@ public: return std::make_shared(target_scale.value_or(DataTypeDateTime64::default_scale), std::move(timezone)); } else if constexpr (std::is_same_v) - { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} cannot be used with {}", getName(), arguments[0].type->getName()); - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected result type in datetime add interval function"); } @@ -643,29 +583,18 @@ public: WhichDataType which(from_type); if (which.isDate()) - { - return DateTimeAddIntervalImpl, Transform>::execute( - Transform{}, arguments, result_type); - } + return DateTimeAddIntervalImpl, Transform>::execute(Transform{}, arguments, result_type, 0); else if (which.isDate32()) - { - return DateTimeAddIntervalImpl, Transform>::execute( - Transform{}, arguments, result_type); - } + return DateTimeAddIntervalImpl, Transform>::execute(Transform{}, arguments, result_type, 0); else if (which.isDateTime()) - { - return DateTimeAddIntervalImpl, Transform>::execute( - Transform{}, arguments, result_type); - } + return DateTimeAddIntervalImpl, Transform>::execute(Transform{}, arguments, result_type, 0); else if (const auto * datetime64_type = assert_cast(from_type)) { auto from_scale = datetime64_type->getScale(); - return DateTimeAddIntervalImpl, Transform>::execute( - Transform{}, arguments, result_type, from_scale); + return DateTimeAddIntervalImpl, Transform>::execute(Transform{}, arguments, result_type, from_scale); } else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}", - arguments[0].type->getName(), getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}", arguments[0].type->getName(), getName()); } }; diff --git a/src/Functions/SubtractSubSeconds.cpp b/src/Functions/SubtractSubSeconds.cpp deleted file mode 100644 index b1c47700d13..00000000000 --- a/src/Functions/SubtractSubSeconds.cpp +++ /dev/null @@ -1,28 +0,0 @@ -#include -#include - - -namespace DB -{ - -using FunctionSubtractNanoseconds = FunctionDateOrDateTimeAddInterval; -REGISTER_FUNCTION(SubtractNanoseconds) -{ - factory.registerFunction(); -} - -using FunctionSubtractMicroseconds = FunctionDateOrDateTimeAddInterval; -REGISTER_FUNCTION(SubtractMicroseconds) -{ - factory.registerFunction(); -} - -using FunctionSubtractMilliseconds = FunctionDateOrDateTimeAddInterval; -REGISTER_FUNCTION(SubtractMilliseconds) -{ - factory.registerFunction(); -} - -} - - diff --git a/src/Functions/addMicroseconds.cpp b/src/Functions/addMicroseconds.cpp new file mode 100644 index 00000000000..0dcd6b4452f --- /dev/null +++ b/src/Functions/addMicroseconds.cpp @@ -0,0 +1,16 @@ +#include +#include + + +namespace DB +{ + +using FunctionAddMicroseconds = FunctionDateOrDateTimeAddInterval; +REGISTER_FUNCTION(AddMicroseconds) +{ + factory.registerFunction(); +} + +} + + diff --git a/src/Functions/addMilliseconds.cpp b/src/Functions/addMilliseconds.cpp new file mode 100644 index 00000000000..0e2b696d367 --- /dev/null +++ b/src/Functions/addMilliseconds.cpp @@ -0,0 +1,16 @@ +#include +#include + + +namespace DB +{ + +using FunctionAddMilliseconds = FunctionDateOrDateTimeAddInterval; +REGISTER_FUNCTION(AddMilliseconds) +{ + factory.registerFunction(); +} + +} + + diff --git a/src/Functions/addNanoseconds.cpp b/src/Functions/addNanoseconds.cpp new file mode 100644 index 00000000000..93eadc814d9 --- /dev/null +++ b/src/Functions/addNanoseconds.cpp @@ -0,0 +1,16 @@ +#include +#include + + +namespace DB +{ + +using FunctionAddNanoseconds = FunctionDateOrDateTimeAddInterval; +REGISTER_FUNCTION(AddNanoseconds) +{ + factory.registerFunction(); +} + +} + + diff --git a/src/Functions/addSubSeconds.cpp b/src/Functions/addSubSeconds.cpp deleted file mode 100644 index fa901ad4dcd..00000000000 --- a/src/Functions/addSubSeconds.cpp +++ /dev/null @@ -1,28 +0,0 @@ -#include -#include - - -namespace DB -{ - -using FunctionAddNanoseconds = FunctionDateOrDateTimeAddInterval; -REGISTER_FUNCTION(AddNanoseconds) -{ - factory.registerFunction(); -} - -using FunctionAddMicroseconds = FunctionDateOrDateTimeAddInterval; -REGISTER_FUNCTION(AddMicroseconds) -{ - factory.registerFunction(); -} - -using FunctionAddMilliseconds = FunctionDateOrDateTimeAddInterval; -REGISTER_FUNCTION(AddMilliseconds) -{ - factory.registerFunction(); -} - -} - - diff --git a/src/Functions/subtractMicroseconds.cpp b/src/Functions/subtractMicroseconds.cpp new file mode 100644 index 00000000000..7a274a5021d --- /dev/null +++ b/src/Functions/subtractMicroseconds.cpp @@ -0,0 +1,16 @@ +#include +#include + + +namespace DB +{ + +using FunctionSubtractMicroseconds = FunctionDateOrDateTimeAddInterval; +REGISTER_FUNCTION(SubtractMicroseconds) +{ + factory.registerFunction(); +} + +} + + diff --git a/src/Functions/subtractMilliseconds.cpp b/src/Functions/subtractMilliseconds.cpp new file mode 100644 index 00000000000..a563e18b6bc --- /dev/null +++ b/src/Functions/subtractMilliseconds.cpp @@ -0,0 +1,16 @@ +#include +#include + + +namespace DB +{ + +using FunctionSubtractMilliseconds = FunctionDateOrDateTimeAddInterval; +REGISTER_FUNCTION(SubtractMilliseconds) +{ + factory.registerFunction(); +} + +} + + diff --git a/src/Functions/subtractNanoseconds.cpp b/src/Functions/subtractNanoseconds.cpp new file mode 100644 index 00000000000..fffb4eae37a --- /dev/null +++ b/src/Functions/subtractNanoseconds.cpp @@ -0,0 +1,16 @@ +#include +#include + + +namespace DB +{ + +using FunctionSubtractNanoseconds = FunctionDateOrDateTimeAddInterval; +REGISTER_FUNCTION(SubtractNanoseconds) +{ + factory.registerFunction(); +} + +} + + From fd93bc74bd8919eab7a2c364dcabb941516149c1 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 20 Oct 2023 18:03:16 +0200 Subject: [PATCH 453/634] Fix digest check in replicated ddl worker --- src/Databases/DatabaseReplicatedWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 0ffedeb58f1..5f103a52a61 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -128,7 +128,7 @@ void DatabaseReplicatedDDLWorker::initializeReplication() } std::lock_guard lock{database->metadata_mutex}; - if (!database->checkDigestValid(context)) + if (!database->checkDigestValid(context, false)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Inconsistent database metadata after reconnection to ZooKeeper"); } From ead8b7d71d7b8a059004c5789ee81764a0c95069 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 20 Oct 2023 18:28:55 +0200 Subject: [PATCH 454/634] Test parallel replicas with rollup --- .../02901_parallel_replicas_rollup.reference | 15 ++++ .../02901_parallel_replicas_rollup.sh | 74 +++++++++++++++++++ 2 files changed, 89 insertions(+) create mode 100644 tests/queries/0_stateless/02901_parallel_replicas_rollup.reference create mode 100755 tests/queries/0_stateless/02901_parallel_replicas_rollup.sh diff --git a/tests/queries/0_stateless/02901_parallel_replicas_rollup.reference b/tests/queries/0_stateless/02901_parallel_replicas_rollup.reference new file mode 100644 index 00000000000..29004fde9d4 --- /dev/null +++ b/tests/queries/0_stateless/02901_parallel_replicas_rollup.reference @@ -0,0 +1,15 @@ +1 +02901_parallel_replicas_rollup-default Used parallel replicas: true +0 0 0 6 +2019 0 0 2 +2019 1 0 2 +2019 1 5 1 +2019 1 15 1 +2020 0 0 4 +2020 1 0 2 +2020 1 5 1 +2020 1 15 1 +2020 10 0 2 +2020 10 5 1 +2020 10 15 1 +02901_parallel_replicas_rollup2-default Used parallel replicas: true diff --git a/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh b/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh new file mode 100755 index 00000000000..f23b80348c1 --- /dev/null +++ b/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh @@ -0,0 +1,74 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function were_parallel_replicas_used () +{ + $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" + + # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it + $CLICKHOUSE_CLIENT --query " + SELECT + initial_query_id, + concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used + FROM system.query_log + WHERE event_date >= yesterday() + AND initial_query_id = '$1' + GROUP BY initial_query_id + ORDER BY min(event_time_microseconds) ASC + FORMAT TSV" +} + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS nested" +$CLICKHOUSE_CLIENT --query "CREATE TABLE nested (x UInt8) ENGINE = MergeTree ORDER BY () AS Select 1"; + +query_id="02901_parallel_replicas_rollup-$CLICKHOUSE_DATABASE" +$CLICKHOUSE_CLIENT \ + --query_id "${query_id}" \ + --max_parallel_replicas 3 \ + --prefer_localhost_replica 1 \ + --use_hedged_requests 0 \ + --cluster_for_parallel_replicas "parallel_replicas" \ + --allow_experimental_parallel_reading_from_replicas 1 \ + --parallel_replicas_for_non_replicated_merge_tree 1 \ + --parallel_replicas_min_number_of_rows_per_replica 0 \ + --query " + SELECT 1 FROM nested + GROUP BY 1 WITH ROLLUP + ORDER BY max((SELECT 1 WHERE 0)); +"; +were_parallel_replicas_used $query_id +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS nested" + + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS days" +$CLICKHOUSE_CLIENT --query " + CREATE TABLE days + ( + year Int64, + month Int64, + day Int64 + ) + ENGINE = MergeTree() + ORDER BY year"; +$CLICKHOUSE_CLIENT --query " + INSERT INTO days VALUES (2019, 1, 5), (2019, 1, 15), (2020, 1, 5), (2020, 1, 15), (2020, 10, 5), (2020, 10, 15); +"; + +# Note that we enforce ordering of the final output because it's not guaranteed by GROUP BY ROLLUP, only the values of count() are +query_id="02901_parallel_replicas_rollup2-$CLICKHOUSE_DATABASE" +$CLICKHOUSE_CLIENT \ + --query_id "${query_id}" \ + --max_parallel_replicas 3 \ + --prefer_localhost_replica 1 \ + --use_hedged_requests 0 \ + --cluster_for_parallel_replicas "parallel_replicas" \ + --allow_experimental_parallel_reading_from_replicas 1 \ + --parallel_replicas_for_non_replicated_merge_tree 1 \ + --parallel_replicas_min_number_of_rows_per_replica 0 \ + --query "SELECT * FROM (SELECT year, month, day, count(*) FROM days GROUP BY year, month, day WITH ROLLUP) ORDER BY 1, 2, 3"; + +were_parallel_replicas_used $query_id +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS days" From fda36deec3dd040b6885de0367a6ce0d178dc1c9 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 20 Oct 2023 16:30:44 +0000 Subject: [PATCH 455/634] Fix normalize ASTSelectWithUnionQuery strip FORMAT of the query --- src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp | 6 ++++++ .../0_stateless/02896_union_distinct_http_format.reference | 4 ++++ .../0_stateless/02896_union_distinct_http_format.sh | 7 +++++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02896_union_distinct_http_format.reference create mode 100755 tests/queries/0_stateless/02896_union_distinct_http_format.sh diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp index cbdd656fb8c..f48b987561e 100644 --- a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp @@ -29,7 +29,13 @@ void NormalizeSelectWithUnionQueryMatcher::getSelectsFromUnionListNode(ASTPtr as void NormalizeSelectWithUnionQueryMatcher::visit(ASTPtr & ast, Data & data) { if (auto * select_union = ast->as()) + { + /// The rewrite of ASTSelectWithUnionQuery may strip the format info, so + /// we need to keep and restore it. + auto format = select_union->format; visit(*select_union, data); + select_union->format = format; + } } void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data & data) diff --git a/tests/queries/0_stateless/02896_union_distinct_http_format.reference b/tests/queries/0_stateless/02896_union_distinct_http_format.reference new file mode 100644 index 00000000000..3ca2cb39dc1 --- /dev/null +++ b/tests/queries/0_stateless/02896_union_distinct_http_format.reference @@ -0,0 +1,4 @@ +┌─1─┐ +│ 1 │ +│ 2 │ +└───┘ diff --git a/tests/queries/0_stateless/02896_union_distinct_http_format.sh b/tests/queries/0_stateless/02896_union_distinct_http_format.sh new file mode 100755 index 00000000000..8a79e00ac30 --- /dev/null +++ b/tests/queries/0_stateless/02896_union_distinct_http_format.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo 'SELECT 1 UNION DISTINCT SELECT 2 FORMAT PrettyCompactMonoBlock' | curl 'http://localhost:8123/' --data-binary @- From 5819bcd07a1ed424bc33f81c5f2b9145ca059514 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 20 Oct 2023 18:39:48 +0200 Subject: [PATCH 456/634] Support asynchronous inserts for native protocol (#54730) * support async insert for native protocol * use separate queue for async inserts via native protocol * fix test * better logging for async inserts and more tests * disable mixed internal and external data in async inserts * fix tests * fix quota in async inserts * disable async insert for secondary query of distributed --- src/Client/ClientBase.cpp | 17 +- src/Client/LocalConnection.cpp | 2 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 2 +- .../ClickHouseDictionarySource.cpp | 5 +- .../InterpreterShowAccessEntitiesQuery.cpp | 2 +- .../Access/InterpreterShowPrivilegesQuery.cpp | 2 +- src/Interpreters/AsynchronousInsertLog.cpp | 9 + src/Interpreters/AsynchronousInsertLog.h | 5 +- src/Interpreters/AsynchronousInsertQueue.cpp | 438 ++++++++++++------ src/Interpreters/AsynchronousInsertQueue.h | 92 +++- .../InterpreterKillQueryQuery.cpp | 2 +- .../InterpreterShowColumnsQuery.cpp | 2 +- .../InterpreterShowEngineQuery.cpp | 2 +- .../InterpreterShowFunctionsQuery.cpp | 2 +- .../InterpreterShowIndexesQuery.cpp | 2 +- .../InterpreterShowProcesslistQuery.cpp | 2 +- .../InterpreterShowTablesQuery.cpp | 2 +- src/Interpreters/executeQuery.cpp | 35 +- src/Interpreters/executeQuery.h | 13 +- .../fuzzers/execute_query_fuzzer.cpp | 2 +- src/Interpreters/loadMetadata.cpp | 10 +- .../Executors/StreamingFormatExecutor.cpp | 29 +- .../Executors/StreamingFormatExecutor.h | 3 + .../Transforms/SquashingChunksTransform.h | 1 - src/Server/GRPCServer.cpp | 2 +- src/Server/TCPHandler.cpp | 120 ++++- src/Server/TCPHandler.h | 5 + .../StorageSystemAsynchronousInserts.cpp | 2 +- .../0_stateless/02456_async_inserts_logs.sh | 7 +- ...sable_async_with_dedup_correctly.reference | 2 +- ...4_async_insert_native_protocol_1.reference | 7 + .../02884_async_insert_native_protocol_1.sh | 41 ++ ...4_async_insert_native_protocol_2.reference | 7 + .../02884_async_insert_native_protocol_2.sh | 32 ++ ...4_async_insert_native_protocol_3.reference | 13 + .../02884_async_insert_native_protocol_3.sh | 41 ++ ...4_async_insert_native_protocol_4.reference | 8 + .../02884_async_insert_native_protocol_4.sh | 34 ++ ...02884_async_insert_skip_settings.reference | 5 + .../02884_async_insert_skip_settings.sql | 45 ++ 40 files changed, 797 insertions(+), 255 deletions(-) create mode 100644 tests/queries/0_stateless/02884_async_insert_native_protocol_1.reference create mode 100755 tests/queries/0_stateless/02884_async_insert_native_protocol_1.sh create mode 100644 tests/queries/0_stateless/02884_async_insert_native_protocol_2.reference create mode 100755 tests/queries/0_stateless/02884_async_insert_native_protocol_2.sh create mode 100644 tests/queries/0_stateless/02884_async_insert_native_protocol_3.reference create mode 100755 tests/queries/0_stateless/02884_async_insert_native_protocol_3.sh create mode 100644 tests/queries/0_stateless/02884_async_insert_native_protocol_4.reference create mode 100755 tests/queries/0_stateless/02884_async_insert_native_protocol_4.sh create mode 100644 tests/queries/0_stateless/02884_async_insert_skip_settings.reference create mode 100644 tests/queries/0_stateless/02884_async_insert_skip_settings.sql diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a5a2a6ab328..a350654cdda 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -107,6 +107,7 @@ namespace ErrorCodes extern const int CANNOT_OPEN_FILE; extern const int FILE_ALREADY_EXISTS; extern const int USER_SESSION_LIMIT_EXCEEDED; + extern const int NOT_IMPLEMENTED; } } @@ -1432,7 +1433,7 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars } /// Process the query that requires transferring data blocks to the server. - const auto parsed_insert_query = parsed_query->as(); + const auto & parsed_insert_query = parsed_query->as(); if ((!parsed_insert_query.data && !parsed_insert_query.infile) && (is_interactive || (!stdin_is_a_tty && std_in.eof()))) { const auto & settings = global_context->getSettingsRef(); @@ -1878,10 +1879,20 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin if (insert && insert->select) insert->tryFindInputFunction(input_function); - bool is_async_insert = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); + bool is_async_insert_with_inlined_data = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); + + if (is_async_insert_with_inlined_data) + { + bool have_data_in_stdin = !is_interactive && !stdin_is_a_tty && !std_in.eof(); + bool have_external_data = have_data_in_stdin || insert->infile; + + if (have_external_data) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Processing async inserts with both inlined and external data (from stdin or infile) is not supported"); + } /// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately. - if (insert && (!insert->select || input_function) && !insert->watch && !is_async_insert) + if (insert && (!insert->select || input_function) && !insert->watch && !is_async_insert_with_inlined_data) { if (input_function && insert->format.empty()) throw Exception(ErrorCodes::INVALID_USAGE_OF_INPUT, "FORMAT must be specified for function input()"); diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index c05f039ffb6..15ba1bcd199 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -125,7 +125,7 @@ void LocalConnection::sendQuery( try { - state->io = executeQuery(state->query, query_context, false, state->stage); + state->io = executeQuery(state->query, query_context, false, state->stage).second; if (state->io.pipeline.pushing()) { diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 7409e80ac8c..3d10e66e964 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -75,7 +75,7 @@ static BlockIO tryToExecuteQuery(const String & query_to_execute, ContextMutable if (!database.empty()) query_context->setCurrentDatabase(database); - return executeQuery("/*" + comment + "*/ " + query_to_execute, query_context, true); + return executeQuery("/*" + comment + "*/ " + query_to_execute, query_context, true).second; } catch (...) { diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 2dc7f6145b3..92fae2bc495 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -168,8 +168,7 @@ QueryPipeline ClickHouseDictionarySource::createStreamForQuery(const String & qu if (configuration.is_local) { - pipeline = executeQuery(query, context_copy, true).pipeline; - + pipeline = executeQuery(query, context_copy, true).second.pipeline; pipeline.convertStructureTo(empty_sample_block.getColumnsWithTypeAndName()); } else @@ -191,7 +190,7 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re if (configuration.is_local) { - return readInvalidateQuery(executeQuery(request, context_copy, true).pipeline); + return readInvalidateQuery(executeQuery(request, context_copy, true).second.pipeline); } else { diff --git a/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp index e7b9237b680..b0937dc2f66 100644 --- a/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp @@ -23,7 +23,7 @@ InterpreterShowAccessEntitiesQuery::InterpreterShowAccessEntitiesQuery(const AST BlockIO InterpreterShowAccessEntitiesQuery::execute() { - return executeQuery(getRewrittenQuery(), getContext(), true); + return executeQuery(getRewrittenQuery(), getContext(), true).second; } diff --git a/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp b/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp index 05aa74d7dc4..213e3c813fa 100644 --- a/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp @@ -12,7 +12,7 @@ InterpreterShowPrivilegesQuery::InterpreterShowPrivilegesQuery(const ASTPtr & qu BlockIO InterpreterShowPrivilegesQuery::execute() { - return executeQuery("SELECT * FROM system.privileges", context, true); + return executeQuery("SELECT * FROM system.privileges", context, true).second; } } diff --git a/src/Interpreters/AsynchronousInsertLog.cpp b/src/Interpreters/AsynchronousInsertLog.cpp index d7c9059d9de..092862bb2b1 100644 --- a/src/Interpreters/AsynchronousInsertLog.cpp +++ b/src/Interpreters/AsynchronousInsertLog.cpp @@ -24,6 +24,13 @@ NamesAndTypesList AsynchronousInsertLogElement::getNamesAndTypes() {"FlushError", static_cast(Status::FlushError)}, }); + auto type_data_kind = std::make_shared( + DataTypeEnum8::Values + { + {"Parsed", static_cast(DataKind::Parsed)}, + {"Preprocessed", static_cast(DataKind::Preprocessed)}, + }); + return { {"event_date", std::make_shared()}, @@ -39,6 +46,7 @@ NamesAndTypesList AsynchronousInsertLogElement::getNamesAndTypes() {"rows", std::make_shared()}, {"exception", std::make_shared()}, {"status", type_status}, + {"data_kind", type_data_kind}, {"flush_time", std::make_shared()}, {"flush_time_microseconds", std::make_shared(6)}, @@ -64,6 +72,7 @@ void AsynchronousInsertLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(rows); columns[i++]->insert(exception); columns[i++]->insert(status); + columns[i++]->insert(data_kind); columns[i++]->insert(flush_time); columns[i++]->insert(flush_time_microseconds); diff --git a/src/Interpreters/AsynchronousInsertLog.h b/src/Interpreters/AsynchronousInsertLog.h index a76db78d3ea..3a93b29dabe 100644 --- a/src/Interpreters/AsynchronousInsertLog.h +++ b/src/Interpreters/AsynchronousInsertLog.h @@ -1,6 +1,6 @@ #pragma once -#include "Common/Exception.h" +#include #include #include #include @@ -31,6 +31,9 @@ struct AsynchronousInsertLogElement String exception; Status status{}; + using DataKind = AsynchronousInsertQueue::DataKind; + DataKind data_kind{}; + time_t flush_time{}; Decimal64 flush_time_microseconds{}; String flush_query_id; diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index a343fae6251..70ce2df813c 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -58,26 +58,53 @@ namespace ErrorCodes extern const int UNKNOWN_EXCEPTION; extern const int UNKNOWN_FORMAT; extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } -AsynchronousInsertQueue::InsertQuery::InsertQuery(const ASTPtr & query_, const Settings & settings_, const std::optional & user_id_, const std::vector & current_roles_) +static const NameSet settings_to_skip +{ + /// We don't consider this setting because it is only for deduplication, + /// which means we can put two inserts with different tokens in the same block safely. + "insert_deduplication_token", + "log_comment", +}; + +AsynchronousInsertQueue::InsertQuery::InsertQuery( + const ASTPtr & query_, + const std::optional & user_id_, + const std::vector & current_roles_, + const Settings & settings_, + DataKind data_kind_) : query(query_->clone()) , query_str(queryToString(query)) - , settings(settings_) , user_id(user_id_) , current_roles(current_roles_) - , hash(calculateHash()) + , settings(settings_) + , data_kind(data_kind_) { -} + SipHash siphash; -AsynchronousInsertQueue::InsertQuery::InsertQuery(const InsertQuery & other) - : query(other.query->clone()) - , query_str(other.query_str) - , settings(other.settings) - , user_id(other.user_id) - , current_roles(other.current_roles) - , hash(other.hash) -{ + siphash.update(data_kind); + query->updateTreeHash(siphash); + + if (user_id) + { + siphash.update(*user_id); + for (const auto & current_role : current_roles) + siphash.update(current_role); + } + + for (const auto & setting : settings.allChanged()) + { + if (settings_to_skip.contains(setting.getName())) + continue; + + setting_changes.emplace_back(setting.getName(), setting.getValue()); + siphash.update(setting.getName()); + applyVisitor(FieldVisitorHash(siphash), setting.getValue()); + } + + hash = siphash.get128(); } AsynchronousInsertQueue::InsertQuery & @@ -90,46 +117,29 @@ AsynchronousInsertQueue::InsertQuery::operator=(const InsertQuery & other) user_id = other.user_id; current_roles = other.current_roles; settings = other.settings; + data_kind = other.data_kind; hash = other.hash; + setting_changes = other.setting_changes; } return *this; } -UInt128 AsynchronousInsertQueue::InsertQuery::calculateHash() const -{ - SipHash siphash; - query->updateTreeHash(siphash); - - if (user_id) - { - siphash.update(*user_id); - for (const auto & current_role : current_roles) - siphash.update(current_role); - } - - for (const auto & setting : settings.allChanged()) - { - /// We don't consider this setting because it is only for deduplication, - /// which means we can put two inserts with different tokens in the same block safely. - if (setting.getName() == "insert_deduplication_token") - continue; - siphash.update(setting.getName()); - applyVisitor(FieldVisitorHash(siphash), setting.getValue()); - } - - return siphash.get128(); -} - bool AsynchronousInsertQueue::InsertQuery::operator==(const InsertQuery & other) const { - return query_str == other.query_str && user_id == other.user_id && current_roles == other.current_roles && settings == other.settings; + return toTupleCmp() == other.toTupleCmp(); } -AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_, const String & async_dedup_token_, MemoryTracker * user_memory_tracker_) - : bytes(std::move(bytes_)) +AsynchronousInsertQueue::InsertData::Entry::Entry( + DataChunk && chunk_, + String && query_id_, + const String & async_dedup_token_, + const String & format_, + MemoryTracker * user_memory_tracker_) + : chunk(std::move(chunk_)) , query_id(std::move(query_id_)) , async_dedup_token(async_dedup_token_) + , format(format_) , user_memory_tracker(user_memory_tracker_) , create_time(std::chrono::system_clock::now()) { @@ -146,7 +156,7 @@ void AsynchronousInsertQueue::InsertData::Entry::finish(std::exception_ptr excep // Each entry in the list may correspond to a different user, // so we need to switch current thread's MemoryTracker. MemoryTrackerSwitcher switcher(user_memory_tracker); - bytes = ""; + chunk = {}; } if (exception_) @@ -216,15 +226,12 @@ void AsynchronousInsertQueue::scheduleDataProcessingJob(const InsertQuery & key, }); } -AsynchronousInsertQueue::PushResult -AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) +void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const ContextPtr & query_context) { - query = query->clone(); - const auto & settings = query_context->getSettingsRef(); auto & insert_query = query->as(); insert_query.async_insert_flush = true; - InterpreterInsertQuery interpreter(query, query_context, settings.insert_allow_materialized_columns); + InterpreterInsertQuery interpreter(query, query_context, query_context->getSettingsRef().insert_allow_materialized_columns); auto table = interpreter.getTable(insert_query); auto sample_block = interpreter.getSampleBlock(insert_query, table, table->getInMemoryMetadataPtr()); @@ -235,6 +242,13 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) /// InterpreterInsertQuery::getTable() -> ITableFunction::execute(). if (insert_query.table_id) query_context->checkAccess(AccessType::INSERT, insert_query.table_id, sample_block.getNames()); +} + +AsynchronousInsertQueue::PushResult +AsynchronousInsertQueue::pushQueryWithInlinedData(ASTPtr query, ContextPtr query_context) +{ + query = query->clone(); + preprocessInsertQuery(query, query_context); String bytes; { @@ -245,7 +259,7 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) auto read_buf = getReadBufferFromASTInsertQuery(query); LimitReadBuffer limit_buf( - *read_buf, settings.async_insert_max_data_size, + *read_buf, query_context->getSettingsRef().async_insert_max_data_size, /*throw_exception=*/ false, /*exact_limit=*/ {}); WriteBufferFromString write_buf(bytes); @@ -270,9 +284,35 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) } } - auto entry = std::make_shared(std::move(bytes), query_context->getCurrentQueryId(), settings.insert_deduplication_token, CurrentThread::getUserMemoryTracker()); + return pushDataChunk(std::move(query), std::move(bytes), std::move(query_context)); +} - InsertQuery key{query, settings, query_context->getUserID(), query_context->getCurrentRoles()}; +AsynchronousInsertQueue::PushResult +AsynchronousInsertQueue::pushQueryWithBlock(ASTPtr query, Block block, ContextPtr query_context) +{ + query = query->clone(); + preprocessInsertQuery(query, query_context); + return pushDataChunk(std::move(query), std::move(block), std::move(query_context)); +} + +AsynchronousInsertQueue::PushResult +AsynchronousInsertQueue::pushDataChunk(ASTPtr query, DataChunk chunk, ContextPtr query_context) +{ + const auto & settings = query_context->getSettingsRef(); + auto & insert_query = query->as(); + + auto data_kind = chunk.getDataKind(); + auto entry = std::make_shared( + std::move(chunk), query_context->getCurrentQueryId(), + settings.insert_deduplication_token, insert_query.format, + CurrentThread::getUserMemoryTracker()); + + /// If data is parsed on client we don't care of format which is written + /// in INSERT query. Replace it to put all such queries into one bucket in queue. + if (data_kind == DataKind::Preprocessed) + insert_query.format = "Native"; + + InsertQuery key{query, query_context->getUserID(), query_context->getCurrentRoles(), settings, data_kind}; InsertDataPtr data_to_process; std::future insert_future; @@ -292,7 +332,7 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) auto queue_it = it->second; auto & data = queue_it->second.data; - size_t entry_data_size = entry->bytes.size(); + size_t entry_data_size = entry->chunk.byteSize(); assert(data); data->size_in_bytes += entry_data_size; @@ -456,6 +496,13 @@ catch (...) tryLogCurrentException("AsynchronousInsertQueue", "Failed to add elements to AsynchronousInsertLog"); } +String serializeQuery(const IAST & query, size_t max_length) +{ + return query.hasSecretParts() + ? query.formatForLogging(max_length) + : wipeSensitiveDataAndCutToLength(serializeAST(query), max_length); +} + } // static @@ -469,6 +516,7 @@ try const auto * log = &Poco::Logger::get("AsynchronousInsertQueue"); const auto & insert_query = assert_cast(*key.query); + auto insert_context = Context::createCopy(global_context); bool internal = false; // To enable logging this query bool async_insert = true; @@ -478,6 +526,10 @@ try /// 'resetParser' doesn't work for parallel parsing. key.settings.set("input_format_parallel_parsing", false); + /// It maybe insert into distributed table. + /// It doesn't make sense to make insert into destination tables asynchronous. + key.settings.set("async_insert", false); + insert_context->makeQueryContext(); /// Access rights must be checked for the user who executed the initial INSERT query. @@ -491,6 +543,7 @@ try auto insert_query_id = insert_context->getCurrentQueryId(); auto query_start_time = std::chrono::system_clock::now(); + Stopwatch start_watch{CLOCK_MONOTONIC}; insert_context->setQueryKind(ClientInfo::QueryKind::INITIAL_QUERY); insert_context->setInitialQueryStartTime(query_start_time); @@ -499,36 +552,45 @@ try DB::CurrentThread::QueryScope query_scope_holder(insert_context); - size_t log_queries_cut_to_length = insert_context->getSettingsRef().log_queries_cut_to_length; - String query_for_logging = insert_query.hasSecretParts() - ? insert_query.formatForLogging(log_queries_cut_to_length) - : wipeSensitiveDataAndCutToLength(serializeAST(insert_query), log_queries_cut_to_length); + auto query_for_logging = serializeQuery(*key.query, insert_context->getSettingsRef().log_queries_cut_to_length); /// We add it to the process list so /// a) it appears in system.processes /// b) can be cancelled if we want to /// c) has an associated process list element where runtime metrics are stored - auto process_list_entry - = insert_context->getProcessList().insert(query_for_logging, key.query.get(), insert_context, start_watch.getStart()); + auto process_list_entry = insert_context->getProcessList().insert( + query_for_logging, + key.query.get(), + insert_context, + start_watch.getStart()); + auto query_status = process_list_entry->getQueryStatus(); insert_context->setProcessListElement(std::move(query_status)); - String query_database{}; - String query_table{}; + String query_database; + String query_table; + if (insert_query.table_id) { query_database = insert_query.table_id.getDatabaseName(); query_table = insert_query.table_id.getTableName(); insert_context->setInsertionTable(insert_query.table_id); } + std::unique_ptr interpreter; QueryPipeline pipeline; QueryLogElement query_log_elem; + auto async_insert_log = global_context->getAsynchronousInsertLog(); + std::vector log_elements; + if (async_insert_log) + log_elements.reserve(data->entries.size()); + try { interpreter = std::make_unique( key.query, insert_context, key.settings.insert_allow_materialized_columns, false, false, true); + pipeline = interpreter->execute().pipeline; chassert(pipeline.pushing()); @@ -550,92 +612,39 @@ try throw; } - auto header = pipeline.getHeader(); - auto format = getInputFormatFromASTInsertQuery(key.query, false, header, insert_context, nullptr); - - size_t total_rows = 0; - InsertData::EntryPtr current_entry; - String current_exception; - - auto on_error = [&](const MutableColumns & result_columns, Exception & e) + auto add_entry_to_log = [&]( + const auto & entry, const auto & entry_query_for_logging, + const auto & exception, size_t num_rows, size_t num_bytes) { - current_exception = e.displayText(); - LOG_ERROR(log, "Failed parsing for query '{}' with query id {}. {}", - key.query_str, current_entry->query_id, current_exception); + if (!async_insert_log) + return; - for (const auto & column : result_columns) - if (column->size() > total_rows) - column->popBack(column->size() - total_rows); + AsynchronousInsertLogElement elem; + elem.event_time = timeInSeconds(entry->create_time); + elem.event_time_microseconds = timeInMicroseconds(entry->create_time); + elem.query_for_logging = entry_query_for_logging; + elem.database = query_database; + elem.table = query_table; + elem.format = entry->format; + elem.query_id = entry->query_id; + elem.bytes = num_bytes; + elem.rows = num_rows; + elem.exception = exception; + elem.data_kind = entry->chunk.getDataKind(); - current_entry->finish(std::current_exception()); - return 0; - }; - - std::shared_ptr adding_defaults_transform; - if (insert_context->getSettingsRef().input_format_defaults_for_omitted_fields && insert_query.table_id) - { - StoragePtr storage = DatabaseCatalog::instance().getTable(insert_query.table_id, insert_context); - auto metadata_snapshot = storage->getInMemoryMetadataPtr(); - const auto & columns = metadata_snapshot->getColumns(); - if (columns.hasDefaults()) - adding_defaults_transform = std::make_shared(header, columns, *format, insert_context); - } - - auto insert_log = global_context->getAsynchronousInsertLog(); - std::vector log_elements; - - if (insert_log) - log_elements.reserve(data->entries.size()); - - StreamingFormatExecutor executor(header, format, std::move(on_error), std::move(adding_defaults_transform)); - std::unique_ptr last_buffer; - auto chunk_info = std::make_shared(); - for (const auto & entry : data->entries) - { - auto buffer = std::make_unique(entry->bytes); - current_entry = entry; - auto bytes_size = entry->bytes.size(); - size_t num_rows = executor.execute(*buffer); - total_rows += num_rows; - chunk_info->offsets.push_back(total_rows); - chunk_info->tokens.push_back(entry->async_dedup_token); - - /// Keep buffer, because it still can be used - /// in destructor, while resetting buffer at next iteration. - last_buffer = std::move(buffer); - - if (insert_log) + /// If there was a parsing error, + /// the entry won't be flushed anyway, + /// so add the log element immediately. + if (!elem.exception.empty()) { - AsynchronousInsertLogElement elem; - elem.event_time = timeInSeconds(entry->create_time); - elem.event_time_microseconds = timeInMicroseconds(entry->create_time); - elem.query_for_logging = query_for_logging; - elem.database = query_database; - elem.table = query_table; - elem.format = insert_query.format; - elem.query_id = entry->query_id; - elem.bytes = bytes_size; - elem.rows = num_rows; - elem.exception = current_exception; - current_exception.clear(); - - /// If there was a parsing error, - /// the entry won't be flushed anyway, - /// so add the log element immediately. - if (!elem.exception.empty()) - { - elem.status = AsynchronousInsertLogElement::ParsingError; - insert_log->add(std::move(elem)); - } - else - { - log_elements.push_back(elem); - } + elem.status = AsynchronousInsertLogElement::ParsingError; + async_insert_log->add(std::move(elem)); } - } - - format->addBuffer(std::move(last_buffer)); - ProfileEvents::increment(ProfileEvents::AsyncInsertRows, total_rows); + else + { + log_elements.push_back(elem); + } + }; auto finish_entries = [&] { @@ -648,11 +657,21 @@ try if (!log_elements.empty()) { auto flush_time = std::chrono::system_clock::now(); - appendElementsToLogSafe(*insert_log, std::move(log_elements), flush_time, insert_query_id, ""); + appendElementsToLogSafe(*async_insert_log, std::move(log_elements), flush_time, insert_query_id, ""); } }; - if (total_rows == 0) + Chunk chunk; + auto header = pipeline.getHeader(); + + if (key.data_kind == DataKind::Parsed) + chunk = processEntriesWithParsing(key, data->entries, header, insert_context, log, add_entry_to_log); + else + chunk = processPreprocessedEntries(key, data->entries, header, insert_context, add_entry_to_log); + + ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); + + if (chunk.getNumRows() == 0) { finish_entries(); return; @@ -660,9 +679,8 @@ try try { - auto chunk = Chunk(executor.getResultColumns(), total_rows); - chunk.setChunkInfo(std::move(chunk_info)); - size_t total_bytes = chunk.bytes(); + size_t num_rows = chunk.getNumRows(); + size_t num_bytes = chunk.bytes(); auto source = std::make_shared(header, std::move(chunk)); pipeline.complete(Pipe(std::move(source))); @@ -670,8 +688,7 @@ try CompletedPipelineExecutor completed_executor(pipeline); completed_executor.execute(); - LOG_INFO(log, "Flushed {} rows, {} bytes for query '{}'", - total_rows, total_bytes, key.query_str); + LOG_INFO(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); bool pulling_pipeline = false; logQueryFinish(query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); @@ -684,7 +701,7 @@ try { auto exception = getCurrentExceptionMessage(false); auto flush_time = std::chrono::system_clock::now(); - appendElementsToLogSafe(*insert_log, std::move(log_elements), flush_time, insert_query_id, exception); + appendElementsToLogSafe(*async_insert_log, std::move(log_elements), flush_time, insert_query_id, exception); } throw; } @@ -708,6 +725,133 @@ catch (...) finishWithException(key.query, data->entries, Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception")); } +template +Chunk AsynchronousInsertQueue::processEntriesWithParsing( + const InsertQuery & key, + const std::list & entries, + const Block & header, + const ContextPtr & insert_context, + const Poco::Logger * logger, + LogFunc && add_to_async_insert_log) +{ + size_t total_rows = 0; + InsertData::EntryPtr current_entry; + String current_exception; + + const auto & insert_query = assert_cast(*key.query); + auto format = getInputFormatFromASTInsertQuery(key.query, false, header, insert_context, nullptr); + std::shared_ptr adding_defaults_transform; + + if (insert_context->getSettingsRef().input_format_defaults_for_omitted_fields && insert_query.table_id) + { + StoragePtr storage = DatabaseCatalog::instance().getTable(insert_query.table_id, insert_context); + auto metadata_snapshot = storage->getInMemoryMetadataPtr(); + const auto & columns = metadata_snapshot->getColumns(); + if (columns.hasDefaults()) + adding_defaults_transform = std::make_shared(header, columns, *format, insert_context); + } + + auto on_error = [&](const MutableColumns & result_columns, Exception & e) + { + current_exception = e.displayText(); + LOG_ERROR(logger, "Failed parsing for query '{}' with query id {}. {}", + key.query_str, current_entry->query_id, current_exception); + + for (const auto & column : result_columns) + if (column->size() > total_rows) + column->popBack(column->size() - total_rows); + + current_entry->finish(std::current_exception()); + return 0; + }; + + StreamingFormatExecutor executor(header, format, std::move(on_error), std::move(adding_defaults_transform)); + std::unique_ptr last_buffer; + auto chunk_info = std::make_shared(); + auto query_for_logging = serializeQuery(*key.query, insert_context->getSettingsRef().log_queries_cut_to_length); + + for (const auto & entry : entries) + { + current_entry = entry; + + const auto * bytes = entry->chunk.asString(); + if (!bytes) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected entry with data kind Parsed. Got: {}", entry->chunk.getDataKind()); + + auto buffer = std::make_unique(*bytes); + size_t num_bytes = bytes->size(); + size_t num_rows = executor.execute(*buffer); + + /// Keep buffer, because it still can be used + /// in destructor, while resetting buffer at next iteration. + last_buffer = std::move(buffer); + + total_rows += num_rows; + chunk_info->offsets.push_back(total_rows); + chunk_info->tokens.push_back(entry->async_dedup_token); + + add_to_async_insert_log(entry, query_for_logging, current_exception, num_rows, num_bytes); + current_exception.clear(); + } + + format->addBuffer(std::move(last_buffer)); + + Chunk chunk(executor.getResultColumns(), total_rows); + chunk.setChunkInfo(std::move(chunk_info)); + return chunk; +} + +template +Chunk AsynchronousInsertQueue::processPreprocessedEntries( + const InsertQuery & key, + const std::list & entries, + const Block & header, + const ContextPtr & insert_context, + LogFunc && add_to_async_insert_log) +{ + size_t total_rows = 0; + auto chunk_info = std::make_shared(); + auto result_columns = header.cloneEmptyColumns(); + + std::unordered_map format_to_query; + + auto get_query_by_format = [&](const String & format) -> const String & + { + auto [it, inserted] = format_to_query.try_emplace(format); + if (!inserted) + return it->second; + + auto query = key.query->clone(); + assert_cast(*query).format = format; + it->second = serializeQuery(*query, insert_context->getSettingsRef().log_queries_cut_to_length); + return it->second; + }; + + for (const auto & entry : entries) + { + const auto * block = entry->chunk.asBlock(); + if (!block) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected entry with data kind Preprocessed. Got: {}", entry->chunk.getDataKind()); + + auto columns = block->getColumns(); + for (size_t i = 0, s = columns.size(); i < s; ++i) + result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); + + total_rows += block->rows(); + chunk_info->offsets.push_back(total_rows); + chunk_info->tokens.push_back(entry->async_dedup_token); + + const auto & query_for_logging = get_query_by_format(entry->format); + add_to_async_insert_log(entry, query_for_logging, "", block->rows(), block->bytes()); + } + + Chunk chunk(std::move(result_columns), total_rows); + chunk.setChunkInfo(std::move(chunk_info)); + return chunk; +} + template void AsynchronousInsertQueue::finishWithException( const ASTPtr & query, const std::list & entries, const E & exception) diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index 2b92e336d09..99394d0fb14 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -1,13 +1,16 @@ #pragma once +#include #include #include #include #include #include #include +#include #include +#include namespace DB { @@ -38,11 +41,23 @@ public: /// Read buffer that contains extracted /// from query data in case of too much data. std::unique_ptr insert_data_buffer; + + /// Block that contains received by Native + /// protocol data in case of too much data. + Block insert_block; + }; + + enum class DataKind + { + Parsed = 0, + Preprocessed = 1, }; /// Force flush the whole queue. void flushAll(); - PushResult push(ASTPtr query, ContextPtr query_context); + + PushResult pushQueryWithInlinedData(ASTPtr query, ContextPtr query_context); + PushResult pushQueryWithBlock(ASTPtr query, Block block, ContextPtr query_context); size_t getPoolSize() const { return pool_size; } private: @@ -52,18 +67,55 @@ private: public: ASTPtr query; String query_str; - Settings settings; std::optional user_id; std::vector current_roles; + Settings settings; + + DataKind data_kind; UInt128 hash; - InsertQuery(const ASTPtr & query_, const Settings & settings_, const std::optional & user_id_, const std::vector & current_roles_); - InsertQuery(const InsertQuery & other); + InsertQuery( + const ASTPtr & query_, + const std::optional & user_id_, + const std::vector & current_roles_, + const Settings & settings_, + DataKind data_kind_); + + InsertQuery(const InsertQuery & other) { *this = other; } InsertQuery & operator=(const InsertQuery & other); bool operator==(const InsertQuery & other) const; private: - UInt128 calculateHash() const; + auto toTupleCmp() const { return std::tie(data_kind, query_str, user_id, current_roles, setting_changes); } + + std::vector setting_changes; + }; + + struct DataChunk : public std::variant + { + using std::variant::variant; + + size_t byteSize() const + { + return std::visit([](const T & arg) + { + if constexpr (std::is_same_v) + return arg.bytes(); + else + return arg.size(); + }, *this); + } + + DataKind getDataKind() const + { + if (std::holds_alternative(*this)) + return DataKind::Preprocessed; + else + return DataKind::Parsed; + } + + const String * asString() const { return std::get_if(this); } + const Block * asBlock() const { return std::get_if(this); } }; struct InsertData @@ -71,13 +123,19 @@ private: struct Entry { public: - String bytes; + DataChunk chunk; const String query_id; const String async_dedup_token; + const String format; MemoryTracker * const user_memory_tracker; const std::chrono::time_point create_time; - Entry(String && bytes_, String && query_id_, const String & async_dedup_token, MemoryTracker * user_memory_tracker_); + Entry( + DataChunk && chunk_, + String && query_id_, + const String & async_dedup_token_, + const String & format_, + MemoryTracker * user_memory_tracker_); void finish(std::exception_ptr exception_ = nullptr); std::future getFuture() { return promise.get_future(); } @@ -158,11 +216,31 @@ private: Poco::Logger * log = &Poco::Logger::get("AsynchronousInsertQueue"); + PushResult pushDataChunk(ASTPtr query, DataChunk chunk, ContextPtr query_context); + void preprocessInsertQuery(const ASTPtr & query, const ContextPtr & query_context); + void processBatchDeadlines(size_t shard_num); void scheduleDataProcessingJob(const InsertQuery & key, InsertDataPtr data, ContextPtr global_context); static void processData(InsertQuery key, InsertDataPtr data, ContextPtr global_context); + template + static Chunk processEntriesWithParsing( + const InsertQuery & key, + const std::list & entries, + const Block & header, + const ContextPtr & insert_context, + const Poco::Logger * logger, + LogFunc && add_to_async_insert_log); + + template + static Chunk processPreprocessedEntries( + const InsertQuery & key, + const std::list & entries, + const Block & header, + const ContextPtr & insert_context, + LogFunc && add_to_async_insert_log); + template static void finishWithException(const ASTPtr & query, const std::list & entries, const E & exception); diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 590b7fe37b8..1c2e3ff6777 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -420,7 +420,7 @@ Block InterpreterKillQueryQuery::getSelectResult(const String & columns, const S if (where_expression) select_query += " WHERE " + queryToString(where_expression); - auto io = executeQuery(select_query, getContext(), true); + auto io = executeQuery(select_query, getContext(), true).second; PullingPipelineExecutor executor(io.pipeline); Block res; while (!res && executor.pull(res)); diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index 55891f51063..d14a36ef7e1 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -162,7 +162,7 @@ WHERE BlockIO InterpreterShowColumnsQuery::execute() { - return executeQuery(getRewrittenQuery(), getContext(), true); + return executeQuery(getRewrittenQuery(), getContext(), true).second; } diff --git a/src/Interpreters/InterpreterShowEngineQuery.cpp b/src/Interpreters/InterpreterShowEngineQuery.cpp index 8fd829f39ec..a2367e9bfdf 100644 --- a/src/Interpreters/InterpreterShowEngineQuery.cpp +++ b/src/Interpreters/InterpreterShowEngineQuery.cpp @@ -12,7 +12,7 @@ namespace DB BlockIO InterpreterShowEnginesQuery::execute() { - return executeQuery("SELECT * FROM system.table_engines ORDER BY name", getContext(), true); + return executeQuery("SELECT * FROM system.table_engines ORDER BY name", getContext(), true).second; } } diff --git a/src/Interpreters/InterpreterShowFunctionsQuery.cpp b/src/Interpreters/InterpreterShowFunctionsQuery.cpp index efadb929451..ace22ca4bb6 100644 --- a/src/Interpreters/InterpreterShowFunctionsQuery.cpp +++ b/src/Interpreters/InterpreterShowFunctionsQuery.cpp @@ -15,7 +15,7 @@ InterpreterShowFunctionsQuery::InterpreterShowFunctionsQuery(const ASTPtr & quer BlockIO InterpreterShowFunctionsQuery::execute() { - return executeQuery(getRewrittenQuery(), getContext(), true); + return executeQuery(getRewrittenQuery(), getContext(), true).second; } String InterpreterShowFunctionsQuery::getRewrittenQuery() diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index 149420006fb..9b36f1496e7 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -107,7 +107,7 @@ ORDER BY index_type, expression, column_name, seq_in_index;)", database, table, BlockIO InterpreterShowIndexesQuery::execute() { - return executeQuery(getRewrittenQuery(), getContext(), true); + return executeQuery(getRewrittenQuery(), getContext(), true).second; } diff --git a/src/Interpreters/InterpreterShowProcesslistQuery.cpp b/src/Interpreters/InterpreterShowProcesslistQuery.cpp index f9241368a8f..4ed5f4171c6 100644 --- a/src/Interpreters/InterpreterShowProcesslistQuery.cpp +++ b/src/Interpreters/InterpreterShowProcesslistQuery.cpp @@ -12,7 +12,7 @@ namespace DB BlockIO InterpreterShowProcesslistQuery::execute() { - return executeQuery("SELECT * FROM system.processes ORDER BY elapsed DESC", getContext(), true); + return executeQuery("SELECT * FROM system.processes ORDER BY elapsed DESC", getContext(), true).second; } } diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 5fe0a862e05..97bd8e7a8b7 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -188,7 +188,7 @@ BlockIO InterpreterShowTablesQuery::execute() return res; } - return executeQuery(getRewrittenQuery(), getContext(), true); + return executeQuery(getRewrittenQuery(), getContext(), true).second; } /// (*) Sorting is strictly speaking not necessary but 1. it is convenient for users, 2. SQL currently does not allow to diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 861332c65fd..decda4c62f9 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -926,12 +926,10 @@ static std::tuple executeQueryImpl( reason = "asynchronous insert queue is not configured"; else if (insert_query->select) reason = "insert query has select"; - else if (!insert_query->hasInlinedData()) - reason = "insert query doesn't have inlined data"; - else + else if (insert_query->hasInlinedData()) async_insert = true; - if (!async_insert) + if (!reason.empty()) LOG_DEBUG(logger, "Setting async_insert=1, but INSERT query will be executed synchronously (reason: {})", reason); } @@ -954,7 +952,7 @@ static std::tuple executeQueryImpl( quota->checkExceeded(QuotaType::ERRORS); } - auto result = queue->push(ast, context); + auto result = queue->pushQueryWithInlinedData(ast, context); if (result.status == AsynchronousInsertQueue::PushResult::OK) { @@ -1228,19 +1226,20 @@ static std::tuple executeQueryImpl( throw; } - return std::make_tuple(ast, std::move(res)); + return std::make_tuple(std::move(ast), std::move(res)); } -BlockIO executeQuery( +std::pair executeQuery( const String & query, ContextMutablePtr context, bool internal, QueryProcessingStage::Enum stage) { ASTPtr ast; - BlockIO streams; - std::tie(ast, streams) = executeQueryImpl(query.data(), query.data() + query.size(), context, internal, stage, nullptr); + BlockIO res; + + std::tie(ast, res) = executeQueryImpl(query.data(), query.data() + query.size(), context, internal, stage, nullptr); if (const auto * ast_query_with_output = dynamic_cast(ast.get())) { @@ -1249,26 +1248,12 @@ BlockIO executeQuery( : context->getDefaultFormat(); if (format_name == "Null") - streams.null_format = true; + res.null_format = true; } - return streams; + return std::make_pair(std::move(ast), std::move(res)); } -BlockIO executeQuery( - bool allow_processors, - const String & query, - ContextMutablePtr context, - bool internal, - QueryProcessingStage::Enum stage) -{ - if (!allow_processors) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Flag allow_processors is deprecated for executeQuery"); - - return executeQuery(query, context, internal, stage); -} - - void executeQuery( ReadBuffer & istr, WriteBuffer & ostr, diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index a31d4f2f08c..6f14f54d7d6 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -43,7 +43,7 @@ void executeQuery( /// More low-level function for server-to-server interaction. /// Prepares a query for execution but doesn't execute it. -/// Returns a pair of block streams which, when used, will result in query execution. +/// Returns a pair of parsed query and BlockIO which, when used, will result in query execution. /// This means that the caller can to the extent control the query execution pipeline. /// /// To execute: @@ -55,22 +55,13 @@ void executeQuery( /// /// Correctly formatting the results (according to INTO OUTFILE and FORMAT sections) /// must be done separately. -BlockIO executeQuery( +std::pair executeQuery( const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out. ContextMutablePtr context, /// DB, tables, data types, storage engines, functions, aggregate functions... bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList. QueryProcessingStage::Enum stage = QueryProcessingStage::Complete /// To which stage the query must be executed. ); -/// Old interface with allow_processors flag. For compatibility. -BlockIO executeQuery( - bool allow_processors, /// If can use processors pipeline - const String & query, - ContextMutablePtr context, - bool internal = false, - QueryProcessingStage::Enum stage = QueryProcessingStage::Complete -); - /// Executes BlockIO returned from executeQuery(...) /// if built pipeline does not require any input and does not produce any output. void executeTrivialBlockIO(BlockIO & streams, ContextPtr context); diff --git a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp index f12c01120cf..0f6bfc1ae58 100644 --- a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp +++ b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp @@ -42,7 +42,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) static bool initialized = initialize(); (void) initialized; - auto io = DB::executeQuery(input, context, true, QueryProcessingStage::Complete); + auto io = DB::executeQuery(input, context, true, QueryProcessingStage::Complete).second; PullingPipelineExecutor executor(io.pipeline); Block res; diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index aeb45c08bea..faa1dcda2c0 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -282,7 +282,7 @@ static void convertOrdinaryDatabaseToAtomic(Poco::Logger * log, ContextMutablePt LOG_INFO(log, "Will convert database {} from Ordinary to Atomic", name_quoted); String create_database_query = fmt::format("CREATE DATABASE IF NOT EXISTS {}", tmp_name_quoted); - auto res = executeQuery(create_database_query, context, true); + auto res = executeQuery(create_database_query, context, true).second; executeTrivialBlockIO(res, context); res = {}; auto tmp_database = DatabaseCatalog::instance().getDatabase(tmp_name); @@ -322,7 +322,7 @@ static void convertOrdinaryDatabaseToAtomic(Poco::Logger * log, ContextMutablePt String tmp_qualified_quoted_name = id.getFullTableName(); String move_table_query = fmt::format("RENAME TABLE {} TO {}", qualified_quoted_name, tmp_qualified_quoted_name); - res = executeQuery(move_table_query, context, true); + res = executeQuery(move_table_query, context, true).second; executeTrivialBlockIO(res, context); res = {}; } @@ -334,12 +334,12 @@ static void convertOrdinaryDatabaseToAtomic(Poco::Logger * log, ContextMutablePt String drop_query = fmt::format("DROP DATABASE {}", name_quoted); context->setSetting("force_remove_data_recursively_on_drop", false); - res = executeQuery(drop_query, context, true); + res = executeQuery(drop_query, context, true).second; executeTrivialBlockIO(res, context); res = {}; String rename_query = fmt::format("RENAME DATABASE {} TO {}", tmp_name_quoted, name_quoted); - res = executeQuery(rename_query, context, true); + res = executeQuery(rename_query, context, true).second; executeTrivialBlockIO(res, context); LOG_INFO(log, "Finished database engine conversion of {}", name_quoted); @@ -409,7 +409,7 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons /// Reload database just in case (and update logger name) String detach_query = fmt::format("DETACH DATABASE {}", backQuoteIfNeed(database_name)); - auto res = executeQuery(detach_query, context, true); + auto res = executeQuery(detach_query, context, true).second; executeTrivialBlockIO(res, context); res = {}; diff --git a/src/Processors/Executors/StreamingFormatExecutor.cpp b/src/Processors/Executors/StreamingFormatExecutor.cpp index 46818989032..281961f7c7c 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.cpp +++ b/src/Processors/Executors/StreamingFormatExecutor.cpp @@ -66,21 +66,9 @@ size_t StreamingFormatExecutor::execute() return new_rows; case IProcessor::Status::PortFull: - { - auto chunk = port.pull(); - if (adding_defaults_transform) - adding_defaults_transform->transform(chunk); - - auto chunk_rows = chunk.getNumRows(); - new_rows += chunk_rows; - - auto columns = chunk.detachColumns(); - - for (size_t i = 0, s = columns.size(); i < s; ++i) - result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); - + new_rows += insertChunk(port.pull()); break; - } + case IProcessor::Status::NeedData: case IProcessor::Status::Async: case IProcessor::Status::ExpandPipeline: @@ -107,4 +95,17 @@ size_t StreamingFormatExecutor::execute() } } +size_t StreamingFormatExecutor::insertChunk(Chunk chunk) +{ + size_t chunk_rows = chunk.getNumRows(); + if (adding_defaults_transform) + adding_defaults_transform->transform(chunk); + + auto columns = chunk.detachColumns(); + for (size_t i = 0, s = columns.size(); i < s; ++i) + result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); + + return chunk_rows; +} + } diff --git a/src/Processors/Executors/StreamingFormatExecutor.h b/src/Processors/Executors/StreamingFormatExecutor.h index f5a1562a340..3aa90ab0360 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.h +++ b/src/Processors/Executors/StreamingFormatExecutor.h @@ -33,6 +33,9 @@ public: /// Execute with provided read buffer. size_t execute(ReadBuffer & buffer); + /// Inserts into result columns already preprocessed chunk. + size_t insertChunk(Chunk chunk); + /// Releases currently accumulated columns. MutableColumns getResultColumns(); diff --git a/src/Processors/Transforms/SquashingChunksTransform.h b/src/Processors/Transforms/SquashingChunksTransform.h index df13f539b90..f82e9e46a61 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -22,7 +22,6 @@ protected: GenerateResult onGenerate() override; void onFinish() override; - private: SquashingTransform squashing; Chunk cur_chunk; diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 4d9f5c983c5..812c2b5489d 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -949,7 +949,7 @@ namespace query_end = insert_query->data; } String query(begin, query_end); - io = ::DB::executeQuery(true, query, query_context); + io = ::DB::executeQuery(query, query_context).second; } void Call::processInput() diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 8303ee2c9a1..4908bf82b46 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1,3 +1,7 @@ +#include "Interpreters/AsynchronousInsertQueue.h" +#include "Interpreters/Context_fwd.h" +#include "Interpreters/SquashingTransform.h" +#include "Parsers/ASTInsertQuery.h" #include #include #include @@ -99,6 +103,7 @@ namespace DB::ErrorCodes extern const int AUTHENTICATION_FAILED; extern const int QUERY_WAS_CANCELLED; extern const int CLIENT_INFO_DOES_NOT_MATCH; + extern const int TIMEOUT_EXCEEDED; extern const int SUPPORT_IS_DISABLED; extern const int UNSUPPORTED_METHOD; extern const int FUNCTION_NOT_ALLOWED; @@ -497,7 +502,7 @@ void TCPHandler::runImpl() }); /// Processing Query - state.io = executeQuery(state.query, query_context, false, state.stage); + std::tie(state.parsed_query, state.io) = executeQuery(state.query, query_context, false, state.stage); after_check_cancelled.restart(); after_send_progress.restart(); @@ -810,35 +815,66 @@ void TCPHandler::skipData() throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); } +void TCPHandler::startInsertQuery() +{ + /// Send ColumnsDescription for insertion table + if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA) + { + const auto & table_id = query_context->getInsertionTable(); + if (query_context->getSettingsRef().input_format_defaults_for_omitted_fields) + { + if (!table_id.empty()) + { + auto storage_ptr = DatabaseCatalog::instance().getTable(table_id, query_context); + sendTableColumns(storage_ptr->getInMemoryMetadataPtr()->getColumns()); + } + } + } + + /// Send block to the client - table structure. + sendData(state.io.pipeline.getHeader()); + sendLogs(); +} + +AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(AsynchronousInsertQueue & insert_queue) +{ + using PushResult = AsynchronousInsertQueue::PushResult; + + startInsertQuery(); + SquashingTransform squashing(0, query_context->getSettingsRef().async_insert_max_data_size); + + while (readDataNext()) + { + auto result = squashing.add(std::move(state.block_for_insert)); + if (result) + { + return PushResult + { + .status = PushResult::TOO_MUCH_DATA, + .insert_block = std::move(result), + }; + } + } + + auto result = squashing.add({}); + return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); +} void TCPHandler::processInsertQuery() { size_t num_threads = state.io.pipeline.getNumThreads(); - auto run_executor = [&](auto & executor) + auto run_executor = [&](auto & executor, Block processed_data) { /// Made above the rest of the lines, - /// so that in case of `writePrefix` function throws an exception, + /// so that in case of `start` function throws an exception, /// client receive exception before sending data. executor.start(); - /// Send ColumnsDescription for insertion table - if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA) - { - const auto & table_id = query_context->getInsertionTable(); - if (query_context->getSettingsRef().input_format_defaults_for_omitted_fields) - { - if (!table_id.empty()) - { - auto storage_ptr = DatabaseCatalog::instance().getTable(table_id, query_context); - sendTableColumns(storage_ptr->getInMemoryMetadataPtr()->getColumns()); - } - } - } - - /// Send block to the client - table structure. - sendData(executor.getHeader()); - sendLogs(); + if (processed_data) + executor.push(std::move(processed_data)); + else + startInsertQuery(); while (readDataNext()) executor.push(std::move(state.block_for_insert)); @@ -849,15 +885,55 @@ void TCPHandler::processInsertQuery() executor.finish(); }; + Block processed_block; + const auto & settings = query_context->getSettingsRef(); + + auto * insert_queue = query_context->getAsynchronousInsertQueue(); + const auto & insert_query = assert_cast(*state.parsed_query); + + bool async_insert_enabled = settings.async_insert; + if (insert_query.table_id) + if (auto table = DatabaseCatalog::instance().tryGetTable(insert_query.table_id, query_context)) + async_insert_enabled |= table->areAsynchronousInsertsEnabled(); + + if (insert_queue && async_insert_enabled && !insert_query.select) + { + auto result = processAsyncInsertQuery(*insert_queue); + if (result.status == AsynchronousInsertQueue::PushResult::OK) + { + if (settings.wait_for_async_insert) + { + size_t timeout_ms = settings.wait_for_async_insert_timeout.totalMilliseconds(); + auto wait_status = result.future.wait_for(std::chrono::milliseconds(timeout_ms)); + + if (wait_status == std::future_status::deferred) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: got future in deferred state"); + + if (wait_status == std::future_status::timeout) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Wait for async insert timeout ({} ms) exceeded)", timeout_ms); + + result.future.get(); + } + + sendInsertProfileEvents(); + return; + } + else if (result.status == AsynchronousInsertQueue::PushResult::TOO_MUCH_DATA) + { + LOG_DEBUG(log, "Setting async_insert=1, but INSERT query will be executed synchronously because it has too much data"); + processed_block = std::move(result.insert_block); + } + } + if (num_threads > 1) { PushingAsyncPipelineExecutor executor(state.io.pipeline); - run_executor(executor); + run_executor(executor, std::move(processed_block)); } else { PushingPipelineExecutor executor(state.io.pipeline); - run_executor(executor); + run_executor(executor, processed_block); } sendInsertProfileEvents(); diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 7ebb605e1c9..9fd243baa6c 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -21,6 +21,7 @@ #include #include "IServer.h" +#include "Interpreters/AsynchronousInsertQueue.h" #include "Server/TCPProtocolStackData.h" #include "Storages/MergeTree/RequestResponse.h" #include "base/types.h" @@ -73,6 +74,8 @@ struct QueryState /// Query text. String query; + /// Parsed query + ASTPtr parsed_query; /// Streams of blocks, that are processing the query. BlockIO io; @@ -247,7 +250,9 @@ private: [[noreturn]] void receiveUnexpectedTablesStatusRequest(); /// Process INSERT query + void startInsertQuery(); void processInsertQuery(); + AsynchronousInsertQueue::PushResult processAsyncInsertQuery(AsynchronousInsertQueue & insert_queue); /// Process a request that does not require the receiving of data blocks from the client void processOrdinaryQuery(); diff --git a/src/Storages/System/StorageSystemAsynchronousInserts.cpp b/src/Storages/System/StorageSystemAsynchronousInserts.cpp index 15258ccfd7f..ec3a9d92f30 100644 --- a/src/Storages/System/StorageSystemAsynchronousInserts.cpp +++ b/src/Storages/System/StorageSystemAsynchronousInserts.cpp @@ -82,7 +82,7 @@ void StorageSystemAsynchronousInserts::fillData(MutableColumns & res_columns, Co for (const auto & entry : data->entries) { arr_query_id.push_back(entry->query_id); - arr_bytes.push_back(entry->bytes.size()); + arr_bytes.push_back(entry->chunk.byteSize()); } res_columns[i++]->insert(arr_query_id); diff --git a/tests/queries/0_stateless/02456_async_inserts_logs.sh b/tests/queries/0_stateless/02456_async_inserts_logs.sh index 43cd73d7231..3251c777dfc 100755 --- a/tests/queries/0_stateless/02456_async_inserts_logs.sh +++ b/tests/queries/0_stateless/02456_async_inserts_logs.sh @@ -33,7 +33,12 @@ ${CLICKHOUSE_CLIENT} -q " SELECT table, format, bytes, rows, empty(exception), status, status = 'ParsingError' ? flush_time_microseconds = 0 : flush_time_microseconds > event_time_microseconds AS time_ok FROM system.asynchronous_insert_log - WHERE database = '$CLICKHOUSE_DATABASE' OR query ILIKE 'INSERT INTO FUNCTION%$CLICKHOUSE_DATABASE%' + WHERE + ( + database = '$CLICKHOUSE_DATABASE' AND table = 't_async_inserts_logs' + OR query ILIKE 'INSERT INTO FUNCTION%$CLICKHOUSE_DATABASE%t_async_inserts_logs%' + ) + AND data_kind = 'Parsed' ORDER BY table, status, format" ${CLICKHOUSE_CLIENT} -q "DROP TABLE t_async_inserts_logs" diff --git a/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.reference b/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.reference index 014be4ce1a9..3999dc78a15 100644 --- a/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.reference +++ b/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.reference @@ -1,6 +1,6 @@ 0 1 -1 +0 1 2 3 diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_1.reference b/tests/queries/0_stateless/02884_async_insert_native_protocol_1.reference new file mode 100644 index 00000000000..44f4e24d7df --- /dev/null +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_1.reference @@ -0,0 +1,7 @@ +NOT_IMPLEMENTED +2 +1 aaa +2 bbb +3 ccc +Ok 2 Preprocessed JSONEachRow +Ok 1 Parsed JSONEachRow diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_1.sh b/tests/queries/0_stateless/02884_async_insert_native_protocol_1.sh new file mode 100755 index 00000000000..82e2bb709f9 --- /dev/null +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_1.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT -n -q " + DROP TABLE IF EXISTS t_async_insert_native_1; + CREATE TABLE t_async_insert_native_1 (id UInt64, s String) ENGINE = MergeTree ORDER BY id; +" + +async_insert_options="--async_insert 1 --wait_for_async_insert 0 --async_insert_busy_timeout_ms 1000000" + +echo '{"id": 1, "s": "aaa"} {"id": 2, "s": "bbb"}' | $CLICKHOUSE_CLIENT $async_insert_options -q 'INSERT INTO t_async_insert_native_1 FORMAT JSONEachRow' +$CLICKHOUSE_CLIENT $async_insert_options -q 'INSERT INTO t_async_insert_native_1 FORMAT JSONEachRow {"id": 3, "s": "ccc"}' + +# Mixed inlined and external data is not supported. +echo '{"id": 1, "s": "aaa"}' \ + | $CLICKHOUSE_CLIENT $async_insert_options -q 'INSERT INTO t_async_insert_native_1 FORMAT JSONEachRow {"id": 2, "s": "bbb"}' 2>&1 \ + | grep -o "NOT_IMPLEMENTED" + +$CLICKHOUSE_CLIENT -n -q " + SELECT sum(length(entries.bytes)) FROM system.asynchronous_inserts + WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_async_insert_native_1'; + + SYSTEM FLUSH ASYNC INSERT QUEUE; + + SELECT * FROM t_async_insert_native_1 ORDER BY id; + + SYSTEM FLUSH LOGS; + + SELECT status, rows, data_kind, format + FROM system.asynchronous_insert_log + WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_async_insert_native_1' + ORDER BY event_time_microseconds; + + DROP TABLE t_async_insert_native_1; +" diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_2.reference b/tests/queries/0_stateless/02884_async_insert_native_protocol_2.reference new file mode 100644 index 00000000000..75c015cfab8 --- /dev/null +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_2.reference @@ -0,0 +1,7 @@ +1 aaa +2 bbb +3 ccc +4 ddd +5 eee +JSONEachRow Ok 2 Preprocessed JSONEachRow +Values Ok 3 Preprocessed Values diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_2.sh b/tests/queries/0_stateless/02884_async_insert_native_protocol_2.sh new file mode 100755 index 00000000000..b9b1854eaef --- /dev/null +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_2.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT -n -q " + DROP TABLE IF EXISTS t_async_insert_native_2; + CREATE TABLE t_async_insert_native_2 (id UInt64, s String) ENGINE = MergeTree ORDER BY id; +" + +async_insert_options="--async_insert 1 --wait_for_async_insert 1" + +echo '{"id": 1, "s": "aaa"} {"id": 2, "s": "bbb"}' | $CLICKHOUSE_CLIENT $async_insert_options -q 'INSERT INTO t_async_insert_native_2 FORMAT JSONEachRow' & +echo "(3, 'ccc') (4, 'ddd') (5, 'eee')" | $CLICKHOUSE_CLIENT $async_insert_options -q 'INSERT INTO t_async_insert_native_2 FORMAT Values' & + +wait + +$CLICKHOUSE_CLIENT -n -q " + SELECT * FROM t_async_insert_native_2 ORDER BY id; + + SYSTEM FLUSH LOGS; + + SELECT format, status, rows, data_kind, format + FROM system.asynchronous_insert_log + WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_async_insert_native_2' + ORDER BY format; + + DROP TABLE t_async_insert_native_2; +" diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_3.reference b/tests/queries/0_stateless/02884_async_insert_native_protocol_3.reference new file mode 100644 index 00000000000..8d5d2846308 --- /dev/null +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_3.reference @@ -0,0 +1,13 @@ +Native 3 +Values 1 +1 aaa +2 bbb +3 ccc +4 ddd +5 eee +6 fff +7 ggg +JSONEachRow Ok 2 Preprocessed JSONEachRow +Values Ok 3 Preprocessed Values +CSV Ok 1 Preprocessed CSV +Values Ok 1 Parsed Values diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_3.sh b/tests/queries/0_stateless/02884_async_insert_native_protocol_3.sh new file mode 100755 index 00000000000..abe6be9e2bc --- /dev/null +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_3.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT -n -q " + DROP TABLE IF EXISTS t_async_insert_native_3; + CREATE TABLE t_async_insert_native_3 (id UInt64, s String) ENGINE = MergeTree ORDER BY id; +" + +async_insert_options="--async_insert 1 --wait_for_async_insert 0 --async_insert_busy_timeout_ms 1000000" + +echo '{"id": 1, "s": "aaa"} {"id": 2, "s": "bbb"}' | $CLICKHOUSE_CLIENT $async_insert_options -q 'INSERT INTO t_async_insert_native_3 FORMAT JSONEachRow' +echo "(3, 'ccc') (4, 'ddd') (5, 'eee')" | $CLICKHOUSE_CLIENT $async_insert_options -q 'INSERT INTO t_async_insert_native_3 FORMAT Values' +echo '6,"fff"' | $CLICKHOUSE_CLIENT $async_insert_options -q 'INSERT INTO t_async_insert_native_3 FORMAT CSV' +$CLICKHOUSE_CLIENT $async_insert_options -q "INSERT INTO t_async_insert_native_3 VALUES (7, 'ggg')" + +wait + +$CLICKHOUSE_CLIENT -n -q " + SELECT format, length(entries.bytes) FROM system.asynchronous_inserts + WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_async_insert_native_3' + ORDER BY format; + + SYSTEM FLUSH ASYNC INSERT QUEUE; + + SELECT * FROM t_async_insert_native_3 ORDER BY id; + + SYSTEM FLUSH LOGS; + + SELECT format, status, rows, data_kind, format + FROM system.asynchronous_insert_log + WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_async_insert_native_3' + ORDER BY event_time_microseconds; + + DROP TABLE t_async_insert_native_3; +" diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_4.reference b/tests/queries/0_stateless/02884_async_insert_native_protocol_4.reference new file mode 100644 index 00000000000..aaabca43547 --- /dev/null +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_4.reference @@ -0,0 +1,8 @@ +0 +1 +1 +2 +3 +4 +5 +Values Ok 1 Preprocessed Values diff --git a/tests/queries/0_stateless/02884_async_insert_native_protocol_4.sh b/tests/queries/0_stateless/02884_async_insert_native_protocol_4.sh new file mode 100755 index 00000000000..9118c11315c --- /dev/null +++ b/tests/queries/0_stateless/02884_async_insert_native_protocol_4.sh @@ -0,0 +1,34 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -n -q " + DROP TABLE IF EXISTS t_async_insert_native_4; + CREATE TABLE t_async_insert_native_4 (id UInt64) ENGINE = MergeTree ORDER BY id; +" + +async_insert_options="--async_insert 1 --wait_for_async_insert 1" +CLICKHOUSE_CLIENT_WITH_LOG=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') + +echo "(1)" | $CLICKHOUSE_CLIENT_WITH_LOG $async_insert_options --async_insert_max_data_size 10 \ + -q 'INSERT INTO t_async_insert_native_4 FORMAT Values' 2>&1 \ + | grep -c "too much data" + +echo "(2) (3) (4) (5)" | $CLICKHOUSE_CLIENT_WITH_LOG $async_insert_options --async_insert_max_data_size 10 \ + -q 'INSERT INTO t_async_insert_native_4 FORMAT Values' 2>&1 \ + | grep -c "too much data" + +$CLICKHOUSE_CLIENT -n -q " + SELECT * FROM t_async_insert_native_4 ORDER BY id; + + SYSTEM FLUSH LOGS; + + SELECT format, status, rows, data_kind, format + FROM system.asynchronous_insert_log + WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_async_insert_native_4' + ORDER BY format; + + DROP TABLE t_async_insert_native_4; +" diff --git a/tests/queries/0_stateless/02884_async_insert_skip_settings.reference b/tests/queries/0_stateless/02884_async_insert_skip_settings.reference new file mode 100644 index 00000000000..318966ce93b --- /dev/null +++ b/tests/queries/0_stateless/02884_async_insert_skip_settings.reference @@ -0,0 +1,5 @@ +4 +1 +1 +2 +1 diff --git a/tests/queries/0_stateless/02884_async_insert_skip_settings.sql b/tests/queries/0_stateless/02884_async_insert_skip_settings.sql new file mode 100644 index 00000000000..facd39d1079 --- /dev/null +++ b/tests/queries/0_stateless/02884_async_insert_skip_settings.sql @@ -0,0 +1,45 @@ +-- Tags: no-parallel + +DROP TABLE IF EXISTS t_async_insert_skip_settings SYNC; + +CREATE TABLE t_async_insert_skip_settings (id UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/tables/t_async_insert_skip_settings', '1') +ORDER BY id; + +SET async_insert = 1; +SET async_insert_deduplicate = 1; +SET wait_for_async_insert = 0; +SET async_insert_busy_timeout_ms = 100000; + +SET insert_deduplication_token = '1'; +SET log_comment = 'async_insert_skip_settings_1'; +INSERT INTO t_async_insert_skip_settings VALUES (1); + +SET insert_deduplication_token = '2'; +SET log_comment = 'async_insert_skip_settings_2'; +INSERT INTO t_async_insert_skip_settings VALUES (1); + +SET insert_deduplication_token = '1'; +SET log_comment = 'async_insert_skip_settings_3'; +INSERT INTO t_async_insert_skip_settings VALUES (2); + +SET insert_deduplication_token = '3'; +SET log_comment = 'async_insert_skip_settings_4'; +INSERT INTO t_async_insert_skip_settings VALUES (2); + +SYSTEM FLUSH LOGS; + +SELECT length(entries.bytes) FROM system.asynchronous_inserts +WHERE database = currentDatabase() AND table = 't_async_insert_skip_settings' +ORDER BY first_update; + +SYSTEM FLUSH ASYNC INSERT QUEUE; + +SELECT * FROM t_async_insert_skip_settings ORDER BY id; + +SYSTEM FLUSH LOGS; + +SELECT uniqExact(flush_query_id) FROM system.asynchronous_insert_log +WHERE database = currentDatabase() AND table = 't_async_insert_skip_settings'; + +DROP TABLE t_async_insert_skip_settings SYNC; From 0a1f3f205caa0a951d149a76a4509599476d011d Mon Sep 17 00:00:00 2001 From: Aleksei Filatov <68555560+aalexfvk@users.noreply.github.com> Date: Fri, 20 Oct 2023 20:24:19 +0300 Subject: [PATCH 457/634] Add external HTTP Basic authenticator (#55199) --- .../external-authenticators/http.md | 91 +++++++++++++ .../external-authenticators/index.md | 1 + .../sql-reference/statements/create/user.md | 3 +- src/Access/AccessControl.cpp | 2 +- src/Access/AccessControl.h | 2 +- src/Access/Authentication.cpp | 18 ++- src/Access/Authentication.h | 9 +- src/Access/AuthenticationData.cpp | 22 +++- src/Access/AuthenticationData.h | 16 ++- src/Access/Common/AuthenticationType.cpp | 5 + src/Access/Common/AuthenticationType.h | 3 + .../Common/HTTPAuthenticationScheme.cpp | 31 +++++ src/Access/Common/HTTPAuthenticationScheme.h | 16 +++ src/Access/ExternalAuthenticators.cpp | 71 ++++++++++- src/Access/ExternalAuthenticators.h | 13 +- src/Access/HTTPAuthClient.h | 93 ++++++++++++++ src/Access/IAccessStorage.cpp | 16 ++- src/Access/IAccessStorage.h | 41 +++++- src/Access/LDAPAccessStorage.cpp | 6 +- src/Access/LDAPAccessStorage.h | 2 +- src/Access/MultipleAccessStorage.cpp | 10 +- src/Access/MultipleAccessStorage.h | 2 +- src/Access/SettingsAuthResponseParser.cpp | 45 +++++++ src/Access/SettingsAuthResponseParser.h | 28 ++++ src/Access/UsersConfigAccessStorage.cpp | 19 ++- src/Interpreters/Session.cpp | 9 +- src/Interpreters/Session.h | 4 + src/Interpreters/SessionLog.cpp | 3 +- src/Parsers/Access/ASTAuthenticationData.cpp | 16 +++ src/Parsers/Access/ParserCreateUserQuery.cpp | 21 +++ .../__init__.py | 0 .../configs/config.xml | 14 ++ .../configs/users.xml | 64 ++++++++++ .../http_auth_server.py | 82 ++++++++++++ .../test_external_http_authenticator/test.py | 120 ++++++++++++++++++ .../02117_show_create_table_system.reference | 2 +- .../02713_create_user_substitutions.reference | 2 + .../02713_create_user_substitutions.sh | 6 +- 38 files changed, 866 insertions(+), 42 deletions(-) create mode 100644 docs/en/operations/external-authenticators/http.md create mode 100644 src/Access/Common/HTTPAuthenticationScheme.cpp create mode 100644 src/Access/Common/HTTPAuthenticationScheme.h create mode 100644 src/Access/HTTPAuthClient.h create mode 100644 src/Access/SettingsAuthResponseParser.cpp create mode 100644 src/Access/SettingsAuthResponseParser.h create mode 100644 tests/integration/test_external_http_authenticator/__init__.py create mode 100644 tests/integration/test_external_http_authenticator/configs/config.xml create mode 100644 tests/integration/test_external_http_authenticator/configs/users.xml create mode 100644 tests/integration/test_external_http_authenticator/http_auth_server.py create mode 100644 tests/integration/test_external_http_authenticator/test.py diff --git a/docs/en/operations/external-authenticators/http.md b/docs/en/operations/external-authenticators/http.md new file mode 100644 index 00000000000..24e9cf7cdf6 --- /dev/null +++ b/docs/en/operations/external-authenticators/http.md @@ -0,0 +1,91 @@ +--- +slug: /en/operations/external-authenticators/http +title: "HTTP" +--- +import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md'; + + + +HTTP server can be used to authenticate ClickHouse users. HTTP authentication can only be used as an external authenticator for existing users, which are defined in `users.xml` or in local access control paths. Currently, [Basic](https://datatracker.ietf.org/doc/html/rfc7617) authentication scheme using GET method is supported. + +## HTTP authentication server definition {#http-auth-server-definition} + +To define HTTP authentication server you must add `http_authentication_servers` section to the `config.xml`. + +**Example** +```xml + + + + + http://localhost:8000/auth + 1000 + 1000 + 1000 + 3 + 50 + 1000 + + + + +``` + +Note, that you can define multiple HTTP servers inside the `http_authentication_servers` section using distinct names. + +**Parameters** +- `uri` - URI for making authentication request + +Timeouts in milliseconds on the socket used for communicating with the server: +- `connection_timeout_ms` - Default: 1000 ms. +- `receive_timeout_ms` - Default: 1000 ms. +- `send_timeout_ms` - Default: 1000 ms. + +Retry parameters: +- `max_tries` - The maximum number of attempts to make an authentication request. Default: 3 +- `retry_initial_backoff_ms` - The backoff initial interval on retry. Default: 50 ms +- `retry_max_backoff_ms` - The maximum backoff interval. Default: 1000 ms + +### Enabling HTTP authentication in `users.xml` {#enabling-http-auth-in-users-xml} + +In order to enable HTTP authentication for the user, specify `http_authentication` section instead of `password` or similar sections in the user definition. + +Parameters: +- `server` - Name of the HTTP authentication server configured in the main `config.xml` file as described previously. +- `scheme` - HTTP authentication scheme. `Basic` is only supported now. Default: Basic + +Example (goes into `users.xml`): +```xml + + + + + + basic_server + basic + + + +``` + +:::note +Note that HTTP authentication cannot be used alongside with any other authentication mechanism. The presence of any other sections like `password` alongside `http_authentication` will force ClickHouse to shutdown. +::: + +### Enabling HTTP authentication using SQL {#enabling-http-auth-using-sql} + +When [SQL-driven Access Control and Account Management](/docs/en/guides/sre/user-management/index.md#access-control) is enabled in ClickHouse, users identified by HTTP authentication can also be created using SQL statements. + +```sql +CREATE USER my_user IDENTIFIED WITH HTTP SERVER 'basic_server' SCHEME 'Basic' +``` + +...or, `Basic` is default without explicit scheme definition + +```sql +CREATE USER my_user IDENTIFIED WITH HTTP SERVER 'basic_server' +``` + +### Passing session settings {#passing-session-settings} + +If a response body from HTTP authentication server has JSON format and contains `settings` sub-object, ClickHouse will try parse its key: value pairs as string values and set them as session settings for authenticated user's current session. If parsing is failed, a response body from server will be ignored. diff --git a/docs/en/operations/external-authenticators/index.md b/docs/en/operations/external-authenticators/index.md index bbfb733cf10..f644613641c 100644 --- a/docs/en/operations/external-authenticators/index.md +++ b/docs/en/operations/external-authenticators/index.md @@ -16,3 +16,4 @@ The following external authenticators and directories are supported: - [LDAP](./ldap.md#external-authenticators-ldap) [Authenticator](./ldap.md#ldap-external-authenticator) and [Directory](./ldap.md#ldap-external-user-directory) - Kerberos [Authenticator](./kerberos.md#external-authenticators-kerberos) - [SSL X.509 authentication](./ssl-x509.md#ssl-external-authentication) +- HTTP [Authenticator](./http.md) \ No newline at end of file diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index 20cd9a47e98..96f75908c35 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -12,7 +12,7 @@ Syntax: ``` sql CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [, name2 [ON CLUSTER cluster_name2] ...] - [NOT IDENTIFIED | IDENTIFIED {[WITH {no_password | plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'}] + [NOT IDENTIFIED | IDENTIFIED {[WITH {no_password | plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']}] [HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] [VALID UNTIL datetime] [IN access_storage_type] @@ -40,6 +40,7 @@ There are multiple ways of user identification: - `IDENTIFIED WITH kerberos` or `IDENTIFIED WITH kerberos REALM 'realm'` - `IDENTIFIED WITH ssl_certificate CN 'mysite.com:user'` - `IDENTIFIED WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa', KEY 'another_public_key' TYPE 'ssh-ed25519'` +- `IDENTIFIED WITH http SERVER 'http_server'` or `IDENTIFIED WITH http SERVER 'http_server' SCHEME 'basic'` - `IDENTIFIED BY 'qwerty'` Password complexity requirements can be edited in [config.xml](/docs/en/operations/configuration-files). Below is an example configuration that requires passwords to be at least 12 characters long and contain 1 number. Each password complexity rule requires a regex to match against passwords and a description of the rule. diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index bf123534feb..d50bbdecf90 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -567,7 +567,7 @@ AccessChangesNotifier & AccessControl::getChangesNotifier() } -UUID AccessControl::authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address) const +AuthResult AccessControl::authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address) const { try { diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 74bcc119184..76431ab4928 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -118,7 +118,7 @@ public: scope_guard subscribeForChanges(const UUID & id, const OnChangedHandler & handler) const; scope_guard subscribeForChanges(const std::vector & ids, const OnChangedHandler & handler) const; - UUID authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address) const; + AuthResult authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address) const; /// Makes a backup of access entities. void restoreFromBackup(RestorerFromBackup & restorer) override; diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp index 07a75db730e..372334bb152 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -85,7 +85,11 @@ namespace } -bool Authentication::areCredentialsValid(const Credentials & credentials, const AuthenticationData & auth_data, const ExternalAuthenticators & external_authenticators) +bool Authentication::areCredentialsValid( + const Credentials & credentials, + const AuthenticationData & auth_data, + const ExternalAuthenticators & external_authenticators, + SettingsChanges & settings) { if (!credentials.isReady()) return false; @@ -100,6 +104,7 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const case AuthenticationType::DOUBLE_SHA1_PASSWORD: case AuthenticationType::BCRYPT_PASSWORD: case AuthenticationType::LDAP: + case AuthenticationType::HTTP: throw Authentication::Require("ClickHouse Basic Authentication"); case AuthenticationType::KERBEROS: @@ -133,6 +138,7 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const case AuthenticationType::BCRYPT_PASSWORD: case AuthenticationType::LDAP: case AuthenticationType::KERBEROS: + case AuthenticationType::HTTP: throw Authentication::Require("ClickHouse Basic Authentication"); case AuthenticationType::SSL_CERTIFICATE: @@ -177,6 +183,14 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const case AuthenticationType::BCRYPT_PASSWORD: return checkPasswordBcrypt(basic_credentials->getPassword(), auth_data.getPasswordHashBinary()); + case AuthenticationType::HTTP: + switch (auth_data.getHTTPAuthenticationScheme()) + { + case HTTPAuthenticationScheme::BASIC: + return external_authenticators.checkHTTPBasicCredentials( + auth_data.getHTTPAuthenticationServerName(), *basic_credentials, settings); + } + case AuthenticationType::MAX: break; } @@ -192,6 +206,7 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const case AuthenticationType::DOUBLE_SHA1_PASSWORD: case AuthenticationType::BCRYPT_PASSWORD: case AuthenticationType::LDAP: + case AuthenticationType::HTTP: throw Authentication::Require("ClickHouse Basic Authentication"); case AuthenticationType::KERBEROS: @@ -218,6 +233,7 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const case AuthenticationType::DOUBLE_SHA1_PASSWORD: case AuthenticationType::BCRYPT_PASSWORD: case AuthenticationType::LDAP: + case AuthenticationType::HTTP: throw Authentication::Require("ClickHouse Basic Authentication"); case AuthenticationType::KERBEROS: diff --git a/src/Access/Authentication.h b/src/Access/Authentication.h index d1e00a28ebb..ffc497cc442 100644 --- a/src/Access/Authentication.h +++ b/src/Access/Authentication.h @@ -14,12 +14,19 @@ namespace ErrorCodes class Credentials; class ExternalAuthenticators; +class SettingsChanges; /// TODO: Try to move this checking to Credentials. struct Authentication { /// Checks the credentials (passwords, readiness, etc.) - static bool areCredentialsValid(const Credentials & credentials, const AuthenticationData & auth_data, const ExternalAuthenticators & external_authenticators); + /// If necessary, makes a request to external authenticators and fills in the session settings if they were + /// returned by the authentication server + static bool areCredentialsValid( + const Credentials & credentials, + const AuthenticationData & auth_data, + const ExternalAuthenticators & external_authenticators, + SettingsChanges & settings); // A signaling class used to communicate requirements for credentials. template diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp index 483c0ecb4d6..1036e1e027e 100644 --- a/src/Access/AuthenticationData.cpp +++ b/src/Access/AuthenticationData.cpp @@ -105,7 +105,8 @@ bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs) return (lhs.type == rhs.type) && (lhs.password_hash == rhs.password_hash) && (lhs.ldap_server_name == rhs.ldap_server_name) && (lhs.kerberos_realm == rhs.kerberos_realm) && (lhs.ssl_certificate_common_names == rhs.ssl_certificate_common_names) - && (lhs.ssh_keys == rhs.ssh_keys); + && (lhs.ssh_keys == rhs.ssh_keys) && (lhs.http_auth_scheme == rhs.http_auth_scheme) + && (lhs.http_auth_server_name == rhs.http_auth_server_name); } @@ -128,6 +129,7 @@ void AuthenticationData::setPassword(const String & password_) case AuthenticationType::KERBEROS: case AuthenticationType::SSL_CERTIFICATE: case AuthenticationType::SSH_KEY: + case AuthenticationType::HTTP: throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot specify password for authentication type {}", toString(type)); case AuthenticationType::MAX: @@ -232,6 +234,7 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash) case AuthenticationType::KERBEROS: case AuthenticationType::SSL_CERTIFICATE: case AuthenticationType::SSH_KEY: + case AuthenticationType::HTTP: throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot specify password binary hash for authentication type {}", toString(type)); case AuthenticationType::MAX: @@ -326,6 +329,12 @@ std::shared_ptr AuthenticationData::toAST() const throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL"); #endif } + case AuthenticationType::HTTP: + { + node->children.push_back(std::make_shared(getHTTPAuthenticationServerName())); + node->children.push_back(std::make_shared(toString(getHTTPAuthenticationScheme()))); + break; + } case AuthenticationType::NO_PASSWORD: [[fallthrough]]; case AuthenticationType::MAX: @@ -484,6 +493,17 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que auth_data.setSSLCertificateCommonNames(std::move(common_names)); } + else if (query.type == AuthenticationType::HTTP) + { + String server = checkAndGetLiteralArgument(args[0], "http_auth_server_name"); + auto scheme = HTTPAuthenticationScheme::BASIC; // Default scheme + + if (args_size > 1) + scheme = parseHTTPAuthenticationScheme(checkAndGetLiteralArgument(args[1], "scheme")); + + auth_data.setHTTPAuthenticationServerName(server); + auth_data.setHTTPAuthenticationScheme(scheme); + } else { throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected ASTAuthenticationData structure"); diff --git a/src/Access/AuthenticationData.h b/src/Access/AuthenticationData.h index 75859ed7c52..feef4d71d66 100644 --- a/src/Access/AuthenticationData.h +++ b/src/Access/AuthenticationData.h @@ -1,13 +1,14 @@ #pragma once #include -#include -#include +#include #include +#include +#include +#include #include #include -#include namespace DB { @@ -61,6 +62,12 @@ public: const std::vector & getSSHKeys() const { return ssh_keys; } void setSSHKeys(std::vector && ssh_keys_) { ssh_keys = std::forward>(ssh_keys_); } + HTTPAuthenticationScheme getHTTPAuthenticationScheme() const { return http_auth_scheme; } + void setHTTPAuthenticationScheme(HTTPAuthenticationScheme scheme) { http_auth_scheme = scheme; } + + const String & getHTTPAuthenticationServerName() const { return http_auth_server_name; } + void setHTTPAuthenticationServerName(const String & name) { http_auth_server_name = name; } + friend bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs); friend bool operator !=(const AuthenticationData & lhs, const AuthenticationData & rhs) { return !(lhs == rhs); } @@ -88,6 +95,9 @@ private: boost::container::flat_set ssl_certificate_common_names; String salt; std::vector ssh_keys; + /// HTTP authentication properties + String http_auth_server_name; + HTTPAuthenticationScheme http_auth_scheme = HTTPAuthenticationScheme::BASIC; }; } diff --git a/src/Access/Common/AuthenticationType.cpp b/src/Access/Common/AuthenticationType.cpp index b1f410d9285..a61dc33fc0e 100644 --- a/src/Access/Common/AuthenticationType.cpp +++ b/src/Access/Common/AuthenticationType.cpp @@ -67,6 +67,11 @@ const AuthenticationTypeInfo & AuthenticationTypeInfo::get(AuthenticationType ty static const auto info = make_info("SSH_KEY"); return info; } + case AuthenticationType::HTTP: + { + static const auto info = make_info("HTTP"); + return info; + } case AuthenticationType::MAX: break; } diff --git a/src/Access/Common/AuthenticationType.h b/src/Access/Common/AuthenticationType.h index 2bdef30830c..e81b9e6e439 100644 --- a/src/Access/Common/AuthenticationType.h +++ b/src/Access/Common/AuthenticationType.h @@ -37,6 +37,9 @@ enum class AuthenticationType /// The check is performed on server side by decrypting the data and comparing with the original string. SSH_KEY, + /// Authentication through HTTP protocol + HTTP, + MAX, }; diff --git a/src/Access/Common/HTTPAuthenticationScheme.cpp b/src/Access/Common/HTTPAuthenticationScheme.cpp new file mode 100644 index 00000000000..fb9601eeb8f --- /dev/null +++ b/src/Access/Common/HTTPAuthenticationScheme.cpp @@ -0,0 +1,31 @@ +#include "HTTPAuthenticationScheme.h" + +#include +#include +#include + +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + + +String toString(HTTPAuthenticationScheme scheme) +{ + return String(magic_enum::enum_name(scheme)); +} + +HTTPAuthenticationScheme parseHTTPAuthenticationScheme(const String & scheme_str) +{ + auto scheme = magic_enum::enum_cast(Poco::toUpper(scheme_str)); + if (!scheme) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown HTTP authentication scheme: {}. Possible value is 'BASIC'", scheme_str); + return *scheme; +} + +} diff --git a/src/Access/Common/HTTPAuthenticationScheme.h b/src/Access/Common/HTTPAuthenticationScheme.h new file mode 100644 index 00000000000..84d39ac5a26 --- /dev/null +++ b/src/Access/Common/HTTPAuthenticationScheme.h @@ -0,0 +1,16 @@ +#pragma once + +#include + +namespace DB +{ + +enum class HTTPAuthenticationScheme +{ + BASIC, +}; + + +String toString(HTTPAuthenticationScheme scheme); +HTTPAuthenticationScheme parseHTTPAuthenticationScheme(const String & scheme_str); +} diff --git a/src/Access/ExternalAuthenticators.cpp b/src/Access/ExternalAuthenticators.cpp index e4d4d2acd06..9439a17dabd 100644 --- a/src/Access/ExternalAuthenticators.cpp +++ b/src/Access/ExternalAuthenticators.cpp @@ -1,11 +1,14 @@ #include #include +#include +#include #include -#include +#include #include +#include -#include #include +#include #include #include @@ -230,6 +233,24 @@ void parseKerberosParams(GSSAcceptorContext::Params & params, const Poco::Util:: params.keytab = config.getString("kerberos.keytab", ""); } +HTTPAuthClientParams parseHTTPAuthParams(const Poco::Util::AbstractConfiguration & config, const String & prefix) +{ + HTTPAuthClientParams http_auth_params; + + http_auth_params.uri = config.getString(prefix + ".uri"); + + size_t connection_timeout_ms = config.getInt(prefix + ".connection_timeout_ms", 1000); + size_t receive_timeout_ms = config.getInt(prefix + ".receive_timeout_ms", 1000); + size_t send_timeout_ms = config.getInt(prefix + ".send_timeout_ms", 1000); + http_auth_params.timeouts = ConnectionTimeouts{connection_timeout_ms, receive_timeout_ms, send_timeout_ms}; + + http_auth_params.max_tries = config.getInt(prefix + ".max_tries", 3); + http_auth_params.retry_initial_backoff_ms = config.getInt(prefix + ".retry_initial_backoff_ms", 50); + http_auth_params.retry_max_backoff_ms = config.getInt(prefix + ".retry_max_backoff_ms", 1000); + + return http_auth_params; +} + } void parseLDAPRoleSearchParams(LDAPClient::RoleSearchParams & params, const Poco::Util::AbstractConfiguration & config, const String & prefix) @@ -265,6 +286,9 @@ void ExternalAuthenticators::setConfiguration(const Poco::Util::AbstractConfigur std::size_t ldap_servers_key_count = 0; std::size_t kerberos_keys_count = 0; + std::size_t http_auth_server_keys_count = 0; + + const String http_auth_servers_config = "http_authentication_servers"; for (auto key : all_keys) { @@ -276,6 +300,7 @@ void ExternalAuthenticators::setConfiguration(const Poco::Util::AbstractConfigur ldap_servers_key_count += (key == "ldap_servers"); kerberos_keys_count += (key == "kerberos"); + http_auth_server_keys_count += (key == http_auth_servers_config); } if (ldap_servers_key_count > 1) @@ -284,6 +309,25 @@ void ExternalAuthenticators::setConfiguration(const Poco::Util::AbstractConfigur if (kerberos_keys_count > 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple kerberos sections are not allowed"); + if (http_auth_server_keys_count > 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple http_authentication_servers sections are not allowed"); + + Poco::Util::AbstractConfiguration::Keys http_auth_server_names; + config.keys(http_auth_servers_config, http_auth_server_names); + http_auth_servers.clear(); + for (const auto & http_auth_server_name : http_auth_server_names) + { + String prefix = fmt::format("{}.{}", http_auth_servers_config, http_auth_server_name); + try + { + http_auth_servers[http_auth_server_name] = parseHTTPAuthParams(config, prefix); + } + catch (...) + { + tryLogCurrentException(log, "Could not parse HTTP auth server" + backQuote(http_auth_server_name)); + } + } + Poco::Util::AbstractConfiguration::Keys ldap_server_names; config.keys("ldap_servers", ldap_server_names); ldap_client_params_blueprint.clear(); @@ -490,4 +534,27 @@ GSSAcceptorContext::Params ExternalAuthenticators::getKerberosParams() const return kerberos_params.value(); } +HTTPAuthClientParams ExternalAuthenticators::getHTTPAuthenticationParams(const String& server) const +{ + std::scoped_lock lock{mutex}; + + const auto it = http_auth_servers.find(server); + if (it == http_auth_servers.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "HTTP server '{}' is not configured", server); + return it->second; +} + +bool ExternalAuthenticators::checkHTTPBasicCredentials( + const String & server, const BasicCredentials & credentials, SettingsChanges & settings) const +{ + auto params = getHTTPAuthenticationParams(server); + HTTPBasicAuthClient client(params); + + auto [is_ok, settings_from_auth_server] = client.authenticate(credentials.getUserName(), credentials.getPassword()); + + if (is_ok) + std::ranges::move(settings_from_auth_server, std::back_inserter(settings)); + + return is_ok; +} } diff --git a/src/Access/ExternalAuthenticators.h b/src/Access/ExternalAuthenticators.h index 7b47c9351fd..46c51f0d2f3 100644 --- a/src/Access/ExternalAuthenticators.h +++ b/src/Access/ExternalAuthenticators.h @@ -1,11 +1,14 @@ #pragma once -#include #include #include +#include +#include #include -#include #include +#include + +#include #include #include @@ -27,6 +30,8 @@ namespace Poco namespace DB { +class SettingsChanges; + class ExternalAuthenticators { public: @@ -37,10 +42,13 @@ public: bool checkLDAPCredentials(const String & server, const BasicCredentials & credentials, const LDAPClient::RoleSearchParamsList * role_search_params = nullptr, LDAPClient::SearchResultsList * role_search_results = nullptr) const; bool checkKerberosCredentials(const String & realm, const GSSAcceptorContext & credentials) const; + bool checkHTTPBasicCredentials(const String & server, const BasicCredentials & credentials, SettingsChanges & settings) const; GSSAcceptorContext::Params getKerberosParams() const; private: + HTTPAuthClientParams getHTTPAuthenticationParams(const String& server) const; + struct LDAPCacheEntry { UInt128 last_successful_params_hash = 0; @@ -56,6 +64,7 @@ private: LDAPParams ldap_client_params_blueprint TSA_GUARDED_BY(mutex) ; mutable LDAPCaches ldap_caches TSA_GUARDED_BY(mutex) ; std::optional kerberos_params TSA_GUARDED_BY(mutex) ; + std::unordered_map http_auth_servers TSA_GUARDED_BY(mutex) ; void resetImpl() TSA_REQUIRES(mutex); }; diff --git a/src/Access/HTTPAuthClient.h b/src/Access/HTTPAuthClient.h new file mode 100644 index 00000000000..5a8a3031a84 --- /dev/null +++ b/src/Access/HTTPAuthClient.h @@ -0,0 +1,93 @@ +#pragma once +#include +#include + +#include +#include + + +namespace DB +{ + +struct HTTPAuthClientParams +{ + Poco::URI uri; + ConnectionTimeouts timeouts; + size_t max_tries; + size_t retry_initial_backoff_ms; + size_t retry_max_backoff_ms; +}; + +template +class HTTPAuthClient +{ +public: + using Result = TResponseParser::Result; + + HTTPAuthClient(const HTTPAuthClientParams & params, const TResponseParser & parser_ = TResponseParser{}) + : timeouts{params.timeouts} + , max_tries{params.max_tries} + , retry_initial_backoff_ms{params.retry_initial_backoff_ms} + , retry_max_backoff_ms{params.retry_max_backoff_ms} + , uri{params.uri} + , parser{parser_} + { + } + + Result authenticateRequest(Poco::Net::HTTPRequest & request) const + { + auto session = makeHTTPSession(uri, timeouts); + Poco::Net::HTTPResponse response; + + auto milliseconds_to_wait = retry_initial_backoff_ms; + for (size_t attempt = 0; attempt < max_tries; ++attempt) + { + bool last_attempt = attempt + 1 >= max_tries; + try + { + session->sendRequest(request); + auto & body_stream = session->receiveResponse(response); + return parser.parse(response, &body_stream); + } + catch (const Poco::Exception &) // TODO: make retries smarter + { + if (last_attempt) + throw; + + sleepForMilliseconds(milliseconds_to_wait); + milliseconds_to_wait = std::min(milliseconds_to_wait * 2, retry_max_backoff_ms); + } + } + UNREACHABLE(); + } + + const Poco::URI & getURI() const { return uri; } + +private: + const ConnectionTimeouts timeouts; + const size_t max_tries; + const size_t retry_initial_backoff_ms; + const size_t retry_max_backoff_ms; + const Poco::URI uri; + TResponseParser parser; +}; + + +template +class HTTPBasicAuthClient : private HTTPAuthClient +{ +public: + using HTTPAuthClient::HTTPAuthClient; + using Result = HTTPAuthClient::Result; + + Result authenticate(const String & user_name, const String & password) const + { + Poco::Net::HTTPRequest request{Poco::Net::HTTPRequest::HTTP_GET, this->getURI().getPathAndQuery()}; + Poco::Net::HTTPBasicCredentials basic_credentials{user_name, password}; + basic_credentials.authenticate(request); + + return this->authenticateRequest(request); + } +}; + +} diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index fcc4ea79e65..222f38b41b6 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -488,7 +488,7 @@ bool IAccessStorage::updateImpl(const UUID & id, const UpdateFunc &, bool throw_ } -UUID IAccessStorage::authenticate( +AuthResult IAccessStorage::authenticate( const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, @@ -499,7 +499,7 @@ UUID IAccessStorage::authenticate( } -std::optional IAccessStorage::authenticate( +std::optional IAccessStorage::authenticate( const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, @@ -511,7 +511,7 @@ std::optional IAccessStorage::authenticate( } -std::optional IAccessStorage::authenticateImpl( +std::optional IAccessStorage::authenticateImpl( const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, @@ -523,6 +523,7 @@ std::optional IAccessStorage::authenticateImpl( { if (auto user = tryRead(*id)) { + AuthResult auth_result { .user_id = *id }; if (!isAddressAllowed(*user, address)) throwAddressNotAllowed(address); @@ -531,10 +532,10 @@ std::optional IAccessStorage::authenticateImpl( ((auth_type == AuthenticationType::PLAINTEXT_PASSWORD) && !allow_plaintext_password)) throwAuthenticationTypeNotAllowed(auth_type); - if (!areCredentialsValid(*user, credentials, external_authenticators)) + if (!areCredentialsValid(*user, credentials, external_authenticators, auth_result.settings)) throwInvalidCredentials(); - return id; + return auth_result; } } @@ -548,7 +549,8 @@ std::optional IAccessStorage::authenticateImpl( bool IAccessStorage::areCredentialsValid( const User & user, const Credentials & credentials, - const ExternalAuthenticators & external_authenticators) const + const ExternalAuthenticators & external_authenticators, + SettingsChanges & settings) const { if (!credentials.isReady()) return false; @@ -564,7 +566,7 @@ bool IAccessStorage::areCredentialsValid( return false; } - return Authentication::areCredentialsValid(credentials, user.auth_data, external_authenticators); + return Authentication::areCredentialsValid(credentials, user.auth_data, external_authenticators, settings); } diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index b82537df639..797318438e1 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -5,10 +5,12 @@ #include #include #include +#include + +#include #include #include #include -#include namespace Poco { class Logger; } @@ -23,6 +25,14 @@ enum class AuthenticationType; class BackupEntriesCollector; class RestorerFromBackup; +/// Result of authentication +struct AuthResult +{ + UUID user_id; + /// Session settings received from authentication server (if any) + SettingsChanges settings; +}; + /// Contains entities, i.e. instances of classes derived from IAccessEntity. /// The implementations of this class MUST be thread-safe. class IAccessStorage : public boost::noncopyable @@ -171,8 +181,19 @@ public: /// Finds a user, check the provided credentials and returns the ID of the user if they are valid. /// Throws an exception if no such user or credentials are invalid. - UUID authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool allow_no_password, bool allow_plaintext_password) const; - std::optional authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const; + AuthResult authenticate( + const Credentials & credentials, + const Poco::Net::IPAddress & address, + const ExternalAuthenticators & external_authenticators, + bool allow_no_password, + bool allow_plaintext_password) const; + std::optional authenticate( + const Credentials & credentials, + const Poco::Net::IPAddress & address, + const ExternalAuthenticators & external_authenticators, + bool throw_if_user_not_exists, + bool allow_no_password, + bool allow_plaintext_password) const; /// Returns true if this storage can be stored to or restored from a backup. virtual bool isBackupAllowed() const { return false; } @@ -190,8 +211,18 @@ protected: virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); virtual bool removeImpl(const UUID & id, bool throw_if_not_exists); virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); - virtual std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const; - virtual bool areCredentialsValid(const User & user, const Credentials & credentials, const ExternalAuthenticators & external_authenticators) const; + virtual std::optional authenticateImpl( + const Credentials & credentials, + const Poco::Net::IPAddress & address, + const ExternalAuthenticators & external_authenticators, + bool throw_if_user_not_exists, + bool allow_no_password, + bool allow_plaintext_password) const; + virtual bool areCredentialsValid( + const User & user, + const Credentials & credentials, + const ExternalAuthenticators & external_authenticators, + SettingsChanges & settings) const; virtual bool isAddressAllowed(const User & user, const Poco::Net::IPAddress & address) const; static UUID generateRandomID(); Poco::Logger * getLogger() const; diff --git a/src/Access/LDAPAccessStorage.cpp b/src/Access/LDAPAccessStorage.cpp index f4e4fcd9c3a..9be28b763e0 100644 --- a/src/Access/LDAPAccessStorage.cpp +++ b/src/Access/LDAPAccessStorage.cpp @@ -450,7 +450,7 @@ std::optional> LDAPAccessStorage::readNameWi } -std::optional LDAPAccessStorage::authenticateImpl( +std::optional LDAPAccessStorage::authenticateImpl( const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, @@ -503,7 +503,9 @@ std::optional LDAPAccessStorage::authenticateImpl( updateAssignedRolesNoLock(*id, user->getName(), external_roles); } - return id; + if (id) + return AuthResult{ .user_id = *id }; + return std::nullopt; } } diff --git a/src/Access/LDAPAccessStorage.h b/src/Access/LDAPAccessStorage.h index 478a91a86d3..b01f58c724a 100644 --- a/src/Access/LDAPAccessStorage.h +++ b/src/Access/LDAPAccessStorage.h @@ -48,7 +48,7 @@ private: // IAccessStorage implementations. virtual std::vector findAllImpl(AccessEntityType type) const override; virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; virtual std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - virtual std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; + virtual std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; void setConfiguration(const Poco::Util::AbstractConfiguration & config, const String & prefix); void processRoleChange(const UUID & id, const AccessEntityPtr & entity); diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index 7cc8c20e47b..aa9329d488c 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -415,7 +415,7 @@ bool MultipleAccessStorage::updateImpl(const UUID & id, const UpdateFunc & updat } -std::optional +std::optional MultipleAccessStorage::authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, @@ -426,14 +426,14 @@ MultipleAccessStorage::authenticateImpl(const Credentials & credentials, const P { const auto & storage = (*storages)[i]; bool is_last_storage = (i == storages->size() - 1); - auto id = storage->authenticate(credentials, address, external_authenticators, + auto auth_result = storage->authenticate(credentials, address, external_authenticators, (throw_if_user_not_exists && is_last_storage), allow_no_password, allow_plaintext_password); - if (id) + if (auth_result) { std::lock_guard lock{mutex}; - ids_cache.set(*id, storage); - return id; + ids_cache.set(auth_result->user_id, storage); + return auth_result; } } diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 940606948a0..005e6e2b9cd 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -67,7 +67,7 @@ protected: bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; - std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; + std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; private: using Storages = std::vector; diff --git a/src/Access/SettingsAuthResponseParser.cpp b/src/Access/SettingsAuthResponseParser.cpp new file mode 100644 index 00000000000..62d15f1dcfc --- /dev/null +++ b/src/Access/SettingsAuthResponseParser.cpp @@ -0,0 +1,45 @@ +#include "SettingsAuthResponseParser.h" + +#include +#include + +#include +#include +#include + +namespace DB +{ + +SettingsAuthResponseParser::Result +SettingsAuthResponseParser::parse(const Poco::Net::HTTPResponse & response, std::istream * body_stream) const +{ + Result result; + + if (response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_OK) + return result; + result.is_ok = true; + + if (!body_stream) + return result; + + Poco::JSON::Parser parser; + Poco::JSON::Object::Ptr parsed_body; + + try + { + Poco::Dynamic::Var json = parser.parse(*body_stream); + Poco::JSON::Object::Ptr obj = json.extract(); + Poco::JSON::Object::Ptr settings_obj = obj->getObject(settings_key); + + if (settings_obj) + for (const auto & [key, value] : *settings_obj) + result.settings.emplace_back(key, settingStringToValueUtil(key, value)); + } + catch (...) + { + LOG_INFO(&Poco::Logger::get("HTTPAuthentication"), "Failed to parse settings from authentication response. Skip it."); + } + return result; +} + +} diff --git a/src/Access/SettingsAuthResponseParser.h b/src/Access/SettingsAuthResponseParser.h new file mode 100644 index 00000000000..686f10e5e20 --- /dev/null +++ b/src/Access/SettingsAuthResponseParser.h @@ -0,0 +1,28 @@ +#pragma once +#include + +#include + +namespace Poco::Net +{ +class HTTPResponse; +} + +namespace DB +{ +/// Class for parsing authentication response containing session settings +class SettingsAuthResponseParser +{ + static constexpr auto settings_key = "settings"; + +public: + struct Result + { + bool is_ok = false; + SettingsChanges settings; + }; + + Result parse(const Poco::Net::HTTPResponse & response, std::istream * body_stream) const; +}; + +} diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 72b4ead0c57..952a1064829 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -135,18 +135,22 @@ namespace const auto ssh_keys_config = user_config + ".ssh_keys"; bool has_ssh_keys = config.has(ssh_keys_config); - size_t num_password_fields = has_no_password + has_password_plaintext + has_password_sha256_hex + has_password_double_sha1_hex + has_ldap + has_kerberos + has_certificates + has_ssh_keys; + const auto http_auth_config = user_config + ".http_authentication"; + bool has_http_auth = config.has(http_auth_config); + + size_t num_password_fields = has_no_password + has_password_plaintext + has_password_sha256_hex + has_password_double_sha1_hex + + has_ldap + has_kerberos + has_certificates + has_ssh_keys + has_http_auth; if (num_password_fields > 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "More than one field of 'password', 'password_sha256_hex', " - "'password_double_sha1_hex', 'no_password', 'ldap', 'kerberos', 'ssl_certificates', 'ssh_keys' " - "are used to specify authentication info for user {}. " + "'password_double_sha1_hex', 'no_password', 'ldap', 'kerberos', 'ssl_certificates', 'ssh_keys', " + "'http_authentication' are used to specify authentication info for user {}. " "Must be only one of them.", user_name); if (num_password_fields < 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Either 'password' or 'password_sha256_hex' " "or 'password_double_sha1_hex' or 'no_password' or 'ldap' or 'kerberos " - "or 'ssl_certificates' or 'ssh_keys' must be specified for user {}.", user_name); + "or 'ssl_certificates' or 'ssh_keys' or 'http_authentication' must be specified for user {}.", user_name); if (has_password_plaintext) { @@ -248,6 +252,13 @@ namespace throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL"); #endif } + else if (has_http_auth) + { + user->auth_data = AuthenticationData{AuthenticationType::HTTP}; + user->auth_data.setHTTPAuthenticationServerName(config.getString(http_auth_config + ".server")); + auto scheme = config.getString(http_auth_config + ".scheme"); + user->auth_data.setHTTPAuthenticationScheme(parseHTTPAuthenticationScheme(scheme)); + } auto auth_type = user->auth_data.getType(); if (((auth_type == AuthenticationType::NO_PASSWORD) && !allow_no_password) || diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 7105e18ce18..9d68e72e977 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -351,7 +351,10 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So try { - user_id = global_context->getAccessControl().authenticate(credentials_, address.host()); + auto auth_result = global_context->getAccessControl().authenticate(credentials_, address.host()); + user_id = auth_result.user_id; + settings_from_auth_server = auth_result.settings; + LOG_DEBUG(log, "{} Authenticated with global context as user {}", toString(auth_id), toString(*user_id)); } @@ -523,6 +526,10 @@ ContextMutablePtr Session::makeSessionContext() {}, session_context->getSettingsRef().max_sessions_for_user); + // Use QUERY source as for SET query for a session + session_context->checkSettingsConstraints(settings_from_auth_server, SettingSource::QUERY); + session_context->applySettingsChanges(settings_from_auth_server); + recordLoginSucess(session_context); return session_context; diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 43e54474bbd..af8148698d3 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -119,6 +119,10 @@ private: SessionTracker::SessionTrackerHandle session_tracker_handle; + /// Settings received from authentication server during authentication process + /// to set when creating a session context + SettingsChanges settings_from_auth_server; + Poco::Logger * log = nullptr; }; diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index e058d9c0a0b..2fc5c5b982b 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -89,9 +89,10 @@ NamesAndTypesList SessionLogElement::getNamesAndTypes() AUTH_TYPE_NAME_AND_VALUE(AuthType::SSH_KEY), AUTH_TYPE_NAME_AND_VALUE(AuthType::SSL_CERTIFICATE), AUTH_TYPE_NAME_AND_VALUE(AuthType::BCRYPT_PASSWORD), + AUTH_TYPE_NAME_AND_VALUE(AuthType::HTTP), }); #undef AUTH_TYPE_NAME_AND_VALUE - static_assert(static_cast(AuthenticationType::MAX) == 9); + static_assert(static_cast(AuthenticationType::MAX) == 10); auto interface_type_column = std::make_shared( DataTypeEnum8::Values diff --git a/src/Parsers/Access/ASTAuthenticationData.cpp b/src/Parsers/Access/ASTAuthenticationData.cpp index db9a7b1fc75..3a62480dc0c 100644 --- a/src/Parsers/Access/ASTAuthenticationData.cpp +++ b/src/Parsers/Access/ASTAuthenticationData.cpp @@ -55,6 +55,7 @@ void ASTAuthenticationData::formatImpl(const FormatSettings & settings, FormatSt bool salt = false; bool parameter = false; bool parameters = false; + bool scheme = false; if (type) { @@ -124,6 +125,14 @@ void ASTAuthenticationData::formatImpl(const FormatSettings & settings, FormatSt parameters = true; break; } + case AuthenticationType::HTTP: + { + prefix = "SERVER"; + parameter = true; + if (children.size() == 2) + scheme = true; + break; + } case AuthenticationType::NO_PASSWORD: [[fallthrough]]; case AuthenticationType::MAX: throw Exception(ErrorCodes::LOGICAL_ERROR, "AST: Unexpected authentication type {}", toString(*type)); @@ -186,6 +195,13 @@ void ASTAuthenticationData::formatImpl(const FormatSettings & settings, FormatSt child->format(settings); } } + + if (scheme) + { + settings.ostr << " SCHEME "; + children[1]->format(settings); + } + } bool ASTAuthenticationData::hasSecretParts() const diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 8bcbc83d4b4..6fec40ec84a 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -80,6 +80,8 @@ namespace bool expect_kerberos_realm = false; bool expect_common_names = false; bool expect_public_ssh_key = false; + bool expect_http_auth_server = false; + if (ParserKeyword{"WITH"}.ignore(pos, expected)) { @@ -97,6 +99,8 @@ namespace expect_common_names = true; else if (check_type == AuthenticationType::SSH_KEY) expect_public_ssh_key = true; + else if (check_type == AuthenticationType::HTTP) + expect_http_auth_server = true; else if (check_type != AuthenticationType::NO_PASSWORD) expect_password = true; @@ -134,6 +138,7 @@ namespace ASTPtr parsed_salt; ASTPtr common_names; ASTPtr public_ssh_keys; + ASTPtr http_auth_scheme; if (expect_password || expect_hash) { @@ -178,6 +183,19 @@ namespace if (!ParserList{std::make_unique(), std::make_unique(TokenType::Comma), false}.parse(pos, common_names, expected)) return false; } + else if (expect_http_auth_server) + { + if (!ParserKeyword{"SERVER"}.ignore(pos, expected)) + return false; + if (!ParserStringAndSubstitution{}.parse(pos, value, expected)) + return false; + + if (ParserKeyword{"SCHEME"}.ignore(pos, expected)) + { + if (!ParserStringAndSubstitution{}.parse(pos, http_auth_scheme, expected)) + return false; + } + } auth_data = std::make_shared(); @@ -197,6 +215,9 @@ namespace if (public_ssh_keys) auth_data->children = std::move(public_ssh_keys->children); + if (http_auth_scheme) + auth_data->children.push_back(std::move(http_auth_scheme)); + return true; }); } diff --git a/tests/integration/test_external_http_authenticator/__init__.py b/tests/integration/test_external_http_authenticator/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_external_http_authenticator/configs/config.xml b/tests/integration/test_external_http_authenticator/configs/config.xml new file mode 100644 index 00000000000..ab73af91bbf --- /dev/null +++ b/tests/integration/test_external_http_authenticator/configs/config.xml @@ -0,0 +1,14 @@ + + + + http://localhost:8000/basic_auth + 1000 + 1000 + 1000 + 3 + 50 + 1000 + + + auth_ + diff --git a/tests/integration/test_external_http_authenticator/configs/users.xml b/tests/integration/test_external_http_authenticator/configs/users.xml new file mode 100644 index 00000000000..54f418eec19 --- /dev/null +++ b/tests/integration/test_external_http_authenticator/configs/users.xml @@ -0,0 +1,64 @@ + + + + + + + + + basic_server + basic + + 1 + + ::/0 + + default + default + + + + basic_server + basic + + + ::/0 + + default + default + + + + basic_server + basic + + + ::/0 + + default + default + + + + basic_server + basic + + + ::/0 + + default + default + + + + basic_server + basic + + + ::/0 + + default + default + + + diff --git a/tests/integration/test_external_http_authenticator/http_auth_server.py b/tests/integration/test_external_http_authenticator/http_auth_server.py new file mode 100644 index 00000000000..6437468c417 --- /dev/null +++ b/tests/integration/test_external_http_authenticator/http_auth_server.py @@ -0,0 +1,82 @@ +import base64 +import http.server +import json + + +GOOD_PASSWORD = "good_password" +USER_RESPONSES = { + "test_user_1": {"settings": {"auth_user": "'test_user'", "auth_num": "UInt64_15"}}, + "test_user_2": {}, + "test_user_3": "", + "test_user_4": "not json string", +} + + +class RequestHandler(http.server.BaseHTTPRequestHandler): + def decode_basic(self, data): + decoded_data = base64.b64decode(data).decode("utf-8") + return decoded_data.split(":", 1) + + def do_AUTHHEAD(self): + self.send_response(http.HTTPStatus.UNAUTHORIZED) + self.send_header("WWW-Authenticate", 'Basic realm="Test"') + self.send_header("Content-type", "text/html") + self.end_headers() + + def do_ACCESS_GRANTED(self, user: str) -> None: + self.send_response(http.HTTPStatus.OK) + body = "" + + response = USER_RESPONSES.get(user) + + if isinstance(response, dict): + body = json.dumps(response) + else: + body = response or "" + + body_raw = body.encode("utf-8") + self.send_header("Content-Type", "application/json") + self.send_header("Content-Length", len(body_raw)) + self.end_headers() + self.wfile.write(body_raw) + + def do_GET(self): + if self.path == "/health": + self.send_response(http.HTTPStatus.OK) + self.send_header("Content-Type", "text/plain") + self.end_headers() + self.wfile.write(b"OK") + + elif self.path == "/basic_auth": + auth_header = self.headers.get("Authorization") + + if auth_header is None: + self.do_AUTHHEAD() + return + + auth_scheme, data = auth_header.split(" ", 1) + + if auth_scheme != "Basic": + print(auth_scheme) + self.do_AUTHHEAD() + return + + user_name, password = self.decode_basic(data) + if password == GOOD_PASSWORD: + self.do_ACCESS_GRANTED(user_name) + else: + self.do_AUTHHEAD() + + +if __name__ == "__main__": + httpd = http.server.HTTPServer( + ( + "0.0.0.0", + 8000, + ), + RequestHandler, + ) + try: + httpd.serve_forever() + finally: + httpd.server_close() diff --git a/tests/integration/test_external_http_authenticator/test.py b/tests/integration/test_external_http_authenticator/test.py new file mode 100644 index 00000000000..ca5225142f8 --- /dev/null +++ b/tests/integration/test_external_http_authenticator/test.py @@ -0,0 +1,120 @@ +import json +import logging +import pytest +import os +import sys + + +from .http_auth_server import GOOD_PASSWORD, USER_RESPONSES + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "node", + main_configs=["configs/config.xml"], + user_configs=["configs/users.xml"], +) +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +def run_echo_server(): + tmp = instance.copy_file_to_container( + os.path.join(SCRIPT_DIR, "http_auth_server.py"), + "/http_auth_server.py", + ) + + tmp = instance.exec_in_container( + [ + "bash", + "-c", + "python3 /http_auth_server.py > /http_auth_server.log 2>&1", + ], + detach=True, + user="root", + ) + + for _ in range(0, 10): + ping_response = instance.exec_in_container( + ["curl", "-s", f"http://localhost:8000/health"], + nothrow=True, + ) + logging.debug(f"Reply1: {ping_response}") + if ping_response == "OK": + return + + raise Exception("Echo server is not responding") + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + run_echo_server() + yield cluster + finally: + cluster.shutdown() + + +def test_user_from_config_basic_auth_pass(started_cluster): + assert ( + instance.query("SHOW CREATE USER good_user") + == "CREATE USER good_user IDENTIFIED WITH http SERVER \\'basic_server\\' SCHEME \\'BASIC\\' SETTINGS PROFILE default\n" + ) + assert ( + instance.query( + "SELECT currentUser()", user="good_user", password="good_password" + ) + == "good_user\n" + ) + + +def test_user_create_basic_auth_pass(started_cluster): + instance.query( + "CREATE USER basic_user IDENTIFIED WITH HTTP SERVER 'basic_server' SCHEME 'BASIC'" + ) + + assert ( + instance.query("SHOW CREATE USER basic_user") + == "CREATE USER basic_user IDENTIFIED WITH http SERVER \\'basic_server\\' SCHEME \\'BASIC\\'\n" + ) + assert ( + instance.query( + "SELECT currentUser()", user="basic_user", password=GOOD_PASSWORD + ) + == "basic_user\n" + ) + + instance.query("DROP USER basic_user") + + +def test_basic_auth_failed(started_cluster): + assert "good_user: Authentication failed" in instance.query_and_get_error( + "SELECT currentUser()", user="good_user", password="bad_password" + ) + + +def test_session_settings_from_auth_response(started_cluster): + for user, response in USER_RESPONSES.items(): + query_id = f"test_query_{user}" + assert ( + instance.query( + "SELECT currentUser()", + user=user, + password="good_password", + query_id=query_id, + ) + == f"{user}\n" + ) + instance.query("SYSTEM FLUSH LOGS") + + res = instance.query( + f"select Settings from system.query_log where type = 'QueryFinish' and query_id = '{query_id}' FORMAT JSON" + ) + + res = json.loads(res) + query_settings = res["data"][0]["Settings"] + + if isinstance(response, dict): + for key, value in response.get("settings", {}).items(): + assert query_settings.get(key) == value diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index b5b31e44594..9ed905a0df8 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -1121,7 +1121,7 @@ CREATE TABLE system.users `name` String, `id` UUID, `storage` String, - `auth_type` Enum8('no_password' = 0, 'plaintext_password' = 1, 'sha256_password' = 2, 'double_sha1_password' = 3, 'ldap' = 4, 'kerberos' = 5, 'ssl_certificate' = 6, 'bcrypt_password' = 7, 'ssh_key' = 8), + `auth_type` Enum8('no_password' = 0, 'plaintext_password' = 1, 'sha256_password' = 2, 'double_sha1_password' = 3, 'ldap' = 4, 'kerberos' = 5, 'ssl_certificate' = 6, 'bcrypt_password' = 7, 'ssh_key' = 8, 'http' = 9), `auth_params` String, `host_ip` Array(String), `host_names` Array(String), diff --git a/tests/queries/0_stateless/02713_create_user_substitutions.reference b/tests/queries/0_stateless/02713_create_user_substitutions.reference index f9b5cc495b5..e06e4815841 100644 --- a/tests/queries/0_stateless/02713_create_user_substitutions.reference +++ b/tests/queries/0_stateless/02713_create_user_substitutions.reference @@ -9,3 +9,5 @@ CREATE USER user9_02713 IDENTIFIED WITH ldap SERVER \'qwerty9\' CREATE USER user10_02713 IDENTIFIED WITH kerberos REALM \'qwerty10\' CREATE USER user11_02713 IDENTIFIED WITH ssl_certificate CN \'qwerty11\', \'qwerty12\' +CREATE USER user12_02713 IDENTIFIED WITH http SERVER \'basic_server\' SCHEME \'BASIC\' +CREATE USER user13_02713 IDENTIFIED WITH http SERVER \'basic_server\' SCHEME \'BASIC\' diff --git a/tests/queries/0_stateless/02713_create_user_substitutions.sh b/tests/queries/0_stateless/02713_create_user_substitutions.sh index 42926335acb..1e2cfd1620a 100755 --- a/tests/queries/0_stateless/02713_create_user_substitutions.sh +++ b/tests/queries/0_stateless/02713_create_user_substitutions.sh @@ -26,6 +26,8 @@ $CLICKHOUSE_CLIENT --param_hash=\$2a\$12\$wuohz0HFSBBNE8huN0Yx6.kmWrefiYVKeMp4gs $CLICKHOUSE_CLIENT --param_server=qwerty9 -q "CREATE USER user9_02713 IDENTIFIED WITH LDAP SERVER {server:String}"; $CLICKHOUSE_CLIENT --param_realm=qwerty10 -q "CREATE USER user10_02713 IDENTIFIED WITH KERBEROS REALM {realm:String}"; $CLICKHOUSE_CLIENT --param_cert1=qwerty11 --param_cert2=qwerty12 -q "CREATE USER user11_02713 IDENTIFIED WITH SSL_CERTIFICATE CN {cert1:String}, {cert2:String}"; +$CLICKHOUSE_CLIENT --param_server=basic_server -q "CREATE USER user12_02713 IDENTIFIED WITH http SERVER {server:String}" +$CLICKHOUSE_CLIENT --param_server=basic_server -q "CREATE USER user13_02713 IDENTIFIED WITH http SERVER {server:String} SCHEME 'Basic'" $CLICKHOUSE_CLIENT --user=user1_02713 --password=qwerty1 -q "SELECT 1"; $CLICKHOUSE_CLIENT --user=user2_02713 --password=qwerty2 -q "SELECT 2"; @@ -39,5 +41,7 @@ $CLICKHOUSE_CLIENT --user=user8_02713 --password=qwerty8 -q "SELECT 8"; $CLICKHOUSE_CLIENT -q "SHOW CREATE USER user9_02713"; $CLICKHOUSE_CLIENT -q "SHOW CREATE USER user10_02713"; $CLICKHOUSE_CLIENT -q "SHOW CREATE USER user11_02713"; +$CLICKHOUSE_CLIENT -q "SHOW CREATE USER user12_02713"; +$CLICKHOUSE_CLIENT -q "SHOW CREATE USER user13_02713"; -$CLICKHOUSE_CLIENT -q "DROP USER user1_02713, user2_02713, user3_02713, user4_02713, user5_02713, user6_02713, user7_02713, user8_02713, user9_02713, user10_02713, user11_02713"; +$CLICKHOUSE_CLIENT -q "DROP USER user1_02713, user2_02713, user3_02713, user4_02713, user5_02713, user6_02713, user7_02713, user8_02713, user9_02713, user10_02713, user11_02713, user12_02713, user13_02713"; From 414b5ed8639f7830c4af03be965feaff420d36e1 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 20 Oct 2023 17:31:33 +0000 Subject: [PATCH 458/634] fix test --- .../0_stateless/02896_union_distinct_http_format.reference | 2 +- tests/queries/0_stateless/02896_union_distinct_http_format.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02896_union_distinct_http_format.reference b/tests/queries/0_stateless/02896_union_distinct_http_format.reference index 3ca2cb39dc1..15c34d719e3 100644 --- a/tests/queries/0_stateless/02896_union_distinct_http_format.reference +++ b/tests/queries/0_stateless/02896_union_distinct_http_format.reference @@ -1,4 +1,4 @@ -┌─1─┐ +┌─a─┐ │ 1 │ │ 2 │ └───┘ diff --git a/tests/queries/0_stateless/02896_union_distinct_http_format.sh b/tests/queries/0_stateless/02896_union_distinct_http_format.sh index 8a79e00ac30..1501801325d 100755 --- a/tests/queries/0_stateless/02896_union_distinct_http_format.sh +++ b/tests/queries/0_stateless/02896_union_distinct_http_format.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo 'SELECT 1 UNION DISTINCT SELECT 2 FORMAT PrettyCompactMonoBlock' | curl 'http://localhost:8123/' --data-binary @- +curl -d@- -sS "${CLICKHOUSE_URL}" <<< 'SELECT 1 as a UNION DISTINCT SELECT 2 as a order by a FORMAT PrettyCompactMonoBlock' From c7e147896ee8bd96c9a6c0a667e45446603496d3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 20 Oct 2023 20:02:20 +0200 Subject: [PATCH 459/634] (temporarily) use different paths for replicas --- tests/config/install.sh | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/config/install.sh b/tests/config/install.sh index 9e3b235515d..a5037bfb64e 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -169,11 +169,17 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] chgrp clickhouse /etc/clickhouse-server2 sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server1 sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 + rm /etc/clickhouse-server1/config.d/macros.xml rm /etc/clickhouse-server2/config.d/macros.xml sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server1/config.d/macros.xml sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|s1|s2|" > /etc/clickhouse-server2/config.d/macros.xml + rm /etc/clickhouse-server1/config.d/transactions.xml + rm /etc/clickhouse-server2/config.d/transactions.xml + sudo -u clickhouse cat /etc/clickhouse-server/config.d/transactions.xml | sed "s|/test/clickhouse/txn|/test/clickhouse/txn1|" > /etc/clickhouse-server1/config.d/transactions.xml + sudo -u clickhouse cat /etc/clickhouse-server/config.d/transactions.xml | sed "s|/test/clickhouse/txn|/test/clickhouse/txn2|" > /etc/clickhouse-server2/config.d/transactions.xml + sudo mkdir -p /var/lib/clickhouse1 sudo mkdir -p /var/lib/clickhouse2 sudo chown clickhouse /var/lib/clickhouse1 From 8a0eae21f5487cd56dfe845afe7c3857757812c3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 20 Oct 2023 20:45:16 +0200 Subject: [PATCH 460/634] Update stress.py --- tests/ci/stress.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 8cc40839706..48ca92b339b 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -133,7 +133,7 @@ def prepare_for_hung_check(drop_databases: bool) -> bool: logging.info("Will terminate gdb (if any)") call_with_retry("kill -TERM $(pidof gdb)") # Sometimes there is a message `Child process was stopped by signal 19` in logs after stopping gdb - call_with_retry("kill -CONT $(lsof -ti:9000)") + call_with_retry("kill -CONT $(cat /var/run/clickhouse-server/clickhouse-server.pid)") # ThreadFuzzer significantly slows down server and causes false-positive hung check failures call_with_retry("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'") From 8cc0dc17eb1010973f48e93a9aacf2a9b5206647 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 20 Oct 2023 18:50:48 +0000 Subject: [PATCH 461/634] Try to fix possible segfault in Native ORC input format --- .../Impl/NativeORCBlockInputFormat.cpp | 29 ++++++++++++------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 1b6cde11be7..11b42e08c9c 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -496,16 +496,21 @@ readColumnWithStringData(const orc::ColumnVectorBatch * orc_column, const orc::T const auto * orc_str_column = dynamic_cast(orc_column); size_t reserver_size = 0; for (size_t i = 0; i < orc_str_column->numElements; ++i) - reserver_size += orc_str_column->length[i] + 1; + { + if (!orc_str_column->hasNulls || orc_str_column->notNull[i]) + reserver_size += orc_str_column->length[i]; + reserver_size += 1; + } + column_chars_t.reserve(reserver_size); column_offsets.reserve(orc_str_column->numElements); size_t curr_offset = 0; for (size_t i = 0; i < orc_str_column->numElements; ++i) { - const auto * buf = orc_str_column->data[i]; - if (buf) + if (!orc_str_column->hasNulls || orc_str_column->notNull[i]) { + const auto * buf = orc_str_column->data[i]; size_t buf_size = orc_str_column->length[i]; column_chars_t.insert_assume_reserved(buf, buf + buf_size); curr_offset += buf_size; @@ -531,7 +536,7 @@ readColumnWithFixedStringData(const orc::ColumnVectorBatch * orc_column, const o const auto * orc_str_column = dynamic_cast(orc_column); for (size_t i = 0; i < orc_str_column->numElements; ++i) { - if (orc_str_column->data[i]) + if (!orc_str_column->hasNulls || orc_str_column->notNull[i]) column_chars_t.insert_assume_reserved(orc_str_column->data[i], orc_str_column->data[i] + orc_str_column->length[i]); else column_chars_t.resize_fill(column_chars_t.size() + fixed_len); @@ -580,7 +585,7 @@ readIPv6ColumnFromBinaryData(const orc::ColumnVectorBatch * orc_column, const or for (size_t i = 0; i < orc_str_column->numElements; ++i) { /// If at least one value size is not 16 bytes, fallback to reading String column and further cast to IPv6. - if (orc_str_column->data[i] && orc_str_column->length[i] != sizeof(IPv6)) + if ((!orc_str_column->hasNulls || orc_str_column->notNull[i]) && orc_str_column->length[i] != sizeof(IPv6)) return readColumnWithStringData(orc_column, orc_type, column_name); } @@ -591,10 +596,10 @@ readIPv6ColumnFromBinaryData(const orc::ColumnVectorBatch * orc_column, const or for (size_t i = 0; i < orc_str_column->numElements; ++i) { - if (!orc_str_column->data[i]) [[unlikely]] - ipv6_column.insertDefault(); - else + if (!orc_str_column->hasNulls || orc_str_column->notNull[i]) ipv6_column.insertData(orc_str_column->data[i], orc_str_column->length[i]); + else + ipv6_column.insertDefault(); } return {std::move(internal_column), std::move(internal_type), column_name}; @@ -628,9 +633,7 @@ static ColumnWithTypeAndName readColumnWithBigNumberFromBinaryData( for (size_t i = 0; i < orc_str_column->numElements; ++i) { - if (!orc_str_column->data[i]) [[unlikely]] - integer_column.insertDefault(); - else + if (!orc_str_column->hasNulls || orc_str_column->notNull[i]) { if (sizeof(typename ColumnType::ValueType) != orc_str_column->length[i]) throw Exception( @@ -642,6 +645,10 @@ static ColumnWithTypeAndName readColumnWithBigNumberFromBinaryData( integer_column.insertData(orc_str_column->data[i], orc_str_column->length[i]); } + else + { + integer_column.insertDefault(); + } } return {std::move(internal_column), column_type, column_name}; } From f377f4f5948a1857f87254d1c9b2c7dd51a78882 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 20 Oct 2023 19:06:01 +0000 Subject: [PATCH 462/634] Automatic style fix --- tests/ci/stress.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 48ca92b339b..9db9f25cbca 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -133,7 +133,9 @@ def prepare_for_hung_check(drop_databases: bool) -> bool: logging.info("Will terminate gdb (if any)") call_with_retry("kill -TERM $(pidof gdb)") # Sometimes there is a message `Child process was stopped by signal 19` in logs after stopping gdb - call_with_retry("kill -CONT $(cat /var/run/clickhouse-server/clickhouse-server.pid)") + call_with_retry( + "kill -CONT $(cat /var/run/clickhouse-server/clickhouse-server.pid)" + ) # ThreadFuzzer significantly slows down server and causes false-positive hung check failures call_with_retry("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'") From 8c7973d6559c08c878c4b98f91183689ad606b75 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 20 Oct 2023 21:10:11 +0200 Subject: [PATCH 463/634] trigger CI From e5bfc7d3fc6ff04ae6d0e283cd80fc8abee4a914 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 14 Oct 2023 12:17:35 +0200 Subject: [PATCH 464/634] Improved DateTime64 parsing Allow to parse unambigous short DateTime64 Throw error on invalid DT64 representation (as for DateTime) --- src/IO/ReadHelpers.cpp | 45 ++++++++++++--- src/IO/ReadHelpers.h | 56 ++++++++++++++----- .../0_stateless/01268_DateTime64_in_WHERE.sql | 2 +- ..._toDateTime_from_string_clamping.reference | 2 +- .../02373_datetime64_monotonicity.queries | 8 +-- .../02373_datetime64_monotonicity.reference | 16 +++--- .../02889_datetime64_from_string.reference | 3 + .../02889_datetime64_from_string.sql | 5 ++ 8 files changed, 99 insertions(+), 38 deletions(-) create mode 100644 tests/queries/0_stateless/02889_datetime64_from_string.reference create mode 100644 tests/queries/0_stateless/02889_datetime64_from_string.sql diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index fa2400a08a5..f3ea4b29a1b 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1139,7 +1139,7 @@ template void readDateTextFallback(LocalDate &, ReadBuffer &); template bool readDateTextFallback(LocalDate &, ReadBuffer &); -template +template ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut) { static constexpr bool throw_exception = std::is_same_v; @@ -1155,11 +1155,30 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D char * s_pos = s; /** Read characters, that could represent unix timestamp. - * Only unix timestamp of at least 5 characters is supported. + * Only unix timestamp of at least 5 characters is supported by default, exception is thrown for a shorter one + * unless `treat_ambiguous_as_timestamp` == true: e.g when parsing a string like '123.23' there is no ambiguity. * Then look at 5th character. If it is a number - treat whole as unix timestamp. * If it is not a number - then parse datetime in YYYY-MM-DD hh:mm:ss or YYYY-MM-DD format. */ + int negative_multiplier = 1; + + if (!buf.eof() && *buf.position() == '-') + { + if constexpr (dt64_mode) + { + negative_multiplier = -1; + ++buf.position(); + } + else + { + if constexpr (throw_exception) + throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime"); + else + return false; + } + } + /// A piece similar to unix timestamp, maybe scaled to subsecond precision. while (s_pos < s + date_time_broken_down_length && !buf.eof() && isNumericASCII(*buf.position())) { @@ -1169,7 +1188,8 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D } /// 2015-01-01 01:02:03 or 2015-01-01 - if (s_pos == s + 4 && !buf.eof() && !isNumericASCII(*buf.position())) + /// if negative, it is a timestamp with no ambiguity + if (negative_multiplier == 1 && s_pos == s + 4 && !buf.eof() && !isNumericASCII(*buf.position())) { const auto already_read_length = s_pos - s; const size_t remaining_date_size = date_broken_down_length - already_read_length; @@ -1220,27 +1240,34 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D } else { - if (s_pos - s >= 5) + datetime = 0; + bool too_short = s_pos - s <= 4; + + if (!too_short || dt64_mode) { /// Not very efficient. - datetime = 0; for (const char * digit_pos = s; digit_pos < s_pos; ++digit_pos) datetime = datetime * 10 + *digit_pos - '0'; } - else + datetime *= negative_multiplier; + + if (too_short && negative_multiplier != -1) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse datetime"); + throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime"); else return false; } + } return ReturnType(true); } -template void readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &); -template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &); +template void readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &); +template void readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &); +template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &); +template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &); void skipJSONField(ReadBuffer & buf, StringRef name_of_field) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 415f4e100d2..fcd8550462f 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -909,15 +909,28 @@ inline T parseFromString(std::string_view str) } -template +template ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut); /** In YYYY-MM-DD hh:mm:ss or YYYY-MM-DD format, according to specified time zone. * As an exception, also supported parsing of unix timestamp in form of decimal number. */ -template +template inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut) { + static constexpr bool throw_exception = std::is_same_v; + + if constexpr (!dt64_mode) + { + if (!isNumericASCII(*buf.position())) + { + if constexpr (throw_exception) + throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse datetime"); + else + return false; + } + } + /// Optimistic path, when whole value is in buffer. const char * s = buf.position(); @@ -965,19 +978,32 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons return readIntTextImpl(datetime, buf); } else - return readDateTimeTextFallback(datetime, buf, date_lut); + return readDateTimeTextFallback(datetime, buf, date_lut); } template inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut) { - time_t whole; - if (!readDateTimeTextImpl(whole, buf, date_lut)) + time_t whole = 0; + bool is_negative_timestamp = (!buf.eof() && *buf.position() == '-'); + bool is_empty = buf.eof(); + + if (!is_empty) { - return ReturnType(false); + try + { + readDateTimeTextImpl(whole, buf, date_lut); + } + catch (const DB::ParsingException & exception) + { + if (is_empty) + datetime64 = 0; + else if (buf.eof() || *buf.position() != '.') + throw exception; + } } - int negative_multiplier = 1; + int negative_fraction_multiplier = 1; DB::DecimalUtils::DecimalComponents components{static_cast(whole), 0}; @@ -1005,18 +1031,18 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re while (!buf.eof() && isNumericASCII(*buf.position())) ++buf.position(); - /// Fractional part (subseconds) is treated as positive by users - /// (as DateTime64 itself is a positive, although underlying decimal is negative) - /// setting fractional part to be negative when whole is 0 results in wrong value, - /// so we multiply result by -1. - if (components.whole < 0 && components.fractional != 0) + /// Fractional part (subseconds) is treated as positive by users, but represented as a negative number. + /// E.g. `1925-12-12 13:14:15.123` is represented internally as timestamp `-1390214744.877`. + /// Thus need to convert . to .<1-0.> + /// Also, setting fractional part to be negative when whole is 0 results in wrong value, in this case multiply result by -1. + if (!is_negative_timestamp && components.whole < 0 && components.fractional != 0) { const auto scale_multiplier = DecimalUtils::scaleMultiplier(scale); ++components.whole; components.fractional = scale_multiplier - components.fractional; if (!components.whole) { - negative_multiplier = -1; + negative_fraction_multiplier = -1; } } } @@ -1032,13 +1058,13 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re bool is_ok = true; if constexpr (std::is_same_v) { - datetime64 = DecimalUtils::decimalFromComponents(components, scale) * negative_multiplier; + datetime64 = DecimalUtils::decimalFromComponents(components, scale) * negative_fraction_multiplier; } else { is_ok = DecimalUtils::tryGetDecimalFromComponents(components, scale, datetime64); if (is_ok) - datetime64 *= negative_multiplier; + datetime64 *= negative_fraction_multiplier; } return ReturnType(is_ok); diff --git a/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql b/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql index 63d9a11daaa..3e859717873 100644 --- a/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql +++ b/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql @@ -5,7 +5,7 @@ WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT materiali WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE DT64 = materialize(S); -- {serverError 43} WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE materialize(S) = DT64; -- {serverError 43} -SELECT * WHERE toDateTime64(123.345, 3) == 'ABCD'; -- {serverError 53} -- invalid DateTime64 string +SELECT * WHERE toDateTime64(123.345, 3) == 'ABCD'; -- {serverError 41} -- invalid DateTime64 string SELECT * WHERE toDateTime64(123.345, 3) == '2020-02-05 14:34:12.33333333333333333333333333333333333333333333333333333333'; SELECT 'in SELECT'; diff --git a/tests/queries/0_stateless/01702_toDateTime_from_string_clamping.reference b/tests/queries/0_stateless/01702_toDateTime_from_string_clamping.reference index ecea0a9f69f..c07ef5ca631 100644 --- a/tests/queries/0_stateless/01702_toDateTime_from_string_clamping.reference +++ b/tests/queries/0_stateless/01702_toDateTime_from_string_clamping.reference @@ -1,4 +1,4 @@ -1940-10-09 21:13:17.6 +1940-10-09 21:13:16.4 2284-06-04 23:46:43.6 2299-12-31 23:40:00.1 1900-01-01 00:00:00.9 diff --git a/tests/queries/0_stateless/02373_datetime64_monotonicity.queries b/tests/queries/0_stateless/02373_datetime64_monotonicity.queries index 212198c89de..404e3391205 100644 --- a/tests/queries/0_stateless/02373_datetime64_monotonicity.queries +++ b/tests/queries/0_stateless/02373_datetime64_monotonicity.queries @@ -46,11 +46,11 @@ SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= '2020-01- SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= '2020-01-01 00:00:01.1' settings force_index_by_date = 1, force_primary_key = 1; create table dt64_monot_test_string(date_time String, x String) Engine=MergeTree order by date_time; -insert into dt64_monot_test_string select '2020-01-01 00:00', '' from numbers(1); -insert into dt64_monot_test_string select '2020-01-01 00:00:00.000000' , '' from numbers(10); +insert into dt64_monot_test_string select '2020-01-01 00:00:00.000000001', '' from numbers(1); +insert into dt64_monot_test_string select '2020-01-01 00:00:00.000', '' from numbers(10); -SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,3) = '1970-01-01 00:00:00.000000000'; -SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,3) = '1970-01-01 00:00:00.000000001'; +SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00.000000000'; +SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,3) = '2020-01-01 00:00:00.000000001'; SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00'; drop table dt64_monot_test; diff --git a/tests/queries/0_stateless/02373_datetime64_monotonicity.reference b/tests/queries/0_stateless/02373_datetime64_monotonicity.reference index d9c310bdbc9..935ee685cc9 100644 --- a/tests/queries/0_stateless/02373_datetime64_monotonicity.reference +++ b/tests/queries/0_stateless/02373_datetime64_monotonicity.reference @@ -17,8 +17,8 @@ Asia/Tehran 10 0 0 -0 -0 +10 +11 10 UTC @@ -40,8 +40,8 @@ UTC 10 10 10 -1 -1 +10 +11 10 Canada/Atlantic @@ -63,8 +63,8 @@ Canada/Atlantic 10 10 10 -0 -0 +10 +11 10 Europe/Berlin @@ -86,7 +86,7 @@ Europe/Berlin 10 10 9 -0 -0 +10 +11 10 diff --git a/tests/queries/0_stateless/02889_datetime64_from_string.reference b/tests/queries/0_stateless/02889_datetime64_from_string.reference new file mode 100644 index 00000000000..e6e2208ed4c --- /dev/null +++ b/tests/queries/0_stateless/02889_datetime64_from_string.reference @@ -0,0 +1,3 @@ +1969-12-31 23:57:57.000 +1970-01-01 00:00:23.900 +1969-12-31 23:59:36.100 diff --git a/tests/queries/0_stateless/02889_datetime64_from_string.sql b/tests/queries/0_stateless/02889_datetime64_from_string.sql new file mode 100644 index 00000000000..50c29de19bd --- /dev/null +++ b/tests/queries/0_stateless/02889_datetime64_from_string.sql @@ -0,0 +1,5 @@ +SELECT toDateTime64('-123', 3, 'UTC'); -- Allowed: no year starts with '-' +SELECT toDateTime64('23.9', 3, 'UTC'); -- Allowed: no year has a dot in notation +SELECT toDateTime64('-23.9', 3, 'UTC'); -- Allowed + +SELECT toDateTime64('1234', 3, 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } From 7d54c1fb001f2dab29ce11d090a4bb7d5fcfe85d Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 20 Oct 2023 22:25:58 +0200 Subject: [PATCH 465/634] check EOF before reading buffer --- src/IO/ReadHelpers.cpp | 2 +- src/IO/ReadHelpers.h | 6 ++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index f3ea4b29a1b..ad41e8afd65 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1156,7 +1156,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D /** Read characters, that could represent unix timestamp. * Only unix timestamp of at least 5 characters is supported by default, exception is thrown for a shorter one - * unless `treat_ambiguous_as_timestamp` == true: e.g when parsing a string like '123.23' there is no ambiguity. + * (unless parsing a string like '1.23' or '-12': there is no ambiguity, it is a DT64 timestamp). * Then look at 5th character. If it is a number - treat whole as unix timestamp. * If it is not a number - then parse datetime in YYYY-MM-DD hh:mm:ss or YYYY-MM-DD format. */ diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index fcd8550462f..40f812050db 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -922,7 +922,7 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons if constexpr (!dt64_mode) { - if (!isNumericASCII(*buf.position())) + if (!buf.eof() && !isNumericASCII(*buf.position())) { if constexpr (throw_exception) throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse datetime"); @@ -996,9 +996,7 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re } catch (const DB::ParsingException & exception) { - if (is_empty) - datetime64 = 0; - else if (buf.eof() || *buf.position() != '.') + if (buf.eof() || *buf.position() != '.') throw exception; } } From 7ec4b99e944e89d86ed870d22c8d698743d9ed37 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 21 Oct 2023 03:14:22 +0200 Subject: [PATCH 466/634] Revert partial result --- docs/en/operations/settings/settings.md | 12 - src/Client/ClientBase.cpp | 36 +- src/Client/ClientBase.h | 15 - src/Core/Settings.h | 4 - src/Interpreters/Aggregator.cpp | 23 -- src/Interpreters/Aggregator.h | 2 - src/Processors/Chunk.cpp | 3 +- .../Executors/CompletedPipelineExecutor.cpp | 4 +- src/Processors/Executors/ExecutingGraph.cpp | 1 + .../Executors/ExecutionThreadContext.h | 13 +- src/Processors/Executors/ExecutorTasks.cpp | 15 +- src/Processors/Executors/ExecutorTasks.h | 2 +- src/Processors/Executors/PipelineExecutor.cpp | 5 +- src/Processors/Executors/PipelineExecutor.h | 5 +- .../PullingAsyncPipelineExecutor.cpp | 7 +- .../Executors/PullingAsyncPipelineExecutor.h | 2 +- .../Executors/PullingPipelineExecutor.cpp | 2 +- .../PushingAsyncPipelineExecutor.cpp | 2 +- .../Executors/PushingPipelineExecutor.cpp | 2 +- src/Processors/Formats/IOutputFormat.cpp | 110 ++---- src/Processors/Formats/IOutputFormat.h | 13 +- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 34 +- .../Formats/Impl/PrettyBlockOutputFormat.h | 6 - .../Impl/PrettyCompactBlockOutputFormat.cpp | 3 +- src/Processors/Formats/LazyOutputFormat.h | 5 +- src/Processors/IProcessor.cpp | 5 - src/Processors/IProcessor.h | 23 -- src/Processors/LimitTransform.cpp | 9 +- src/Processors/LimitTransform.h | 10 - .../QueryPlan/BuildQueryPipelineSettings.cpp | 11 +- .../QueryPlan/BuildQueryPipelineSettings.h | 3 - src/Processors/QueryPlan/QueryPlan.cpp | 5 - src/Processors/QueryPlan/SortingStep.h | 2 - .../AggregatingPartialResultTransform.cpp | 47 --- .../AggregatingPartialResultTransform.h | 29 -- .../Transforms/AggregatingTransform.cpp | 16 +- .../Transforms/AggregatingTransform.h | 21 -- .../Transforms/ExpressionTransform.cpp | 8 - .../Transforms/ExpressionTransform.h | 5 - .../LimitPartialResultTransform.cpp | 42 --- .../Transforms/LimitPartialResultTransform.h | 36 -- .../Transforms/LimitsCheckingTransform.cpp | 1 - .../Transforms/LimitsCheckingTransform.h | 2 - .../MergeSortingPartialResultTransform.cpp | 54 --- .../MergeSortingPartialResultTransform.h | 28 -- .../Transforms/MergeSortingTransform.cpp | 12 - .../Transforms/MergeSortingTransform.h | 11 - .../Transforms/PartialResultTransform.cpp | 80 ----- .../Transforms/PartialResultTransform.h | 57 ---- src/QueryPipeline/Pipe.cpp | 214 +++--------- src/QueryPipeline/Pipe.h | 25 +- src/QueryPipeline/QueryPipeline.cpp | 52 +-- src/QueryPipeline/QueryPipeline.h | 7 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 20 +- src/QueryPipeline/QueryPipelineBuilder.h | 9 - src/Server/TCPHandler.cpp | 9 +- .../0_stateless/02010_lc_native.python | 291 +++++++++++++--- .../02210_processors_profile_log.reference | 1 - .../02458_insert_select_progress_tcp.python | 244 ++++++++++++-- .../02750_settings_alias_tcp_protocol.python | 212 +++++++++++- ...rting_result_during_query_execution.python | 95 ------ ...ng_result_during_query_execution.reference | 38 --- ...l_sorting_result_during_query_execution.sh | 8 - ...ating_result_during_query_execution.python | 129 -------- ...ng_result_during_query_execution.reference | 88 ----- ...gregating_result_during_query_execution.sh | 8 - .../queries/0_stateless/helpers/tcp_client.py | 313 ------------------ 67 files changed, 775 insertions(+), 1831 deletions(-) delete mode 100644 src/Processors/Transforms/AggregatingPartialResultTransform.cpp delete mode 100644 src/Processors/Transforms/AggregatingPartialResultTransform.h delete mode 100644 src/Processors/Transforms/LimitPartialResultTransform.cpp delete mode 100644 src/Processors/Transforms/LimitPartialResultTransform.h delete mode 100644 src/Processors/Transforms/MergeSortingPartialResultTransform.cpp delete mode 100644 src/Processors/Transforms/MergeSortingPartialResultTransform.h delete mode 100644 src/Processors/Transforms/PartialResultTransform.cpp delete mode 100644 src/Processors/Transforms/PartialResultTransform.h delete mode 100755 tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.python delete mode 100644 tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.reference delete mode 100755 tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.sh delete mode 100644 tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.python delete mode 100644 tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.reference delete mode 100755 tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.sh delete mode 100644 tests/queries/0_stateless/helpers/tcp_client.py diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ccbf54843e4..5c9472600b8 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4708,18 +4708,6 @@ SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_pars └─────────────────────┴──────────────────────────┘ ``` -## partial_result_update_duration_ms - -Interval (in milliseconds) for sending updates with partial data about the result table to the client (in interactive mode) during query execution. Setting to 0 disables partial results. Only supported for single-threaded GROUP BY without key, ORDER BY, LIMIT and OFFSET. - -:::note -It's an experimental feature. Enable `allow_experimental_partial_result` setting first to use it. -::: - -## max_rows_in_partial_result - -Maximum rows to show in the partial result after every real-time update while the query runs (use partial result limit + OFFSET as a value in case of OFFSET in the query). - ## validate_tcp_client_information {#validate-tcp-client-information} Determines whether validation of client information enabled when query packet is received from a client using a TCP connection. diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a350654cdda..16229c5e44c 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -449,20 +449,7 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) if (!block) return; - if (block.rows() == 0 && partial_result_mode == PartialResultMode::Active) - { - partial_result_mode = PartialResultMode::Inactive; - if (is_interactive) - { - progress_indication.clearProgressOutput(*tty_buf); - std::cout << "Full result:" << std::endl; - progress_indication.writeProgress(*tty_buf); - } - } - - if (partial_result_mode == PartialResultMode::Inactive) - processed_rows += block.rows(); - + processed_rows += block.rows(); /// Even if all blocks are empty, we still need to initialize the output stream to write empty resultset. initOutputFormat(block, parsed_query); @@ -472,20 +459,13 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) return; - if (!is_interactive && partial_result_mode == PartialResultMode::Active) - return; - /// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker. if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout)) progress_indication.clearProgressOutput(*tty_buf); try { - if (partial_result_mode == PartialResultMode::Active) - output_format->writePartialResult(materializeBlock(block)); - else - output_format->write(materializeBlock(block)); - + output_format->write(materializeBlock(block)); written_first_block = true; } catch (const Exception &) @@ -549,9 +529,6 @@ void ClientBase::onProfileInfo(const ProfileInfo & profile_info) void ClientBase::initOutputFormat(const Block & block, ASTPtr parsed_query) try { - if (partial_result_mode == PartialResultMode::NotInit) - partial_result_mode = PartialResultMode::Active; - if (!output_format) { /// Ignore all results when fuzzing as they can be huge. @@ -994,14 +971,6 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa const auto & settings = global_context->getSettingsRef(); const Int32 signals_before_stop = settings.partial_result_on_first_cancel ? 2 : 1; - bool has_partial_result_setting = settings.partial_result_update_duration_ms.totalMilliseconds() > 0; - - if (has_partial_result_setting) - { - partial_result_mode = PartialResultMode::NotInit; - if (is_interactive) - std::cout << "Partial result:" << std::endl; - } int retries_left = 10; while (retries_left) @@ -1828,7 +1797,6 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin } processed_rows = 0; - partial_result_mode = PartialResultMode::Inactive; written_first_block = false; progress_indication.resetProgress(); profile_events.watch.restart(); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 604c8cf4d5c..2156aae7181 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -275,21 +275,6 @@ protected: size_t processed_rows = 0; /// How many rows have been read or written. bool print_num_processed_rows = false; /// Whether to print the number of processed rows at - enum class PartialResultMode: UInt8 - { - /// Query doesn't show partial result before the first block with 0 rows. - /// The first block with 0 rows initializes the output table format using its header. - NotInit, - - /// Query shows partial result after the first and before the second block with 0 rows. - /// The second block with 0 rows indicates that that receiving blocks with partial result has been completed and next blocks will be with the full result. - Active, - - /// Query doesn't show partial result at all. - Inactive, - }; - PartialResultMode partial_result_mode = PartialResultMode::Inactive; - bool print_stack_trace = false; /// The last exception that was received from the server. Is used for the /// return code in batch mode. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b58705614d8..f1b031eaf93 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -314,10 +314,6 @@ class IColumn; \ M(Bool, partial_result_on_first_cancel, false, "Allows query to return a partial result after cancel.", 0) \ \ - M(Bool, allow_experimental_partial_result, 0, "Enable experimental feature: partial results for running queries.", 0) \ - M(Milliseconds, partial_result_update_duration_ms, 0, "Interval (in milliseconds) for sending updates with partial data about the result table to the client (in interactive mode) during query execution. Setting to 0 disables partial results. Only supported for single-threaded GROUP BY without key, ORDER BY, LIMIT and OFFSET.", 0) \ - M(UInt64, max_rows_in_partial_result, 10, "Maximum rows to show in the partial result after every real-time update while the query runs (use partial result limit + OFFSET as a value in case of OFFSET in the query).", 0) \ - \ M(Bool, ignore_on_cluster_for_replicated_udf_queries, false, "Ignore ON CLUSTER clause for replicated UDF management queries.", 0) \ M(Bool, ignore_on_cluster_for_replicated_access_entities_queries, false, "Ignore ON CLUSTER clause for replicated access entities management queries.", 0) \ /** Settings for testing hedged requests */ \ diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index e16064db713..ac0ebc44bec 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2334,29 +2334,6 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va return block; } -Block Aggregator::prepareBlockAndFillWithoutKeySnapshot(AggregatedDataVariants & data_variants) const -{ - size_t rows = 1; - bool final = true; - - auto && out_cols - = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), data_variants.aggregates_pools, final, rows); - auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; - - AggregatedDataWithoutKey & data = data_variants.without_key; - - /// Always single-thread. It's safe to pass current arena from 'aggregates_pool'. - for (size_t insert_i = 0; insert_i < params.aggregates_size; ++insert_i) - aggregate_functions[insert_i]->insertResultInto( - data + offsets_of_aggregate_states[insert_i], - *final_aggregate_columns[insert_i], - data_variants.aggregates_pool); - - Block block = finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows); - - return block; -} - template Aggregator::ConvertToBlockRes Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 4acf73ce50f..ab53f76d2ce 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1217,7 +1217,6 @@ private: friend class ConvertingAggregatedToChunksSource; friend class ConvertingAggregatedToChunksWithMergingSource; friend class AggregatingInOrderTransform; - friend class AggregatingPartialResultTransform; /// Data structure of source blocks. Block header; @@ -1402,7 +1401,6 @@ private: std::atomic * is_cancelled = nullptr) const; Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const; - Block prepareBlockAndFillWithoutKeySnapshot(AggregatedDataVariants & data_variants) const; BlocksList prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, ThreadPool * thread_pool) const; template diff --git a/src/Processors/Chunk.cpp b/src/Processors/Chunk.cpp index cd442085eca..3839a8963b2 100644 --- a/src/Processors/Chunk.cpp +++ b/src/Processors/Chunk.cpp @@ -14,8 +14,7 @@ namespace ErrorCodes extern const int POSITION_OUT_OF_BOUND; } -Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_) - : columns(std::move(columns_)), num_rows(num_rows_) +Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_) : columns(std::move(columns_)), num_rows(num_rows_) { checkNumRowsIsConsistent(); } diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index c30586e194e..598a51bf0c7 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -75,7 +75,7 @@ void CompletedPipelineExecutor::execute() if (interactive_timeout_ms) { data = std::make_unique(); - data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms); + data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element); data->executor->setReadProgressCallback(pipeline.getReadProgressCallback()); /// Avoid passing this to lambda, copy ptr to data instead. @@ -105,7 +105,7 @@ void CompletedPipelineExecutor::execute() } else { - PipelineExecutor executor(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms); + PipelineExecutor executor(pipeline.processors, pipeline.process_list_element); executor.setReadProgressCallback(pipeline.getReadProgressCallback()); executor.execute(pipeline.getNumThreads(), pipeline.getConcurrencyControl()); } diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 6a946b4a4b9..27f6a454b24 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -260,6 +260,7 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue { pid = updated_processors.top(); updated_processors.pop(); + /// In this method we have ownership on node. auto & node = *nodes[pid]; diff --git a/src/Processors/Executors/ExecutionThreadContext.h b/src/Processors/Executors/ExecutionThreadContext.h index 85788a70771..eb048f8ab09 100644 --- a/src/Processors/Executors/ExecutionThreadContext.h +++ b/src/Processors/Executors/ExecutionThreadContext.h @@ -30,12 +30,6 @@ private: /// Callback for read progress. ReadProgressCallback * read_progress_callback = nullptr; - /// Timer that stops optimization of running local tasks instead of queuing them. - /// It provides local progress for each IProcessor task, allowing the partial result of the request to be always sended to the user. - Stopwatch watch; - /// Time period that limits the maximum allowed duration for optimizing the scheduling of local tasks within the executor - const UInt64 partial_result_duration_ms; - public: #ifndef NDEBUG /// Time for different processing stages. @@ -68,13 +62,8 @@ public: void setException(std::exception_ptr exception_) { exception = exception_; } void rethrowExceptionIfHas(); - bool needWatchRestartForPartialResultProgress() { return partial_result_duration_ms != 0 && partial_result_duration_ms < watch.elapsedMilliseconds(); } - void restartWatch() { watch.restart(); } - - explicit ExecutionThreadContext(size_t thread_number_, bool profile_processors_, bool trace_processors_, ReadProgressCallback * callback, UInt64 partial_result_duration_ms_) + explicit ExecutionThreadContext(size_t thread_number_, bool profile_processors_, bool trace_processors_, ReadProgressCallback * callback) : read_progress_callback(callback) - , watch(CLOCK_MONOTONIC) - , partial_result_duration_ms(partial_result_duration_ms_) , thread_number(thread_number_) , profile_processors(profile_processors_) , trace_processors(trace_processors_) diff --git a/src/Processors/Executors/ExecutorTasks.cpp b/src/Processors/Executors/ExecutorTasks.cpp index 08920592391..e61d225a968 100644 --- a/src/Processors/Executors/ExecutorTasks.cpp +++ b/src/Processors/Executors/ExecutorTasks.cpp @@ -108,15 +108,8 @@ void ExecutorTasks::pushTasks(Queue & queue, Queue & async_queue, ExecutionThrea { context.setTask(nullptr); - /// If sending partial results is allowed and local tasks scheduling optimization is repeated longer than the limit - /// or new task need to send partial result later, skip optimization for this iteration. - /// Otherwise take local task from queue if has one. - if ((!queue.empty() && queue.front()->processor->isPartialResultProcessor()) - || context.needWatchRestartForPartialResultProgress()) - { - context.restartWatch(); - } - else if (!queue.empty() && !context.hasAsyncTasks()) + /// Take local task from queue if has one. + if (!queue.empty() && !context.hasAsyncTasks()) { context.setTask(queue.front()); queue.pop(); @@ -146,7 +139,7 @@ void ExecutorTasks::pushTasks(Queue & queue, Queue & async_queue, ExecutionThrea } } -void ExecutorTasks::init(size_t num_threads_, size_t use_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback, UInt64 partial_result_duration_ms) +void ExecutorTasks::init(size_t num_threads_, size_t use_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback) { num_threads = num_threads_; use_threads = use_threads_; @@ -158,7 +151,7 @@ void ExecutorTasks::init(size_t num_threads_, size_t use_threads_, bool profile_ executor_contexts.reserve(num_threads); for (size_t i = 0; i < num_threads; ++i) - executor_contexts.emplace_back(std::make_unique(i, profile_processors, trace_processors, callback, partial_result_duration_ms)); + executor_contexts.emplace_back(std::make_unique(i, profile_processors, trace_processors, callback)); } } diff --git a/src/Processors/Executors/ExecutorTasks.h b/src/Processors/Executors/ExecutorTasks.h index ab6d5e91411..d35f8de94d1 100644 --- a/src/Processors/Executors/ExecutorTasks.h +++ b/src/Processors/Executors/ExecutorTasks.h @@ -58,7 +58,7 @@ public: void tryGetTask(ExecutionThreadContext & context); void pushTasks(Queue & queue, Queue & async_queue, ExecutionThreadContext & context); - void init(size_t num_threads_, size_t use_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback, UInt64 partial_result_duration_ms); + void init(size_t num_threads_, size_t use_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback); void fill(Queue & queue); void upscale(size_t use_threads_); diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 77779e2cec2..37af391fba3 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -33,9 +33,8 @@ namespace ErrorCodes } -PipelineExecutor::PipelineExecutor(std::shared_ptr & processors, QueryStatusPtr elem, UInt64 partial_result_duration_ms_) +PipelineExecutor::PipelineExecutor(std::shared_ptr & processors, QueryStatusPtr elem) : process_list_element(std::move(elem)) - , partial_result_duration_ms(partial_result_duration_ms_) { if (process_list_element) { @@ -329,7 +328,7 @@ void PipelineExecutor::initializeExecution(size_t num_threads, bool concurrency_ Queue queue; graph->initializeExecution(queue); - tasks.init(num_threads, use_threads, profile_processors, trace_processors, read_progress_callback.get(), partial_result_duration_ms); + tasks.init(num_threads, use_threads, profile_processors, trace_processors, read_progress_callback.get()); tasks.fill(queue); if (num_threads > 1) diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index 6cb0e6c4ac1..dee12dad282 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -33,7 +33,7 @@ public: /// During pipeline execution new processors can appear. They will be added to existing set. /// /// Explicit graph representation is built in constructor. Throws if graph is not correct. - explicit PipelineExecutor(std::shared_ptr & processors, QueryStatusPtr elem, UInt64 partial_result_duration_ms_ = 0); + explicit PipelineExecutor(std::shared_ptr & processors, QueryStatusPtr elem); ~PipelineExecutor(); /// Execute pipeline in multiple threads. Must be called once. @@ -90,9 +90,6 @@ private: ReadProgressCallbackPtr read_progress_callback; - /// Duration between sending partial result through the pipeline - const UInt64 partial_result_duration_ms; - using Queue = std::queue; void initializeExecution(size_t num_threads, bool concurrency_control); /// Initialize executor contexts and task_queue. diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 95a2022bf93..345bec395b2 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -41,13 +41,12 @@ struct PullingAsyncPipelineExecutor::Data } }; -PullingAsyncPipelineExecutor::PullingAsyncPipelineExecutor(QueryPipeline & pipeline_, bool has_partial_result_setting) : pipeline(pipeline_) +PullingAsyncPipelineExecutor::PullingAsyncPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) { if (!pipeline.pulling()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for PullingAsyncPipelineExecutor must be pulling"); - lazy_format = std::make_shared(pipeline.output->getHeader(), /*is_partial_result_protocol_active*/ has_partial_result_setting); - + lazy_format = std::make_shared(pipeline.output->getHeader()); pipeline.complete(lazy_format); } @@ -104,7 +103,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) if (!data) { data = std::make_unique(); - data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms); + data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element); data->executor->setReadProgressCallback(pipeline.getReadProgressCallback()); data->lazy_format = lazy_format.get(); diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.h b/src/Processors/Executors/PullingAsyncPipelineExecutor.h index 202ecbf281b..361bcc0155c 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.h @@ -21,7 +21,7 @@ struct ProfileInfo; class PullingAsyncPipelineExecutor { public: - explicit PullingAsyncPipelineExecutor(QueryPipeline & pipeline_, bool has_partial_result_setting = false); + explicit PullingAsyncPipelineExecutor(QueryPipeline & pipeline_); ~PullingAsyncPipelineExecutor(); /// Get structure of returned block or chunk. diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index f79f15c19bf..cbf73c5cb07 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -44,7 +44,7 @@ bool PullingPipelineExecutor::pull(Chunk & chunk) { if (!executor) { - executor = std::make_shared(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms); + executor = std::make_shared(pipeline.processors, pipeline.process_list_element); executor->setReadProgressCallback(pipeline.getReadProgressCallback()); } diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index f3ed24e7e96..a816ab9ca7f 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -167,7 +167,7 @@ void PushingAsyncPipelineExecutor::start() started = true; data = std::make_unique(); - data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms); + data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element); data->executor->setReadProgressCallback(pipeline.getReadProgressCallback()); data->source = pushing_source.get(); diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index f2b018792c7..696932932df 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -87,7 +87,7 @@ void PushingPipelineExecutor::start() return; started = true; - executor = std::make_shared(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms); + executor = std::make_shared(pipeline.processors, pipeline.process_list_element); executor->setReadProgressCallback(pipeline.getReadProgressCallback()); if (!executor->executeStep(&input_wait_flag)) diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index e691e32a7bc..88a6fb1e92f 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -1,89 +1,40 @@ #include #include -#include namespace DB { -IOutputFormat::IOutputFormat(const Block & header_, WriteBuffer & out_, bool is_partial_result_protocol_active_) - : IProcessor({header_, header_, header_, header_}, {}) - , out(out_) - , is_partial_result_protocol_active(is_partial_result_protocol_active_) +IOutputFormat::IOutputFormat(const Block & header_, WriteBuffer & out_) + : IProcessor({header_, header_, header_}, {}), out(out_) { } -void IOutputFormat::setCurrentChunk(InputPort & input, PortKind kind) -{ - current_chunk = input.pull(true); - current_block_kind = kind; - has_input = true; -} - -IOutputFormat::Status IOutputFormat::prepareMainAndPartialResult() -{ - bool need_data = false; - for (auto kind : {Main, PartialResult}) - { - auto & input = getPort(kind); - - if (input.isFinished()) - continue; - - if (kind == PartialResult && main_input_activated) - { - input.close(); - continue; - } - - input.setNeeded(); - need_data = true; - - if (!input.hasData()) - continue; - - setCurrentChunk(input, kind); - return Status::Ready; - } - - if (need_data) - return Status::NeedData; - - return Status::Finished; -} - -IOutputFormat::Status IOutputFormat::prepareTotalsAndExtremes() -{ - for (auto kind : {Totals, Extremes}) - { - auto & input = getPort(kind); - - if (!input.isConnected() || input.isFinished()) - continue; - - input.setNeeded(); - if (!input.hasData()) - return Status::NeedData; - - setCurrentChunk(input, kind); - return Status::Ready; - } - - return Status::Finished; -} - IOutputFormat::Status IOutputFormat::prepare() { if (has_input) return Status::Ready; - auto status = prepareMainAndPartialResult(); - if (status != Status::Finished) - return status; + for (auto kind : {Main, Totals, Extremes}) + { + auto & input = getPort(kind); - status = prepareTotalsAndExtremes(); - if (status != Status::Finished) - return status; + if (kind != Main && !input.isConnected()) + continue; + + if (input.isFinished()) + continue; + + input.setNeeded(); + + if (!input.hasData()) + return Status::NeedData; + + current_chunk = input.pull(true); + current_block_kind = kind; + has_input = true; + return Status::Ready; + } finished = true; @@ -132,18 +83,8 @@ void IOutputFormat::work() case Main: result_rows += current_chunk.getNumRows(); result_bytes += current_chunk.allocatedBytes(); - if (is_partial_result_protocol_active && !main_input_activated && current_chunk.hasRows()) - { - /// Sending an empty block signals to the client that partial results are terminated, - /// and only data from the main pipeline will be forwarded. - consume(Chunk(current_chunk.cloneEmptyColumns(), 0)); - main_input_activated = true; - } consume(std::move(current_chunk)); break; - case PartialResult: - consumePartialResult(std::move(current_chunk)); - break; case Totals: writeSuffixIfNeeded(); if (auto totals = prepareTotals(std::move(current_chunk))) @@ -178,15 +119,6 @@ void IOutputFormat::write(const Block & block) flush(); } -void IOutputFormat::writePartialResult(const Block & block) -{ - writePrefixIfNeeded(); - consumePartialResult(Chunk(block.getColumns(), block.rows())); - - if (auto_flush) - flush(); -} - void IOutputFormat::finalize() { if (finalized) diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index e642132fb64..cae2ab7691e 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -23,9 +23,9 @@ class WriteBuffer; class IOutputFormat : public IProcessor { public: - enum PortKind { Main = 0, Totals = 1, Extremes = 2, PartialResult = 3 }; + enum PortKind { Main = 0, Totals = 1, Extremes = 2 }; - IOutputFormat(const Block & header_, WriteBuffer & out_, bool is_partial_result_protocol_active_ = false); + IOutputFormat(const Block & header_, WriteBuffer & out_); Status prepare() override; void work() override; @@ -54,7 +54,6 @@ public: /// TODO: separate formats and processors. void write(const Block & block); - void writePartialResult(const Block & block); void finalize(); @@ -122,7 +121,6 @@ protected: virtual void consume(Chunk) = 0; virtual void consumeTotals(Chunk) {} virtual void consumeExtremes(Chunk) {} - virtual void consumePartialResult(Chunk) {} virtual void finalizeImpl() {} virtual void finalizeBuffers() {} virtual void writePrefix() {} @@ -176,7 +174,6 @@ protected: Chunk current_chunk; PortKind current_block_kind = PortKind::Main; - bool main_input_activated = false; bool has_input = false; bool finished = false; bool finalized = false; @@ -191,15 +188,9 @@ protected: Statistics statistics; private: - void setCurrentChunk(InputPort & input, PortKind kind); - IOutputFormat::Status prepareMainAndPartialResult(); - IOutputFormat::Status prepareTotalsAndExtremes(); - size_t rows_read_before = 0; bool are_totals_written = false; - bool is_partial_result_protocol_active = false; - /// Counters for consumed chunks. Are used for QueryLog. size_t result_rows = 0; size_t result_bytes = 0; diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 6fa891297f6..14648e68f94 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -134,8 +134,7 @@ void PrettyBlockOutputFormat::write(Chunk chunk, PortKind port_kind) { if (total_rows >= format_settings.pretty.max_rows) { - if (port_kind != PortKind::PartialResult) - total_rows += chunk.getNumRows(); + total_rows += chunk.getNumRows(); return; } if (mono_block) @@ -316,8 +315,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind } writeString(bottom_separator_s, out); - if (port_kind != PortKind::PartialResult) - total_rows += num_rows; + total_rows += num_rows; } @@ -390,34 +388,6 @@ void PrettyBlockOutputFormat::consumeExtremes(Chunk chunk) write(std::move(chunk), PortKind::Extremes); } -void PrettyBlockOutputFormat::clearLastLines(size_t lines_number) -{ - /// http://en.wikipedia.org/wiki/ANSI_escape_code - #define MOVE_TO_PREV_LINE "\033[A" - #define CLEAR_TO_END_OF_LINE "\033[K" - - static const char * clear_prev_line = MOVE_TO_PREV_LINE \ - CLEAR_TO_END_OF_LINE; - - /// Move cursor to the beginning of line - writeCString("\r", out); - - for (size_t line = 0; line < lines_number; ++line) - { - writeCString(clear_prev_line, out); - } -} - -void PrettyBlockOutputFormat::consumePartialResult(Chunk chunk) -{ - if (prev_partial_block_rows > 0) - /// number of rows + header line + footer line - clearLastLines(prev_partial_block_rows + 2); - - prev_partial_block_rows = chunk.getNumRows(); - write(std::move(chunk), PortKind::PartialResult); -} - void PrettyBlockOutputFormat::writeMonoChunkIfNeeded() { diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index 92466dce3ff..dfb23ac63f9 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -28,12 +28,7 @@ protected: void consumeTotals(Chunk) override; void consumeExtremes(Chunk) override; - void clearLastLines(size_t lines_number); - void consumePartialResult(Chunk) override; - size_t total_rows = 0; - size_t prev_partial_block_rows = 0; - size_t row_number_width = 7; // "10000. " const FormatSettings format_settings; @@ -60,7 +55,6 @@ protected: void resetFormatterImpl() override { total_rows = 0; - prev_partial_block_rows = 0; } private: diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 3a04d86b1ad..2ba9ec725e2 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -194,8 +194,7 @@ void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind po writeBottom(max_widths); - if (port_kind != PortKind::PartialResult) - total_rows += num_rows; + total_rows += num_rows; } diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index bbcfdbb7193..9cf609ed2d7 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -14,8 +14,8 @@ class LazyOutputFormat : public IOutputFormat { public: - explicit LazyOutputFormat(const Block & header, bool is_partial_result_protocol_active = false) - : IOutputFormat(header, out, is_partial_result_protocol_active), queue(2) {} + explicit LazyOutputFormat(const Block & header) + : IOutputFormat(header, out), queue(2) {} String getName() const override { return "LazyOutputFormat"; } @@ -49,7 +49,6 @@ protected: void consumeTotals(Chunk chunk) override { totals = std::move(chunk); } void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); } - void consumePartialResult(Chunk chunk) override { consume(std::move(chunk)); } private: diff --git a/src/Processors/IProcessor.cpp b/src/Processors/IProcessor.cpp index 2f294a32531..8b160153733 100644 --- a/src/Processors/IProcessor.cpp +++ b/src/Processors/IProcessor.cpp @@ -40,10 +40,5 @@ std::string IProcessor::statusToName(Status status) UNREACHABLE(); } -ProcessorPtr IProcessor::getPartialResultProcessorPtr(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) -{ - return current_processor->getPartialResultProcessor(current_processor, partial_result_limit, partial_result_duration_ms); -} - } diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 51a0bb1c121..c6bef186877 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -164,8 +164,6 @@ public: static std::string statusToName(Status status); - static ProcessorPtr getPartialResultProcessorPtr(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms); - /** Method 'prepare' is responsible for all cheap ("instantaneous": O(1) of data volume, no wait) calculations. * * It may access input and output ports, @@ -237,22 +235,6 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'expandPipeline' is not implemented for {} processor", getName()); } - enum class PartialResultStatus - { - /// Processor currently doesn't support work with the partial result pipeline. - NotSupported, - - /// Processor can be skipped in the partial result pipeline. - SkipSupported, - - /// Processor creates a light-weight copy of itself in the partial result pipeline. - /// The copy can create snapshots of the original processor or transform small blocks of data in the same way as the original processor - FullSupported, - }; - - virtual bool isPartialResultProcessor() const { return false; } - virtual PartialResultStatus getPartialResultProcessorSupportStatus() const { return PartialResultStatus::NotSupported; } - /// In case if query was cancelled executor will wait till all processors finish their jobs. /// Generally, there is no reason to check this flag. However, it may be reasonable for long operations (e.g. i/o). bool isCancelled() const { return is_cancelled.load(std::memory_order_acquire); } @@ -387,11 +369,6 @@ public: protected: virtual void onCancel() {} - virtual ProcessorPtr getPartialResultProcessor(const ProcessorPtr & /*current_processor*/, UInt64 /*partial_result_limit*/, UInt64 /*partial_result_duration_ms*/) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getPartialResultProcessor' is not implemented for {} processor", getName()); - } - private: /// For: /// - elapsed_us diff --git a/src/Processors/LimitTransform.cpp b/src/Processors/LimitTransform.cpp index b2bf3c28eee..5e24062d67a 100644 --- a/src/Processors/LimitTransform.cpp +++ b/src/Processors/LimitTransform.cpp @@ -1,5 +1,5 @@ #include -#include + namespace DB { @@ -180,6 +180,7 @@ LimitTransform::Status LimitTransform::preparePair(PortsData & data) return Status::NeedData; data.current_chunk = input.pull(true); + auto rows = data.current_chunk.getNumRows(); if (rows_before_limit_at_least && !data.input_port_has_counter) @@ -366,11 +367,5 @@ bool LimitTransform::sortColumnsEqualAt(const ColumnRawPtrs & current_chunk_sort return true; } -ProcessorPtr LimitTransform::getPartialResultProcessor(const ProcessorPtr & /*current_processor*/, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) -{ - const auto & header = inputs.front().getHeader(); - return std::make_shared(header, partial_result_limit, partial_result_duration_ms, limit, offset); -} - } diff --git a/src/Processors/LimitTransform.h b/src/Processors/LimitTransform.h index cfacc9634f9..33ff968985f 100644 --- a/src/Processors/LimitTransform.h +++ b/src/Processors/LimitTransform.h @@ -55,8 +55,6 @@ private: ColumnRawPtrs extractSortColumns(const Columns & columns) const; bool sortColumnsEqualAt(const ColumnRawPtrs & current_chunk_sort_columns, UInt64 current_chunk_row_num) const; - ProcessorPtr getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) override; - public: LimitTransform( const Block & header_, UInt64 limit_, UInt64 offset_, size_t num_streams = 1, @@ -75,14 +73,6 @@ public: void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } void setInputPortHasCounter(size_t pos) { ports_data[pos].input_port_has_counter = true; } - - PartialResultStatus getPartialResultProcessorSupportStatus() const override - { - /// Currently LimitPartialResultTransform support only single-thread work. - bool is_partial_result_supported = inputs.size() == 1 && outputs.size() == 1; - - return is_partial_result_supported ? PartialResultStatus::FullSupported : PartialResultStatus::NotSupported; - } }; } diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp index cd911e4cdf4..65e464f3075 100644 --- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp +++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp @@ -14,16 +14,7 @@ namespace ErrorCodes BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(ContextPtr from) { BuildQueryPipelineSettings settings; - - const auto & context_settings = from->getSettingsRef(); - settings.partial_result_limit = context_settings.max_rows_in_partial_result; - settings.partial_result_duration_ms = context_settings.partial_result_update_duration_ms.totalMilliseconds(); - if (settings.partial_result_duration_ms && !context_settings.allow_experimental_partial_result) - throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, - "Partial results are not allowed by default, it's an experimental feature. " - "Setting 'allow_experimental_partial_result' must be enabled to use 'partial_result_update_duration_ms'"); - - settings.actions_settings = ExpressionActionsSettings::fromSettings(context_settings, CompileExpressions::yes); + settings.actions_settings = ExpressionActionsSettings::fromSettings(from->getSettingsRef(), CompileExpressions::yes); settings.process_list_element = from->getProcessListElement(); settings.progress_callback = from->getProgressCallback(); return settings; diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h index 0410bf925d1..3b5e4e06953 100644 --- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h +++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h @@ -19,9 +19,6 @@ struct BuildQueryPipelineSettings QueryStatusPtr process_list_element; ProgressCallback progress_callback = nullptr; - UInt64 partial_result_limit = 0; - UInt64 partial_result_duration_ms = 0; - const ExpressionActionsSettings & getActionsSettings() const { return actions_settings; } static BuildQueryPipelineSettings fromContext(ContextPtr from); }; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index ec82c233ce4..2d2dc66a8c9 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -168,8 +168,6 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline( QueryPipelineBuilderPtr last_pipeline; - bool has_partial_result_setting = build_pipeline_settings.partial_result_duration_ms > 0; - std::stack stack; stack.push(Frame{.node = root}); @@ -196,9 +194,6 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline( } else stack.push(Frame{.node = frame.node->children[next_child]}); - - if (has_partial_result_setting && last_pipeline && !last_pipeline->isPartialResultActive()) - last_pipeline->activatePartialResult(build_pipeline_settings.partial_result_limit, build_pipeline_settings.partial_result_duration_ms); } last_pipeline->setProgressCallback(build_pipeline_settings.progress_callback); diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index a72cab05754..371a24ac6f2 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -27,8 +27,6 @@ public: size_t max_bytes_before_external_sort = 0; TemporaryDataOnDiskScopePtr tmp_data = nullptr; size_t min_free_disk_space = 0; - UInt64 partial_result_limit = 0; - UInt64 partial_result_duration_ms = 0; explicit Settings(const Context & context); explicit Settings(size_t max_block_size_); diff --git a/src/Processors/Transforms/AggregatingPartialResultTransform.cpp b/src/Processors/Transforms/AggregatingPartialResultTransform.cpp deleted file mode 100644 index cf8ce72e096..00000000000 --- a/src/Processors/Transforms/AggregatingPartialResultTransform.cpp +++ /dev/null @@ -1,47 +0,0 @@ -#include - -namespace DB -{ - -AggregatingPartialResultTransform::AggregatingPartialResultTransform( - const Block & input_header, const Block & output_header, AggregatingTransformPtr aggregating_transform_, - UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_) - : PartialResultTransform(input_header, output_header, partial_result_limit_, partial_result_duration_ms_) - , aggregating_transform(std::move(aggregating_transform_)) - , transform_aggregator(input_header, aggregating_transform->params->params) - {} - -void AggregatingPartialResultTransform::transformPartialResult(Chunk & chunk) -{ - auto & params = aggregating_transform->params->params; - - bool no_more_keys = false; - AggregatedDataVariants variants; - ColumnRawPtrs key_columns(params.keys_size); - Aggregator::AggregateColumns aggregate_columns(params.aggregates_size); - - const UInt64 num_rows = chunk.getNumRows(); - transform_aggregator.executeOnBlock(chunk.detachColumns(), 0, num_rows, variants, key_columns, aggregate_columns, no_more_keys); - - auto transformed_block = transform_aggregator.convertToBlocks(variants, /*final*/ true, /*max_threads*/ 1).front(); - - chunk = convertToChunk(transformed_block); -} - -PartialResultTransform::ShaphotResult AggregatingPartialResultTransform::getRealProcessorSnapshot() -{ - std::lock_guard lock(aggregating_transform->snapshot_mutex); - if (aggregating_transform->is_generate_initialized) - return {{}, SnaphotStatus::Stopped}; - - if (aggregating_transform->variants.empty()) - return {{}, SnaphotStatus::NotReady}; - - auto & snapshot_aggregator = aggregating_transform->params->aggregator; - auto & snapshot_variants = aggregating_transform->many_data->variants; - auto block = snapshot_aggregator.prepareBlockAndFillWithoutKeySnapshot(*snapshot_variants.at(0)); - - return {convertToChunk(block), SnaphotStatus::Ready}; -} - -} diff --git a/src/Processors/Transforms/AggregatingPartialResultTransform.h b/src/Processors/Transforms/AggregatingPartialResultTransform.h deleted file mode 100644 index f7bac3a5394..00000000000 --- a/src/Processors/Transforms/AggregatingPartialResultTransform.h +++ /dev/null @@ -1,29 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -class AggregatingPartialResultTransform : public PartialResultTransform -{ -public: - using AggregatingTransformPtr = std::shared_ptr; - - AggregatingPartialResultTransform( - const Block & input_header, const Block & output_header, AggregatingTransformPtr aggregating_transform_, - UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_); - - String getName() const override { return "AggregatingPartialResultTransform"; } - - void transformPartialResult(Chunk & chunk) override; - ShaphotResult getRealProcessorSnapshot() override; - -private: - AggregatingTransformPtr aggregating_transform; - Aggregator transform_aggregator; -}; - -} diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 9546d396523..bf475c57d36 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -1,4 +1,3 @@ -#include #include #include @@ -10,6 +9,7 @@ #include + namespace ProfileEvents { extern const Event ExternalAggregationMerge; @@ -660,8 +660,6 @@ void AggregatingTransform::consume(Chunk chunk) src_rows += num_rows; src_bytes += chunk.bytes(); - std::lock_guard lock(snapshot_mutex); - if (params->params.only_merge) { auto block = getInputs().front().getHeader().cloneWithColumns(chunk.detachColumns()); @@ -681,10 +679,6 @@ void AggregatingTransform::initGenerate() if (is_generate_initialized.load(std::memory_order_acquire)) return; - std::lock_guard lock(snapshot_mutex); - if (is_generate_initialized.load(std::memory_order_relaxed)) - return; - is_generate_initialized.store(true, std::memory_order_release); /// If there was no data, and we aggregate without keys, and we must return single row with the result of empty aggregation. @@ -815,12 +809,4 @@ void AggregatingTransform::initGenerate() } } -ProcessorPtr AggregatingTransform::getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) -{ - const auto & input_header = inputs.front().getHeader(); - const auto & output_header = outputs.front().getHeader(); - auto aggregating_processor = std::dynamic_pointer_cast(current_processor); - return std::make_shared(input_header, output_header, std::move(aggregating_processor), partial_result_limit, partial_result_duration_ms); -} - } diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 7b13b1a34f6..3420cdeaa50 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -170,23 +170,9 @@ public: void work() override; Processors expandPipeline() override; - PartialResultStatus getPartialResultProcessorSupportStatus() const override - { - /// Currently AggregatingPartialResultTransform support only single-thread aggregation without key. - - /// TODO: check that insert results from aggregator.prepareBlockAndFillWithoutKey return values without - /// changing of the aggregator state when aggregation with keys will be supported in AggregatingPartialResultTransform. - bool is_partial_result_supported = params->params.keys_size == 0 /// Aggregation without key. - && many_data->variants.size() == 1; /// Use only one stream for aggregation. - - return is_partial_result_supported ? PartialResultStatus::FullSupported : PartialResultStatus::NotSupported; - } - protected: void consume(Chunk chunk); - ProcessorPtr getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) override; - private: /// To read the data that was flushed into the temporary data file. Processors processors; @@ -226,13 +212,6 @@ private: bool is_consume_started = false; - friend class AggregatingPartialResultTransform; - /// The mutex protects variables that are used for creating a snapshot of the current processor. - /// The current implementation of AggregatingPartialResultTransform uses the 'is_generate_initialized' variable to check - /// whether the processor has started sending data through the main pipeline, and the corresponding partial result processor should stop creating snapshots. - /// Additionally, the mutex protects the 'params->aggregator' and 'many_data->variants' variables, which are used to get data from them for a snapshot. - std::mutex snapshot_mutex; - void initGenerate(); }; diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index bd92267a733..0d3341b000c 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -25,14 +25,6 @@ void ExpressionTransform::transform(Chunk & chunk) chunk.setColumns(block.getColumns(), num_rows); } -ProcessorPtr ExpressionTransform::getPartialResultProcessor(const ProcessorPtr & /*current_processor*/, UInt64 /*partial_result_limit*/, UInt64 /*partial_result_duration_ms*/) -{ - const auto & header = getInputPort().getHeader(); - auto result = std::make_shared(header, expression); - result->setDescription("(Partial result)"); - return result; -} - ConvertingTransform::ConvertingTransform(const Block & header_, ExpressionActionsPtr expression_) : ExceptionKeepingTransform(header_, ExpressionTransform::transformHeader(header_, expression_->getActionsDAG())) , expression(std::move(expression_)) diff --git a/src/Processors/Transforms/ExpressionTransform.h b/src/Processors/Transforms/ExpressionTransform.h index 8250f25f0f8..791c7d7ba73 100644 --- a/src/Processors/Transforms/ExpressionTransform.h +++ b/src/Processors/Transforms/ExpressionTransform.h @@ -26,15 +26,10 @@ public: static Block transformHeader(Block header, const ActionsDAG & expression); - PartialResultStatus getPartialResultProcessorSupportStatus() const override { return PartialResultStatus::FullSupported; } - protected: void transform(Chunk & chunk) override; - ProcessorPtr getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) override; - private: - ExpressionActionsPtr expression; }; diff --git a/src/Processors/Transforms/LimitPartialResultTransform.cpp b/src/Processors/Transforms/LimitPartialResultTransform.cpp deleted file mode 100644 index c9eaa9dc7dd..00000000000 --- a/src/Processors/Transforms/LimitPartialResultTransform.cpp +++ /dev/null @@ -1,42 +0,0 @@ -#include -#include - -namespace DB -{ - -LimitPartialResultTransform::LimitPartialResultTransform( - const Block & header, - UInt64 partial_result_limit_, - UInt64 partial_result_duration_ms_, - UInt64 limit_, - UInt64 offset_) - : PartialResultTransform(header, partial_result_limit_, partial_result_duration_ms_) - , limit(limit_) - , offset(offset_) - {} - -void LimitPartialResultTransform::transformPartialResult(Chunk & chunk) -{ - UInt64 num_rows = chunk.getNumRows(); - if (num_rows < offset || limit == 0) - { - chunk = {}; - return; - } - - UInt64 length = std::min(limit, num_rows - offset); - - /// Check if some rows should be removed - if (length < num_rows) - { - UInt64 num_columns = chunk.getNumColumns(); - auto columns = chunk.detachColumns(); - - for (UInt64 i = 0; i < num_columns; ++i) - columns[i] = columns[i]->cut(offset, length); - - chunk.setColumns(std::move(columns), length); - } -} - -} diff --git a/src/Processors/Transforms/LimitPartialResultTransform.h b/src/Processors/Transforms/LimitPartialResultTransform.h deleted file mode 100644 index 3a0116b624d..00000000000 --- a/src/Processors/Transforms/LimitPartialResultTransform.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class LimitTransform; - -/// Currently support only single thread implementation with one input and one output ports -class LimitPartialResultTransform : public PartialResultTransform -{ -public: - using LimitTransformPtr = std::shared_ptr; - - LimitPartialResultTransform( - const Block & header, - UInt64 partial_result_limit_, - UInt64 partial_result_duration_ms_, - UInt64 limit_, - UInt64 offset_); - - String getName() const override { return "LimitPartialResultTransform"; } - - void transformPartialResult(Chunk & chunk) override; - /// LimitsTransform doesn't have a state which can be snapshoted - ShaphotResult getRealProcessorSnapshot() override { return {{}, SnaphotStatus::Stopped}; } - -private: - UInt64 limit; - UInt64 offset; - - LimitTransformPtr limit_transform; -}; - -} diff --git a/src/Processors/Transforms/LimitsCheckingTransform.cpp b/src/Processors/Transforms/LimitsCheckingTransform.cpp index 0557f3f291e..02d2fef808c 100644 --- a/src/Processors/Transforms/LimitsCheckingTransform.cpp +++ b/src/Processors/Transforms/LimitsCheckingTransform.cpp @@ -1,5 +1,4 @@ #include -#include #include namespace DB diff --git a/src/Processors/Transforms/LimitsCheckingTransform.h b/src/Processors/Transforms/LimitsCheckingTransform.h index eabb988dab6..2f96a17c17b 100644 --- a/src/Processors/Transforms/LimitsCheckingTransform.h +++ b/src/Processors/Transforms/LimitsCheckingTransform.h @@ -33,8 +33,6 @@ public: void setQuota(const std::shared_ptr & quota_) { quota = quota_; } - PartialResultStatus getPartialResultProcessorSupportStatus() const override { return PartialResultStatus::SkipSupported; } - protected: void transform(Chunk & chunk) override; diff --git a/src/Processors/Transforms/MergeSortingPartialResultTransform.cpp b/src/Processors/Transforms/MergeSortingPartialResultTransform.cpp deleted file mode 100644 index 44b34ce3f58..00000000000 --- a/src/Processors/Transforms/MergeSortingPartialResultTransform.cpp +++ /dev/null @@ -1,54 +0,0 @@ -#include - -namespace DB -{ - -MergeSortingPartialResultTransform::MergeSortingPartialResultTransform( - const Block & header, MergeSortingTransformPtr merge_sorting_transform_, - UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_) - : PartialResultTransform(header, partial_result_limit_, partial_result_duration_ms_) - , merge_sorting_transform(std::move(merge_sorting_transform_)) - {} - -PartialResultTransform::ShaphotResult MergeSortingPartialResultTransform::getRealProcessorSnapshot() -{ - std::lock_guard lock(merge_sorting_transform->snapshot_mutex); - if (merge_sorting_transform->generated_prefix) - return {{}, SnaphotStatus::Stopped}; - - if (merge_sorting_transform->chunks.empty()) - return {{}, SnaphotStatus::NotReady}; - - /// Sort all input data - merge_sorting_transform->remerge(); - - /// It's possible that we had only empty chunks before remerge - if (merge_sorting_transform->chunks.empty()) - return {{}, SnaphotStatus::NotReady}; - - /// Add a copy of the first `partial_result_limit` rows to a generated_chunk - /// to send it later as a partial result in the next prepare stage of the current processor - auto generated_columns = merge_sorting_transform->chunks[0].cloneEmptyColumns(); - - size_t total_rows = 0; - for (const auto & merged_chunk : merge_sorting_transform->chunks) - { - size_t rows = std::min(merged_chunk.getNumRows(), partial_result_limit - total_rows); - if (rows == 0) - break; - - for (size_t position = 0; position < generated_columns.size(); ++position) - { - auto column = merged_chunk.getColumns()[position]; - generated_columns[position]->insertRangeFrom(*column, 0, rows); - } - - total_rows += rows; - } - - auto partial_result = Chunk(std::move(generated_columns), total_rows, merge_sorting_transform->chunks[0].getChunkInfo()); - merge_sorting_transform->enrichChunkWithConstants(partial_result); - return {std::move(partial_result), SnaphotStatus::Ready}; -} - -} diff --git a/src/Processors/Transforms/MergeSortingPartialResultTransform.h b/src/Processors/Transforms/MergeSortingPartialResultTransform.h deleted file mode 100644 index 781aa8e1265..00000000000 --- a/src/Processors/Transforms/MergeSortingPartialResultTransform.h +++ /dev/null @@ -1,28 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class MergeSortingPartialResultTransform : public PartialResultTransform -{ -public: - using MergeSortingTransformPtr = std::shared_ptr; - - MergeSortingPartialResultTransform( - const Block & header, MergeSortingTransformPtr merge_sorting_transform_, - UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_); - - String getName() const override { return "MergeSortingPartialResultTransform"; } - - /// MergeSortingTransform always receives chunks in a sorted state, so transformation is not needed - void transformPartialResult(Chunk & /*chunk*/) override {} - ShaphotResult getRealProcessorSnapshot() override; - -private: - MergeSortingTransformPtr merge_sorting_transform; -}; - -} diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index e801e5e16d5..de77711d129 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -137,8 +136,6 @@ void MergeSortingTransform::consume(Chunk chunk) /// If there were only const columns in sort description, then there is no need to sort. /// Return the chunk as is. - std::lock_guard lock(snapshot_mutex); - if (description.empty()) { generated_chunk = std::move(chunk); @@ -216,8 +213,6 @@ void MergeSortingTransform::serialize() void MergeSortingTransform::generate() { - std::lock_guard lock(snapshot_mutex); - if (!generated_prefix) { size_t num_tmp_files = tmp_data ? tmp_data->getStreams().size() : 0; @@ -278,11 +273,4 @@ void MergeSortingTransform::remerge() sum_bytes_in_blocks = new_sum_bytes_in_blocks; } -ProcessorPtr MergeSortingTransform::getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) -{ - const auto & header = inputs.front().getHeader(); - auto merge_sorting_processor = std::dynamic_pointer_cast(current_processor); - return std::make_shared(header, std::move(merge_sorting_processor), partial_result_limit, partial_result_duration_ms); -} - } diff --git a/src/Processors/Transforms/MergeSortingTransform.h b/src/Processors/Transforms/MergeSortingTransform.h index 67f098b4362..e8c180b6903 100644 --- a/src/Processors/Transforms/MergeSortingTransform.h +++ b/src/Processors/Transforms/MergeSortingTransform.h @@ -33,8 +33,6 @@ public: String getName() const override { return "MergeSortingTransform"; } - PartialResultStatus getPartialResultProcessorSupportStatus() const override { return PartialResultStatus::FullSupported; } - protected: void consume(Chunk chunk) override; void serialize() override; @@ -42,8 +40,6 @@ protected: Processors expandPipeline() override; - ProcessorPtr getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) override; - private: size_t max_bytes_before_remerge; double remerge_lowered_memory_bytes_ratio; @@ -63,13 +59,6 @@ private: void remerge(); ProcessorPtr external_merging_sorted; - - friend class MergeSortingPartialResultTransform; - /// The mutex protects variables that are used for creating a snapshot of the current processor. - /// The current implementation of MergeSortingPartialResultTransform uses the 'generated_prefix' variable to check - /// whether the processor has started sending data through the main pipeline, and the corresponding partial result processor should stop creating snapshots. - /// Additionally, the mutex protects the 'chunks' variable and all variables in the 'remerge' function, which is used to transition 'chunks' to a sorted state. - std::mutex snapshot_mutex; }; } diff --git a/src/Processors/Transforms/PartialResultTransform.cpp b/src/Processors/Transforms/PartialResultTransform.cpp deleted file mode 100644 index 97ff79dee54..00000000000 --- a/src/Processors/Transforms/PartialResultTransform.cpp +++ /dev/null @@ -1,80 +0,0 @@ -#include - -namespace DB -{ - - -PartialResultTransform::PartialResultTransform(const Block & header, UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_) - : PartialResultTransform(header, header, partial_result_limit_, partial_result_duration_ms_) {} - -PartialResultTransform::PartialResultTransform(const Block & input_header, const Block & output_header, UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_) - : IProcessor({input_header}, {output_header}) - , input(inputs.front()) - , output(outputs.front()) - , partial_result_limit(partial_result_limit_) - , partial_result_duration_ms(partial_result_duration_ms_) - , watch(CLOCK_MONOTONIC) - {} - -IProcessor::Status PartialResultTransform::prepare() -{ - if (output.isFinished()) - { - input.close(); - return Status::Finished; - } - - if (finished_getting_snapshots) - { - output.finish(); - return Status::Finished; - } - - if (!output.canPush()) - { - input.setNotNeeded(); - return Status::PortFull; - } - - /// If input data from previous partial result processor is finished then - /// PartialResultTransform ready to create snapshots and send them as a partial result - if (input.isFinished()) - { - if (partial_result.snapshot_status == SnaphotStatus::Ready) - { - partial_result.snapshot_status = SnaphotStatus::NotReady; - output.push(std::move(partial_result.chunk)); - return Status::PortFull; - } - - return Status::Ready; - } - - input.setNeeded(); - if (!input.hasData()) - return Status::NeedData; - - partial_result.chunk = input.pull(); - transformPartialResult(partial_result.chunk); - if (partial_result.chunk.getNumRows() > 0) - { - output.push(std::move(partial_result.chunk)); - return Status::PortFull; - } - - return Status::NeedData; -} - -void PartialResultTransform::work() -{ - if (partial_result_duration_ms < watch.elapsedMilliseconds()) - { - partial_result = getRealProcessorSnapshot(); - if (partial_result.snapshot_status == SnaphotStatus::Stopped) - finished_getting_snapshots = true; - - watch.restart(); - } -} - -} diff --git a/src/Processors/Transforms/PartialResultTransform.h b/src/Processors/Transforms/PartialResultTransform.h deleted file mode 100644 index 4fe87638f38..00000000000 --- a/src/Processors/Transforms/PartialResultTransform.h +++ /dev/null @@ -1,57 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -/// Processors of this type are used to construct an auxiliary pipeline with processors corresponding to those in the main pipeline. -/// These processors work in two modes: -/// 1) Creating a snapshot of the corresponding processor from the main pipeline once per partial_result_duration_ms (period in milliseconds), and then sending the snapshot through the partial result pipeline. -/// 2) Transforming small blocks of data in the same way as the original processor and sending the transformed data through the partial result pipeline. -/// All processors of this type rely on the invariant that a new block from the previous processor of the partial result pipeline overwrites information about the previous block of the same previous processor. -class PartialResultTransform : public IProcessor -{ -public: - PartialResultTransform(const Block & header, UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_); - PartialResultTransform(const Block & input_header, const Block & output_header, UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_); - - String getName() const override { return "PartialResultTransform"; } - - Status prepare() override; - void work() override; - - bool isPartialResultProcessor() const override { return true; } - -protected: - enum class SnaphotStatus - { - NotReady, // Waiting for data from the previous partial result processor or awaiting a timer before creating the snapshot. - Ready, // Current partial result processor has received a snapshot from the processor in the main pipeline. - Stopped, // The processor from the main pipeline has started sending data, and the pipeline for partial results should use data from the next processors of the main pipeline. - }; - - struct ShaphotResult - { - Chunk chunk; - SnaphotStatus snapshot_status; - }; - - InputPort & input; - OutputPort & output; - - UInt64 partial_result_limit; - UInt64 partial_result_duration_ms; - - ShaphotResult partial_result = {{}, SnaphotStatus::NotReady}; - - bool finished_getting_snapshots = false; - - virtual void transformPartialResult(Chunk & /*chunk*/) = 0; - virtual ShaphotResult getRealProcessorSnapshot() = 0; // { return {{}, SnaphotStatus::Stopped}; } - -private: - Stopwatch watch; -}; - -} diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index b22c135e865..46b5db08ca1 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -12,7 +12,6 @@ #include #include #include -#include #include @@ -170,9 +169,12 @@ Pipe::Pipe(ProcessorPtr source) { checkSource(*source); + if (collected_processors) + collected_processors->emplace_back(source); + output_ports.push_back(&source->getOutputs().front()); header = output_ports.front()->getHeader(); - addProcessor(std::move(source)); + processors->emplace_back(std::move(source)); max_parallel_streams = 1; } @@ -364,11 +366,11 @@ void Pipe::addSource(ProcessorPtr source) else assertBlocksHaveEqualStructure(header, source_header, "Pipes"); - output_ports.push_back(&source->getOutputs().front()); - if (isPartialResultActive()) - partial_result_ports.push_back(nullptr); + if (collected_processors) + collected_processors->emplace_back(source); - addProcessor(std::move(source)); + output_ports.push_back(&source->getOutputs().front()); + processors->emplace_back(std::move(source)); max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } @@ -386,9 +388,11 @@ void Pipe::addTotalsSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); - totals_port = &source->getOutputs().front(); + if (collected_processors) + collected_processors->emplace_back(source); - addProcessor(std::move(source)); + totals_port = &source->getOutputs().front(); + processors->emplace_back(std::move(source)); } void Pipe::addExtremesSource(ProcessorPtr source) @@ -404,20 +408,11 @@ void Pipe::addExtremesSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); + if (collected_processors) + collected_processors->emplace_back(source); + extremes_port = &source->getOutputs().front(); - - addProcessor(std::move(source)); -} - -void Pipe::activatePartialResult(UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_) -{ - if (is_partial_result_active) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Partial result for Pipe should be initialized only once"); - - is_partial_result_active = true; - partial_result_limit = partial_result_limit_; - partial_result_duration_ms = partial_result_duration_ms_; - partial_result_ports.assign(output_ports.size(), nullptr); + processors->emplace_back(std::move(source)); } static void dropPort(OutputPort *& port, Processors & processors, Processors * collected_processors) @@ -445,15 +440,6 @@ void Pipe::dropExtremes() dropPort(extremes_port, *processors, collected_processors); } -void Pipe::dropPartialResult() -{ - for (auto & port : partial_result_ports) - dropPort(port, *processors, collected_processors); - - is_partial_result_active = false; - partial_result_ports.clear(); -} - void Pipe::addTransform(ProcessorPtr transform) { addTransform(std::move(transform), static_cast(nullptr), static_cast(nullptr)); @@ -484,8 +470,6 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort if (extremes) extremes_port = extremes; - addPartialResultTransform(transform); - size_t next_output = 0; for (auto & input : inputs) { @@ -536,7 +520,10 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); - addProcessor(std::move(transform)); + if (collected_processors) + collected_processors->emplace_back(transform); + + processors->emplace_back(std::move(transform)); max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } @@ -573,8 +560,6 @@ void Pipe::addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes_port = nullptr; } - addPartialResultTransform(transform); - bool found_totals = false; bool found_extremes = false; @@ -624,119 +609,14 @@ void Pipe::addTransform(ProcessorPtr transform, InputPort * totals, InputPort * if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); - addProcessor(std::move(transform)); + if (collected_processors) + collected_processors->emplace_back(transform); + + processors->emplace_back(std::move(transform)); max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } -void Pipe::addPartialResultSimpleTransform(const ProcessorPtr & transform, size_t partial_result_port_id) -{ - if (isPartialResultActive()) - { - auto & partial_result_port = partial_result_ports[partial_result_port_id]; - auto partial_result_status = transform->getPartialResultProcessorSupportStatus(); - - if (partial_result_status == IProcessor::PartialResultStatus::NotSupported) - dropPort(partial_result_port, *processors, collected_processors); - - if (partial_result_status != IProcessor::PartialResultStatus::FullSupported) - return; - - auto partial_result_transform = IProcessor::getPartialResultProcessorPtr(transform, partial_result_limit, partial_result_duration_ms); - - connectPartialResultPort(partial_result_port, partial_result_transform->getInputs().front()); - - partial_result_port = &partial_result_transform->getOutputs().front(); - - addProcessor(std::move(partial_result_transform)); - } -} - -void Pipe::addPartialResultTransform(const ProcessorPtr & transform) -{ - if (isPartialResultActive()) - { - size_t new_outputs_size = 0; - for (const auto & output : transform->getOutputs()) - { - /// We do not use totals_port and extremes_port in partial result - if ((totals_port && totals_port == &output) || (extremes_port && extremes_port == &output)) - continue; - ++new_outputs_size; - } - - auto partial_result_status = transform->getPartialResultProcessorSupportStatus(); - - if (partial_result_status == IProcessor::PartialResultStatus::SkipSupported && new_outputs_size != partial_result_ports.size()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot skip transform {} in the partial result part of the Pipe because it has {} output ports, but the partial result part expects {} output ports", - transform->getName(), - new_outputs_size, - partial_result_ports.size()); - - if (partial_result_status == IProcessor::PartialResultStatus::NotSupported) - { - for (auto & partial_result_port : partial_result_ports) - dropPort(partial_result_port, *processors, collected_processors); - - partial_result_ports.assign(new_outputs_size, nullptr); - return; - } - - if (partial_result_status != IProcessor::PartialResultStatus::FullSupported) - return; - - auto partial_result_transform = IProcessor::getPartialResultProcessorPtr(transform, partial_result_limit, partial_result_duration_ms); - auto & inputs = partial_result_transform->getInputs(); - - if (inputs.size() != partial_result_ports.size()) - { - WriteBufferFromOwnString out; - if (processors && !processors->empty()) - printPipeline(*processors, out); - - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot add partial result transform {} to Pipe because it has {} input ports, but {} expected\n{}", - partial_result_transform->getName(), - inputs.size(), - partial_result_ports.size(), out.str()); - } - - size_t next_port = 0; - for (auto & input : inputs) - { - connectPartialResultPort(partial_result_ports[next_port], input); - ++next_port; - } - - partial_result_ports.assign(new_outputs_size, nullptr); - - next_port = 0; - for (auto & new_partial_result_port : partial_result_transform->getOutputs()) - { - partial_result_ports[next_port] = &new_partial_result_port; - ++next_port; - } - - addProcessor(std::move(partial_result_transform)); - } -} - -void Pipe::connectPartialResultPort(OutputPort * partial_result_port, InputPort & partial_result_transform_port) -{ - if (partial_result_port == nullptr) - { - auto source = std::make_shared(getHeader()); - partial_result_port = &source->getPort(); - - addProcessor(std::move(source)); - } - - connect(*partial_result_port, partial_result_transform_port); -} - void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) { if (output_ports.empty()) @@ -744,7 +624,7 @@ void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) Block new_header; - auto add_transform = [&](OutputPort *& port, size_t partial_result_port_id, StreamType stream_type) + auto add_transform = [&](OutputPort *& port, StreamType stream_type) { if (!port) return; @@ -780,22 +660,19 @@ void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) { connect(*port, transform->getInputs().front()); port = &transform->getOutputs().front(); - if (stream_type == StreamType::Main) - addPartialResultSimpleTransform(transform, partial_result_port_id); - addProcessor(std::move(transform)); + if (collected_processors) + collected_processors->emplace_back(transform); + + processors->emplace_back(std::move(transform)); } }; - size_t partial_result_port_id = 0; for (auto & port : output_ports) - { - add_transform(port, partial_result_port_id, StreamType::Main); - ++partial_result_port_id; - } + add_transform(port, StreamType::Main); - add_transform(totals_port, 0, StreamType::Totals); - add_transform(extremes_port, 0, StreamType::Extremes); + add_transform(totals_port, StreamType::Totals); + add_transform(extremes_port, StreamType::Extremes); header = std::move(new_header); } @@ -816,7 +693,6 @@ void Pipe::addChains(std::vector chains) dropTotals(); dropExtremes(); - dropPartialResult(); size_t max_parallel_streams_for_chains = 0; @@ -835,21 +711,18 @@ void Pipe::addChains(std::vector chains) auto added_processors = Chain::getProcessors(std::move(chains[i])); for (auto & transform : added_processors) - addProcessor(std::move(transform)); + { + if (collected_processors) + collected_processors->emplace_back(transform); + + processors->emplace_back(std::move(transform)); + } } header = std::move(new_header); max_parallel_streams = std::max(max_parallel_streams, max_parallel_streams_for_chains); } -void Pipe::addProcessor(ProcessorPtr processor) -{ - if (collected_processors) - collected_processors->emplace_back(processor); - - processors->emplace_back(std::move(processor)); -} - void Pipe::resize(size_t num_streams, bool force, bool strict) { if (output_ports.empty()) @@ -910,9 +783,6 @@ void Pipe::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) add_transform(totals_port, StreamType::Totals); add_transform(extremes_port, StreamType::Extremes); - for (auto & port : partial_result_ports) - add_transform(port, StreamType::PartialResult); - output_ports.clear(); header.clear(); } @@ -922,9 +792,6 @@ void Pipe::transform(const Transformer & transformer, bool check_ports) if (output_ports.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot transform empty Pipe"); - /// TODO: Add functionality to work with partial result ports in transformer. - dropPartialResult(); - auto new_processors = transformer(output_ports); /// Create hash table with new processors. @@ -1014,10 +881,5 @@ void Pipe::transform(const Transformer & transformer, bool check_ports) max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } -OutputPort * Pipe::getPartialResultPort(size_t pos) const -{ - return partial_result_ports.empty() ? nullptr : partial_result_ports[pos]; -} - } diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index a6bd46a325b..09931e38578 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -48,9 +48,6 @@ public: OutputPort * getOutputPort(size_t pos) const { return output_ports[pos]; } OutputPort * getTotalsPort() const { return totals_port; } OutputPort * getExtremesPort() const { return extremes_port; } - OutputPort * getPartialResultPort(size_t pos) const; - - bool isPartialResultActive() { return is_partial_result_active; } /// Add processor to list, add it output ports to output_ports. /// Processor shouldn't have input ports, output ports shouldn't be connected. @@ -61,13 +58,9 @@ public: void addTotalsSource(ProcessorPtr source); void addExtremesSource(ProcessorPtr source); - /// Activate sending partial result during main pipeline execution - void activatePartialResult(UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_); - - /// Drop totals, extremes and partial result (create NullSink for them). + /// Drop totals and extremes (create NullSink for them). void dropTotals(); void dropExtremes(); - void dropPartialResult(); /// Add processor to list. It should have size() input ports with compatible header. /// Output ports should have same headers. @@ -76,16 +69,11 @@ public: void addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes); void addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes); - void addPartialResultTransform(const ProcessorPtr & transform); - void addPartialResultSimpleTransform(const ProcessorPtr & transform, size_t partial_result_port_id); - void connectPartialResultPort(OutputPort * partial_result_port, InputPort & partial_result_transform_port); - enum class StreamType { Main = 0, /// Stream for query data. There may be several streams of this type. Totals, /// Stream for totals. No more than one. Extremes, /// Stream for extremes. No more than one. - PartialResult, /// Stream for partial result data. There may be several streams of this type. }; using ProcessorGetter = std::function; @@ -121,17 +109,10 @@ private: Block header; std::shared_ptr processors; - /// If the variable is true, then each time a processor is added pipe will try - /// to add processor which will send partial result from original processor - bool is_partial_result_active = false; - UInt64 partial_result_limit = 0; - UInt64 partial_result_duration_ms = 0; - - /// Output ports. Totals, extremes and partial results are allowed to be empty. + /// Output ports. Totals and extremes are allowed to be empty. OutputPortRawPtrs output_ports; OutputPort * totals_port = nullptr; OutputPort * extremes_port = nullptr; - OutputPortRawPtrs partial_result_ports; /// It is the max number of processors which can be executed in parallel for each step. /// Usually, it's the same as the number of output ports. @@ -147,8 +128,6 @@ private: static Pipe unitePipes(Pipes pipes, Processors * collected_processors, bool allow_empty_header); void setSinks(const Pipe::ProcessorGetterWithStreamKind & getter); - void addProcessor(ProcessorPtr processor); - friend class QueryPipelineBuilder; friend class QueryPipeline; }; diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 4ce0aa029be..935c006c217 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -73,8 +73,7 @@ static void checkPulling( Processors & processors, OutputPort * output, OutputPort * totals, - OutputPort * extremes, - OutputPort * partial_result) + OutputPort * extremes) { if (!output || output->isConnected()) throw Exception( @@ -91,15 +90,9 @@ static void checkPulling( ErrorCodes::LOGICAL_ERROR, "Cannot create pulling QueryPipeline because its extremes port is connected"); - if (partial_result && partial_result->isConnected()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot create pulling QueryPipeline because its partial_result port is connected"); - bool found_output = false; bool found_totals = false; bool found_extremes = false; - bool found_partial_result = false; for (const auto & processor : processors) { for (const auto & in : processor->getInputs()) @@ -113,8 +106,6 @@ static void checkPulling( found_totals = true; else if (extremes && &out == extremes) found_extremes = true; - else if (partial_result && &out == partial_result) - found_partial_result = true; else checkOutput(out, processor, processors); } @@ -132,10 +123,6 @@ static void checkPulling( throw Exception( ErrorCodes::LOGICAL_ERROR, "Cannot create pulling QueryPipeline because its extremes port does not belong to any processor"); - if (partial_result && !found_partial_result) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot create pulling QueryPipeline because its partial result port does not belong to any processor"); } static void checkCompleted(Processors & processors) @@ -338,20 +325,17 @@ QueryPipeline::QueryPipeline( std::shared_ptr processors_, OutputPort * output_, OutputPort * totals_, - OutputPort * extremes_, - OutputPort * partial_result_) + OutputPort * extremes_) : resources(std::move(resources_)) , processors(std::move(processors_)) , output(output_) , totals(totals_) , extremes(extremes_) - , partial_result(partial_result_) { - checkPulling(*processors, output, totals, extremes, partial_result); + checkPulling(*processors, output, totals, extremes); } QueryPipeline::QueryPipeline(Pipe pipe) - : partial_result_duration_ms(pipe.partial_result_duration_ms) { if (pipe.numOutputPorts() > 0) { @@ -359,11 +343,8 @@ QueryPipeline::QueryPipeline(Pipe pipe) output = pipe.getOutputPort(0); totals = pipe.getTotalsPort(); extremes = pipe.getExtremesPort(); - partial_result = pipe.getPartialResultPort(0); - num_threads = pipe.max_parallel_streams; - processors = std::move(pipe.processors); - checkPulling(*processors, output, totals, extremes, partial_result); + checkPulling(*processors, output, totals, extremes); } else { @@ -395,7 +376,6 @@ QueryPipeline::QueryPipeline(std::shared_ptr format) auto & format_main = format->getPort(IOutputFormat::PortKind::Main); auto & format_totals = format->getPort(IOutputFormat::PortKind::Totals); auto & format_extremes = format->getPort(IOutputFormat::PortKind::Extremes); - auto & format_partial_result = format->getPort(IOutputFormat::PortKind::PartialResult); if (!totals) { @@ -411,21 +391,12 @@ QueryPipeline::QueryPipeline(std::shared_ptr format) processors->emplace_back(std::move(source)); } - if (!partial_result) - { - auto source = std::make_shared(format_partial_result.getHeader()); - partial_result = &source->getPort(); - processors->emplace_back(std::move(source)); - } - connect(*totals, format_totals); connect(*extremes, format_extremes); - connect(*partial_result, format_partial_result); input = &format_main; totals = nullptr; extremes = nullptr; - partial_result = nullptr; output_format = format.get(); @@ -453,7 +424,6 @@ void QueryPipeline::complete(std::shared_ptr sink) drop(totals, *processors); drop(extremes, *processors); - drop(partial_result, *processors); connect(*output, sink->getPort()); processors->emplace_back(std::move(sink)); @@ -469,7 +439,6 @@ void QueryPipeline::complete(Chain chain) drop(totals, *processors); drop(extremes, *processors); - drop(partial_result, *processors); processors->reserve(processors->size() + chain.getProcessors().size() + 1); for (auto processor : chain.getProcessors()) @@ -495,7 +464,6 @@ void QueryPipeline::complete(Pipe pipe) pipe.resize(1); pipe.dropExtremes(); pipe.dropTotals(); - pipe.dropPartialResult(); connect(*pipe.getOutputPort(0), *input); input = nullptr; @@ -524,13 +492,11 @@ void QueryPipeline::complete(std::shared_ptr format) addMaterializing(output, *processors); addMaterializing(totals, *processors); addMaterializing(extremes, *processors); - addMaterializing(partial_result, *processors); } auto & format_main = format->getPort(IOutputFormat::PortKind::Main); auto & format_totals = format->getPort(IOutputFormat::PortKind::Totals); auto & format_extremes = format->getPort(IOutputFormat::PortKind::Extremes); - auto & format_partial_result = format->getPort(IOutputFormat::PortKind::PartialResult); if (!totals) { @@ -546,22 +512,13 @@ void QueryPipeline::complete(std::shared_ptr format) processors->emplace_back(std::move(source)); } - if (!partial_result) - { - auto source = std::make_shared(format_partial_result.getHeader()); - partial_result = &source->getPort(); - processors->emplace_back(std::move(source)); - } - connect(*output, format_main); connect(*totals, format_totals); connect(*extremes, format_extremes); - connect(*partial_result, format_partial_result); output = nullptr; totals = nullptr; extremes = nullptr; - partial_result = nullptr; initRowsBeforeLimit(format.get()); output_format = format.get(); @@ -733,7 +690,6 @@ void QueryPipeline::convertStructureTo(const ColumnsWithTypeAndName & columns) addExpression(output, actions, *processors); addExpression(totals, actions, *processors); addExpression(extremes, actions, *processors); - addExpression(partial_result, actions, *processors); } std::unique_ptr QueryPipeline::getReadProgressCallback() const diff --git a/src/QueryPipeline/QueryPipeline.h b/src/QueryPipeline/QueryPipeline.h index 20e58bc0f59..f14cf61aac2 100644 --- a/src/QueryPipeline/QueryPipeline.h +++ b/src/QueryPipeline/QueryPipeline.h @@ -75,8 +75,7 @@ public: std::shared_ptr processors_, OutputPort * output_, OutputPort * totals_ = nullptr, - OutputPort * extremes_ = nullptr, - OutputPort * partial_result_ = nullptr); + OutputPort * extremes_ = nullptr); bool initialized() const { return !processors->empty(); } /// When initialized, exactly one of the following is true. @@ -155,7 +154,6 @@ private: OutputPort * output = nullptr; OutputPort * totals = nullptr; OutputPort * extremes = nullptr; - OutputPort * partial_result = nullptr; QueryStatusPtr process_list_element; @@ -164,9 +162,6 @@ private: size_t num_threads = 0; bool concurrency_control = false; - UInt64 partial_result_limit = 0; - UInt64 partial_result_duration_ms = 0; - friend class PushingPipelineExecutor; friend class PullingPipelineExecutor; friend class PushingAsyncPipelineExecutor; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index e176e8585f5..f9726339872 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -110,15 +110,6 @@ void QueryPipelineBuilder::init(QueryPipeline & pipeline) pipe.header = {}; } - if (pipeline.partial_result) - { - /// Set partial result ports only after activation because when activated, it is set to nullptr - pipe.activatePartialResult(pipeline.partial_result_limit, pipeline.partial_result_duration_ms); - pipe.partial_result_ports = {pipeline.partial_result}; - } - else - pipe.dropPartialResult(); - pipe.totals_port = pipeline.totals; pipe.extremes_port = pipeline.extremes; pipe.max_parallel_streams = pipeline.num_threads; @@ -361,10 +352,6 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped left->checkInitializedAndNotCompleted(); right->checkInitializedAndNotCompleted(); - /// TODO: Support joining of partial results from different pipelines. - left->pipe.dropPartialResult(); - right->pipe.dropPartialResult(); - left->pipe.dropExtremes(); right->pipe.dropExtremes(); if (left->getNumStreams() != 1 || right->getNumStreams() != 1) @@ -377,7 +364,6 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped auto joining = std::make_shared(join, inputs, out_header, max_block_size); - /// TODO: Support partial results in merge pipelines after joining support above. return mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); } @@ -398,10 +384,6 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe left->pipe.dropExtremes(); right->pipe.dropExtremes(); - /// TODO: Support joining of partial results from different pipelines. - left->pipe.dropPartialResult(); - right->pipe.dropPartialResult(); - left->pipe.collected_processors = collected_processors; /// Collect the NEW processors for the right pipeline. @@ -652,7 +634,7 @@ PipelineExecutorPtr QueryPipelineBuilder::execute() if (!isCompleted()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot execute pipeline because it is not completed"); - return std::make_shared(pipe.processors, process_list_element, pipe.partial_result_duration_ms); + return std::make_shared(pipe.processors, process_list_element); } Pipe QueryPipelineBuilder::getPipe(QueryPipelineBuilder pipeline, QueryPlanResourceHolder & resources) diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index cee545ac29d..5d273df7068 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -85,15 +85,6 @@ public: /// Pipeline will be completed after this transformation. void setSinks(const Pipe::ProcessorGetterWithStreamKind & getter); - /// Activate building separate pipeline for sending partial result. - void activatePartialResult(UInt64 partial_result_limit, UInt64 partial_result_duration_ms) - { - pipe.activatePartialResult(partial_result_limit, partial_result_duration_ms); - } - - /// Check if building of a pipeline for sending partial result active. - bool isPartialResultActive() { return pipe.isPartialResultActive(); } - /// Add totals which returns one chunk with single row with defaults. void addDefaultTotals(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 4908bf82b46..6f8164ec19f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -965,14 +965,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors() std::unique_lock progress_lock(task_callback_mutex, std::defer_lock); { - const auto & settings = query_context->getSettingsRef(); - bool has_partial_result_setting = settings.partial_result_update_duration_ms.totalMilliseconds() > 0; - if (has_partial_result_setting && !settings.allow_experimental_partial_result) - throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, - "Partial results are not allowed by default, it's an experimental feature. " - "Setting 'allow_experimental_partial_result' must be enabled to use 'partial_result_update_duration_ms'"); - - PullingAsyncPipelineExecutor executor(pipeline, has_partial_result_setting); + PullingAsyncPipelineExecutor executor(pipeline); CurrentMetrics::Increment query_thread_metric_increment{CurrentMetrics::QueryThread}; Block block; diff --git a/tests/queries/0_stateless/02010_lc_native.python b/tests/queries/0_stateless/02010_lc_native.python index 219fdf04472..6c4220855c8 100755 --- a/tests/queries/0_stateless/02010_lc_native.python +++ b/tests/queries/0_stateless/02010_lc_native.python @@ -1,33 +1,227 @@ #!/usr/bin/env python3 # -*- coding: utf-8 -*- +import socket import os -import sys +import uuid -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, "helpers")) +CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "127.0.0.1") +CLICKHOUSE_PORT = int(os.environ.get("CLICKHOUSE_PORT_TCP", "900000")) +CLICKHOUSE_DATABASE = os.environ.get("CLICKHOUSE_DATABASE", "default") +CLIENT_NAME = "simple native protocol" -from tcp_client import ( - TCPClient, - CLICKHOUSE_DATABASE, - writeVarUInt, - writeStringBinary, - serializeBlockInfo, - assertPacket, -) + +def writeVarUInt(x, ba): + for _ in range(0, 9): + byte = x & 0x7F + if x > 0x7F: + byte |= 0x80 + + ba.append(byte) + + x >>= 7 + if x == 0: + return + + +def writeStringBinary(s, ba): + b = bytes(s, "utf-8") + writeVarUInt(len(s), ba) + ba.extend(b) + + +def readStrict(s, size=1): + res = bytearray() + while size: + cur = s.recv(size) + # if not res: + # raise "Socket is closed" + size -= len(cur) + res.extend(cur) + + return res + + +def readUInt(s, size=1): + res = readStrict(s, size) + val = 0 + for i in range(len(res)): + val += res[i] << (i * 8) + return val + + +def readUInt8(s): + return readUInt(s) + + +def readUInt16(s): + return readUInt(s, 2) + + +def readUInt32(s): + return readUInt(s, 4) + + +def readUInt64(s): + return readUInt(s, 8) + + +def readVarUInt(s): + x = 0 + for i in range(9): + byte = readStrict(s)[0] + x |= (byte & 0x7F) << (7 * i) + + if not byte & 0x80: + return x + + return x + + +def readStringBinary(s): + size = readVarUInt(s) + s = readStrict(s, size) + return s.decode("utf-8") + + +def sendHello(s): + ba = bytearray() + writeVarUInt(0, ba) # Hello + writeStringBinary(CLIENT_NAME, ba) + writeVarUInt(21, ba) + writeVarUInt(9, ba) + writeVarUInt(54449, ba) + writeStringBinary("default", ba) # database + writeStringBinary("default", ba) # user + writeStringBinary("", ba) # pwd + s.sendall(ba) + + +def receiveHello(s): + p_type = readVarUInt(s) + assert p_type == 0 # Hello + server_name = readStringBinary(s) + # print("Server name: ", server_name) + server_version_major = readVarUInt(s) + # print("Major: ", server_version_major) + server_version_minor = readVarUInt(s) + # print("Minor: ", server_version_minor) + server_revision = readVarUInt(s) + # print("Revision: ", server_revision) + server_timezone = readStringBinary(s) + # print("Timezone: ", server_timezone) + server_display_name = readStringBinary(s) + # print("Display name: ", server_display_name) + server_version_patch = readVarUInt(s) + # print("Version patch: ", server_version_patch) + + +def serializeClientInfo(ba, query_id): + writeStringBinary("default", ba) # initial_user + writeStringBinary(query_id, ba) # initial_query_id + writeStringBinary("127.0.0.1:9000", ba) # initial_address + ba.extend([0] * 8) # initial_query_start_time_microseconds + ba.append(1) # TCP + writeStringBinary("os_user", ba) # os_user + writeStringBinary("client_hostname", ba) # client_hostname + writeStringBinary(CLIENT_NAME, ba) # client_name + writeVarUInt(21, ba) + writeVarUInt(9, ba) + writeVarUInt(54449, ba) + writeStringBinary("", ba) # quota_key + writeVarUInt(0, ba) # distributed_depth + writeVarUInt(1, ba) # client_version_patch + ba.append(0) # No telemetry + + +def sendQuery(s, query): + ba = bytearray() + query_id = uuid.uuid4().hex + writeVarUInt(1, ba) # query + writeStringBinary(query_id, ba) + + ba.append(1) # INITIAL_QUERY + + # client info + serializeClientInfo(ba, query_id) + + writeStringBinary("", ba) # No settings + writeStringBinary("", ba) # No interserver secret + writeVarUInt(2, ba) # Stage - Complete + ba.append(0) # No compression + writeStringBinary(query, ba) # query, finally + s.sendall(ba) + + +def serializeBlockInfo(ba): + writeVarUInt(1, ba) # 1 + ba.append(0) # is_overflows + writeVarUInt(2, ba) # 2 + writeVarUInt(0, ba) # 0 + ba.extend([0] * 4) # bucket_num + + +def sendEmptyBlock(s): + ba = bytearray() + writeVarUInt(2, ba) # Data + writeStringBinary("", ba) + serializeBlockInfo(ba) + writeVarUInt(0, ba) # rows + writeVarUInt(0, ba) # columns + s.sendall(ba) + + +def assertPacket(packet, expected): + assert packet == expected, packet + + +def readHeader(s): + packet_type = readVarUInt(s) + if packet_type == 2: # Exception + raise RuntimeError(readException(s)) + assertPacket(packet_type, 1) # Data + + readStringBinary(s) # external table name + # BlockInfo + assertPacket(readVarUInt(s), 1) # 1 + assertPacket(readUInt8(s), 0) # is_overflows + assertPacket(readVarUInt(s), 2) # 2 + assertPacket(readUInt32(s), 4294967295) # bucket_num + assertPacket(readVarUInt(s), 0) # 0 + columns = readVarUInt(s) # rows + rows = readVarUInt(s) # columns + print("Rows {} Columns {}".format(rows, columns)) + for _ in range(columns): + col_name = readStringBinary(s) + type_name = readStringBinary(s) + print("Column {} type {}".format(col_name, type_name)) + + +def readException(s): + code = readUInt32(s) + name = readStringBinary(s) + text = readStringBinary(s) + readStringBinary(s) # trace + assertPacket(readUInt8(s), 0) # has_nested + return "code {}: {}".format(code, text.replace("DB::Exception:", "")) def insertValidLowCardinalityRow(): - with TCPClient() as client: - client.sendQuery( + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: + s.settimeout(30) + s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) + sendHello(s) + receiveHello(s) + sendQuery( + s, "insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV".format( CLICKHOUSE_DATABASE ), ) # external tables - client.sendEmptyBlock() - client.readHeader() + sendEmptyBlock(s) + readHeader(s) # Data ba = bytearray() @@ -46,25 +240,31 @@ def insertValidLowCardinalityRow(): writeStringBinary("hello", ba) # key ba.extend([1] + [0] * 7) # num_indexes ba.extend([0] * 8) # UInt64 index (0 for 'hello') - client.send(ba) + s.sendall(ba) # Fin block - client.sendEmptyBlock() + sendEmptyBlock(s) - assertPacket(client.readVarUInt(), 5) # End of stream + assertPacket(readVarUInt(s), 5) # End of stream + s.close() def insertLowCardinalityRowWithIndexOverflow(): - with TCPClient() as client: - client.sendQuery( + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: + s.settimeout(30) + s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) + sendHello(s) + receiveHello(s) + sendQuery( + s, "insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV".format( CLICKHOUSE_DATABASE ), ) # external tables - client.sendEmptyBlock() - client.readHeader() + sendEmptyBlock(s) + readHeader(s) # Data ba = bytearray() @@ -83,23 +283,29 @@ def insertLowCardinalityRowWithIndexOverflow(): writeStringBinary("hello", ba) # key ba.extend([1] + [0] * 7) # num_indexes ba.extend([0] * 7 + [1]) # UInt64 index (overflow) - client.send(ba) + s.sendall(ba) - assertPacket(client.readVarUInt(), 2) # Exception - print(client.readException()) + assertPacket(readVarUInt(s), 2) + print(readException(s)) + s.close() def insertLowCardinalityRowWithIncorrectDictType(): - with TCPClient() as client: - client.sendQuery( + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: + s.settimeout(30) + s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) + sendHello(s) + receiveHello(s) + sendQuery( + s, "insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV".format( CLICKHOUSE_DATABASE ), ) # external tables - client.sendEmptyBlock() - client.readHeader() + sendEmptyBlock(s) + readHeader(s) # Data ba = bytearray() @@ -118,23 +324,29 @@ def insertLowCardinalityRowWithIncorrectDictType(): writeStringBinary("hello", ba) # key ba.extend([1] + [0] * 7) # num_indexes ba.extend([0] * 8) # UInt64 index (overflow) - client.send(ba) + s.sendall(ba) - assertPacket(client.readVarUInt(), 2) # Exception - print(client.readException()) + assertPacket(readVarUInt(s), 2) + print(readException(s)) + s.close() def insertLowCardinalityRowWithIncorrectAdditionalKeys(): - with TCPClient() as client: - client.sendQuery( + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: + s.settimeout(30) + s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) + sendHello(s) + receiveHello(s) + sendQuery( + s, "insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV".format( CLICKHOUSE_DATABASE ), ) # external tables - client.sendEmptyBlock() - client.readHeader() + sendEmptyBlock(s) + readHeader(s) # Data ba = bytearray() @@ -153,10 +365,11 @@ def insertLowCardinalityRowWithIncorrectAdditionalKeys(): writeStringBinary("hello", ba) # key ba.extend([1] + [0] * 7) # num_indexes ba.extend([0] * 8) # UInt64 index (0 for 'hello') - client.send(ba) + s.sendall(ba) - assertPacket(client.readVarUInt(), 2) # Exception - print(client.readException()) + assertPacket(readVarUInt(s), 2) + print(readException(s)) + s.close() def main(): diff --git a/tests/queries/0_stateless/02210_processors_profile_log.reference b/tests/queries/0_stateless/02210_processors_profile_log.reference index f480236111f..41543d0706a 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log.reference +++ b/tests/queries/0_stateless/02210_processors_profile_log.reference @@ -38,5 +38,4 @@ LazyOutputFormat 1 1 1 0 0 LimitsCheckingTransform 1 1 1 1 1 NullSource 1 0 0 0 0 NullSource 1 0 0 0 0 -NullSource 0 0 0 0 0 SourceFromSingleChunk 1 0 0 1 1 diff --git a/tests/queries/0_stateless/02458_insert_select_progress_tcp.python b/tests/queries/0_stateless/02458_insert_select_progress_tcp.python index fdc64a8dba8..92240e109c1 100644 --- a/tests/queries/0_stateless/02458_insert_select_progress_tcp.python +++ b/tests/queries/0_stateless/02458_insert_select_progress_tcp.python @@ -1,30 +1,188 @@ #!/usr/bin/env python3 -import json +import socket import os -import sys +import uuid +import json -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, "helpers")) +CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "127.0.0.1") +CLICKHOUSE_PORT = int(os.environ.get("CLICKHOUSE_PORT_TCP", "900000")) +CLICKHOUSE_DATABASE = os.environ.get("CLICKHOUSE_DATABASE", "default") +CLIENT_NAME = "simple native protocol" -from tcp_client import TCPClient + +def writeVarUInt(x, ba): + for _ in range(0, 9): + byte = x & 0x7F + if x > 0x7F: + byte |= 0x80 + + ba.append(byte) + + x >>= 7 + if x == 0: + return + + +def writeStringBinary(s, ba): + b = bytes(s, "utf-8") + writeVarUInt(len(s), ba) + ba.extend(b) + + +def readStrict(s, size=1): + res = bytearray() + while size: + cur = s.recv(size) + # if not res: + # raise "Socket is closed" + size -= len(cur) + res.extend(cur) + + return res + + +def readUInt(s, size=1): + res = readStrict(s, size) + val = 0 + for i in range(len(res)): + val += res[i] << (i * 8) + return val + + +def readUInt8(s): + return readUInt(s) + + +def readUInt16(s): + return readUInt(s, 2) + + +def readUInt32(s): + return readUInt(s, 4) + + +def readUInt64(s): + return readUInt(s, 8) + + +def readVarUInt(s): + x = 0 + for i in range(9): + byte = readStrict(s)[0] + x |= (byte & 0x7F) << (7 * i) + + if not byte & 0x80: + return x + + return x + + +def readStringBinary(s): + size = readVarUInt(s) + s = readStrict(s, size) + return s.decode("utf-8") + + +def sendHello(s): + ba = bytearray() + writeVarUInt(0, ba) # Hello + writeStringBinary(CLIENT_NAME, ba) + writeVarUInt(21, ba) + writeVarUInt(9, ba) + writeVarUInt(54449, ba) + writeStringBinary(CLICKHOUSE_DATABASE, ba) # database + writeStringBinary("default", ba) # user + writeStringBinary("", ba) # pwd + s.sendall(ba) + + +def receiveHello(s): + p_type = readVarUInt(s) + assert p_type == 0 # Hello + server_name = readStringBinary(s) + # print("Server name: ", server_name) + server_version_major = readVarUInt(s) + # print("Major: ", server_version_major) + server_version_minor = readVarUInt(s) + # print("Minor: ", server_version_minor) + server_revision = readVarUInt(s) + # print("Revision: ", server_revision) + server_timezone = readStringBinary(s) + # print("Timezone: ", server_timezone) + server_display_name = readStringBinary(s) + # print("Display name: ", server_display_name) + server_version_patch = readVarUInt(s) + # print("Version patch: ", server_version_patch) + + +def serializeClientInfo(ba, query_id): + writeStringBinary("default", ba) # initial_user + writeStringBinary(query_id, ba) # initial_query_id + writeStringBinary("127.0.0.1:9000", ba) # initial_address + ba.extend([0] * 8) # initial_query_start_time_microseconds + ba.append(1) # TCP + writeStringBinary("os_user", ba) # os_user + writeStringBinary("client_hostname", ba) # client_hostname + writeStringBinary(CLIENT_NAME, ba) # client_name + writeVarUInt(21, ba) + writeVarUInt(9, ba) + writeVarUInt(54449, ba) + writeStringBinary("", ba) # quota_key + writeVarUInt(0, ba) # distributed_depth + writeVarUInt(1, ba) # client_version_patch + ba.append(0) # No telemetry + + +def sendQuery(s, query): + ba = bytearray() + query_id = uuid.uuid4().hex + writeVarUInt(1, ba) # query + writeStringBinary(query_id, ba) + + ba.append(1) # INITIAL_QUERY + + # client info + serializeClientInfo(ba, query_id) + + writeStringBinary("", ba) # No settings + writeStringBinary("", ba) # No interserver secret + writeVarUInt(2, ba) # Stage - Complete + ba.append(0) # No compression + writeStringBinary(query, ba) # query, finally + s.sendall(ba) + + +def serializeBlockInfo(ba): + writeVarUInt(1, ba) # 1 + ba.append(0) # is_overflows + writeVarUInt(2, ba) # 2 + writeVarUInt(0, ba) # 0 + ba.extend([0] * 4) # bucket_num + + +def sendEmptyBlock(s): + ba = bytearray() + writeVarUInt(2, ba) # Data + writeStringBinary("", ba) + serializeBlockInfo(ba) + writeVarUInt(0, ba) # rows + writeVarUInt(0, ba) # columns + s.sendall(ba) + + +def assertPacket(packet, expected): + assert packet == expected, packet class Progress: - def __init__( - self, - read_rows=0, - read_bytes=0, - total_rows_to_read=0, - written_rows=0, - written_bytes=0, - ): + def __init__(self): # NOTE: this is done in ctor to initialize __dict__ - self.read_rows = read_rows - self.read_bytes = read_bytes - self.total_rows_to_read = total_rows_to_read - self.written_rows = written_rows - self.written_bytes = written_bytes + self.read_rows = 0 + self.read_bytes = 0 + self.total_rows_to_read = 0 + self.written_rows = 0 + self.written_bytes = 0 def __str__(self): return json.dumps(self.__dict__) @@ -37,6 +195,13 @@ class Progress: self.written_bytes += b.written_bytes return self + def readPacket(self, s): + self.read_rows += readVarUInt(s) + self.read_bytes += readVarUInt(s) + self.total_rows_to_read += readVarUInt(s) + self.written_rows += readVarUInt(s) + self.written_bytes += readVarUInt(s) + def __bool__(self): return ( self.read_rows > 0 @@ -47,25 +212,52 @@ class Progress: ) +def readProgress(s): + packet_type = readVarUInt(s) + if packet_type == 2: # Exception + raise RuntimeError(readException(s)) + + if packet_type == 5: # End stream + return None + + assertPacket(packet_type, 3) # Progress + + progress = Progress() + progress.readPacket(s) + return progress + + +def readException(s): + code = readUInt32(s) + name = readStringBinary(s) + text = readStringBinary(s) + readStringBinary(s) # trace + assertPacket(readUInt8(s), 0) # has_nested + return "code {}: {}".format(code, text.replace("DB::Exception:", "")) + + def main(): - with TCPClient() as client: + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: + s.settimeout(30) + s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) + sendHello(s) + receiveHello(s) # For 1 second sleep and 1000ms of interactive_delay we definitelly should have non zero progress packet. # NOTE: interactive_delay=0 cannot be used since in this case CompletedPipelineExecutor will not call cancelled callback. - client.sendQuery( + sendQuery( + s, "insert into function null('_ Int') select sleep(1) from numbers(2) settings max_block_size=1, interactive_delay=1000", ) # external tables - client.sendEmptyBlock() + sendEmptyBlock(s) summary_progress = Progress() non_empty_progress_packets = 0 while True: - progress_info = client.readProgress() - if progress_info is None: + progress = readProgress(s) + if progress is None: break - - progress = Progress(*progress_info) summary_progress += progress if progress: non_empty_progress_packets += 1 @@ -76,6 +268,8 @@ def main(): # - 1 or 2 for each SELECT block assert non_empty_progress_packets in (3, 4), f"{non_empty_progress_packets=:}" + s.close() + if __name__ == "__main__": main() diff --git a/tests/queries/0_stateless/02750_settings_alias_tcp_protocol.python b/tests/queries/0_stateless/02750_settings_alias_tcp_protocol.python index 1736807410f..48b27d434ec 100644 --- a/tests/queries/0_stateless/02750_settings_alias_tcp_protocol.python +++ b/tests/queries/0_stateless/02750_settings_alias_tcp_protocol.python @@ -1,23 +1,217 @@ #!/usr/bin/env python3 - +import socket import os -import sys +import uuid +import json -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, "helpers")) +CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "127.0.0.1") +CLICKHOUSE_PORT = int(os.environ.get("CLICKHOUSE_PORT_TCP", "900000")) +CLICKHOUSE_DATABASE = os.environ.get("CLICKHOUSE_DATABASE", "default") +CLIENT_NAME = "simple native protocol" -from tcp_client import TCPClient + +def writeVarUInt(x, ba): + for _ in range(0, 9): + byte = x & 0x7F + if x > 0x7F: + byte |= 0x80 + + ba.append(byte) + + x >>= 7 + if x == 0: + return + + +def writeStringBinary(s, ba): + b = bytes(s, "utf-8") + writeVarUInt(len(s), ba) + ba.extend(b) + + +def readStrict(s, size=1): + res = bytearray() + while size: + cur = s.recv(size) + # if not res: + # raise "Socket is closed" + size -= len(cur) + res.extend(cur) + + return res + + +def readUInt(s, size=1): + res = readStrict(s, size) + val = 0 + for i in range(len(res)): + val += res[i] << (i * 8) + return val + + +def readUInt8(s): + return readUInt(s) + + +def readUInt16(s): + return readUInt(s, 2) + + +def readUInt32(s): + return readUInt(s, 4) + + +def readUInt64(s): + return readUInt(s, 8) + + +def readVarUInt(s): + x = 0 + for i in range(9): + byte = readStrict(s)[0] + x |= (byte & 0x7F) << (7 * i) + + if not byte & 0x80: + return x + + return x + + +def readStringBinary(s): + size = readVarUInt(s) + s = readStrict(s, size) + return s.decode("utf-8") + + +def sendHello(s): + ba = bytearray() + writeVarUInt(0, ba) # Hello + writeStringBinary(CLIENT_NAME, ba) + writeVarUInt(21, ba) + writeVarUInt(9, ba) + writeVarUInt(54449, ba) + writeStringBinary(CLICKHOUSE_DATABASE, ba) # database + writeStringBinary("default", ba) # user + writeStringBinary("", ba) # pwd + s.sendall(ba) + + +def receiveHello(s): + p_type = readVarUInt(s) + assert p_type == 0 # Hello + _server_name = readStringBinary(s) + _server_version_major = readVarUInt(s) + _server_version_minor = readVarUInt(s) + _server_revision = readVarUInt(s) + _server_timezone = readStringBinary(s) + _server_display_name = readStringBinary(s) + _server_version_patch = readVarUInt(s) + + +def serializeClientInfo(ba, query_id): + writeStringBinary("default", ba) # initial_user + writeStringBinary(query_id, ba) # initial_query_id + writeStringBinary("127.0.0.1:9000", ba) # initial_address + ba.extend([0] * 8) # initial_query_start_time_microseconds + ba.append(1) # TCP + writeStringBinary("os_user", ba) # os_user + writeStringBinary("client_hostname", ba) # client_hostname + writeStringBinary(CLIENT_NAME, ba) # client_name + writeVarUInt(21, ba) + writeVarUInt(9, ba) + writeVarUInt(54449, ba) + writeStringBinary("", ba) # quota_key + writeVarUInt(0, ba) # distributed_depth + writeVarUInt(1, ba) # client_version_patch + ba.append(0) # No telemetry + + +def sendQuery(s, query, settings): + ba = bytearray() + query_id = uuid.uuid4().hex + writeVarUInt(1, ba) # query + writeStringBinary(query_id, ba) + + ba.append(1) # INITIAL_QUERY + + # client info + serializeClientInfo(ba, query_id) + + # Settings + for key, value in settings.items(): + writeStringBinary(key, ba) + writeVarUInt(1, ba) # is_important + writeStringBinary(str(value), ba) + writeStringBinary("", ba) # End of settings + + writeStringBinary("", ba) # No interserver secret + writeVarUInt(2, ba) # Stage - Complete + ba.append(0) # No compression + writeStringBinary(query, ba) # query, finally + s.sendall(ba) + + +def serializeBlockInfo(ba): + writeVarUInt(1, ba) # 1 + ba.append(0) # is_overflows + writeVarUInt(2, ba) # 2 + writeVarUInt(0, ba) # 0 + ba.extend([0] * 4) # bucket_num + + +def sendEmptyBlock(s): + ba = bytearray() + writeVarUInt(2, ba) # Data + writeStringBinary("", ba) + serializeBlockInfo(ba) + writeVarUInt(0, ba) # rows + writeVarUInt(0, ba) # columns + s.sendall(ba) + + +def assertPacket(packet, expected): + assert packet == expected, "Got: {}, expected: {}".format(packet, expected) + + +def readResponse(s): + packet_type = readVarUInt(s) + if packet_type == 2: # Exception + raise RuntimeError(readException(s)) + + if packet_type == 1: # Data + return None + if packet_type == 3: # Progress + return None + if packet_type == 5: # End stream + return None + + raise RuntimeError("Unexpected packet: {}".format(packet_type)) + + +def readException(s): + code = readUInt32(s) + _name = readStringBinary(s) + text = readStringBinary(s) + readStringBinary(s) # trace + assertPacket(readUInt8(s), 0) # has_nested + return "code {}: {}".format(code, text.replace("DB::Exception:", "")) def main(): - with TCPClient() as client: - client.sendQuery("select 1", {"replication_alter_partitions_sync": 1}) + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: + s.settimeout(30) + s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) + sendHello(s) + receiveHello(s) + sendQuery(s, "select 1", {"replication_alter_partitions_sync": 1}) # external tables - client.sendEmptyBlock() + sendEmptyBlock(s) - while client.readResponse() is not None: + while readResponse(s) is not None: pass + + s.close() print("OK") diff --git a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.python b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.python deleted file mode 100755 index 61ba0e14605..00000000000 --- a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.python +++ /dev/null @@ -1,95 +0,0 @@ -#!/usr/bin/env python3 -# -*- coding: utf-8 -*- - - -import os -import sys - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, "helpers")) - -from tcp_client import TCPClient - - -def run_query_without_errors(query, support_partial_result): - with TCPClient() as client: - client.sendQuery(query, settings={"allow_experimental_partial_result": True}) - - # external tables - client.sendEmptyBlock() - client.readHeader() - - # Partial result - partial_result = client.readDataWithoutProgress()[0] - if support_partial_result: - assert ( - len(partial_result.value) > 0 - ), "Expected at least one block with a non-empty partial result before getting the full result" - - while True: - assert all( - a >= b - for a, b in zip(partial_result.value, partial_result.value[1:]) - ), "Partial result always should be sorted for this test" - - new_partial_result = client.readDataWithoutProgress( - need_print_info=False - )[0] - if len(new_partial_result.value) == 0: - break - - data_size = len(partial_result.value) - assert all( - partial_result.value[i] <= new_partial_result.value[i] - for i in range(data_size) - ), f"New partial result values should always be greater then old one because a new block contains more information about the full data. New result {new_partial_result}. Previous result {partial_result}" - - partial_result = new_partial_result - else: - block_rows = len(partial_result.value) - assert ( - block_rows == 0 - ), f"Expected only empty partial result block before getting the full result, but block has {block_rows} rows" - - # Full result - full_result = client.readDataWithoutProgress()[0] - - data_size = len(partial_result.value) - assert all( - partial_result.value[i] <= full_result.value[i] for i in range(data_size) - ), f"Full result values should always be greater then partial result values. Full result {full_result}. Partial result {partial_result}" - - for result in full_result.value: - print(result) - - -def main(): - rows_number = 2e7 + 1 - - # Request with partial result limit less then full limit - run_query_without_errors( - f"SELECT number FROM numbers_mt({rows_number}) ORDER BY -number LIMIT 5 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 3", - support_partial_result=True, - ) - - # Request with partial result limit greater then full limit - run_query_without_errors( - f"SELECT number FROM numbers_mt({rows_number}) ORDER BY -number LIMIT 3 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 5", - support_partial_result=True, - ) - - # Request with OFFSET - run_query_without_errors( - f"SELECT number FROM numbers_mt({rows_number}) ORDER BY -number LIMIT 3 OFFSET 1 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 5", - support_partial_result=True, - ) - - # Request with OFFSET greater then partial result limit (partial result pipeline use blocks with less then OFFSET, so there will be no elements in block after LimitPartialResultTransform) - run_query_without_errors( - f"SELECT number FROM numbers_mt({rows_number}) ORDER BY -number LIMIT 3 OFFSET 15 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 5", - support_partial_result=False, - ) - - -if __name__ == "__main__": - main() diff --git a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.reference b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.reference deleted file mode 100644 index dd3a343560f..00000000000 --- a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.reference +++ /dev/null @@ -1,38 +0,0 @@ -Rows 0 Columns 1 -Column number type UInt64 -Rows 3 Columns 1 -Column number type UInt64 -Rows 5 Columns 1 -Column number type UInt64 -20000000 -19999999 -19999998 -19999997 -19999996 -Rows 0 Columns 1 -Column number type UInt64 -Rows 3 Columns 1 -Column number type UInt64 -Rows 3 Columns 1 -Column number type UInt64 -20000000 -19999999 -19999998 -Rows 0 Columns 1 -Column number type UInt64 -Rows 3 Columns 1 -Column number type UInt64 -Rows 3 Columns 1 -Column number type UInt64 -19999999 -19999998 -19999997 -Rows 0 Columns 1 -Column number type UInt64 -Rows 0 Columns 1 -Column number type UInt64 -Rows 3 Columns 1 -Column number type UInt64 -19999985 -19999984 -19999983 diff --git a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.sh b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.sh deleted file mode 100755 index 1ed15197dbf..00000000000 --- a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.sh +++ /dev/null @@ -1,8 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -# We should have correct env vars from shell_config.sh to run this test -python3 "$CURDIR"/02833_partial_sorting_result_during_query_execution.python diff --git a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.python b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.python deleted file mode 100644 index a33c714e89c..00000000000 --- a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.python +++ /dev/null @@ -1,129 +0,0 @@ -#!/usr/bin/env python3 -# -*- coding: utf-8 -*- - - -import os -import sys - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, "helpers")) - -from tcp_client import TCPClient - - -def get_keys(results): - return [key for key, _ in results] - - -def check_new_result(new_results, old_results, invariants, rows_limit): - if rows_limit is not None: - assert ( - len(new_results[0].value) <= rows_limit - ), f"Result should have no more then {rows_limit} rows. But it has {len(new_results[0].value)} rows" - - for new_result, old_result in zip(new_results, old_results): - assert ( - new_result.key == old_result.key - ), f"Keys in blocks should be in the same order. Full results keys {get_keys(full_results)}. Partial results keys {get_keys(partial_results)}" - - key = new_result.key - if key in invariants: - new_value = new_result.value - old_value = old_result.value - assert invariants[key]( - old_value, new_value - ), f"Problem with the invariant between new and old result for key: {key}. New value {new_value}. Old value {old_value}" - - -def run_query_without_errors( - query, support_partial_result, invariants=None, rows_limit=None -): - if invariants is None: - invariants = {} - - with TCPClient() as client: - client.sendQuery(query, settings={"allow_experimental_partial_result": True}) - - # external tables - client.sendEmptyBlock() - client.readHeader() - - # Partial result - partial_results = client.readDataWithoutProgress() - if support_partial_result: - assert ( - len(partial_results) > 0 and len(partial_results[0].value) > 0 - ), "Expected at least one block with a non-empty partial result before getting the full result" - while True: - new_partial_results = client.readDataWithoutProgress( - need_print_info=False - ) - if len(new_partial_results[0].value) == 0: - break - - check_new_result( - new_partial_results, partial_results, invariants, rows_limit - ) - partial_results = new_partial_results - else: - block_rows = len(partial_results[0].value) - assert ( - block_rows == 0 - ), f"Expected only empty partial result block before getting the full result, but block has {block_rows} rows" - - # Full result - full_results = client.readDataWithoutProgress() - if support_partial_result: - check_new_result(full_results, partial_results, invariants, rows_limit) - - for data in full_results: - if isinstance(data.value[0], int): - print(data.key, data.value) - - -def supported_scenarios_without_key(): - rows_number = 2e7 + 1 - - # Simple aggregation query - query = f"select median(number), stddevSamp(number), stddevPop(number), max(number), min(number), any(number), count(number), avg(number), sum(number) from numbers_mt({rows_number}) settings max_threads = 1, partial_result_update_duration_ms = 1" - invariants = { - "median(number)": lambda old_value, new_value: old_value <= new_value, - "max(number)": lambda old_value, new_value: old_value <= new_value, - "min(number)": lambda old_value, new_value: old_value >= new_value, - "count(number)": lambda old_value, new_value: old_value <= new_value, - "avg(number)": lambda old_value, new_value: old_value <= new_value, - "sum(number)": lambda old_value, new_value: old_value <= new_value, - } - run_query_without_errors( - query, support_partial_result=True, invariants=invariants, rows_limit=1 - ) - - # Aggregation query with a nested ORDER BY subquery - query = f"select median(number), stddevSamp(number), stddevPop(number), max(number), min(number), any(number), count(number), avg(number), sum(number) FROM (SELECT number FROM numbers_mt({rows_number}) ORDER BY -number LIMIT 3) settings max_threads = 1, partial_result_update_duration_ms=1" - - # Aggregation receives small partial result blocks from ORDER BY which always sends blocks with bigger values - invariants["min(number)"] = lambda old_value, new_value: old_value <= new_value - run_query_without_errors( - query, support_partial_result=True, invariants=invariants, rows_limit=1 - ) - - -def unsupported_scenarios(): - rows_number = 2e7 + 1 - - # Currently aggregator for partial result supports only single thread aggregation without key - # Update test when multithreading or aggregation with GROUP BY will be supported for partial result updates - multithread_query = f"select sum(number) from numbers_mt({rows_number}) settings max_threads = 2, partial_result_update_duration_ms = 100" - run_query_without_errors(multithread_query, support_partial_result=False) - - group_with_key_query = f"select mod2, sum(number) from numbers_mt({rows_number}) group by number % 2 as mod2 settings max_threads = 1, partial_result_update_duration_ms = 100" - run_query_without_errors(group_with_key_query, support_partial_result=False) - - -def main(): - supported_scenarios_without_key() - unsupported_scenarios() - - -if __name__ == "__main__": - main() diff --git a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.reference b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.reference deleted file mode 100644 index aea61fad42f..00000000000 --- a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.reference +++ /dev/null @@ -1,88 +0,0 @@ -Rows 0 Columns 9 -Column median(number) type Float64 -Column stddevSamp(number) type Float64 -Column stddevPop(number) type Float64 -Column max(number) type UInt64 -Column min(number) type UInt64 -Column any(number) type UInt64 -Column count(number) type UInt64 -Column avg(number) type Float64 -Column sum(number) type UInt64 -Rows 1 Columns 9 -Column median(number) type Float64 -Column stddevSamp(number) type Float64 -Column stddevPop(number) type Float64 -Column max(number) type UInt64 -Column min(number) type UInt64 -Column any(number) type UInt64 -Column count(number) type UInt64 -Column avg(number) type Float64 -Column sum(number) type UInt64 -Rows 1 Columns 9 -Column median(number) type Float64 -Column stddevSamp(number) type Float64 -Column stddevPop(number) type Float64 -Column max(number) type UInt64 -Column min(number) type UInt64 -Column any(number) type UInt64 -Column count(number) type UInt64 -Column avg(number) type Float64 -Column sum(number) type UInt64 -max(number) [20000000] -min(number) [0] -any(number) [0] -count(number) [20000001] -sum(number) [200000010000000] -Rows 0 Columns 9 -Column median(number) type Float64 -Column stddevSamp(number) type Float64 -Column stddevPop(number) type Float64 -Column max(number) type UInt64 -Column min(number) type UInt64 -Column any(number) type UInt64 -Column count(number) type UInt64 -Column avg(number) type Float64 -Column sum(number) type UInt64 -Rows 1 Columns 9 -Column median(number) type Float64 -Column stddevSamp(number) type Float64 -Column stddevPop(number) type Float64 -Column max(number) type UInt64 -Column min(number) type UInt64 -Column any(number) type UInt64 -Column count(number) type UInt64 -Column avg(number) type Float64 -Column sum(number) type UInt64 -Rows 1 Columns 9 -Column median(number) type Float64 -Column stddevSamp(number) type Float64 -Column stddevPop(number) type Float64 -Column max(number) type UInt64 -Column min(number) type UInt64 -Column any(number) type UInt64 -Column count(number) type UInt64 -Column avg(number) type Float64 -Column sum(number) type UInt64 -max(number) [20000000] -min(number) [19999998] -any(number) [20000000] -count(number) [3] -sum(number) [59999997] -Rows 0 Columns 1 -Column sum(number) type UInt64 -Rows 0 Columns 1 -Column sum(number) type UInt64 -Rows 1 Columns 1 -Column sum(number) type UInt64 -sum(number) [200000010000000] -Rows 0 Columns 2 -Column mod2 type UInt8 -Column sum(number) type UInt64 -Rows 0 Columns 2 -Column mod2 type UInt8 -Column sum(number) type UInt64 -Rows 2 Columns 2 -Column mod2 type UInt8 -Column sum(number) type UInt64 -mod2 [0, 1] -sum(number) [100000010000000, 100000000000000] diff --git a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.sh b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.sh deleted file mode 100755 index e70a3c53ec4..00000000000 --- a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.sh +++ /dev/null @@ -1,8 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -# We should have correct env vars from shell_config.sh to run this test -python3 "$CURDIR"/02834_partial_aggregating_result_during_query_execution.python diff --git a/tests/queries/0_stateless/helpers/tcp_client.py b/tests/queries/0_stateless/helpers/tcp_client.py deleted file mode 100644 index fdc4ab28e04..00000000000 --- a/tests/queries/0_stateless/helpers/tcp_client.py +++ /dev/null @@ -1,313 +0,0 @@ -import socket -import os -import uuid -import struct - -CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "127.0.0.1") -CLICKHOUSE_PORT = int(os.environ.get("CLICKHOUSE_PORT_TCP", "900000")) -CLICKHOUSE_DATABASE = os.environ.get("CLICKHOUSE_DATABASE", "default") -CLIENT_NAME = "simple native protocol" - - -def writeVarUInt(x, ba): - for _ in range(0, 9): - byte = x & 0x7F - if x > 0x7F: - byte |= 0x80 - - ba.append(byte) - - x >>= 7 - if x == 0: - return - - -def writeStringBinary(s, ba): - b = bytes(s, "utf-8") - writeVarUInt(len(s), ba) - ba.extend(b) - - -def serializeClientInfo(ba, query_id): - writeStringBinary("default", ba) # initial_user - writeStringBinary(query_id, ba) # initial_query_id - writeStringBinary("127.0.0.1:9000", ba) # initial_address - ba.extend([0] * 8) # initial_query_start_time_microseconds - ba.append(1) # TCP - writeStringBinary("os_user", ba) # os_user - writeStringBinary("client_hostname", ba) # client_hostname - writeStringBinary(CLIENT_NAME, ba) # client_name - writeVarUInt(21, ba) - writeVarUInt(9, ba) - writeVarUInt(54449, ba) - writeStringBinary("", ba) # quota_key - writeVarUInt(0, ba) # distributed_depth - writeVarUInt(1, ba) # client_version_patch - ba.append(0) # No telemetry - - -def serializeBlockInfo(ba): - writeVarUInt(1, ba) # 1 - ba.append(0) # is_overflows - writeVarUInt(2, ba) # 2 - writeVarUInt(0, ba) # 0 - ba.extend([0] * 4) # bucket_num - - -def assertPacket(packet, expected): - assert packet == expected, "Got: {}, expected: {}".format(packet, expected) - - -class Data(object): - def __init__(self, key, value): - self.key = key - self.value = value - - -class TCPClient(object): - def __init__(self, timeout=30): - self.timeout = timeout - self.socket = None - - def __enter__(self): - self.socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - self.socket.settimeout(self.timeout) - self.socket.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) - - self.sendHello() - self.receiveHello() - - return self - - def __exit__(self, exc_type, exc_value, traceback): - if self.socket: - self.socket.close() - - def readStrict(self, size=1): - res = bytearray() - while size: - cur = self.socket.recv(size) - # if not res: - # raise "Socket is closed" - size -= len(cur) - res.extend(cur) - - return res - - def readUInt(self, size=1): - res = self.readStrict(size) - val = 0 - for i in range(len(res)): - val += res[i] << (i * 8) - return val - - def readUInt8(self): - return self.readUInt() - - def readUInt16(self): - return self.readUInt(2) - - def readUInt32(self): - return self.readUInt(4) - - def readUInt64(self): - return self.readUInt(8) - - def readFloat16(self): - return struct.unpack("e", self.readStrict(2)) - - def readFloat32(self): - return struct.unpack("f", self.readStrict(4)) - - def readFloat64(self): - return struct.unpack("d", self.readStrict(8)) - - def readVarUInt(self): - x = 0 - for i in range(9): - byte = self.readStrict()[0] - x |= (byte & 0x7F) << (7 * i) - - if not byte & 0x80: - return x - - return x - - def readStringBinary(self): - size = self.readVarUInt() - s = self.readStrict(size) - return s.decode("utf-8") - - def send(self, byte_array): - self.socket.sendall(byte_array) - - def sendHello(self): - ba = bytearray() - writeVarUInt(0, ba) # Hello - writeStringBinary(CLIENT_NAME, ba) - writeVarUInt(21, ba) - writeVarUInt(9, ba) - writeVarUInt(54449, ba) - writeStringBinary(CLICKHOUSE_DATABASE, ba) # database - writeStringBinary("default", ba) # user - writeStringBinary("", ba) # pwd - self.send(ba) - - def receiveHello(self): - p_type = self.readVarUInt() - assert p_type == 0 # Hello - _server_name = self.readStringBinary() - _server_version_major = self.readVarUInt() - _server_version_minor = self.readVarUInt() - _server_revision = self.readVarUInt() - _server_timezone = self.readStringBinary() - _server_display_name = self.readStringBinary() - _server_version_patch = self.readVarUInt() - - def sendQuery(self, query, settings=None): - if settings == None: - settings = {} # No settings - - ba = bytearray() - query_id = uuid.uuid4().hex - writeVarUInt(1, ba) # query - writeStringBinary(query_id, ba) - - ba.append(1) # INITIAL_QUERY - - # client info - serializeClientInfo(ba, query_id) - - # Settings - for key, value in settings.items(): - writeStringBinary(key, ba) - writeVarUInt(1, ba) # is_important - writeStringBinary(str(value), ba) - writeStringBinary("", ba) # End of settings - - writeStringBinary("", ba) # No interserver secret - writeVarUInt(2, ba) # Stage - Complete - ba.append(0) # No compression - writeStringBinary(query, ba) # query, finally - self.send(ba) - - def sendEmptyBlock(self): - ba = bytearray() - writeVarUInt(2, ba) # Data - writeStringBinary("", ba) - serializeBlockInfo(ba) - writeVarUInt(0, ba) # rows - writeVarUInt(0, ba) # columns - self.send(ba) - - def readException(self): - code = self.readUInt32() - _name = self.readStringBinary() - text = self.readStringBinary() - self.readStringBinary() # trace - assertPacket(self.readUInt8(), 0) # has_nested - return "code {}: {}".format(code, text.replace("DB::Exception:", "")) - - def readPacketType(self): - packet_type = self.readVarUInt() - if packet_type == 2: # Exception - raise RuntimeError(self.readException()) - - return packet_type - - def readResponse(self): - packet_type = self.readPacketType() - if packet_type == 1: # Data - return None - if packet_type == 3: # Progress - return None - if packet_type == 5: # End stream - return None - - raise RuntimeError("Unexpected packet: {}".format(packet_type)) - - def readProgressData(self): - read_rows = self.readVarUInt() - read_bytes = self.readVarUInt() - total_rows_to_read = self.readVarUInt() - written_rows = self.readVarUInt() - written_bytes = self.readVarUInt() - - return read_rows, read_bytes, total_rows_to_read, written_rows, written_bytes - - def readProgress(self): - packet_type = self.readPacketType() - if packet_type == 5: # End stream - return None - assertPacket(packet_type, 3) # Progress - return self.readProgressData() - - def readHeaderInfo(self): - self.readStringBinary() # external table name - # BlockInfo - assertPacket(self.readVarUInt(), 1) # field number 1 - assertPacket(self.readUInt8(), 0) # is_overflows - assertPacket(self.readVarUInt(), 2) # field number 2 - assertPacket(self.readUInt32(), 4294967295) # bucket_num - assertPacket(self.readVarUInt(), 0) # 0 - columns = self.readVarUInt() # rows - rows = self.readVarUInt() # columns - - return columns, rows - - def readHeader(self): - packet_type = self.readPacketType() - assertPacket(packet_type, 1) # Data - - columns, rows = self.readHeaderInfo() - print("Rows {} Columns {}".format(rows, columns)) - for _ in range(columns): - col_name = self.readStringBinary() - type_name = self.readStringBinary() - print("Column {} type {}".format(col_name, type_name)) - - def readRow(self, row_type, rows): - supported_row_types = { - "UInt8": self.readUInt8, - "UInt16": self.readUInt16, - "UInt32": self.readUInt32, - "UInt64": self.readUInt64, - "Float16": self.readFloat16, - "Float32": self.readFloat32, - "Float64": self.readFloat64, - } - if row_type in supported_row_types: - read_type = supported_row_types[row_type] - row = [read_type() for _ in range(rows)] - return row - else: - raise RuntimeError( - "Current python version of tcp client doesn't support the following type of row: {}".format( - row_type - ) - ) - - def readDataWithoutProgress(self, need_print_info=True): - packet_type = self.readPacketType() - while packet_type == 3: # Progress - self.readProgressData() - packet_type = self.readPacketType() - - if packet_type == 5: # End stream - return None - assertPacket(packet_type, 1) # Data - - columns, rows = self.readHeaderInfo() - data = [] - if need_print_info: - print("Rows {} Columns {}".format(rows, columns)) - - for _ in range(columns): - col_name = self.readStringBinary() - type_name = self.readStringBinary() - if need_print_info: - print("Column {} type {}".format(col_name, type_name)) - - data.append(Data(col_name, self.readRow(type_name, rows))) - - return data From cd5c6a743a3369929984c178efced82e06968f29 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 21 Oct 2023 03:16:40 +0200 Subject: [PATCH 467/634] Revert partial result --- src/QueryPipeline/Pipe.cpp | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index 46b5db08ca1..b1c82d7a7e8 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -315,18 +315,6 @@ Pipe Pipe::unitePipes(Pipes pipes, Processors * collected_processors, bool allow for (auto & pipe : pipes) { - if (res.isPartialResultActive() && pipe.isPartialResultActive()) - { - res.partial_result_ports.insert(res.partial_result_ports.end(), pipe.partial_result_ports.begin(), pipe.partial_result_ports.end()); - } - else - { - if (pipe.isPartialResultActive()) - pipe.dropPartialResult(); - if (res.isPartialResultActive()) - res.dropPartialResult(); - } - if (!allow_empty_header || pipe.header) assertCompatibleHeader(pipe.header, res.header, "Pipe::unitePipes"); From 1b0cadebd01707f30b6aa0e755cc4f2a26067142 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sat, 21 Oct 2023 11:46:25 +0000 Subject: [PATCH 468/634] Convert sparse columns to full for window functions --- src/Processors/Transforms/WindowTransform.cpp | 2 +- ...ndow_function_with_sparse_column.reference | 5 +++ ...900_window_function_with_sparse_column.sql | 45 +++++++++++++++++++ 3 files changed, 51 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02900_window_function_with_sparse_column.reference create mode 100644 tests/queries/0_stateless/02900_window_function_with_sparse_column.sql diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index be76971ddcd..9565a073f48 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1066,7 +1066,7 @@ void WindowTransform::appendChunk(Chunk & chunk) auto columns = chunk.detachColumns(); block.original_input_columns = columns; for (auto & column : columns) - column = recursiveRemoveLowCardinality(std::move(column)->convertToFullColumnIfConst()); + column = recursiveRemoveLowCardinality(std::move(column)->convertToFullColumnIfConst()->convertToFullColumnIfSparse()); block.input_columns = std::move(columns); // Initialize output columns. diff --git a/tests/queries/0_stateless/02900_window_function_with_sparse_column.reference b/tests/queries/0_stateless/02900_window_function_with_sparse_column.reference new file mode 100644 index 00000000000..694d752d6a8 --- /dev/null +++ b/tests/queries/0_stateless/02900_window_function_with_sparse_column.reference @@ -0,0 +1,5 @@ +false +false +false + + diff --git a/tests/queries/0_stateless/02900_window_function_with_sparse_column.sql b/tests/queries/0_stateless/02900_window_function_with_sparse_column.sql new file mode 100644 index 00000000000..0b053d3e870 --- /dev/null +++ b/tests/queries/0_stateless/02900_window_function_with_sparse_column.sql @@ -0,0 +1,45 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/55843 +-- These tests pass without the fix when either of +-- - optimize_read_in_window_order = 0 and optimize_read_in_order = 0 +-- - ratio_of_defaults_for_sparse_serialization = 1 +-- However it is better to leave the settings as randomized because we run +-- stateless tests quite a few times during a PR, so if a bug is introduced +-- then there is a big chance of catching it. Furthermore, randomized settings +-- might identify new bugs. + +CREATE TABLE test1 +( + id String, + time DateTime64(9), + key Int64, + value Bool, +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(time) +ORDER BY (key, id, time); + +INSERT INTO test1 VALUES ('id0', now(), 3, false) + +SELECT last_value(value) OVER (PARTITION BY id ORDER BY time ASC) as last_value +FROM test1 +WHERE (key = 3); + +SELECT last_value(value) OVER (ORDER BY time ASC) as last_value +FROM test1 +WHERE (key = 3); + +SELECT last_value(value) OVER (PARTITION BY id ORDER BY time ASC) as last_value +FROM test1; + + + +CREATE TABLE test2 +( + time DateTime, + value String +) +ENGINE = MergeTree +ORDER BY (time) AS SELECT 0, ''; + +SELECT any(value) OVER (ORDER BY time ASC) FROM test2; +SELECT last_value(value) OVER (ORDER BY time ASC) FROM test2; From 852d627f6a1206d4e585324a58503ccb38a7ce72 Mon Sep 17 00:00:00 2001 From: Srikanth Chekuri Date: Sat, 21 Oct 2023 21:37:04 +0530 Subject: [PATCH 469/634] Introduce setting `create_table_empty_primary_key_by_default` If set to true and the table of MergeTree family does not contain ORDER BY and PRIMARY KEY, then an empty tuple is used by default --- src/Core/Settings.h | 1 + .../MergeTree/registerStorageMergeTree.cpp | 17 ++++++++++++++--- 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3b4975c3d64..b282a66fefd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -807,6 +807,7 @@ class IColumn; M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0) \ M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \ M(Bool, print_pretty_type_names, false, "Print pretty type names in DESCRIBE query and toTypeName() function", 0) \ + M(Bool, create_table_empty_primary_key_by_default, false, "Allow to create *MergeTree tables with empty primary key when ORDER BY and PRIMARY KEY not specified", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 0a182789311..f22d86499c2 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -535,9 +535,20 @@ static StoragePtr create(const StorageFactory::Arguments & args) args.storage_def->set(args.storage_def->order_by, args.storage_def->primary_key->clone()); if (!args.storage_def->order_by) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "You must provide an ORDER BY or PRIMARY KEY expression in the table definition. " - "If you don't want this table to be sorted, use ORDER BY/PRIMARY KEY ()"); + { + if (args.getLocalContext()->getSettingsRef().create_table_empty_primary_key_by_default) + { + args.storage_def->set(args.storage_def->order_by, makeASTFunction("tuple")); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "You must provide an ORDER BY or PRIMARY KEY expression in the table definition. " + "If you don't want this table to be sorted, use ORDER BY/PRIMARY KEY (). " + "Otherwise, you can use the setting 'create_table_empty_primary_key_by_default' to " + "automatically add an empty primary key to the table definition"); + } + } /// Get sorting key from engine arguments. /// From 0f2e0ade996a9c723b13676d44d168e9a955b3fd Mon Sep 17 00:00:00 2001 From: Srikanth Chekuri Date: Sat, 21 Oct 2023 21:51:10 +0530 Subject: [PATCH 470/634] Update merge tree doc --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 23ab2699cc1..51393e96c61 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -73,7 +73,7 @@ A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID, ClickHouse uses the sorting key as a primary key if the primary key is not defined explicitly by the `PRIMARY KEY` clause. -Use the `ORDER BY tuple()` syntax, if you do not need sorting. See [Selecting the Primary Key](#selecting-the-primary-key). +Use the `ORDER BY tuple()` syntax, if you do not need sorting Or set `create_table_empty_primary_key_by_default` to `true` to use the `ORDER BY tuple()` syntax by default. See [Selecting the Primary Key](#selecting-the-primary-key). #### PARTITION BY From 9e476eb56e018aeb42dd3ddd059d2f432693a56e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 21 Oct 2023 23:58:56 +0200 Subject: [PATCH 471/634] A test `01019_alter_materialized_view_consistent` is unstable with Analyzer --- tests/analyzer_tech_debt.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index eb124f110bf..e5394307f3b 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -85,3 +85,4 @@ 01940_custom_tld_sharding_key 02815_range_dict_no_direct_join 02861_join_on_nullsafe_compare +01019_alter_materialized_view_consistent From e16911835426272071a3bfa0e56e40382c9f793f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 22 Oct 2023 00:38:16 +0200 Subject: [PATCH 472/634] Remove tests --- .../02876_experimental_partial_result.reference | 0 .../0_stateless/02876_experimental_partial_result.sql | 4 ---- ...ortingPartialResultTransform_empty_block.reference | 0 ...MergeSortingPartialResultTransform_empty_block.sql | 11 ----------- 4 files changed, 15 deletions(-) delete mode 100644 tests/queries/0_stateless/02876_experimental_partial_result.reference delete mode 100644 tests/queries/0_stateless/02876_experimental_partial_result.sql delete mode 100644 tests/queries/0_stateless/02894_MergeSortingPartialResultTransform_empty_block.reference delete mode 100644 tests/queries/0_stateless/02894_MergeSortingPartialResultTransform_empty_block.sql diff --git a/tests/queries/0_stateless/02876_experimental_partial_result.reference b/tests/queries/0_stateless/02876_experimental_partial_result.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02876_experimental_partial_result.sql b/tests/queries/0_stateless/02876_experimental_partial_result.sql deleted file mode 100644 index 8418f07c750..00000000000 --- a/tests/queries/0_stateless/02876_experimental_partial_result.sql +++ /dev/null @@ -1,4 +0,0 @@ - -SET partial_result_update_duration_ms = 10; - -SELECT sum(number) FROM numbers_mt(100_000) SETTINGS max_threads = 1; -- { serverError FUNCTION_NOT_ALLOWED } diff --git a/tests/queries/0_stateless/02894_MergeSortingPartialResultTransform_empty_block.reference b/tests/queries/0_stateless/02894_MergeSortingPartialResultTransform_empty_block.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02894_MergeSortingPartialResultTransform_empty_block.sql b/tests/queries/0_stateless/02894_MergeSortingPartialResultTransform_empty_block.sql deleted file mode 100644 index 9e665e0ae20..00000000000 --- a/tests/queries/0_stateless/02894_MergeSortingPartialResultTransform_empty_block.sql +++ /dev/null @@ -1,11 +0,0 @@ -drop table if exists data; -create table data (key Int) engine=MergeTree() order by key; -insert into data select * from numbers(1); -insert into data select * from numbers(1); -system stop merges data; --- need sleep to trigger partial results to uncover the bug with empty chunk after remerge due to empty array join, i.e.: --- --- MergeSortingTransform: Re-merging intermediate ORDER BY data (1 blocks with 0 rows) to save memory consumption --- MergeSortingTransform: Memory usage is lowered from 4.26 KiB to 0.00 B --- -select key, sleepEachRow(1) from data array join [] as x order by key settings optimize_read_in_order=0, allow_experimental_partial_result=1, partial_result_update_duration_ms=1, max_threads=1, max_execution_time=0, max_block_size=1; From 5217d645512eec218355812ec5cc9785ed136f4e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 22 Oct 2023 01:53:50 +0200 Subject: [PATCH 473/634] Remove garbage --- src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp | 5 ----- src/Server/TCPHandler.cpp | 3 --- 2 files changed, 8 deletions(-) diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp index 65e464f3075..fb3ed7f80fc 100644 --- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp +++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp @@ -6,11 +6,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int FUNCTION_NOT_ALLOWED; -} - BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(ContextPtr from) { BuildQueryPipelineSettings settings; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 6f8164ec19f..871606c6298 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -10,8 +10,6 @@ #include #include #include -#include -#include #include #include #include @@ -106,7 +104,6 @@ namespace DB::ErrorCodes extern const int TIMEOUT_EXCEEDED; extern const int SUPPORT_IS_DISABLED; extern const int UNSUPPORTED_METHOD; - extern const int FUNCTION_NOT_ALLOWED; } namespace From 660d3df3721cba0026f67dde7cbf1f9cd43d0851 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 22 Oct 2023 03:45:56 +0200 Subject: [PATCH 474/634] Remove C++ templates, because they are stupid --- src/Functions/currentProfiles.cpp | 56 ++++++++++++++++++------------- 1 file changed, 32 insertions(+), 24 deletions(-) diff --git a/src/Functions/currentProfiles.cpp b/src/Functions/currentProfiles.cpp index eee458f4f63..b7350eed4e0 100644 --- a/src/Functions/currentProfiles.cpp +++ b/src/Functions/currentProfiles.cpp @@ -17,39 +17,46 @@ namespace { enum class Kind { - CURRENT_PROFILES, - ENABLED_PROFILES, - DEFAULT_PROFILES, + currentProfiles, + enabledProfiles, + defaultProfiles, }; - template + String toString(Kind kind) + { + switch (kind) + { + case Kind::currentProfiles: return "currentProfiles"; + case Kind::enabledProfiles: return "enabledProfiles"; + case Kind::defaultProfiles: return "defaultProfiles"; + } + } + class FunctionCurrentProfiles : public IFunction { public: - static constexpr auto name = (kind == Kind::CURRENT_PROFILES) ? "currentProfiles" : ((kind == Kind::ENABLED_PROFILES) ? "enabledProfiles" : "defaultProfiles"); - static FunctionPtr create(const ContextPtr & context) { return std::make_shared(context); } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override + { + return false; + } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + String getName() const override + { + return toString(kind); + } - String getName() const override { return name; } - - explicit FunctionCurrentProfiles(const ContextPtr & context) + explicit FunctionCurrentProfiles(const ContextPtr & context, Kind kind_) + : kind(kind_) { const auto & manager = context->getAccessControl(); std::vector profile_ids; - if constexpr (kind == Kind::CURRENT_PROFILES) + + switch (kind) { - profile_ids = context->getCurrentProfiles(); - } - else if constexpr (kind == Kind::ENABLED_PROFILES) - { - profile_ids = context->getEnabledProfiles(); - } - else - { - static_assert(kind == Kind::DEFAULT_PROFILES); - profile_ids = context->getUser()->settings.toProfileIDs(); + case Kind::currentProfiles: profile_ids = context->getCurrentProfiles(); break; + case Kind::enabledProfiles: profile_ids = context->getEnabledProfiles(); break; + case Kind::defaultProfiles: profile_ids = context->getUser()->settings.toProfileIDs(); break; } profile_names = manager.tryReadNames(profile_ids); @@ -75,15 +82,16 @@ namespace } private: + Kind kind; Strings profile_names; }; } REGISTER_FUNCTION(CurrentProfiles) { - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); + factory.registerFunction("currentProfiles", [](ContextPtr context){ return std::make_unique(std::make_shared(context, Kind::currentProfiles)); }); + factory.registerFunction("enabledProfiles", [](ContextPtr context){ return std::make_unique(std::make_shared(context, Kind::enabledProfiles)); }); + factory.registerFunction("defaultProfiles", [](ContextPtr context){ return std::make_unique(std::make_shared(context, Kind::defaultProfiles)); }); } } From 754cfdc68c4582c95d3f1274b9d5223ca599da96 Mon Sep 17 00:00:00 2001 From: Srikanth Chekuri Date: Sun, 22 Oct 2023 10:52:08 +0530 Subject: [PATCH 475/634] Add integration tests for create_table_empty_primary_key_by_default --- .../__init__.py | 0 .../test.py | 64 +++++++++++++++++++ 2 files changed, 64 insertions(+) create mode 100644 tests/integration/test_settings_allow_empty_order_by/__init__.py create mode 100644 tests/integration/test_settings_allow_empty_order_by/test.py diff --git a/tests/integration/test_settings_allow_empty_order_by/__init__.py b/tests/integration/test_settings_allow_empty_order_by/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_settings_allow_empty_order_by/test.py b/tests/integration/test_settings_allow_empty_order_by/test.py new file mode 100644 index 00000000000..9e0e2abc0d5 --- /dev/null +++ b/tests/integration/test_settings_allow_empty_order_by/test.py @@ -0,0 +1,64 @@ +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", stay_alive=True) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_create_table_without_order_by(): + with pytest.raises(QueryRuntimeException): + node.query("CREATE DATABASE IF NOT EXISTS test;") + node.query("DROP TABLE IF EXISTS test.test_empty_order_by;") + node.query( + """ + CREATE TABLE test.test_empty_order_by + ( + `a` UInt8, + ) + ENGINE = MergeTree() + SETTINGS index_granularity = 8192; + """ + ) + +def test_create_table_without_order_by_and_setting_enabled(): + node.query("CREATE DATABASE IF NOT EXISTS test;") + node.query("DROP TABLE IF EXISTS test.test_empty_order_by;") + node.query("SET create_table_empty_primary_key_by_default = true;") + node.query( + """ + CREATE TABLE test.test_empty_order_by + ( + `a` UInt8, + ) + ENGINE = MergeTree() + SETTINGS index_granularity = 8192; + """ + ) + assert "ORDER BY tuple()" in node.query(f"SHOW CREATE TABLE test.test_empty_order_by;") + +def test_create_table_without_order_by_and_setting_enabled_columwise_primary_key(): + node.query("CREATE DATABASE IF NOT EXISTS test;") + node.query("DROP TABLE IF EXISTS test.test_empty_order_by;") + node.query("SET create_table_empty_primary_key_by_default = true;") + node.query( + """ + CREATE TABLE test.test_empty_order_by + ( + `a` UInt8 PRIMARY KEY, + `b` String PRIMARY KEY + ) + ENGINE = MergeTree() + SETTINGS index_granularity = 8192; + """ + ) + assert "ORDER BY (a, b)" in node.query(f"SHOW CREATE TABLE test.test_empty_order_by") \ No newline at end of file From eb959a1e08b7207b2997f36b1ad5ea985c0779e5 Mon Sep 17 00:00:00 2001 From: Srikanth Chekuri Date: Sun, 22 Oct 2023 11:22:33 +0530 Subject: [PATCH 476/634] Revert "Add integration tests for create_table_empty_primary_key_by_default" This reverts commit 754cfdc68c4582c95d3f1274b9d5223ca599da96. --- .../__init__.py | 0 .../test.py | 64 ------------------- 2 files changed, 64 deletions(-) delete mode 100644 tests/integration/test_settings_allow_empty_order_by/__init__.py delete mode 100644 tests/integration/test_settings_allow_empty_order_by/test.py diff --git a/tests/integration/test_settings_allow_empty_order_by/__init__.py b/tests/integration/test_settings_allow_empty_order_by/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_settings_allow_empty_order_by/test.py b/tests/integration/test_settings_allow_empty_order_by/test.py deleted file mode 100644 index 9e0e2abc0d5..00000000000 --- a/tests/integration/test_settings_allow_empty_order_by/test.py +++ /dev/null @@ -1,64 +0,0 @@ -import pytest -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node", stay_alive=True) - - -@pytest.fixture(scope="module", autouse=True) -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_create_table_without_order_by(): - with pytest.raises(QueryRuntimeException): - node.query("CREATE DATABASE IF NOT EXISTS test;") - node.query("DROP TABLE IF EXISTS test.test_empty_order_by;") - node.query( - """ - CREATE TABLE test.test_empty_order_by - ( - `a` UInt8, - ) - ENGINE = MergeTree() - SETTINGS index_granularity = 8192; - """ - ) - -def test_create_table_without_order_by_and_setting_enabled(): - node.query("CREATE DATABASE IF NOT EXISTS test;") - node.query("DROP TABLE IF EXISTS test.test_empty_order_by;") - node.query("SET create_table_empty_primary_key_by_default = true;") - node.query( - """ - CREATE TABLE test.test_empty_order_by - ( - `a` UInt8, - ) - ENGINE = MergeTree() - SETTINGS index_granularity = 8192; - """ - ) - assert "ORDER BY tuple()" in node.query(f"SHOW CREATE TABLE test.test_empty_order_by;") - -def test_create_table_without_order_by_and_setting_enabled_columwise_primary_key(): - node.query("CREATE DATABASE IF NOT EXISTS test;") - node.query("DROP TABLE IF EXISTS test.test_empty_order_by;") - node.query("SET create_table_empty_primary_key_by_default = true;") - node.query( - """ - CREATE TABLE test.test_empty_order_by - ( - `a` UInt8 PRIMARY KEY, - `b` String PRIMARY KEY - ) - ENGINE = MergeTree() - SETTINGS index_granularity = 8192; - """ - ) - assert "ORDER BY (a, b)" in node.query(f"SHOW CREATE TABLE test.test_empty_order_by") \ No newline at end of file From 3f9ffbb451bbdc2897cb0d72c414f195eafebb44 Mon Sep 17 00:00:00 2001 From: Srikanth Chekuri Date: Sun, 22 Oct 2023 12:25:09 +0530 Subject: [PATCH 477/634] Add functional tests for create_table_empty_primary_key_by_default 1. 02902_empty_order_by_throws_error - setting disabled and no order by or primary key - expect error - setting disabled and primary key in the table definition - no error and expect order by as primary key 2. 02903_empty_order_by_with_setting_enabled - setting enabled and no order by or primary key - expect the order by as empty - setting enabled and per-column primary key - expect the order by as column order - setting enabled and primary key in table definition (not per-column or order by) - expect order as primary key - setting enabled and order by in table definition (no primary key) - expect same order by --- ...2902_empty_order_by_throws_error.reference | 2 + .../02902_empty_order_by_throws_error.sh | 18 +++++++++ ...ty_order_by_with_setting_enabled.reference | 4 ++ ...903_empty_order_by_with_setting_enabled.sh | 37 +++++++++++++++++++ 4 files changed, 61 insertions(+) create mode 100644 tests/queries/0_stateless/02902_empty_order_by_throws_error.reference create mode 100755 tests/queries/0_stateless/02902_empty_order_by_throws_error.sh create mode 100644 tests/queries/0_stateless/02903_empty_order_by_with_setting_enabled.reference create mode 100755 tests/queries/0_stateless/02903_empty_order_by_with_setting_enabled.sh diff --git a/tests/queries/0_stateless/02902_empty_order_by_throws_error.reference b/tests/queries/0_stateless/02902_empty_order_by_throws_error.reference new file mode 100644 index 00000000000..2c94e483710 --- /dev/null +++ b/tests/queries/0_stateless/02902_empty_order_by_throws_error.reference @@ -0,0 +1,2 @@ +OK +OK diff --git a/tests/queries/0_stateless/02902_empty_order_by_throws_error.sh b/tests/queries/0_stateless/02902_empty_order_by_throws_error.sh new file mode 100755 index 00000000000..64f5dd1a987 --- /dev/null +++ b/tests/queries/0_stateless/02902_empty_order_by_throws_error.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# setting disabled and no order by or primary key; expect error +$CLICKHOUSE_CLIENT -n --query=" + DROP TABLE IF EXISTS test_empty_order_by; + CREATE TABLE test_empty_order_by(a UInt8) ENGINE = MergeTree() SETTINGS index_granularity = 8192; +" 2>&1 \ | grep -F -q "You must provide an ORDER BY or PRIMARY KEY expression in the table definition." && echo 'OK' || echo 'FAIL' + +# setting disabled and primary key in table definition +$CLICKHOUSE_CLIENT -n --query=" + DROP TABLE IF EXISTS test_empty_order_by; + CREATE TABLE test_empty_order_by(a UInt8) ENGINE = MergeTree() PRIMARY KEY a SETTINGS index_granularity = 8192; + SHOW CREATE TABLE test_empty_order_by; +" 2>&1 \ | grep -F -q "ORDER BY a" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/02903_empty_order_by_with_setting_enabled.reference b/tests/queries/0_stateless/02903_empty_order_by_with_setting_enabled.reference new file mode 100644 index 00000000000..b462a5a7baa --- /dev/null +++ b/tests/queries/0_stateless/02903_empty_order_by_with_setting_enabled.reference @@ -0,0 +1,4 @@ +OK +OK +OK +OK diff --git a/tests/queries/0_stateless/02903_empty_order_by_with_setting_enabled.sh b/tests/queries/0_stateless/02903_empty_order_by_with_setting_enabled.sh new file mode 100755 index 00000000000..7ac9b488be5 --- /dev/null +++ b/tests/queries/0_stateless/02903_empty_order_by_with_setting_enabled.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# setting enabled and no order by or primary key +${CLICKHOUSE_CLIENT} -n --query=" + SET create_table_empty_primary_key_by_default = true; + DROP TABLE IF EXISTS test_empty_order_by; + CREATE TABLE test_empty_order_by(a UInt8) ENGINE = MergeTree() SETTINGS index_granularity = 8192; + SHOW CREATE TABLE test_empty_order_by; +" 2>&1 \ | grep -F -q "ORDER BY tuple()" && echo 'OK' || echo 'FAIL' + +# setting enabled and per-column primary key +${CLICKHOUSE_CLIENT} -n --query=" + SET create_table_empty_primary_key_by_default = true; + DROP TABLE IF EXISTS test_empty_order_by; + CREATE TABLE test_empty_order_by(a UInt8 PRIMARY KEY, b String PRIMARY KEY) ENGINE = MergeTree() SETTINGS index_granularity = 8192; + SHOW CREATE TABLE test_empty_order_by; +" 2>&1 \ | grep -F -q "ORDER BY (a, b)" && echo 'OK' || echo 'FAIL' + +# setting enabled and primary key in table definition (not per-column or order by) +${CLICKHOUSE_CLIENT} -n --query=" + SET create_table_empty_primary_key_by_default = true; + DROP TABLE IF EXISTS test_empty_order_by; + CREATE TABLE test_empty_order_by(a UInt8, b String) ENGINE = MergeTree() PRIMARY KEY (a) SETTINGS index_granularity = 8192; + SHOW CREATE TABLE test_empty_order_by; +" 2>&1 \ | grep -F -q "ORDER BY a" && echo 'OK' || echo 'FAIL' + +# setting enabled and order by in table definition (no primary key) +${CLICKHOUSE_CLIENT} -n --query=" + SET create_table_empty_primary_key_by_default = true; + DROP TABLE IF EXISTS test_empty_order_by; + CREATE TABLE test_empty_order_by(a UInt8, b String) ENGINE = MergeTree() ORDER BY (a, b) SETTINGS index_granularity = 8192; + SHOW CREATE TABLE test_empty_order_by; +" 2>&1 \ | grep -F -q "ORDER BY (a, b)" && echo 'OK' || echo 'FAIL' From 53a49164d61c6f920e8e8361d7bcaa4bb301e8f7 Mon Sep 17 00:00:00 2001 From: wxybear Date: Sun, 22 Oct 2023 16:24:01 +0800 Subject: [PATCH 478/634] fix: StorageNull supports subcolumns --- src/Storages/StorageNull.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index f6dd7064a22..f7ee936db8d 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -46,6 +46,8 @@ public: bool supportsParallelInsert() const override { return true; } + bool supportsSubcolumns() const override { return true; } + SinkToStoragePtr write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, bool) override { return std::make_shared(metadata_snapshot->getSampleBlock()); From c2215cb5f0647947395fe8363954f02205ea34eb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 19 Oct 2023 21:52:23 +0000 Subject: [PATCH 479/634] Support string arguments --- .../functions/date-time-functions.md | 32 +- src/Functions/FunctionBinaryArithmetic.h | 32 +- .../FunctionDateOrDateTimeAddInterval.h | 307 +++++++++++++----- .../Transforms/FillingTransform.cpp | 6 +- ...btract_interval_with_string_date.reference | 125 +++++++ ...add_subtract_interval_with_string_date.sql | 139 ++++++++ 6 files changed, 529 insertions(+), 112 deletions(-) create mode 100644 tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.reference create mode 100644 tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 8db971fb926..b7e63f14f10 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -2393,44 +2393,48 @@ Accepts an additional, optional `precision` parameter after the `timezone` param ## addYears, addQuarters, addMonths, addWeeks, addDays, addHours, addMinutes, addSeconds, addMilliseconds, addMicroseconds, addNanoseconds -These functions add units of the interval specified by the function name to a date or a date with time. A date or date with time is returned. +These functions add units of the interval specified by the function name to a date, a date with time or a date or date with time encoded as string. A date or date with time is returned. Example: ``` sql WITH - toDate('2018-01-01') AS date, - toDateTime('2018-01-01 00:00:00') AS date_time + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string SELECT addYears(date, 1) AS add_years_with_date, - addYears(date_time, 1) AS add_years_with_date_time + addYears(date_time, 1) AS add_years_with_date_time, + addYears(date_time_string, 1) AS add_years_with_date_time_string ``` ``` text -┌─add_years_with_date─┬─add_years_with_date_time─┐ -│ 2019-01-01 │ 2019-01-01 00:00:00 │ -└─────────────────────┴──────────────────────────┘ +┌─add_years_with_date─┬─add_years_with_date_time─┬─add_years_with_date_time_string─┐ +│ 2025-01-01 │ 2025-01-01 00:00:00 │ 2025-01-01 00:00:00.000 │ +└─────────────────────┴──────────────────────────┴─────────────────────────────────┘ ``` ## subtractYears, subtractQuarters, subtractMonths, subtractWeeks, subtractDays, subtractHours, subtractMinutes, subtractSeconds, subtractMilliseconds, subtractMicroseconds, subtractNanoseconds -These functions subtract units of the interval specified by the function name from a date or a date with time. A date or date with time is returned. +These functions subtract units of the interval specified by the function name from a date, a date with time or a date or date with time encoded as string. A date or date with time is returned. Example: ``` sql WITH - toDate('2019-01-01') AS date, - toDateTime('2019-01-01 00:00:00') AS date_time + toDate('2024-01-01') AS date, + toDateTime('2024-01-01 00:00:00') AS date_time, + '2024-01-01 00:00:00' AS date_time_string SELECT subtractYears(date, 1) AS subtract_years_with_date, - subtractYears(date_time, 1) AS subtract_years_with_date_time + subtractYears(date_time, 1) AS subtract_years_with_date_time, + subtractYears(date_time_string, 1) AS subtract_years_with_date_time_string ``` ``` text -┌─subtract_years_with_date─┬─subtract_years_with_date_time─┐ -│ 2018-01-01 │ 2018-01-01 00:00:00 │ -└──────────────────────────┴───────────────────────────────┘ +┌─subtract_years_with_date─┬─subtract_years_with_date_time─┬─subtract_years_with_date_time_string─┐ +│ 2023-01-01 │ 2023-01-01 00:00:00 │ 2023-01-01 00:00:00.000 │ +└──────────────────────────┴───────────────────────────────┴──────────────────────────────────────┘ ``` ## timeSlots(StartTime, Duration,\[, Size\]) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 33ed22b9fbd..2dd6c582157 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -796,21 +796,21 @@ class FunctionBinaryArithmetic : public IFunction static FunctionOverloadResolverPtr getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { - bool first_is_date_or_datetime = isDateOrDate32(type0) || isDateTime(type0) || isDateTime64(type0); - bool second_is_date_or_datetime = isDateOrDate32(type1) || isDateTime(type1) || isDateTime64(type1); + bool first_arg_is_date_or_datetime_or_string = isDateOrDate32OrDateTimeOrDateTime64(type0) || isString(type0); + bool second_arg_is_date_or_datetime_or_string = isDateOrDate32OrDateTimeOrDateTime64(type1) || isString(type1); - /// Exactly one argument must be Date or DateTime - if (first_is_date_or_datetime == second_is_date_or_datetime) + /// Exactly one argument must be Date or DateTime or String + if (first_arg_is_date_or_datetime_or_string == second_arg_is_date_or_datetime_or_string) return {}; - /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. + /// Special case when the function is plus or minus, one of arguments is Date or DateTime or String and another is Interval. /// We construct another function (example: addMonths) and call it. if constexpr (!is_plus && !is_minus) return {}; - const DataTypePtr & type_time = first_is_date_or_datetime ? type0 : type1; - const DataTypePtr & type_interval = first_is_date_or_datetime ? type1 : type0; + const DataTypePtr & type_time = first_arg_is_date_or_datetime_or_string ? type0 : type1; + const DataTypePtr & type_interval = first_arg_is_date_or_datetime_or_string ? type1 : type0; bool interval_is_number = isNumber(type_interval); @@ -823,7 +823,7 @@ class FunctionBinaryArithmetic : public IFunction return {}; } - if (second_is_date_or_datetime && is_minus) + if (second_arg_is_date_or_datetime_or_string && is_minus) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong order of arguments for function {}: " "argument of type Interval cannot be first", name); @@ -848,11 +848,11 @@ class FunctionBinaryArithmetic : public IFunction static FunctionOverloadResolverPtr getFunctionForDateTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { - bool first_is_date_or_datetime = isDateOrDate32(type0) || isDateTime(type0) || isDateTime64(type0); - bool second_is_date_or_datetime = isDateOrDate32(type1) || isDateTime(type1) || isDateTime64(type1); + bool first_arg_is_date_or_datetime = isDateOrDate32OrDateTimeOrDateTime64(type0); + bool second_arg_is_date_or_datetime = isDateOrDate32OrDateTimeOrDateTime64(type1); /// Exactly one argument must be Date or DateTime - if (first_is_date_or_datetime == second_is_date_or_datetime) + if (first_arg_is_date_or_datetime == second_arg_is_date_or_datetime) return {}; if (!isTuple(type0) && !isTuple(type1)) @@ -863,7 +863,7 @@ class FunctionBinaryArithmetic : public IFunction if constexpr (!is_plus && !is_minus) return {}; - if (isTuple(type0) && second_is_date_or_datetime && is_minus) + if (isTuple(type0) && second_arg_is_date_or_datetime && is_minus) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong order of arguments for function {}: " "argument of Tuple type cannot be first", name); @@ -1123,7 +1123,7 @@ class FunctionBinaryArithmetic : public IFunction ColumnsWithTypeAndName new_arguments = arguments; /// Interval argument must be second. - if (isDateOrDate32(arguments[1].type) || isDateTime(arguments[1].type) || isDateTime64(arguments[1].type)) + if (isDateOrDate32OrDateTimeOrDateTime64(arguments[1].type) || isString(arguments[1].type)) std::swap(new_arguments[0], new_arguments[1]); /// Change interval argument type to its representation @@ -1511,7 +1511,7 @@ public: } } - /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. + /// Special case when the function is plus or minus, one of arguments is Date/DateTime/String and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1], context)) { ColumnsWithTypeAndName new_arguments(2); @@ -1520,7 +1520,7 @@ public: new_arguments[i].type = arguments[i]; /// Interval argument must be second. - if (isDateOrDate32(new_arguments[1].type) || isDateTime(new_arguments[1].type) || isDateTime64(new_arguments[1].type)) + if (isDateOrDate32OrDateTimeOrDateTime64(new_arguments[1].type) || isString(new_arguments[1].type)) std::swap(new_arguments[0], new_arguments[1]); /// Change interval argument to its representation @@ -2104,7 +2104,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A return executeAggregateAddition(arguments, result_type, input_rows_count); } - /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. + /// Special case when the function is plus or minus, one of arguments is Date/DateTime/String and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0].type, arguments[1].type, context)) { return executeDateTimeIntervalPlusMinus(arguments, result_type, input_rows_count, function_builder); diff --git a/src/Functions/FunctionDateOrDateTimeAddInterval.h b/src/Functions/FunctionDateOrDateTimeAddInterval.h index 4828dd1cc72..2ba53172178 100644 --- a/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -8,7 +8,9 @@ #include #include #include +#include +#include #include #include @@ -16,7 +18,9 @@ #include #include +#include #include +#include namespace DB @@ -47,268 +51,345 @@ struct AddNanosecondsImpl { static constexpr auto name = "addNanoseconds"; - static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16 scale) { Int64 multiplier = DecimalUtils::scaleMultiplier(9 - scale); return DateTime64(DecimalUtils::multiplyAdd(t.value, multiplier, delta)); } - static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16) { Int64 multiplier = DecimalUtils::scaleMultiplier(9); return DateTime64(DecimalUtils::multiplyAdd(static_cast(t), multiplier, delta)); } - static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, const DateLUTImpl &, UInt16) { throw Exception(ErrorCodes::LOGICAL_ERROR, "addNanoseconds() cannot be used with Date"); } - static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, const DateLUTImpl &, UInt16) { throw Exception(ErrorCodes::LOGICAL_ERROR, "addNanoseconds() cannot be used with Date32"); } + static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) + { + ReadBufferFromString buf(s); + DateTime64 t; + parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone); + return execute(t, delta, time_zone, utc_time_zone, scale); + } }; struct AddMicrosecondsImpl { static constexpr auto name = "addMicroseconds"; - static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16 scale) { Int64 multiplier = DecimalUtils::scaleMultiplier(std::abs(6 - scale)); return DateTime64(scale <= 6 ? DecimalUtils::multiplyAdd(t.value, multiplier, delta) : DecimalUtils::multiplyAdd(delta, multiplier, t.value)); } - static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16) { Int64 multiplier = DecimalUtils::scaleMultiplier(6); return DateTime64(DecimalUtils::multiplyAdd(static_cast(t), multiplier, delta)); } - static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, const DateLUTImpl &, UInt16) { throw Exception(ErrorCodes::LOGICAL_ERROR, "addMicroseconds() cannot be used with Date"); } - static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, const DateLUTImpl &, UInt16) { throw Exception(ErrorCodes::LOGICAL_ERROR, "addMicroseconds() cannot be used with Date32"); } + static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) + { + ReadBufferFromString buf(s); + DateTime64 t; + parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone); + return execute(t, delta, time_zone, utc_time_zone, scale); + } }; struct AddMillisecondsImpl { static constexpr auto name = "addMilliseconds"; - static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16 scale) { Int64 multiplier = DecimalUtils::scaleMultiplier(std::abs(3 - scale)); return DateTime64(scale <= 3 ? DecimalUtils::multiplyAdd(t.value, multiplier, delta) : DecimalUtils::multiplyAdd(delta, multiplier, t.value)); } - static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED DateTime64 execute(UInt32 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16) { Int64 multiplier = DecimalUtils::scaleMultiplier(3); return DateTime64(DecimalUtils::multiplyAdd(static_cast(t), multiplier, delta)); } - static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED Int8 execute(UInt16, Int64, const DateLUTImpl &, const DateLUTImpl &, UInt16) { throw Exception(ErrorCodes::LOGICAL_ERROR, "addMilliseconds() cannot be used with Date"); } - static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED Int8 execute(Int32, Int64, const DateLUTImpl &, const DateLUTImpl &, UInt16) { throw Exception(ErrorCodes::LOGICAL_ERROR, "addMilliseconds() cannot be used with Date32"); } + static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) + { + ReadBufferFromString buf(s); + DateTime64 t; + parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone); + return execute(t, delta, time_zone, utc_time_zone, scale); + } }; struct AddSecondsImpl { static constexpr auto name = "addSeconds"; - static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16 scale) { return DateTime64(DecimalUtils::multiplyAdd(delta, DecimalUtils::scaleMultiplier(scale), t.value)); } - static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16) { return static_cast(t + delta); } - static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { // use default datetime64 scale static_assert(DataTypeDateTime64::default_scale == 3); return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta) * 1000; } - static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return static_cast(time_zone.fromDayNum(DayNum(d)) + delta); } + static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) + { + ReadBufferFromString buf(s); + DateTime64 t; + parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone); + return execute(t, delta, time_zone, utc_time_zone, scale); + } }; struct AddMinutesImpl { static constexpr auto name = "addMinutes"; - static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16 scale) { return t + 60 * delta * DecimalUtils::scaleMultiplier(scale); } - static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16) { return static_cast(t + delta * 60); } - static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { // use default datetime64 scale static_assert(DataTypeDateTime64::default_scale == 3); return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 60) * 1000; } - static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return static_cast(time_zone.fromDayNum(DayNum(d)) + delta * 60); } + static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) + { + ReadBufferFromString buf(s); + DateTime64 t; + parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone); + return execute(t, delta, time_zone, utc_time_zone, scale); + } }; struct AddHoursImpl { static constexpr auto name = "addHours"; - static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, UInt16 scale) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16 scale) { return t + 3600 * delta * DecimalUtils::scaleMultiplier(scale); } - static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16) { return static_cast(t + delta * 3600); } - static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { // use default datetime64 scale static_assert(DataTypeDateTime64::default_scale == 3); return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 3600) * 1000; } - static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return static_cast(time_zone.fromDayNum(DayNum(d)) + delta * 3600); } + static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) + { + ReadBufferFromString buf(s); + DateTime64 t; + parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone); + return execute(t, delta, time_zone, utc_time_zone, scale); + } }; struct AddDaysImpl { static constexpr auto name = "addDays"; - static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16 scale) { auto multiplier = DecimalUtils::scaleMultiplier(scale); auto d = std::div(t, multiplier); return time_zone.addDays(d.quot, delta) * multiplier + d.rem; } - static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return static_cast(time_zone.addDays(t, delta)); } - static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16) { return d + delta; } - static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16) { return static_cast(d + delta); } + static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) + { + ReadBufferFromString buf(s); + DateTime64 t; + parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone); + return execute(t, delta, time_zone, utc_time_zone, scale); + } }; struct AddWeeksImpl { static constexpr auto name = "addWeeks"; - static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16 scale) { auto multiplier = DecimalUtils::scaleMultiplier(scale); auto d = std::div(t, multiplier); return time_zone.addDays(d.quot, delta * 7) * multiplier + d.rem; } - static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return static_cast(time_zone.addWeeks(t, delta)); } - static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16) { return static_cast(d + delta * 7); } - static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, UInt16) + static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &, const DateLUTImpl &, UInt16) { return static_cast(d + delta * 7); } + static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) + { + ReadBufferFromString buf(s); + DateTime64 t; + parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone); + return execute(t, delta, time_zone, utc_time_zone, scale); + } }; struct AddMonthsImpl { static constexpr auto name = "addMonths"; - static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16 scale) { auto multiplier = DecimalUtils::scaleMultiplier(scale); auto d = std::div(t, multiplier); return time_zone.addMonths(d.quot, delta) * multiplier + d.rem; } - static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return static_cast(time_zone.addMonths(t, delta)); } - static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return time_zone.addMonths(DayNum(d), delta); } - static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return time_zone.addMonths(ExtendedDayNum(d), delta); } + static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) + { + ReadBufferFromString buf(s); + DateTime64 t; + parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone); + return execute(t, delta, time_zone, utc_time_zone, scale); + } }; struct AddQuartersImpl { static constexpr auto name = "addQuarters"; - static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16 scale) { auto multiplier = DecimalUtils::scaleMultiplier(scale); auto d = std::div(t, multiplier); return time_zone.addQuarters(d.quot, delta) * multiplier + d.rem; } - static UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return static_cast(time_zone.addQuarters(t, delta)); } - static UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return time_zone.addQuarters(DayNum(d), delta); } - static Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return time_zone.addQuarters(ExtendedDayNum(d), delta); } + static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) + { + ReadBufferFromString buf(s); + DateTime64 t; + parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone); + return execute(t, delta, time_zone, utc_time_zone, scale); + } }; struct AddYearsImpl { static constexpr auto name = "addYears"; - static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) + static NO_SANITIZE_UNDEFINED DateTime64 execute(DateTime64 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16 scale) { auto multiplier = DecimalUtils::scaleMultiplier(scale); auto d = std::div(t, multiplier); return time_zone.addYears(d.quot, delta) * multiplier + d.rem; } - static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return static_cast(time_zone.addYears(t, delta)); } - static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return time_zone.addYears(DayNum(d), delta); } - static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, UInt16) + static NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl &, UInt16) { return time_zone.addYears(ExtendedDayNum(d), delta); } + static DateTime64 execute(std::string_view s, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) + { + ReadBufferFromString buf(s); + DateTime64 t; + parseDateTime64BestEffort(t, scale, buf, time_zone, utc_time_zone); + return execute(t, delta, time_zone, utc_time_zone, scale); + } }; template @@ -317,10 +398,10 @@ struct SubtractIntervalImpl : public Transform using Transform::Transform; template - NO_SANITIZE_UNDEFINED auto execute(T t, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) const + NO_SANITIZE_UNDEFINED auto execute(T t, Int64 delta, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone, UInt16 scale) const { /// Signed integer overflow is Ok. - return Transform::execute(t, -delta, time_zone, scale); + return Transform::execute(t, -delta, time_zone, utc_time_zone, scale); } }; @@ -346,40 +427,56 @@ struct Processor : transform(std::move(transform_)) {} - template - void NO_INLINE vectorConstant(const FromVectorType & vec_from, ToVectorType & vec_to, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) const + template + void NO_INLINE vectorConstant(const FromColumnType & col_from, ToColumnType & col_to, Int64 delta, const DateLUTImpl & time_zone, UInt16 scale) const { - size_t size = vec_from.size(); - vec_to.resize(size); + static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); - for (size_t i = 0; i < size; ++i) - vec_to[i] = transform.execute(vec_from[i], checkOverflow(delta), time_zone, scale); + if constexpr (std::is_same_v) + { + const auto & offsets_from = col_from.getOffsets(); + auto & vec_to = col_to.getData(); + + size_t size = offsets_from.size(); + vec_to.resize(size); + + for (size_t i = 0 ; i < size; ++i) + { + std::string_view from = col_from.getDataAt(i).toView(); + vec_to[i] = transform.execute(from, checkOverflow(delta), time_zone, utc_time_zone, scale); + } + } + else + { + const auto & vec_from = col_from.getData(); + auto & vec_to = col_to.getData(); + + size_t size = vec_from.size(); + vec_to.resize(size); + + for (size_t i = 0; i < size; ++i) + vec_to[i] = transform.execute(vec_from[i], checkOverflow(delta), time_zone, utc_time_zone, scale); + } } - template - void vectorVector(const FromVectorType & vec_from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone, UInt16 scale) const + template + void vectorVector(const FromColumnType & col_from, ToColumnType & col_to, const IColumn & delta, const DateLUTImpl & time_zone, UInt16 scale) const { - size_t size = vec_from.size(); - vec_to.resize(size); - castTypeToEither< ColumnUInt8, ColumnUInt16, ColumnUInt32, ColumnUInt64, ColumnInt8, ColumnInt16, ColumnInt32, ColumnInt64, ColumnFloat32, ColumnFloat64>( - &delta, [&](const auto & column){ vectorVector(vec_from, vec_to, column, time_zone, scale, size); return true; }); + &delta, [&](const auto & column){ vectorVector(col_from, col_to, column, time_zone, scale); return true; }); } - template - void constantVector(const FromType & from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone, UInt16 scale) const + template + void constantVector(const FromType & from, ToColumnType & col_to, const IColumn & delta, const DateLUTImpl & time_zone, UInt16 scale) const { - size_t size = delta.size(); - vec_to.resize(size); - castTypeToEither< ColumnUInt8, ColumnUInt16, ColumnUInt32, ColumnUInt64, ColumnInt8, ColumnInt16, ColumnInt32, ColumnInt64, ColumnFloat32, ColumnFloat64>( - &delta, [&](const auto & column){ constantVector(from, vec_to, column, time_zone, scale, size); return true; }); + &delta, [&](const auto & column){ constantVector(from, col_to, column, time_zone, scale); return true; }); } private: @@ -392,20 +489,65 @@ private: throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow"); } - template + template NO_INLINE NO_SANITIZE_UNDEFINED void vectorVector( - const FromVectorType & vec_from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, UInt16 scale, size_t size) const + const FromColumnType & col_from, ToColumnType & col_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, UInt16 scale) const { - for (size_t i = 0; i < size; ++i) - vec_to[i] = transform.execute(vec_from[i], checkOverflow(delta.getData()[i]), time_zone, scale); + static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); + + if constexpr (std::is_same_v) + { + const auto & offsets_from = col_from.getOffsets(); + auto & vec_to = col_to.getData(); + + size_t size = offsets_from.size(); + vec_to.resize(size); + + for (size_t i = 0 ; i < size; ++i) + { + std::string_view from = col_from.getDataAt(i).toView(); + vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone, utc_time_zone, scale); + } + } + else + { + const auto & vec_from = col_from.getData(); + auto & vec_to = col_to.getData(); + + size_t size = vec_from.size(); + vec_to.resize(size); + + for (size_t i = 0; i < size; ++i) + vec_to[i] = transform.execute(vec_from[i], checkOverflow(delta.getData()[i]), time_zone, utc_time_zone, scale); + } } - template + template NO_INLINE NO_SANITIZE_UNDEFINED void constantVector( - const FromType & from, ToVectorType & vec_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, UInt16 scale, size_t size) const + const FromType & from, ToColumnType & col_to, const DeltaColumnType & delta, const DateLUTImpl & time_zone, UInt16 scale) const { - for (size_t i = 0; i < size; ++i) - vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone, scale); + static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); + + if constexpr (std::is_same_v) + { + auto & vec_to = col_to.getData(); + + size_t size = delta.size(); + vec_to.resize(size); + + for (size_t i = 0; i < size; ++i) + vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone, utc_time_zone, scale); + } + else + { + auto & vec_to = col_to.getData(); + + size_t size = delta.size(); + vec_to.resize(size); + + for (size_t i = 0; i < size; ++i) + vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone, utc_time_zone, scale); + } } }; @@ -432,15 +574,15 @@ struct DateTimeAddIntervalImpl if (const auto * sources = checkAndGetColumn(&source_column)) { if (const auto * delta_const_column = typeid_cast(&delta_column)) - processor.vectorConstant(sources->getData(), col_to->getData(), delta_const_column->getInt(0), time_zone, scale); + processor.vectorConstant(*sources, *col_to, delta_const_column->getInt(0), time_zone, scale); else - processor.vectorVector(sources->getData(), col_to->getData(), delta_column, time_zone, scale); + processor.vectorVector(*sources, *col_to, delta_column, time_zone, scale); } else if (const auto * sources_const = checkAndGetColumnConst(&source_column)) { processor.constantVector( sources_const->template getValue(), - col_to->getData(), delta_column, time_zone, scale); + *col_to, delta_column, time_zone, scale); } else { @@ -486,15 +628,15 @@ public: if (arguments.size() == 2) { - if (!isDateOrDate32OrDateTimeOrDateTime64(arguments[0].type)) + if (!isDateOrDate32OrDateTimeOrDateTime64(arguments[0].type) && !isString(arguments[0].type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. " - "Should be a date or a date with time", arguments[0].type->getName(), getName()); + "Must be a date, a date with time or a String", arguments[0].type->getName(), getName()); } else { if (!WhichDataType(arguments[0].type).isDateTime()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. " - "Should be a DateTime", arguments[0].type->getName(), getName()); + "Must be a DateTime", arguments[0].type->getName(), getName()); if (!WhichDataType(arguments[2].type).isString()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of third argument of function {}. " @@ -516,6 +658,8 @@ public: return resolveReturnType(arguments); case TypeIndex::DateTime64: return resolveReturnType(arguments); + case TypeIndex::String: + return resolveReturnType(arguments); default: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid type of 1st argument of function {}: " "{}, expected: Date, DateTime or DateTime64.", getName(), arguments[0].type->getName()); @@ -525,7 +669,7 @@ public: /// Helper templates to deduce return type based on argument type, since some overloads may promote or denote types, /// e.g. addSeconds(Date, 1) => DateTime template - using TransformExecuteReturnType = decltype(std::declval().execute(FieldType(), 0, std::declval(), 0)); + using TransformExecuteReturnType = decltype(std::declval().execute(FieldType(), 0, std::declval(), std::declval(), 0)); // Deduces RETURN DataType from INPUT DataType, based on return type of Transform{}.execute(INPUT_TYPE, UInt64, DateLUTImpl). // e.g. for Transform-type that has execute()-overload with 'UInt16' input and 'UInt32' return, @@ -588,11 +732,14 @@ public: return DateTimeAddIntervalImpl, Transform>::execute(Transform{}, arguments, result_type, 0); else if (which.isDateTime()) return DateTimeAddIntervalImpl, Transform>::execute(Transform{}, arguments, result_type, 0); - else if (const auto * datetime64_type = assert_cast(from_type)) + else if (which.isDateTime64()) { + const auto * datetime64_type = assert_cast(from_type); auto from_scale = datetime64_type->getScale(); return DateTimeAddIntervalImpl, Transform>::execute(Transform{}, arguments, result_type, from_scale); } + else if (which.isString()) + return DateTimeAddIntervalImpl::execute(Transform{}, arguments, result_type, 3); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}", arguments[0].type->getName(), getName()); } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 4ef17a831bc..9577f7ca7ff 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -54,13 +54,14 @@ template static FillColumnDescription::StepFunction getStepFunction( IntervalKind kind, Int64 step, const DateLUTImpl & date_lut, UInt16 scale = DataTypeDateTime64::default_scale) { + static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); switch (kind) // NOLINT(bugprone-switch-missing-default-case) { #define DECLARE_CASE(NAME) \ case IntervalKind::NAME: \ return [step, scale, &date_lut](Field & field) { \ field = Add##NAME##sImpl::execute(static_cast(\ - field.get()), static_cast(step), date_lut, scale); }; + field.get()), static_cast(step), date_lut, utc_time_zone, scale); }; FOR_EACH_INTERVAL_KIND(DECLARE_CASE) #undef DECLARE_CASE @@ -154,6 +155,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & { const auto & step_dec = descr.fill_step.get &>(); Int64 step = DecimalUtils::convertTo(step_dec.getValue(), step_dec.getScale()); + static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); switch (*descr.step_kind) // NOLINT(bugprone-switch-missing-default-case) { @@ -162,7 +164,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & descr.step_func = [step, &time_zone = date_time64->getTimeZone()](Field & field) \ { \ auto field_decimal = field.get>(); \ - auto res = Add##NAME##sImpl::execute(field_decimal.getValue(), step, time_zone, field_decimal.getScale()); \ + auto res = Add##NAME##sImpl::execute(field_decimal.getValue(), step, time_zone, utc_time_zone, field_decimal.getScale()); \ field = DecimalField(res, field_decimal.getScale()); \ }; \ break; diff --git a/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.reference b/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.reference new file mode 100644 index 00000000000..2de0542191e --- /dev/null +++ b/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.reference @@ -0,0 +1,125 @@ +-- const date, const delta + -- add +2024-10-22 00:00:00.000 2024-10-22 00:00:00.000 +2024-01-22 00:00:00.000 2024-01-22 00:00:00.000 +2023-11-22 00:00:00.000 2023-11-22 00:00:00.000 +2023-10-29 00:00:00.000 2023-10-29 00:00:00.000 +2023-10-23 00:00:00.000 2023-10-23 00:00:00.000 +2023-10-22 01:00:00.000 2023-10-22 01:00:00.000 +2023-10-22 00:01:00.000 2023-10-22 00:01:00.000 +2023-10-22 00:00:01.000 2023-10-22 00:00:01.000 +2023-10-22 00:00:00.001 2023-10-22 00:00:00.001 +2023-10-22 00:00:00.000001 2023-10-22 00:00:00.000001 +2023-10-22 00:00:00.000000001 2023-10-22 00:00:00.000000001 + -- subtract +2022-10-22 00:00:00.000 2022-10-22 00:00:00.000 +2023-07-22 00:00:00.000 2023-07-22 00:00:00.000 +2023-09-22 00:00:00.000 2023-09-22 00:00:00.000 +2023-10-15 00:00:00.000 2023-10-15 00:00:00.000 +2023-10-21 00:00:00.000 2023-10-21 00:00:00.000 +2023-10-21 23:00:00.000 2023-10-21 23:00:00.000 +2023-10-21 23:59:00.000 2023-10-21 23:59:00.000 +2023-10-21 23:59:59.000 2023-10-21 23:59:59.000 +2023-10-21 23:59:59.999 2023-10-21 23:59:59.999 +2023-10-21 23:59:59.999999 2023-10-21 23:59:59.999999 +2023-10-21 23:59:59.999999999 2023-10-21 23:59:59.999999999 +-- non-const date, const delta + -- add +2024-10-22 00:00:00.000 2024-10-22 00:00:00.000 +2024-01-22 00:00:00.000 2024-01-22 00:00:00.000 +2023-11-22 00:00:00.000 2023-11-22 00:00:00.000 +2023-10-29 00:00:00.000 2023-10-29 00:00:00.000 +2023-10-23 00:00:00.000 2023-10-23 00:00:00.000 +2023-10-22 01:00:00.000 2023-10-22 01:00:00.000 +2023-10-22 00:01:00.000 2023-10-22 00:01:00.000 +2023-10-22 00:00:01.000 2023-10-22 00:00:01.000 +2023-10-22 00:00:00.001 2023-10-22 00:00:00.001 +2023-10-22 00:00:00.000001 2023-10-22 00:00:00.000001 +2023-10-22 00:00:00.000000001 2023-10-22 00:00:00.000000001 + -- subtract +2022-10-22 00:00:00.000 2022-10-22 00:00:00.000 +2023-07-22 00:00:00.000 2023-07-22 00:00:00.000 +2023-09-22 00:00:00.000 2023-09-22 00:00:00.000 +2023-10-15 00:00:00.000 2023-10-15 00:00:00.000 +2023-10-21 00:00:00.000 2023-10-21 00:00:00.000 +2023-10-21 23:00:00.000 2023-10-21 23:00:00.000 +2023-10-21 23:59:00.000 2023-10-21 23:59:00.000 +2023-10-21 23:59:59.000 2023-10-21 23:59:59.000 +2023-10-21 23:59:59.999 2023-10-21 23:59:59.999 +2023-10-21 23:59:59.999999 2023-10-21 23:59:59.999999 +2023-10-21 23:59:59.999999999 2023-10-21 23:59:59.999999999 +-- const date, non-const delta + -- add +2024-10-22 00:00:00.000 2024-10-22 00:00:00.000 +2024-01-22 00:00:00.000 2024-01-22 00:00:00.000 +2023-11-22 00:00:00.000 2023-11-22 00:00:00.000 +2023-10-29 00:00:00.000 2023-10-29 00:00:00.000 +2023-10-23 00:00:00.000 2023-10-23 00:00:00.000 +2023-10-22 01:00:00.000 2023-10-22 01:00:00.000 +2023-10-22 00:01:00.000 2023-10-22 00:01:00.000 +2023-10-22 00:00:01.000 2023-10-22 00:00:01.000 +2023-10-22 00:00:00.001 2023-10-22 00:00:00.001 +2023-10-22 00:00:00.000001 2023-10-22 00:00:00.000001 +2023-10-22 00:00:00.000000001 2023-10-22 00:00:00.000000001 + -- subtract +2022-10-22 00:00:00.000 2022-10-22 00:00:00.000 +2023-07-22 00:00:00.000 2023-07-22 00:00:00.000 +2023-09-22 00:00:00.000 2023-09-22 00:00:00.000 +2023-10-15 00:00:00.000 2023-10-15 00:00:00.000 +2023-10-21 00:00:00.000 2023-10-21 00:00:00.000 +2023-10-21 23:00:00.000 2023-10-21 23:00:00.000 +2023-10-21 23:59:00.000 2023-10-21 23:59:00.000 +2023-10-21 23:59:59.000 2023-10-21 23:59:59.000 +2023-10-21 23:59:59.999 2023-10-21 23:59:59.999 +2023-10-21 23:59:59.999999 2023-10-21 23:59:59.999999 +2023-10-21 23:59:59.999999999 2023-10-21 23:59:59.999999999 +-- non-const date, non-const delta + -- add +2024-10-22 00:00:00.000 2024-10-22 00:00:00.000 +2024-01-22 00:00:00.000 2024-01-22 00:00:00.000 +2023-11-22 00:00:00.000 2023-11-22 00:00:00.000 +2023-10-29 00:00:00.000 2023-10-29 00:00:00.000 +2023-10-23 00:00:00.000 2023-10-23 00:00:00.000 +2023-10-22 01:00:00.000 2023-10-22 01:00:00.000 +2023-10-22 00:01:00.000 2023-10-22 00:01:00.000 +2023-10-22 00:00:01.000 2023-10-22 00:00:01.000 +2023-10-22 00:00:00.001 2023-10-22 00:00:00.001 +2023-10-22 00:00:00.000001 2023-10-22 00:00:00.000001 +2023-10-22 00:00:00.000000001 2023-10-22 00:00:00.000000001 + -- subtract +2022-10-22 00:00:00.000 2022-10-22 00:00:00.000 +2023-07-22 00:00:00.000 2023-07-22 00:00:00.000 +2023-09-22 00:00:00.000 2023-09-22 00:00:00.000 +2023-10-15 00:00:00.000 2023-10-15 00:00:00.000 +2023-10-21 00:00:00.000 2023-10-21 00:00:00.000 +2023-10-21 23:00:00.000 2023-10-21 23:00:00.000 +2023-10-21 23:59:00.000 2023-10-21 23:59:00.000 +2023-10-21 23:59:59.000 2023-10-21 23:59:59.000 +2023-10-21 23:59:59.999 2023-10-21 23:59:59.999 +2023-10-21 23:59:59.999999 2023-10-21 23:59:59.999999 +2023-10-21 23:59:59.999999999 2023-10-21 23:59:59.999999999 +-- plus operator + -- add +2024-10-23 00:00:00.000 +2024-01-23 00:00:00.000 +2023-11-23 00:00:00.000 +2023-10-30 00:00:00.000 +2023-10-24 00:00:00.000 +2023-10-23 01:00:00.000 +2023-10-23 00:01:00.000 +2023-10-23 00:00:01.000 +2023-10-23 00:00:00.001 +2023-10-23 00:00:00.000001 +2023-10-23 00:00:00.000000001 + -- subtract +2022-10-23 00:00:00.000 +2023-07-23 00:00:00.000 +2023-09-23 00:00:00.000 +2023-10-16 00:00:00.000 +2023-10-22 00:00:00.000 +2023-10-22 23:00:00.000 +2023-10-22 23:59:00.000 +2023-10-22 23:59:59.000 +2023-10-22 23:59:59.999 +2023-10-22 23:59:59.999999 +2023-10-22 23:59:59.999999999 diff --git a/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.sql b/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.sql new file mode 100644 index 00000000000..da92dea5127 --- /dev/null +++ b/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.sql @@ -0,0 +1,139 @@ +SELECT '-- const date, const delta'; + +SELECT ' -- add'; +SELECT addYears('2023-10-22', 1), addYears('2023-10-22 00:00:00.000', 1); +SELECT addQuarters('2023-10-22', 1), addQuarters('2023-10-22 00:00:00.000', 1); +SELECT addMonths('2023-10-22', 1), addMonths('2023-10-22 00:00:00.000', 1); +SELECT addWeeks('2023-10-22', 1), addWeeks('2023-10-22 00:00:00.000', 1); +SELECT addDays('2023-10-22', 1), addDays('2023-10-22 00:00:00.000', 1); +SELECT addHours('2023-10-22', 1), addHours('2023-10-22 00:00:00.000', 1); +SELECT addMinutes('2023-10-22', 1), addMinutes('2023-10-22 00:00:00.000', 1); +SELECT addSeconds('2023-10-22', 1), addSeconds('2023-10-22 00:00:00.000', 1); +SELECT addMilliseconds('2023-10-22', 1), addMilliseconds('2023-10-22 00:00:00.000', 1); +SELECT addMicroseconds('2023-10-22', 1), addMicroseconds('2023-10-22 00:00:00.000', 1); +SELECT addNanoseconds('2023-10-22', 1), addNanoseconds('2023-10-22 00:00:00.000', 1); + +SELECT ' -- subtract'; +SELECT subtractYears('2023-10-22', 1), subtractYears('2023-10-22 00:00:00.000', 1); +SELECT subtractQuarters('2023-10-22', 1), subtractQuarters('2023-10-22 00:00:00.000', 1); +SELECT subtractMonths('2023-10-22', 1), subtractMonths('2023-10-22 00:00:00.000', 1); +SELECT subtractWeeks('2023-10-22', 1), subtractWeeks('2023-10-22 00:00:00.000', 1); +SELECT subtractDays('2023-10-22', 1), subtractDays('2023-10-22 00:00:00.000', 1); +SELECT subtractHours('2023-10-22', 1), subtractHours('2023-10-22 00:00:00.000', 1); +SELECT subtractMinutes('2023-10-22', 1), subtractMinutes('2023-10-22 00:00:00.000', 1); +SELECT subtractSeconds('2023-10-22', 1), subtractSeconds('2023-10-22 00:00:00.000', 1); +SELECT subtractMilliseconds('2023-10-22', 1), subtractMilliseconds('2023-10-22 00:00:00.000', 1); +SELECT subtractMicroseconds('2023-10-22', 1), subtractMicroseconds('2023-10-22 00:00:00.000', 1); +SELECT subtractNanoseconds('2023-10-22', 1), subtractNanoseconds('2023-10-22 00:00:00.000', 1); + +SELECT '-- non-const date, const delta'; + +SELECT ' -- add'; +SELECT addYears(materialize('2023-10-22'), 1), addYears(materialize('2023-10-22 00:00:00.000'), 1); +SELECT addQuarters(materialize('2023-10-22'), 1), addQuarters(materialize('2023-10-22 00:00:00.000'), 1); +SELECT addMonths(materialize('2023-10-22'), 1), addMonths(materialize('2023-10-22 00:00:00.000'), 1); +SELECT addWeeks(materialize('2023-10-22'), 1), addWeeks(materialize('2023-10-22 00:00:00.000'), 1); +SELECT addDays(materialize('2023-10-22'), 1), addDays(materialize('2023-10-22 00:00:00.000'), 1); +SELECT addHours(materialize('2023-10-22'), 1), addHours(materialize('2023-10-22 00:00:00.000'), 1); +SELECT addMinutes(materialize('2023-10-22'), 1), addMinutes(materialize('2023-10-22 00:00:00.000'), 1); +SELECT addSeconds(materialize('2023-10-22'), 1), addSeconds(materialize('2023-10-22 00:00:00.000'), 1); +SELECT addMilliseconds(materialize('2023-10-22'), 1), addMilliseconds(materialize('2023-10-22 00:00:00.000'), 1); +SELECT addMicroseconds(materialize('2023-10-22'), 1), addMicroseconds(materialize('2023-10-22 00:00:00.000'), 1); +SELECT addNanoseconds(materialize('2023-10-22'), 1), addNanoseconds(materialize('2023-10-22 00:00:00.000'), 1); + +SELECT ' -- subtract'; +SELECT subtractYears(materialize('2023-10-22'), 1), subtractYears(materialize('2023-10-22 00:00:00.000'), 1); +SELECT subtractQuarters(materialize('2023-10-22'), 1), subtractQuarters(materialize('2023-10-22 00:00:00.000'), 1); +SELECT subtractMonths(materialize('2023-10-22'), 1), subtractMonths(materialize('2023-10-22 00:00:00.000'), 1); +SELECT subtractWeeks(materialize('2023-10-22'), 1), subtractWeeks(materialize('2023-10-22 00:00:00.000'), 1); +SELECT subtractDays(materialize('2023-10-22'), 1), subtractDays(materialize('2023-10-22 00:00:00.000'), 1); +SELECT subtractHours(materialize('2023-10-22'), 1), subtractHours(materialize('2023-10-22 00:00:00.000'), 1); +SELECT subtractMinutes(materialize('2023-10-22'), 1), subtractMinutes(materialize('2023-10-22 00:00:00.000'), 1); +SELECT subtractSeconds(materialize('2023-10-22'), 1), subtractSeconds(materialize('2023-10-22 00:00:00.000'), 1); +SELECT subtractMilliseconds(materialize('2023-10-22'), 1), subtractMilliseconds(materialize('2023-10-22 00:00:00.000'), 1); +SELECT subtractMicroseconds(materialize('2023-10-22'), 1), subtractMicroseconds(materialize('2023-10-22 00:00:00.000'), 1); +SELECT subtractNanoseconds(materialize('2023-10-22'), 1), subtractNanoseconds(materialize('2023-10-22 00:00:00.000'), 1); + +SELECT '-- const date, non-const delta'; + +SELECT ' -- add'; +SELECT addYears('2023-10-22', materialize(1)), addYears('2023-10-22 00:00:00.000', materialize(1)); +SELECT addQuarters('2023-10-22', materialize(1)), addQuarters('2023-10-22 00:00:00.000', materialize(1)); +SELECT addMonths('2023-10-22', materialize(1)), addMonths('2023-10-22 00:00:00.000', materialize(1)); +SELECT addWeeks('2023-10-22', materialize(1)), addWeeks('2023-10-22 00:00:00.000', materialize(1)); +SELECT addDays('2023-10-22', materialize(1)), addDays('2023-10-22 00:00:00.000', materialize(1)); +SELECT addHours('2023-10-22', materialize(1)), addHours('2023-10-22 00:00:00.000', materialize(1)); +SELECT addMinutes('2023-10-22', materialize(1)), addMinutes('2023-10-22 00:00:00.000', materialize(1)); +SELECT addSeconds('2023-10-22', materialize(1)), addSeconds('2023-10-22 00:00:00.000', materialize(1)); +SELECT addMilliseconds('2023-10-22', materialize(1)), addMilliseconds('2023-10-22 00:00:00.000', materialize(1)); +SELECT addMicroseconds('2023-10-22', materialize(1)), addMicroseconds('2023-10-22 00:00:00.000', materialize(1)); +SELECT addNanoseconds('2023-10-22', materialize(1)), addNanoseconds('2023-10-22 00:00:00.000', materialize(1)); + +SELECT ' -- subtract'; +SELECT subtractYears('2023-10-22', materialize(1)), subtractYears('2023-10-22 00:00:00.000', materialize(1)); +SELECT subtractQuarters('2023-10-22', materialize(1)), subtractQuarters('2023-10-22 00:00:00.000', materialize(1)); +SELECT subtractMonths('2023-10-22', materialize(1)), subtractMonths('2023-10-22 00:00:00.000', materialize(1)); +SELECT subtractWeeks('2023-10-22', materialize(1)), subtractWeeks('2023-10-22 00:00:00.000', materialize(1)); +SELECT subtractDays('2023-10-22', materialize(1)), subtractDays('2023-10-22 00:00:00.000', materialize(1)); +SELECT subtractHours('2023-10-22', materialize(1)), subtractHours('2023-10-22 00:00:00.000', materialize(1)); +SELECT subtractMinutes('2023-10-22', materialize(1)), subtractMinutes('2023-10-22 00:00:00.000', materialize(1)); +SELECT subtractSeconds('2023-10-22', materialize(1)), subtractSeconds('2023-10-22 00:00:00.000', materialize(1)); +SELECT subtractMilliseconds('2023-10-22', materialize(1)), subtractMilliseconds('2023-10-22 00:00:00.000', materialize(1)); +SELECT subtractMicroseconds('2023-10-22', materialize(1)), subtractMicroseconds('2023-10-22 00:00:00.000', materialize(1)); +SELECT subtractNanoseconds('2023-10-22', materialize(1)), subtractNanoseconds('2023-10-22 00:00:00.000', materialize(1)); + +SELECT '-- non-const date, non-const delta'; + +SELECT ' -- add'; +SELECT addYears(materialize('2023-10-22'), materialize(1)), addYears(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT addQuarters(materialize('2023-10-22'), materialize(1)), addQuarters(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT addMonths(materialize('2023-10-22'), materialize(1)), addMonths(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT addWeeks(materialize('2023-10-22'), materialize(1)), addWeeks(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT addDays(materialize('2023-10-22'), materialize(1)), addDays(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT addHours(materialize('2023-10-22'), materialize(1)), addHours(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT addMinutes(materialize('2023-10-22'), materialize(1)), addMinutes(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT addSeconds(materialize('2023-10-22'), materialize(1)), addSeconds(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT addMilliseconds(materialize('2023-10-22'), materialize(1)), addMilliseconds(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT addMicroseconds(materialize('2023-10-22'), materialize(1)), addMicroseconds(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT addNanoseconds(materialize('2023-10-22'), materialize(1)), addNanoseconds(materialize('2023-10-22 00:00:00.000'), materialize(1)); + +SELECT ' -- subtract'; +SELECT subtractYears(materialize('2023-10-22'), materialize(1)), subtractYears(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT subtractQuarters(materialize('2023-10-22'), materialize(1)), subtractQuarters(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT subtractMonths(materialize('2023-10-22'), materialize(1)), subtractMonths(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT subtractWeeks(materialize('2023-10-22'), materialize(1)), subtractWeeks(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT subtractDays(materialize('2023-10-22'), materialize(1)), subtractDays(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT subtractHours(materialize('2023-10-22'), materialize(1)), subtractHours(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT subtractMinutes(materialize('2023-10-22'), materialize(1)), subtractMinutes(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT subtractSeconds(materialize('2023-10-22'), materialize(1)), subtractSeconds(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT subtractMilliseconds(materialize('2023-10-22'), materialize(1)), subtractMilliseconds(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT subtractMicroseconds(materialize('2023-10-22'), materialize(1)), subtractMicroseconds(materialize('2023-10-22 00:00:00.000'), materialize(1)); +SELECT subtractNanoseconds(materialize('2023-10-22'), materialize(1)), subtractNanoseconds(materialize('2023-10-22 00:00:00.000'), 1); + +SELECT '-- plus operator'; + +SELECT '2023-10-23' + INTERVAL 1 YEAR; +SELECT '2023-10-23' + INTERVAL 1 QUARTER; +SELECT '2023-10-23' + INTERVAL 1 MONTH; +SELECT '2023-10-23' + INTERVAL 1 WEEK; +SELECT '2023-10-23' + INTERVAL 1 DAY; +SELECT '2023-10-23' + INTERVAL 1 HOUR; +SELECT '2023-10-23' + INTERVAL 1 MINUTE; +SELECT '2023-10-23' + INTERVAL 1 SECOND; +SELECT '2023-10-23' + INTERVAL 1 MILLISECOND; +SELECT '2023-10-23' + INTERVAL 1 MICROSECOND; +SELECT '2023-10-23' + INTERVAL 1 NANOSECOND; + +SELECT '-- minus operator'; + +SELECT '2023-10-23' - INTERVAL 1 YEAR; +SELECT '2023-10-23' - INTERVAL 1 QUARTER; +SELECT '2023-10-23' - INTERVAL 1 MONTH; +SELECT '2023-10-23' - INTERVAL 1 WEEK; +SELECT '2023-10-23' - INTERVAL 1 DAY; +SELECT '2023-10-23' - INTERVAL 1 HOUR; +SELECT '2023-10-23' - INTERVAL 1 MINUTE; +SELECT '2023-10-23' - INTERVAL 1 SECOND; +SELECT '2023-10-23' - INTERVAL 1 MILLISECOND; +SELECT '2023-10-23' - INTERVAL 1 MICROSECOND; +SELECT '2023-10-23' - INTERVAL 1 NANOSECOND; From 957a83dbccc2079f0731d8bdf3d726c88ebefb60 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 22 Oct 2023 11:30:58 +0000 Subject: [PATCH 480/634] Fix tests --- tests/queries/0_stateless/01923_ttl_with_modify_column.sql | 2 +- .../02900_add_subtract_interval_with_string_date.reference | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01923_ttl_with_modify_column.sql b/tests/queries/0_stateless/01923_ttl_with_modify_column.sql index ed2812d2a39..650f32fb588 100644 --- a/tests/queries/0_stateless/01923_ttl_with_modify_column.sql +++ b/tests/queries/0_stateless/01923_ttl_with_modify_column.sql @@ -38,6 +38,6 @@ INSERT INTO t_ttl_modify_column VALUES (now()); SELECT sum(rows), groupUniqArray(type) FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_ttl_modify_column' AND column = 'InsertionDateTime' AND active; -ALTER TABLE t_ttl_modify_column MODIFY COLUMN InsertionDateTime String; -- { serverError 43 } +ALTER TABLE t_ttl_modify_column MODIFY COLUMN InsertionDateTime Float32; -- { serverError 43 } DROP TABLE IF EXISTS t_ttl_modify_column; diff --git a/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.reference b/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.reference index 2de0542191e..871abbd890f 100644 --- a/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.reference +++ b/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.reference @@ -99,7 +99,6 @@ 2023-10-21 23:59:59.999999 2023-10-21 23:59:59.999999 2023-10-21 23:59:59.999999999 2023-10-21 23:59:59.999999999 -- plus operator - -- add 2024-10-23 00:00:00.000 2024-01-23 00:00:00.000 2023-11-23 00:00:00.000 @@ -111,7 +110,7 @@ 2023-10-23 00:00:00.001 2023-10-23 00:00:00.000001 2023-10-23 00:00:00.000000001 - -- subtract +-- minus operator 2022-10-23 00:00:00.000 2023-07-23 00:00:00.000 2023-09-23 00:00:00.000 From b2bdb58c09e87f83816b27d492a1cd851fd45725 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 22 Oct 2023 15:23:12 +0000 Subject: [PATCH 481/634] Bump gRPC to v1.39.1 --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index 48d1d322358..d6a8479aae9 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 48d1d322358658d8ac6d9b284de9f4ad1ac2bc6b +Subproject commit d6a8479aae92423fa9b72dc3f8ee4a061da3a5fc From d450314d272576230b2e6ee4a0272b1785e0b158 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 22 Oct 2023 15:40:45 +0000 Subject: [PATCH 482/634] Fix include --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index d6a8479aae9..80e8fd63fef 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit d6a8479aae92423fa9b72dc3f8ee4a061da3a5fc +Subproject commit 80e8fd63fef4a8d35c1abe612854f238eadadf0a From 77e7f71e02bb71e2f571d0897c9dc4fb684d6db3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 22 Oct 2023 19:55:17 +0000 Subject: [PATCH 483/634] Add sanity check to RPNBuilderFunctionTreNode --- src/Storages/MergeTree/RPNBuilder.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 29bcd8b87f1..1edfac65da4 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -396,6 +396,12 @@ size_t RPNBuilderFunctionTreeNode::getArgumentsSize() const RPNBuilderTreeNode RPNBuilderFunctionTreeNode::getArgumentAt(size_t index) const { + size_t total_arguments = getArgumentsSize(); + if (index >= total_arguments) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "RPNBuilderFunctionTreeNode has {} arguments, attempted to get argument at index {}", + total_arguments, index); + if (ast_node) { const auto * ast_function = assert_cast(ast_node); From 20316c34e9c3ce37f84e922d29e2e3fab94b423d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 22 Oct 2023 20:25:43 +0000 Subject: [PATCH 484/634] Bump gRPC to v1.42.0 --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index 80e8fd63fef..bef8212d1e0 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 80e8fd63fef4a8d35c1abe612854f238eadadf0a +Subproject commit bef8212d1e01f99e406c282ceab3d42da08e09ce From 80a048d5f89d7b7a49463ceea41d05d745f33c89 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 22 Oct 2023 20:31:22 +0000 Subject: [PATCH 485/634] Tempoarily revert string support in plus and minus operators --- src/Functions/FunctionBinaryArithmetic.h | 32 +++++++++---------- .../01923_ttl_with_modify_column.sql | 2 +- ...btract_interval_with_string_date.reference | 24 -------------- ...add_subtract_interval_with_string_date.sql | 28 ---------------- 4 files changed, 17 insertions(+), 69 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 2dd6c582157..33ed22b9fbd 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -796,21 +796,21 @@ class FunctionBinaryArithmetic : public IFunction static FunctionOverloadResolverPtr getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { - bool first_arg_is_date_or_datetime_or_string = isDateOrDate32OrDateTimeOrDateTime64(type0) || isString(type0); - bool second_arg_is_date_or_datetime_or_string = isDateOrDate32OrDateTimeOrDateTime64(type1) || isString(type1); + bool first_is_date_or_datetime = isDateOrDate32(type0) || isDateTime(type0) || isDateTime64(type0); + bool second_is_date_or_datetime = isDateOrDate32(type1) || isDateTime(type1) || isDateTime64(type1); - /// Exactly one argument must be Date or DateTime or String - if (first_arg_is_date_or_datetime_or_string == second_arg_is_date_or_datetime_or_string) + /// Exactly one argument must be Date or DateTime + if (first_is_date_or_datetime == second_is_date_or_datetime) return {}; - /// Special case when the function is plus or minus, one of arguments is Date or DateTime or String and another is Interval. + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. /// We construct another function (example: addMonths) and call it. if constexpr (!is_plus && !is_minus) return {}; - const DataTypePtr & type_time = first_arg_is_date_or_datetime_or_string ? type0 : type1; - const DataTypePtr & type_interval = first_arg_is_date_or_datetime_or_string ? type1 : type0; + const DataTypePtr & type_time = first_is_date_or_datetime ? type0 : type1; + const DataTypePtr & type_interval = first_is_date_or_datetime ? type1 : type0; bool interval_is_number = isNumber(type_interval); @@ -823,7 +823,7 @@ class FunctionBinaryArithmetic : public IFunction return {}; } - if (second_arg_is_date_or_datetime_or_string && is_minus) + if (second_is_date_or_datetime && is_minus) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong order of arguments for function {}: " "argument of type Interval cannot be first", name); @@ -848,11 +848,11 @@ class FunctionBinaryArithmetic : public IFunction static FunctionOverloadResolverPtr getFunctionForDateTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { - bool first_arg_is_date_or_datetime = isDateOrDate32OrDateTimeOrDateTime64(type0); - bool second_arg_is_date_or_datetime = isDateOrDate32OrDateTimeOrDateTime64(type1); + bool first_is_date_or_datetime = isDateOrDate32(type0) || isDateTime(type0) || isDateTime64(type0); + bool second_is_date_or_datetime = isDateOrDate32(type1) || isDateTime(type1) || isDateTime64(type1); /// Exactly one argument must be Date or DateTime - if (first_arg_is_date_or_datetime == second_arg_is_date_or_datetime) + if (first_is_date_or_datetime == second_is_date_or_datetime) return {}; if (!isTuple(type0) && !isTuple(type1)) @@ -863,7 +863,7 @@ class FunctionBinaryArithmetic : public IFunction if constexpr (!is_plus && !is_minus) return {}; - if (isTuple(type0) && second_arg_is_date_or_datetime && is_minus) + if (isTuple(type0) && second_is_date_or_datetime && is_minus) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong order of arguments for function {}: " "argument of Tuple type cannot be first", name); @@ -1123,7 +1123,7 @@ class FunctionBinaryArithmetic : public IFunction ColumnsWithTypeAndName new_arguments = arguments; /// Interval argument must be second. - if (isDateOrDate32OrDateTimeOrDateTime64(arguments[1].type) || isString(arguments[1].type)) + if (isDateOrDate32(arguments[1].type) || isDateTime(arguments[1].type) || isDateTime64(arguments[1].type)) std::swap(new_arguments[0], new_arguments[1]); /// Change interval argument type to its representation @@ -1511,7 +1511,7 @@ public: } } - /// Special case when the function is plus or minus, one of arguments is Date/DateTime/String and another is Interval. + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1], context)) { ColumnsWithTypeAndName new_arguments(2); @@ -1520,7 +1520,7 @@ public: new_arguments[i].type = arguments[i]; /// Interval argument must be second. - if (isDateOrDate32OrDateTimeOrDateTime64(new_arguments[1].type) || isString(new_arguments[1].type)) + if (isDateOrDate32(new_arguments[1].type) || isDateTime(new_arguments[1].type) || isDateTime64(new_arguments[1].type)) std::swap(new_arguments[0], new_arguments[1]); /// Change interval argument to its representation @@ -2104,7 +2104,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A return executeAggregateAddition(arguments, result_type, input_rows_count); } - /// Special case when the function is plus or minus, one of arguments is Date/DateTime/String and another is Interval. + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0].type, arguments[1].type, context)) { return executeDateTimeIntervalPlusMinus(arguments, result_type, input_rows_count, function_builder); diff --git a/tests/queries/0_stateless/01923_ttl_with_modify_column.sql b/tests/queries/0_stateless/01923_ttl_with_modify_column.sql index 650f32fb588..ed2812d2a39 100644 --- a/tests/queries/0_stateless/01923_ttl_with_modify_column.sql +++ b/tests/queries/0_stateless/01923_ttl_with_modify_column.sql @@ -38,6 +38,6 @@ INSERT INTO t_ttl_modify_column VALUES (now()); SELECT sum(rows), groupUniqArray(type) FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_ttl_modify_column' AND column = 'InsertionDateTime' AND active; -ALTER TABLE t_ttl_modify_column MODIFY COLUMN InsertionDateTime Float32; -- { serverError 43 } +ALTER TABLE t_ttl_modify_column MODIFY COLUMN InsertionDateTime String; -- { serverError 43 } DROP TABLE IF EXISTS t_ttl_modify_column; diff --git a/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.reference b/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.reference index 871abbd890f..fa82844c9f3 100644 --- a/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.reference +++ b/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.reference @@ -98,27 +98,3 @@ 2023-10-21 23:59:59.999 2023-10-21 23:59:59.999 2023-10-21 23:59:59.999999 2023-10-21 23:59:59.999999 2023-10-21 23:59:59.999999999 2023-10-21 23:59:59.999999999 --- plus operator -2024-10-23 00:00:00.000 -2024-01-23 00:00:00.000 -2023-11-23 00:00:00.000 -2023-10-30 00:00:00.000 -2023-10-24 00:00:00.000 -2023-10-23 01:00:00.000 -2023-10-23 00:01:00.000 -2023-10-23 00:00:01.000 -2023-10-23 00:00:00.001 -2023-10-23 00:00:00.000001 -2023-10-23 00:00:00.000000001 --- minus operator -2022-10-23 00:00:00.000 -2023-07-23 00:00:00.000 -2023-09-23 00:00:00.000 -2023-10-16 00:00:00.000 -2023-10-22 00:00:00.000 -2023-10-22 23:00:00.000 -2023-10-22 23:59:00.000 -2023-10-22 23:59:59.000 -2023-10-22 23:59:59.999 -2023-10-22 23:59:59.999999 -2023-10-22 23:59:59.999999999 diff --git a/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.sql b/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.sql index da92dea5127..6a02840617b 100644 --- a/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.sql +++ b/tests/queries/0_stateless/02900_add_subtract_interval_with_string_date.sql @@ -109,31 +109,3 @@ SELECT subtractSeconds(materialize('2023-10-22'), materialize(1)), subtractSecon SELECT subtractMilliseconds(materialize('2023-10-22'), materialize(1)), subtractMilliseconds(materialize('2023-10-22 00:00:00.000'), materialize(1)); SELECT subtractMicroseconds(materialize('2023-10-22'), materialize(1)), subtractMicroseconds(materialize('2023-10-22 00:00:00.000'), materialize(1)); SELECT subtractNanoseconds(materialize('2023-10-22'), materialize(1)), subtractNanoseconds(materialize('2023-10-22 00:00:00.000'), 1); - -SELECT '-- plus operator'; - -SELECT '2023-10-23' + INTERVAL 1 YEAR; -SELECT '2023-10-23' + INTERVAL 1 QUARTER; -SELECT '2023-10-23' + INTERVAL 1 MONTH; -SELECT '2023-10-23' + INTERVAL 1 WEEK; -SELECT '2023-10-23' + INTERVAL 1 DAY; -SELECT '2023-10-23' + INTERVAL 1 HOUR; -SELECT '2023-10-23' + INTERVAL 1 MINUTE; -SELECT '2023-10-23' + INTERVAL 1 SECOND; -SELECT '2023-10-23' + INTERVAL 1 MILLISECOND; -SELECT '2023-10-23' + INTERVAL 1 MICROSECOND; -SELECT '2023-10-23' + INTERVAL 1 NANOSECOND; - -SELECT '-- minus operator'; - -SELECT '2023-10-23' - INTERVAL 1 YEAR; -SELECT '2023-10-23' - INTERVAL 1 QUARTER; -SELECT '2023-10-23' - INTERVAL 1 MONTH; -SELECT '2023-10-23' - INTERVAL 1 WEEK; -SELECT '2023-10-23' - INTERVAL 1 DAY; -SELECT '2023-10-23' - INTERVAL 1 HOUR; -SELECT '2023-10-23' - INTERVAL 1 MINUTE; -SELECT '2023-10-23' - INTERVAL 1 SECOND; -SELECT '2023-10-23' - INTERVAL 1 MILLISECOND; -SELECT '2023-10-23' - INTERVAL 1 MICROSECOND; -SELECT '2023-10-23' - INTERVAL 1 NANOSECOND; From 6fbc5277b6995ac06ddd2cb34b5591e84453790e Mon Sep 17 00:00:00 2001 From: jetgm Date: Mon, 23 Oct 2023 09:56:19 +0800 Subject: [PATCH 486/634] improve query --- src/Interpreters/InterpreterShowTablesQuery.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 94c513aa1e6..e5a286a15e1 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -120,7 +120,9 @@ String InterpreterShowTablesQuery::getRewrittenQuery() if (query.merges) { WriteBufferFromOwnString rewritten_query; - rewritten_query << "SELECT table, database, elapsed, merge_type FROM system.merges"; + rewritten_query << "SELECT table, database, round((elapsed * (1 / progress)) - elapsed, 2) AS estimate_complete, elapsed, " + "round(progress, 2) AS progress, is_mutation, formatReadableSize(total_size_bytes_compressed) AS size, " + "formatReadableSize(memory_usage) AS mem FROM system.merges"; if (!query.like.empty()) { From 839c05e3b8816686d6e0ce3bbb749ca2448f2100 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Fri, 20 Oct 2023 11:25:59 +0200 Subject: [PATCH 487/634] Add load_metadata_threads to describe filesystem cache --- src/Interpreters/InterpreterDescribeCacheQuery.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/InterpreterDescribeCacheQuery.cpp b/src/Interpreters/InterpreterDescribeCacheQuery.cpp index 875c133bb76..2ef108b5d17 100644 --- a/src/Interpreters/InterpreterDescribeCacheQuery.cpp +++ b/src/Interpreters/InterpreterDescribeCacheQuery.cpp @@ -27,6 +27,7 @@ static Block getSampleBlock() ColumnWithTypeAndName{std::make_shared(), "path"}, ColumnWithTypeAndName{std::make_shared>(), "background_download_threads"}, ColumnWithTypeAndName{std::make_shared>(), "enable_bypass_cache_with_threshold"}, + ColumnWithTypeAndName{std::make_shared>(), "load_metadata_threads"}, }; return Block(columns); } @@ -55,6 +56,7 @@ BlockIO InterpreterDescribeCacheQuery::execute() res_columns[i++]->insert(cache->getBasePath()); res_columns[i++]->insert(settings.background_download_threads); res_columns[i++]->insert(settings.enable_bypass_cache_with_threshold); + res_columns[i++]->insert(settings.load_metadata_threads); BlockIO res; size_t num_rows = res_columns[0]->size(); From 08ae38e1301e0d9910702828cd083b97d734d734 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Fri, 20 Oct 2023 12:23:53 +0200 Subject: [PATCH 488/634] Fix 02344_describe_cache test --- tests/queries/0_stateless/02344_describe_cache.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 7f3a881e090..273c7c08907 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1,2 @@ 1 -102400 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/02344_describe_cache_test 2 0 +102400 10000000 33554432 4194304 0 0 0 0 /var/lib/clickhouse/filesystem_caches/02344_describe_cache_test 2 0 1 From c885314bfe2d5e6eddd77dae5e20c8e9ab2c03d3 Mon Sep 17 00:00:00 2001 From: Joris Clement <7713214+joclement@users.noreply.github.com> Date: Mon, 23 Oct 2023 09:41:24 +0200 Subject: [PATCH 489/634] docs: state limitations of `USING` clause. State that it is (currently) not possible to use `USING` with multiple joins. The exception thrown is `Multiple USING statements are not supported`. The relevant code section is https://github.com/ClickHouse/ClickHouse/blob/d08ce131074209fbeb6e3590ae7f2d5baae7fc6a/src/Interpreters/CrossToInnerJoinVisitor.cpp#L191-L192. --- docs/en/sql-reference/statements/select/join.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 38922e964b8..281a1d0436c 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -334,6 +334,7 @@ For multiple `JOIN` clauses in a single `SELECT` query: - Taking all the columns via `*` is available only if tables are joined, not subqueries. - The `PREWHERE` clause is not available. +- The `USING` clause is not available. For `ON`, `WHERE`, and `GROUP BY` clauses: From 8417b64530815d6d123406e01b941d017313580d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 23 Oct 2023 05:27:26 -0300 Subject: [PATCH 490/634] Update array-functions.md --- .../functions/array-functions.md | 27 ++++++++++++++++--- 1 file changed, 24 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 73c51a41dfb..2b6d7e73c34 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1144,11 +1144,32 @@ SELECT arrayFold( x,acc -> acc + x*2, [1, 2, 3, 4], toInt64(3)) AS res; Result: ``` text -┌─arrayFold(lambda(tuple(x, acc), plus(acc, multiply(x, 2))), [1, 2, 3, 4], toInt64(3))─┐ -│ 3 │ -└───────────────────────────────────────────────────────────────────────────────────────┘ +┌─res─┐ +│ 23 │ +└─────┘ ``` +**Example with fibonacci sequence** + +```sql +SELECT arrayFold( x, acc -> (acc.2, acc.2 + acc.1), range(number), (1::Int64, 0::Int64)).1 AS result +FROM numbers(1,10); + +┌─result─┐ +│ 0 │ +│ 1 │ +│ 1 │ +│ 2 │ +│ 3 │ +│ 5 │ +│ 8 │ +│ 13 │ +│ 21 │ +│ 34 │ +└────────┘ +``` + + ## arrayReverse(arr) Returns an array of the same size as the original array containing the elements in reverse order. From bd934575ea94a91bd6985bbc13fa4b6ab7984a12 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 23 Oct 2023 05:29:42 -0300 Subject: [PATCH 491/634] Update array-functions.md --- .../functions/array-functions.md | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 2b6d7e73c34..61536b854ed 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1149,24 +1149,24 @@ Result: └─────┘ ``` -**Example with fibonacci sequence** +**Example with the fibonacci sequence** ```sql -SELECT arrayFold( x, acc -> (acc.2, acc.2 + acc.1), range(number), (1::Int64, 0::Int64)).1 AS result +SELECT arrayFold( x, acc -> (acc.2, acc.2 + acc.1), range(number), (1::Int64, 0::Int64)).1 AS fibonacci FROM numbers(1,10); -┌─result─┐ -│ 0 │ -│ 1 │ -│ 1 │ -│ 2 │ -│ 3 │ -│ 5 │ -│ 8 │ -│ 13 │ -│ 21 │ -│ 34 │ -└────────┘ +┌─fibonacci─┐ +│ 0 │ +│ 1 │ +│ 1 │ +│ 2 │ +│ 3 │ +│ 5 │ +│ 8 │ +│ 13 │ +│ 21 │ +│ 34 │ +└───────────┘ ``` From ba378802089503fa40eabf3c60779dfe33b96971 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 6 Oct 2023 20:10:22 +0300 Subject: [PATCH 492/634] Context added TSA --- src/Common/MutexUtils.h | 80 ++++++ src/Common/SharedLockGuard.h | 29 +++ src/Interpreters/Context.cpp | 471 +++++++++++++++++------------------ src/Interpreters/Context.h | 50 ++-- 4 files changed, 371 insertions(+), 259 deletions(-) create mode 100644 src/Common/MutexUtils.h create mode 100644 src/Common/SharedLockGuard.h diff --git a/src/Common/MutexUtils.h b/src/Common/MutexUtils.h new file mode 100644 index 00000000000..e50dc3a9bab --- /dev/null +++ b/src/Common/MutexUtils.h @@ -0,0 +1,80 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +template +class TSA_CAPABILITY("SharedMutexHelper") SharedMutexHelper +{ +public: + // Exclusive ownership + void lock() TSA_ACQUIRE() /// NOLINT + { + static_cast(this)->lockImpl(); + } + + bool try_lock() TSA_TRY_ACQUIRE(true) /// NOLINT + { + static_cast(this)->tryLockImpl(); + } + + void unlock() TSA_RELEASE() /// NOLINT + { + static_cast(this)->unlockImpl(); + } + + // Shared ownership + void lock_shared() TSA_ACQUIRE_SHARED() /// NOLINT + { + static_cast(this)->lockSharedImpl(); + } + + bool try_lock_shared() TSA_TRY_ACQUIRE_SHARED(true) /// NOLINT + { + static_cast(this)->tryLockSharedImpl(); + } + + void unlock_shared() TSA_RELEASE_SHARED() /// NOLINT + { + static_cast(this)->unlockSharedImpl(); + } + +protected: + void lockImpl() TSA_NO_THREAD_SAFETY_ANALYSIS + { + mutex.lock(); + } + + void tryLockImpl() TSA_NO_THREAD_SAFETY_ANALYSIS + { + mutex.TryLock(); + } + + void unlockImpl() TSA_NO_THREAD_SAFETY_ANALYSIS + { + mutex.unlock(); + } + + void lockSharedImpl() TSA_NO_THREAD_SAFETY_ANALYSIS + { + mutex.lock_shared(); + } + + void tryLockSharedImpl() TSA_NO_THREAD_SAFETY_ANALYSIS + { + mutex.try_lock_shared(); + } + + void unlockSharedImpl() TSA_NO_THREAD_SAFETY_ANALYSIS + { + mutex.unlock_shared(); + } + + MutexType mutex; +}; + +} diff --git a/src/Common/SharedLockGuard.h b/src/Common/SharedLockGuard.h new file mode 100644 index 00000000000..69b1f9469b6 --- /dev/null +++ b/src/Common/SharedLockGuard.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include + +namespace DB +{ + +template +class __attribute__((scoped_lockable)) SharedLockGuard +{ +public: + explicit SharedLockGuard(Mutex & mutex_) TSA_ACQUIRE_SHARED(mutex_) : mutex(mutex_) + { + mutex_.lock(); + } + + ~SharedLockGuard() TSA_RELEASE() + { + mutex.unlock(); + } + +private: + Mutex & mutex; +}; + +} + + diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d0b3e9d1990..b5f0c2701ad 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -181,7 +182,7 @@ struct ContextSharedPart : boost::noncopyable Poco::Logger * log = &Poco::Logger::get("Context"); /// For access of most of shared objects. - mutable SharedMutex mutex; + mutable ContextSharedMutex mutex; /// Separate mutex for access of dictionaries. Separate mutex to avoid locks when server doing request to itself. mutable std::mutex embedded_dictionaries_mutex; mutable std::mutex external_dictionaries_mutex; @@ -193,44 +194,43 @@ struct ContextSharedPart : boost::noncopyable /// Separate mutex for re-initialization of zookeeper session. This operation could take a long time and must not interfere with another operations. mutable std::mutex zookeeper_mutex; - mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper. - ConfigurationPtr zookeeper_config; /// Stores zookeeper configs + mutable zkutil::ZooKeeperPtr zookeeper TSA_GUARDED_BY(zookeeper_mutex); /// Client for ZooKeeper. + ConfigurationPtr zookeeper_config TSA_GUARDED_BY(zookeeper_mutex); /// Stores zookeeper configs #if USE_NURAFT mutable std::mutex keeper_dispatcher_mutex; - mutable std::shared_ptr keeper_dispatcher; + mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); #endif mutable std::mutex auxiliary_zookeepers_mutex; - mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. - ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs + mutable std::map auxiliary_zookeepers TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Map for auxiliary ZooKeeper clients. + ConfigurationPtr auxiliary_zookeepers_config TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Stores auxiliary zookeepers configs String interserver_io_host; /// The host name by which this server is available for other servers. UInt16 interserver_io_port = 0; /// and port. String interserver_scheme; /// http or https MultiVersion interserver_io_credentials; - String path; /// Path to the data directory, with a slash at the end. - String flags_path; /// Path to the directory with some control flags for server maintenance. - String user_files_path; /// Path to the directory with user provided files, usable by 'file' table function. - String dictionaries_lib_path; /// Path to the directory with user provided binaries and libraries for external dictionaries. - String user_scripts_path; /// Path to the directory with user provided scripts. - String filesystem_caches_path; /// Path to the directory with filesystem caches. - ConfigurationPtr config; /// Global configuration settings. - - String tmp_path; /// Path to the temporary files that occur when processing the request. + String path TSA_GUARDED_BY(mutex); /// Path to the data directory, with a slash at the end. + String flags_path TSA_GUARDED_BY(mutex); /// Path to the directory with some control flags for server maintenance. + String user_files_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided files, usable by 'file' table function. + String dictionaries_lib_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided binaries and libraries for external dictionaries. + String user_scripts_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided scripts. + String filesystem_caches_path TSA_GUARDED_BY(mutex); /// Path to the directory with filesystem caches. + ConfigurationPtr config TSA_GUARDED_BY(mutex); /// Global configuration settings. + String tmp_path TSA_GUARDED_BY(mutex); /// Path to the temporary files that occur when processing the request. /// All temporary files that occur when processing the requests accounted here. /// Child scopes for more fine-grained accounting are created per user/query/etc. /// Initialized once during server startup. - TemporaryDataOnDiskScopePtr root_temp_data_on_disk; + TemporaryDataOnDiskScopePtr root_temp_data_on_disk TSA_GUARDED_BY(mutex); - mutable std::unique_ptr embedded_dictionaries; /// Metrica's dictionaries. Have lazy initialization. - mutable std::unique_ptr external_dictionaries_loader; + mutable std::unique_ptr embedded_dictionaries TSA_GUARDED_BY(embedded_dictionaries_mutex); /// Metrica's dictionaries. Have lazy initialization. + mutable std::unique_ptr external_dictionaries_loader TSA_GUARDED_BY(external_dictionaries_mutex); scope_guard models_repository_guard; - ExternalLoaderXMLConfigRepository * external_dictionaries_config_repository = nullptr; - scope_guard dictionaries_xmls; + ExternalLoaderXMLConfigRepository * external_dictionaries_config_repository TSA_GUARDED_BY(external_dictionaries_mutex) = nullptr; + scope_guard dictionaries_xmls TSA_GUARDED_BY(external_dictionaries_mutex); mutable std::unique_ptr external_user_defined_executable_functions_loader; ExternalLoaderXMLConfigRepository * user_defined_executable_functions_config_repository = nullptr; @@ -480,10 +480,30 @@ struct ContextSharedPart : boost::noncopyable } } + void setConfig(const ConfigurationPtr & config_value) + { + if (!config_value) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Set nullptr config is invalid"); + + std::lock_guard lock(mutex); + config = config_value; + access_control->setExternalAuthenticatorsConfig(*config_value); + } + + const Poco::Util::AbstractConfiguration & getConfigRefWithLock(const std::lock_guard &) const TSA_REQUIRES(this->mutex) + { + return config ? *config : Poco::Util::Application::instance().config(); + } + + const Poco::Util::AbstractConfiguration & getConfigRef() const + { + SharedLockGuard lock(mutex); + return config ? *config : Poco::Util::Application::instance().config(); + } /** Perform a complex job of destroying objects in advance. */ - void shutdown() + void shutdown() TSA_NO_THREAD_SAFETY_ANALYSIS { if (shutdown_called) return; @@ -660,6 +680,23 @@ struct ContextSharedPart : boost::noncopyable } }; +void ContextSharedMutex::lockImpl() +{ + ProfileEvents::increment(ProfileEvents::ContextLock); + CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; + Stopwatch watch; + Base::lockImpl(); + ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); +} + +void ContextSharedMutex::lockSharedImpl() +{ + ProfileEvents::increment(ProfileEvents::ContextLock); + CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; + Stopwatch watch; + Base::lockSharedImpl(); + ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); +} ContextData::ContextData() = default; ContextData::ContextData(const ContextData &) = default; @@ -698,7 +735,7 @@ SharedContextHolder Context::createShared() ContextMutablePtr Context::createCopy(const ContextPtr & other) { - auto lock = other->getLocalSharedLock(); + SharedLockGuard lock(other->mutex); return std::shared_ptr(new Context(*other)); } @@ -720,46 +757,6 @@ Context::~Context() = default; InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; } const InterserverIOHandler & Context::getInterserverIOHandler() const { return shared->interserver_io_handler; } -std::unique_lock Context::getGlobalLock() const -{ - ProfileEvents::increment(ProfileEvents::ContextLock); - CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; - Stopwatch watch; - auto lock = std::unique_lock(shared->mutex); - ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); - return lock; -} - -std::shared_lock Context::getGlobalSharedLock() const -{ - ProfileEvents::increment(ProfileEvents::ContextLock); - CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; - Stopwatch watch; - auto lock = std::shared_lock(shared->mutex); - ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); - return lock; -} - -std::unique_lock Context::getLocalLock() const -{ - ProfileEvents::increment(ProfileEvents::ContextLock); - CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; - Stopwatch watch; - auto lock = std::unique_lock(mutex); - ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); - return lock; -} - -std::shared_lock Context::getLocalSharedLock() const -{ - ProfileEvents::increment(ProfileEvents::ContextLock); - CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; - Stopwatch watch; - auto lock = std::shared_lock(mutex); - ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); - return lock; -} - ProcessList & Context::getProcessList() { return shared->process_list; } const ProcessList & Context::getProcessList() const { return shared->process_list; } OvercommitTracker * Context::getGlobalOvercommitTracker() const { return &shared->global_overcommit_tracker; } @@ -783,37 +780,37 @@ String Context::resolveDatabase(const String & database_name) const String Context::getPath() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->path; } String Context::getFlagsPath() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->flags_path; } String Context::getUserFilesPath() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->user_files_path; } String Context::getDictionariesLibPath() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->dictionaries_lib_path; } String Context::getUserScriptsPath() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->user_scripts_path; } String Context::getFilesystemCachesPath() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->filesystem_caches_path; } @@ -821,7 +818,7 @@ Strings Context::getWarnings() const { Strings common_warnings; { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); common_warnings = shared->warnings; } /// Make setting's name ordered @@ -856,7 +853,7 @@ Strings Context::getWarnings() const /// TODO: remove, use `getTempDataOnDisk` VolumePtr Context::getGlobalTemporaryVolume() const { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); /// Calling this method we just bypass the `temp_data_on_disk` and write to the file on the volume directly. /// Volume is the same for `root_temp_data_on_disk` (always set) and `temp_data_on_disk` (if it's set). if (shared->root_temp_data_on_disk) @@ -869,13 +866,13 @@ TemporaryDataOnDiskScopePtr Context::getTempDataOnDisk() const if (temp_data_on_disk) return temp_data_on_disk; - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->root_temp_data_on_disk; } TemporaryDataOnDiskScopePtr Context::getSharedTempDataOnDisk() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->root_temp_data_on_disk; } @@ -888,7 +885,7 @@ void Context::setTempDataOnDisk(TemporaryDataOnDiskScopePtr temp_data_on_disk_) void Context::setPath(const String & path) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); shared->path = path; @@ -910,7 +907,7 @@ void Context::setPath(const String & path) void Context::setFilesystemCachesPath(const String & path) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (!fs::path(path).is_absolute()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filesystem caches path must be absolute: {}", path); @@ -958,7 +955,7 @@ static VolumePtr createLocalSingleDiskVolume(const std::string & path, const Poc void Context::setTemporaryStoragePath(const String & path, size_t max_size) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->root_temp_data_on_disk) throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary storage is already set"); @@ -967,7 +964,7 @@ void Context::setTemporaryStoragePath(const String & path, size_t max_size) if (!shared->tmp_path.ends_with('/')) shared->tmp_path += '/'; - VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path, getConfigRefWithLock(lock)); + VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path, shared->getConfigRefWithLock(lock)); for (const auto & disk : volume->getDisks()) { @@ -1015,7 +1012,7 @@ void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_s setupTmpPath(shared->log, disk->getPath()); } - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->root_temp_data_on_disk) throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary storage is already set"); @@ -1029,7 +1026,7 @@ void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t if (!disk_ptr) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' is not found", cache_disk_name); - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->root_temp_data_on_disk) throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary storage is already set"); @@ -1040,38 +1037,39 @@ void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t LOG_DEBUG(shared->log, "Using file cache ({}) for temporary files", file_cache->getBasePath()); shared->tmp_path = file_cache->getBasePath(); - VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path, getConfigRefWithLock(lock)); + VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path, shared->getConfigRefWithLock(lock)); shared->root_temp_data_on_disk = std::make_shared(volume, file_cache.get(), max_size); } void Context::setFlagsPath(const String & path) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); shared->flags_path = path; } void Context::setUserFilesPath(const String & path) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); shared->user_files_path = path; } void Context::setDictionariesLibPath(const String & path) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); shared->dictionaries_lib_path = path; } void Context::setUserScriptsPath(const String & path) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); shared->user_scripts_path = path; } void Context::addWarningMessage(const String & msg) const { - auto lock = getGlobalLock(); - auto suppress_re = getConfigRefWithLock(lock).getString("warning_supress_regexp", ""); + std::lock_guard lock(shared->mutex); + auto suppress_re = shared->getConfigRefWithLock(lock).getString("warning_supress_regexp", ""); + bool is_supressed = !suppress_re.empty() && re2::RE2::PartialMatch(msg, suppress_re); if (!is_supressed) shared->addWarningMessage(msg); @@ -1079,26 +1077,14 @@ void Context::addWarningMessage(const String & msg) const void Context::setConfig(const ConfigurationPtr & config) { - if (!config) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Set nullptr config is invalid"); - - auto lock = getGlobalLock(); - shared->config = config; - shared->access_control->setExternalAuthenticatorsConfig(*shared->config); -} - -const Poco::Util::AbstractConfiguration & Context::getConfigRefWithLock(const std::unique_lock &) const -{ - return shared->config ? *shared->config : Poco::Util::Application::instance().config(); + shared->setConfig(config); } const Poco::Util::AbstractConfiguration & Context::getConfigRef() const { - auto lock = getGlobalSharedLock(); - return shared->config ? *shared->config : Poco::Util::Application::instance().config(); + return shared->getConfigRef(); } - AccessControl & Context::getAccessControl() { return *shared->access_control; @@ -1111,26 +1097,26 @@ const AccessControl & Context::getAccessControl() const void Context::setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); shared->access_control->setExternalAuthenticatorsConfig(config); } std::unique_ptr Context::makeGSSAcceptorContext() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return std::make_unique(shared->access_control->getExternalAuthenticators().getKerberosParams()); } void Context::setUsersConfig(const ConfigurationPtr & config) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); shared->users_config = config; shared->access_control->setUsersConfig(*shared->users_config); } ConfigurationPtr Context::getUsersConfig() { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->users_config; } @@ -1150,7 +1136,7 @@ void Context::setUser(const UUID & user_id_, const std::optionalgetUserName(); } -void Context::setUserIDWithLock(const UUID & user_id_, const std::unique_lock &) +void Context::setUserIDWithLock(const UUID & user_id_, const std::lock_guard &) { user_id = user_id_; need_recalculate_access = true; @@ -1183,17 +1169,17 @@ void Context::setUserIDWithLock(const UUID & user_id_, const std::unique_lock Context::getUserID() const { - auto lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); return user_id; } -void Context::setCurrentRolesWithLock(const std::vector & current_roles_, const std::unique_lock &) +void Context::setCurrentRolesWithLock(const std::vector & current_roles_, const std::lock_guard &) { if (current_roles_.empty()) current_roles = nullptr; @@ -1204,7 +1190,7 @@ void Context::setCurrentRolesWithLock(const std::vector & current_roles_, void Context::setCurrentRoles(const std::vector & current_roles_) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); setCurrentRolesWithLock(current_roles_, lock); } @@ -1264,7 +1250,7 @@ std::shared_ptr Context::getAccess() const std::optional params; { - auto lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); if (access && !need_recalculate_access) return access; /// No need to recalculate access rights. @@ -1284,7 +1270,7 @@ std::shared_ptr Context::getAccess() const { /// If the parameters of access rights were not changed while we were calculated them /// then we store the new access rights in the Context to allow reusing it later. - auto lock = getLocalLock(); + std::lock_guard lock(mutex); if (get_params() == *params) { access = res; @@ -1312,7 +1298,7 @@ std::optional Context::getQuotaUsage() const return getAccess()->getQuotaUsage(); } -void Context::setCurrentProfileWithLock(const String & profile_name, bool check_constraints, const std::unique_lock & lock) +void Context::setCurrentProfileWithLock(const String & profile_name, bool check_constraints, const std::lock_guard & lock) { try { @@ -1326,13 +1312,13 @@ void Context::setCurrentProfileWithLock(const String & profile_name, bool check_ } } -void Context::setCurrentProfileWithLock(const UUID & profile_id, bool check_constraints, const std::unique_lock & lock) +void Context::setCurrentProfileWithLock(const UUID & profile_id, bool check_constraints, const std::lock_guard & lock) { auto profile_info = getAccessControl().getSettingsProfileInfo(profile_id); setCurrentProfilesWithLock(*profile_info, check_constraints, lock); } -void Context::setCurrentProfilesWithLock(const SettingsProfilesInfo & profiles_info, bool check_constraints, const std::unique_lock & lock) +void Context::setCurrentProfilesWithLock(const SettingsProfilesInfo & profiles_info, bool check_constraints, const std::lock_guard & lock) { if (check_constraints) checkSettingsConstraintsWithLock(profiles_info.settings, SettingSource::PROFILE); @@ -1342,31 +1328,31 @@ void Context::setCurrentProfilesWithLock(const SettingsProfilesInfo & profiles_i void Context::setCurrentProfile(const String & profile_name, bool check_constraints) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); setCurrentProfileWithLock(profile_name, check_constraints, lock); } void Context::setCurrentProfile(const UUID & profile_id, bool check_constraints) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); setCurrentProfileWithLock(profile_id, check_constraints, lock); } void Context::setCurrentProfiles(const SettingsProfilesInfo & profiles_info, bool check_constraints) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); setCurrentProfilesWithLock(profiles_info, check_constraints, lock); } std::vector Context::getCurrentProfiles() const { - auto lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); return settings_constraints_and_current_profiles->current_profiles; } std::vector Context::getEnabledProfiles() const { - auto lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); return settings_constraints_and_current_profiles->enabled_profiles; } @@ -1382,7 +1368,7 @@ ResourceManagerPtr Context::getResourceManager() const ClassifierPtr Context::getWorkloadClassifier() const { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); if (!classifier) classifier = getResourceManager()->acquire(getSettingsRef().workload); return classifier; @@ -1420,7 +1406,7 @@ Tables Context::getExternalTables() const if (isGlobalContext()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have external tables"); - auto lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); Tables res; for (const auto & table : external_tables_mapping) @@ -1447,7 +1433,7 @@ void Context::addExternalTable(const String & table_name, TemporaryTableHolder & if (isGlobalContext()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have external tables"); - auto lock = getLocalLock(); + std::lock_guard lock(mutex); if (external_tables_mapping.end() != external_tables_mapping.find(table_name)) throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Temporary table {} already exists.", backQuoteIfNeed(table_name)); external_tables_mapping.emplace(table_name, std::make_shared(std::move(temporary_table))); @@ -1460,7 +1446,7 @@ std::shared_ptr Context::findExternalTable(const String & std::shared_ptr holder; { - auto lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); auto iter = external_tables_mapping.find(table_name); if (iter == external_tables_mapping.end()) return {}; @@ -1476,7 +1462,7 @@ std::shared_ptr Context::removeExternalTable(const String std::shared_ptr holder; { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); auto iter = external_tables_mapping.find(table_name); if (iter == external_tables_mapping.end()) return {}; @@ -1859,18 +1845,18 @@ bool Context::displaySecretsInShowAndSelect() const Settings Context::getSettings() const { - auto lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); return settings; } void Context::setSettings(const Settings & settings_) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); settings = settings_; need_recalculate_access = true; } -void Context::setSettingWithLock(std::string_view name, const String & value, const std::unique_lock & lock) +void Context::setSettingWithLock(std::string_view name, const String & value, const std::lock_guard & lock) { if (name == "profile") { @@ -1882,7 +1868,7 @@ void Context::setSettingWithLock(std::string_view name, const String & value, co need_recalculate_access = true; } -void Context::setSettingWithLock(std::string_view name, const Field & value, const std::unique_lock & lock) +void Context::setSettingWithLock(std::string_view name, const Field & value, const std::lock_guard & lock) { if (name == "profile") { @@ -1894,7 +1880,7 @@ void Context::setSettingWithLock(std::string_view name, const Field & value, con need_recalculate_access = true; } -void Context::applySettingChangeWithLock(const SettingChange & change, const std::unique_lock & lock) +void Context::applySettingChangeWithLock(const SettingChange & change, const std::lock_guard & lock) { try { @@ -1909,7 +1895,7 @@ void Context::applySettingChangeWithLock(const SettingChange & change, const std } } -void Context::applySettingsChangesWithLock(const SettingsChanges & changes, const std::unique_lock & lock) +void Context::applySettingsChangesWithLock(const SettingsChanges & changes, const std::lock_guard& lock) { for (const SettingChange & change : changes) applySettingChangeWithLock(change, lock); @@ -1918,13 +1904,13 @@ void Context::applySettingsChangesWithLock(const SettingsChanges & changes, cons void Context::setSetting(std::string_view name, const String & value) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); setSettingWithLock(name, value, lock); } void Context::setSetting(std::string_view name, const Field & value) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); setSettingWithLock(name, value, lock); } @@ -1946,7 +1932,7 @@ void Context::applySettingChange(const SettingChange & change) void Context::applySettingsChanges(const SettingsChanges & changes) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); applySettingsChangesWithLock(changes, lock); } @@ -1982,43 +1968,43 @@ void Context::checkMergeTreeSettingsConstraintsWithLock(const MergeTreeSettings void Context::checkSettingsConstraints(const SettingsProfileElements & profile_elements, SettingSource source) const { - auto shared_lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); checkSettingsConstraintsWithLock(profile_elements, source); } void Context::checkSettingsConstraints(const SettingChange & change, SettingSource source) const { - auto shared_lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); checkSettingsConstraintsWithLock(change, source); } void Context::checkSettingsConstraints(const SettingsChanges & changes, SettingSource source) const { - auto shared_lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); getSettingsConstraintsAndCurrentProfilesWithLock()->constraints.check(settings, changes, source); } void Context::checkSettingsConstraints(SettingsChanges & changes, SettingSource source) const { - auto shared_lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); checkSettingsConstraintsWithLock(changes, source); } void Context::clampToSettingsConstraints(SettingsChanges & changes, SettingSource source) const { - auto shared_lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); clampToSettingsConstraintsWithLock(changes, source); } void Context::checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const { - auto shared_lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); checkMergeTreeSettingsConstraintsWithLock(merge_tree_settings, changes); } void Context::resetSettingsToDefaultValue(const std::vector & names) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); for (const String & name: names) settings.setDefaultValue(name); } @@ -2033,13 +2019,13 @@ std::shared_ptr Context::getSettingsCons std::shared_ptr Context::getSettingsConstraintsAndCurrentProfiles() const { - auto lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); return getSettingsConstraintsAndCurrentProfilesWithLock(); } String Context::getCurrentDatabase() const { - auto lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); return current_database; } @@ -2056,7 +2042,7 @@ void Context::setCurrentDatabaseNameInGlobalContext(const String & name) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot set current database for non global context, this method should " "be used during server initialization"); - auto lock = getLocalLock(); + std::lock_guard lock(mutex); if (!current_database.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Default database name cannot be changed in global context without server restart"); @@ -2064,7 +2050,7 @@ void Context::setCurrentDatabaseNameInGlobalContext(const String & name) current_database = name; } -void Context::setCurrentDatabaseWithLock(const String & name, const std::unique_lock &) +void Context::setCurrentDatabaseWithLock(const String & name, const std::lock_guard &) { DatabaseCatalog::instance().assertDatabaseExists(name); current_database = name; @@ -2073,7 +2059,7 @@ void Context::setCurrentDatabaseWithLock(const String & name, const std::unique_ void Context::setCurrentDatabase(const String & name) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); setCurrentDatabaseWithLock(name, lock); } @@ -2253,10 +2239,10 @@ const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() cons ExternalDictionariesLoader & Context::getExternalDictionariesLoader() { std::lock_guard lock(shared->external_dictionaries_mutex); - return getExternalDictionariesLoaderUnlocked(); + return getExternalDictionariesLoaderWithLock(lock); } -ExternalDictionariesLoader & Context::getExternalDictionariesLoaderUnlocked() +ExternalDictionariesLoader & Context::getExternalDictionariesLoaderWithLock(const std::lock_guard &) TSA_REQUIRES(shared->external_dictionaries_mutex) { if (!shared->external_dictionaries_loader) shared->external_dictionaries_loader = @@ -2272,10 +2258,11 @@ const ExternalUserDefinedExecutableFunctionsLoader & Context::getExternalUserDef ExternalUserDefinedExecutableFunctionsLoader & Context::getExternalUserDefinedExecutableFunctionsLoader() { std::lock_guard lock(shared->external_user_defined_executable_functions_mutex); - return getExternalUserDefinedExecutableFunctionsLoaderUnlocked(); + return getExternalUserDefinedExecutableFunctionsLoaderWithLock(lock); } -ExternalUserDefinedExecutableFunctionsLoader & Context::getExternalUserDefinedExecutableFunctionsLoaderUnlocked() +ExternalUserDefinedExecutableFunctionsLoader & +Context::getExternalUserDefinedExecutableFunctionsLoaderWithLock(const std::lock_guard &) TSA_REQUIRES(shared->external_user_defined_executable_functions_mutex) { if (!shared->external_user_defined_executable_functions_loader) shared->external_user_defined_executable_functions_loader = @@ -2315,7 +2302,7 @@ void Context::loadOrReloadDictionaries(const Poco::Util::AbstractConfiguration & std::lock_guard lock(shared->external_dictionaries_mutex); - auto & external_dictionaries_loader = getExternalDictionariesLoaderUnlocked(); + auto & external_dictionaries_loader = getExternalDictionariesLoaderWithLock(lock); external_dictionaries_loader.enableAlwaysLoadEverything(!dictionaries_lazy_load); if (shared->external_dictionaries_config_repository) @@ -2339,7 +2326,7 @@ void Context::loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::Abstr std::lock_guard lock(shared->external_user_defined_executable_functions_mutex); - auto & external_user_defined_executable_functions_loader = getExternalUserDefinedExecutableFunctionsLoaderUnlocked(); + auto & external_user_defined_executable_functions_loader = getExternalUserDefinedExecutableFunctionsLoaderWithLock(lock); if (shared->user_defined_executable_functions_config_repository) { @@ -2361,7 +2348,7 @@ const IUserDefinedSQLObjectsLoader & Context::getUserDefinedSQLObjectsLoader() c shared->user_defined_sql_objects_loader = createUserDefinedSQLObjectsLoader(getGlobalContext()); }); - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return *shared->user_defined_sql_objects_loader; } @@ -2371,7 +2358,7 @@ IUserDefinedSQLObjectsLoader & Context::getUserDefinedSQLObjectsLoader() shared->user_defined_sql_objects_loader = createUserDefinedSQLObjectsLoader(getGlobalContext()); }); - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return *shared->user_defined_sql_objects_loader; } @@ -2413,7 +2400,7 @@ BackupsWorker & Context::getBackupsWorker() const shared->backups_worker.emplace(getGlobalContext(), backup_threads, restore_threads, allow_concurrent_backups, allow_concurrent_restores); }); - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return *shared->backups_worker; } @@ -2457,7 +2444,7 @@ QueryStatusPtr Context::getProcessListElementSafe() const void Context::setUncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->uncompressed_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Uncompressed cache has been already created."); @@ -2467,7 +2454,7 @@ void Context::setUncompressedCache(const String & cache_policy, size_t max_size_ void Context::updateUncompressedCacheConfiguration(const Poco::Util::AbstractConfiguration & config) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (!shared->uncompressed_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Uncompressed cache was not created yet."); @@ -2478,13 +2465,13 @@ void Context::updateUncompressedCacheConfiguration(const Poco::Util::AbstractCon UncompressedCachePtr Context::getUncompressedCache() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->uncompressed_cache; } void Context::clearUncompressedCache() const { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->uncompressed_cache) shared->uncompressed_cache->clear(); @@ -2492,7 +2479,7 @@ void Context::clearUncompressedCache() const void Context::setMarkCache(const String & cache_policy, size_t max_cache_size_in_bytes, double size_ratio) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->mark_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mark cache has been already created."); @@ -2502,7 +2489,7 @@ void Context::setMarkCache(const String & cache_policy, size_t max_cache_size_in void Context::updateMarkCacheConfiguration(const Poco::Util::AbstractConfiguration & config) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (!shared->mark_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mark cache was not created yet."); @@ -2513,13 +2500,13 @@ void Context::updateMarkCacheConfiguration(const Poco::Util::AbstractConfigurati MarkCachePtr Context::getMarkCache() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->mark_cache; } void Context::clearMarkCache() const { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->mark_cache) shared->mark_cache->clear(); @@ -2541,7 +2528,7 @@ ThreadPool & Context::getLoadMarksThreadpool() const void Context::setIndexUncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->index_uncompressed_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Index uncompressed cache has been already created."); @@ -2551,7 +2538,7 @@ void Context::setIndexUncompressedCache(const String & cache_policy, size_t max_ void Context::updateIndexUncompressedCacheConfiguration(const Poco::Util::AbstractConfiguration & config) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (!shared->index_uncompressed_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Index uncompressed cache was not created yet."); @@ -2562,13 +2549,13 @@ void Context::updateIndexUncompressedCacheConfiguration(const Poco::Util::Abstra UncompressedCachePtr Context::getIndexUncompressedCache() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->index_uncompressed_cache; } void Context::clearIndexUncompressedCache() const { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->index_uncompressed_cache) shared->index_uncompressed_cache->clear(); @@ -2576,7 +2563,7 @@ void Context::clearIndexUncompressedCache() const void Context::setIndexMarkCache(const String & cache_policy, size_t max_cache_size_in_bytes, double size_ratio) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->index_mark_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Index mark cache has been already created."); @@ -2586,7 +2573,7 @@ void Context::setIndexMarkCache(const String & cache_policy, size_t max_cache_si void Context::updateIndexMarkCacheConfiguration(const Poco::Util::AbstractConfiguration & config) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); if (!shared->index_mark_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Index mark cache was not created yet."); @@ -2597,13 +2584,13 @@ void Context::updateIndexMarkCacheConfiguration(const Poco::Util::AbstractConfig MarkCachePtr Context::getIndexMarkCache() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->index_mark_cache; } void Context::clearIndexMarkCache() const { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->index_mark_cache) shared->index_mark_cache->clear(); @@ -2611,7 +2598,7 @@ void Context::clearIndexMarkCache() const void Context::setMMappedFileCache(size_t max_cache_size_in_num_entries) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->mmap_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mapped file cache has been already created."); @@ -2621,7 +2608,7 @@ void Context::setMMappedFileCache(size_t max_cache_size_in_num_entries) void Context::updateMMappedFileCacheConfiguration(const Poco::Util::AbstractConfiguration & config) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (!shared->mmap_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mapped file cache was not created yet."); @@ -2632,13 +2619,13 @@ void Context::updateMMappedFileCacheConfiguration(const Poco::Util::AbstractConf MMappedFileCachePtr Context::getMMappedFileCache() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->mmap_cache; } void Context::clearMMappedFileCache() const { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->mmap_cache) shared->mmap_cache->clear(); @@ -2646,7 +2633,7 @@ void Context::clearMMappedFileCache() const void Context::setQueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes, size_t max_entry_size_in_rows) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->query_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Query cache has been already created."); @@ -2656,7 +2643,7 @@ void Context::setQueryCache(size_t max_size_in_bytes, size_t max_entries, size_t void Context::updateQueryCacheConfiguration(const Poco::Util::AbstractConfiguration & config) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (!shared->query_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Query cache was not created yet."); @@ -2670,13 +2657,13 @@ void Context::updateQueryCacheConfiguration(const Poco::Util::AbstractConfigurat QueryCachePtr Context::getQueryCache() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->query_cache; } void Context::clearQueryCache() const { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->query_cache) shared->query_cache->clear(); @@ -2684,7 +2671,7 @@ void Context::clearQueryCache() const void Context::clearCaches() const { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (!shared->uncompressed_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Uncompressed cache was not created yet."); @@ -2826,7 +2813,7 @@ ThrottlerPtr Context::getRemoteReadThrottler() const ThrottlerPtr throttler = shared->remote_read_throttler; if (auto bandwidth = getSettingsRef().max_remote_read_network_bandwidth) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); if (!remote_read_query_throttler) remote_read_query_throttler = std::make_shared(bandwidth, throttler); throttler = remote_read_query_throttler; @@ -2839,7 +2826,7 @@ ThrottlerPtr Context::getRemoteWriteThrottler() const ThrottlerPtr throttler = shared->remote_write_throttler; if (auto bandwidth = getSettingsRef().max_remote_write_network_bandwidth) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); if (!remote_write_query_throttler) remote_write_query_throttler = std::make_shared(bandwidth, throttler); throttler = remote_write_query_throttler; @@ -2852,7 +2839,7 @@ ThrottlerPtr Context::getLocalReadThrottler() const ThrottlerPtr throttler = shared->local_read_throttler; if (auto bandwidth = getSettingsRef().max_local_read_bandwidth) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); if (!local_read_query_throttler) local_read_query_throttler = std::make_shared(bandwidth, throttler); throttler = local_read_query_throttler; @@ -2865,7 +2852,7 @@ ThrottlerPtr Context::getLocalWriteThrottler() const ThrottlerPtr throttler = shared->local_write_throttler; if (auto bandwidth = getSettingsRef().max_local_write_bandwidth) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); if (!local_write_query_throttler) local_write_query_throttler = std::make_shared(bandwidth, throttler); throttler = local_write_query_throttler; @@ -2878,7 +2865,7 @@ ThrottlerPtr Context::getBackupsThrottler() const ThrottlerPtr throttler = shared->backups_server_throttler; if (auto bandwidth = getSettingsRef().max_backup_bandwidth) { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); if (!backups_query_throttler) backups_query_throttler = std::make_shared(bandwidth, throttler); throttler = backups_query_throttler; @@ -2893,7 +2880,7 @@ bool Context::hasDistributedDDL() const void Context::setDDLWorker(std::unique_ptr ddl_worker) { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (shared->ddl_worker) throw Exception(ErrorCodes::LOGICAL_ERROR, "DDL background thread has already been initialized"); ddl_worker->startup(); @@ -2902,7 +2889,7 @@ void Context::setDDLWorker(std::unique_ptr ddl_worker) DDLWorker & Context::getDDLWorker() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->ddl_worker) { if (!hasZooKeeper()) @@ -3018,15 +3005,20 @@ void Context::setSystemZooKeeperLogAfterInitializationIfNeeded() /// This method explicitly sets correct pointer to system log after its initialization. /// TODO get rid of this if possible - std::lock_guard lock(shared->zookeeper_mutex); - if (!shared->system_logs || !shared->system_logs->zookeeper_log) - return; + { + std::lock_guard lock(shared->zookeeper_mutex); + if (!shared->system_logs || !shared->system_logs->zookeeper_log) + return; - if (shared->zookeeper) - shared->zookeeper->setZooKeeperLog(shared->system_logs->zookeeper_log); + if (shared->zookeeper) + shared->zookeeper->setZooKeeperLog(shared->system_logs->zookeeper_log); + } - for (auto & zk : shared->auxiliary_zookeepers) - zk.second->setZooKeeperLog(shared->system_logs->zookeeper_log); + { + std::lock_guard lockAuxiliary(shared->auxiliary_zookeepers_mutex); + for (auto & zk : shared->auxiliary_zookeepers) + zk.second->setZooKeeperLog(shared->system_logs->zookeeper_log); + } } void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) const @@ -3436,7 +3428,7 @@ bool Context::hasTraceCollector() const std::shared_ptr Context::getQueryLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3446,7 +3438,7 @@ std::shared_ptr Context::getQueryLog() const std::shared_ptr Context::getQueryThreadLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3456,7 +3448,7 @@ std::shared_ptr Context::getQueryThreadLog() const std::shared_ptr Context::getQueryViewsLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3466,7 +3458,7 @@ std::shared_ptr Context::getQueryViewsLog() const std::shared_ptr Context::getPartLog(const String & part_database) const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); /// No part log or system logs are shutting down. if (!shared->system_logs) @@ -3484,7 +3476,7 @@ std::shared_ptr Context::getPartLog(const String & part_database) const std::shared_ptr Context::getTraceLog() const { - auto lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); if (!shared->system_logs) return {}; @@ -3495,7 +3487,7 @@ std::shared_ptr Context::getTraceLog() const std::shared_ptr Context::getTextLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3506,7 +3498,7 @@ std::shared_ptr Context::getTextLog() const std::shared_ptr Context::getMetricLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3517,7 +3509,7 @@ std::shared_ptr Context::getMetricLog() const std::shared_ptr Context::getAsynchronousMetricLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3528,7 +3520,7 @@ std::shared_ptr Context::getAsynchronousMetricLog() const std::shared_ptr Context::getOpenTelemetrySpanLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3538,7 +3530,7 @@ std::shared_ptr Context::getOpenTelemetrySpanLog() const std::shared_ptr Context::getSessionLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3549,7 +3541,7 @@ std::shared_ptr Context::getSessionLog() const std::shared_ptr Context::getZooKeeperLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3560,7 +3552,7 @@ std::shared_ptr Context::getZooKeeperLog() const std::shared_ptr Context::getTransactionsInfoLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3571,7 +3563,7 @@ std::shared_ptr Context::getTransactionsInfoLog() const std::shared_ptr Context::getProcessorsProfileLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3581,7 +3573,7 @@ std::shared_ptr Context::getProcessorsProfileLog() const std::shared_ptr Context::getFilesystemCacheLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3599,7 +3591,7 @@ std::shared_ptr Context::getS3QueueLog() const std::shared_ptr Context::getFilesystemReadPrefetchesLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3608,7 +3600,7 @@ std::shared_ptr Context::getFilesystemReadPrefetche std::shared_ptr Context::getAsynchronousInsertLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3618,7 +3610,7 @@ std::shared_ptr Context::getAsynchronousInsertLog() const std::shared_ptr Context::getBackupLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3628,7 +3620,7 @@ std::shared_ptr Context::getBackupLog() const std::vector Context::getSystemLogs() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3638,12 +3630,12 @@ std::vector Context::getSystemLogs() const CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double part_size_ratio) const { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (!shared->compression_codec_selector) { constexpr auto config_name = "compression"; - const auto & config = getConfigRefWithLock(lock); + const auto & config = shared->getConfigRefWithLock(lock); if (config.has(config_name)) shared->compression_codec_selector = std::make_unique(config, "compression"); @@ -3789,11 +3781,11 @@ void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration const MergeTreeSettings & Context::getMergeTreeSettings() const { - auto lock = getLocalLock(); + std::lock_guard lock(shared->mutex); if (!shared->merge_tree_settings) { - const auto & config = getConfigRefWithLock(lock); + const auto & config = shared->getConfigRefWithLock(lock); MergeTreeSettings mt_settings; mt_settings.loadFromConfig("merge_tree", config); shared->merge_tree_settings.emplace(mt_settings); @@ -3804,11 +3796,11 @@ const MergeTreeSettings & Context::getMergeTreeSettings() const const MergeTreeSettings & Context::getReplicatedMergeTreeSettings() const { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (!shared->replicated_merge_tree_settings) { - const auto & config = getConfigRefWithLock(lock); + const auto & config = shared->getConfigRefWithLock(lock); MergeTreeSettings mt_settings; mt_settings.loadFromConfig("merge_tree", config); mt_settings.loadFromConfig("replicated_merge_tree", config); @@ -3820,11 +3812,11 @@ const MergeTreeSettings & Context::getReplicatedMergeTreeSettings() const const StorageS3Settings & Context::getStorageS3Settings() const { - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); if (!shared->storage_s3_settings) { - const auto & config = getConfigRefWithLock(lock); + const auto & config = shared->getConfigRefWithLock(lock); shared->storage_s3_settings.emplace().loadFromConfig("s3", config, getSettingsRef()); } @@ -3927,7 +3919,7 @@ OutputFormatPtr Context::getOutputFormatParallelIfPossible(const String & name, double Context::getUptimeSeconds() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); return shared->uptime_watch.elapsedSeconds(); } @@ -3978,7 +3970,7 @@ void Context::stopServers(const ServerType & server_type) const } -void Context::shutdown() +void Context::shutdown() TSA_NO_THREAD_SAFETY_ANALYSIS { // Disk selector might not be initialized if there was some error during // its initialization. Don't try to initialize it again on shutdown. @@ -4089,7 +4081,7 @@ void Context::addQueryParameters(const NameToNameMap & parameters) void Context::addBridgeCommand(std::unique_ptr cmd) const { - auto lock = getLocalLock(); + std::lock_guard lock(shared->mutex); shared->bridge_commands.emplace_back(std::move(cmd)); } @@ -4314,7 +4306,7 @@ StorageID Context::resolveStorageID(StorageID storage_id, StorageNamespace where StorageID resolved = StorageID::createEmpty(); std::optional exc; { - auto lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); resolved = resolveStorageIDImpl(std::move(storage_id), where, &exc); } if (exc) @@ -4331,7 +4323,7 @@ StorageID Context::tryResolveStorageID(StorageID storage_id, StorageNamespace wh StorageID resolved = StorageID::createEmpty(); { - auto lock = getLocalSharedLock(); + SharedLockGuard lock(mutex); resolved = resolveStorageIDImpl(std::move(storage_id), where, nullptr); } if (resolved && !resolved.hasUUID() && resolved.database_name != DatabaseCatalog::TEMPORARY_DATABASE) @@ -4481,7 +4473,7 @@ MergeTreeTransactionPtr Context::getCurrentTransaction() const bool Context::isServerCompletelyStarted() const { - auto lock = getLocalLock(); + std::lock_guard lock(shared->mutex); assert(getApplicationType() == ApplicationType::SERVER); return shared->is_server_completely_started; } @@ -4489,15 +4481,20 @@ bool Context::isServerCompletelyStarted() const void Context::setServerCompletelyStarted() { { - std::lock_guard lock(shared->zookeeper_mutex); - if (shared->zookeeper) - shared->zookeeper->setServerCompletelyStarted(); + { + std::lock_guard lock(shared->zookeeper_mutex); + if (shared->zookeeper) + shared->zookeeper->setServerCompletelyStarted(); + } - for (auto & zk : shared->auxiliary_zookeepers) - zk.second->setServerCompletelyStarted(); + { + std::lock_guard lock(shared->auxiliary_zookeepers_mutex); + for (auto & zk : shared->auxiliary_zookeepers) + zk.second->setServerCompletelyStarted(); + } } - auto lock = getLocalLock(); + std::lock_guard lock(shared->mutex); assert(global_context.lock().get() == this); assert(!shared->is_server_completely_started); assert(getApplicationType() == ApplicationType::SERVER); @@ -4506,7 +4503,8 @@ void Context::setServerCompletelyStarted() PartUUIDsPtr Context::getPartUUIDs() const { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); + if (!part_uuids) /// For context itself, only this initialization is not const. /// We could have done in constructor. @@ -4572,7 +4570,7 @@ UUID Context::getParallelReplicasGroupUUID() const PartUUIDsPtr Context::getIgnoredPartUUIDs() const { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); if (!ignored_part_uuids) const_cast(ignored_part_uuids) = std::make_shared(); @@ -4596,7 +4594,8 @@ void Context::setAsynchronousInsertQueue(const std::shared_ptrmutex); + if (shared->are_background_executors_initialized) return; @@ -4657,7 +4656,7 @@ void Context::initializeBackgroundExecutorsIfNeeded() bool Context::areBackgroundExecutorsInitialized() { - auto lock = getLocalLock(); + std::lock_guard lock(shared->mutex); return shared->are_background_executors_initialized; } @@ -4798,7 +4797,7 @@ WriteSettings Context::getWriteSettings() const std::shared_ptr Context::getAsyncReadCounters() const { - auto lock = getLocalLock(); + std::lock_guard lock(mutex); if (!async_read_counters) async_read_counters = std::make_shared(); return async_read_counters; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index fac56f3c143..fef34f4f2ac 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -231,6 +232,17 @@ private: std::unique_ptr shared; }; +class ContextSharedMutex : public SharedMutexHelper +{ +private: + using Base = SharedMutexHelper; + friend class SharedMutexHelper; + + void lockImpl(); + + void lockSharedImpl(); +}; + class ContextData { protected: @@ -491,7 +503,7 @@ class Context: public ContextData, public std::enable_shared_from_this { private: /// ContextData mutex - mutable SharedMutex mutex; + mutable ContextSharedMutex mutex; Context(); Context(const Context &); @@ -778,7 +790,6 @@ public: const ExternalDictionariesLoader & getExternalDictionariesLoader() const; ExternalDictionariesLoader & getExternalDictionariesLoader(); - ExternalDictionariesLoader & getExternalDictionariesLoaderUnlocked(); const EmbeddedDictionaries & getEmbeddedDictionaries() const; EmbeddedDictionaries & getEmbeddedDictionaries(); void tryCreateEmbeddedDictionaries(const Poco::Util::AbstractConfiguration & config) const; @@ -786,7 +797,6 @@ public: const ExternalUserDefinedExecutableFunctionsLoader & getExternalUserDefinedExecutableFunctionsLoader() const; ExternalUserDefinedExecutableFunctionsLoader & getExternalUserDefinedExecutableFunctionsLoader(); - ExternalUserDefinedExecutableFunctionsLoader & getExternalUserDefinedExecutableFunctionsLoaderUnlocked(); const IUserDefinedSQLObjectsLoader & getUserDefinedSQLObjectsLoader() const; IUserDefinedSQLObjectsLoader & getUserDefinedSQLObjectsLoader(); void loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::AbstractConfiguration & config); @@ -1220,37 +1230,27 @@ public: const ServerSettings & getServerSettings() const; private: - std::unique_lock getGlobalLock() const; - - std::shared_lock getGlobalSharedLock() const; - - std::unique_lock getLocalLock() const; - - std::shared_lock getLocalSharedLock() const; - - const Poco::Util::AbstractConfiguration & getConfigRefWithLock(const std::unique_lock & lock) const; - std::shared_ptr getSettingsConstraintsAndCurrentProfilesWithLock() const; - void setCurrentProfileWithLock(const String & profile_name, bool check_constraints, const std::unique_lock & lock); + void setCurrentProfileWithLock(const String & profile_name, bool check_constraints, const std::lock_guard & lock); - void setCurrentProfileWithLock(const UUID & profile_id, bool check_constraints, const std::unique_lock & lock); + void setCurrentProfileWithLock(const UUID & profile_id, bool check_constraints, const std::lock_guard & lock); - void setCurrentProfilesWithLock(const SettingsProfilesInfo & profiles_info, bool check_constraints, const std::unique_lock & lock); + void setCurrentProfilesWithLock(const SettingsProfilesInfo & profiles_info, bool check_constraints, const std::lock_guard & lock); - void setCurrentRolesWithLock(const std::vector & current_roles_, const std::unique_lock & lock); + void setCurrentRolesWithLock(const std::vector & current_roles_, const std::lock_guard & lock); - void setSettingWithLock(std::string_view name, const String & value, const std::unique_lock & lock); + void setSettingWithLock(std::string_view name, const String & value, const std::lock_guard & lock); - void setSettingWithLock(std::string_view name, const Field & value, const std::unique_lock & lock); + void setSettingWithLock(std::string_view name, const Field & value, const std::lock_guard & lock); - void applySettingChangeWithLock(const SettingChange & change, const std::unique_lock & lock); + void applySettingChangeWithLock(const SettingChange & change, const std::lock_guard & lock); - void applySettingsChangesWithLock(const SettingsChanges & changes, const std::unique_lock & lock); + void applySettingsChangesWithLock(const SettingsChanges & changes, const std::lock_guard & lock); - void setUserIDWithLock(const UUID & user_id_, const std::unique_lock & lock); + void setUserIDWithLock(const UUID & user_id_, const std::lock_guard & lock); - void setCurrentDatabaseWithLock(const String & name, const std::unique_lock & lock); + void setCurrentDatabaseWithLock(const String & name, const std::lock_guard & lock); void checkSettingsConstraintsWithLock(const SettingsProfileElements & profile_elements, SettingSource source) const; @@ -1264,6 +1264,10 @@ private: void checkMergeTreeSettingsConstraintsWithLock(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const; + ExternalDictionariesLoader & getExternalDictionariesLoaderWithLock(const std::lock_guard & lock); + + ExternalUserDefinedExecutableFunctionsLoader & getExternalUserDefinedExecutableFunctionsLoaderWithLock(const std::lock_guard & lock); + void initGlobal(); void setUserID(const UUID & user_id_); From d363e062f1bd048f46dee788ab24d94bc66d2605 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 18 Oct 2023 13:44:02 +0300 Subject: [PATCH 493/634] Updated implementation --- base/base/defines.h | 1 + src/Common/SharedLockGuard.h | 7 +- src/Interpreters/Context.cpp | 188 +++++++++++++++++++---------------- 3 files changed, 109 insertions(+), 87 deletions(-) diff --git a/base/base/defines.h b/base/base/defines.h index 4d3d8796d21..2019200c120 100644 --- a/base/base/defines.h +++ b/base/base/defines.h @@ -150,6 +150,7 @@ # define TSA_ACQUIRE_SHARED(...) __attribute__((acquire_shared_capability(__VA_ARGS__))) /// function acquires a shared capability, but does not release it # define TSA_TRY_ACQUIRE_SHARED(...) __attribute__((try_acquire_shared_capability(__VA_ARGS__))) /// function tries to acquire a shared capability and returns a boolean value indicating success or failure # define TSA_RELEASE_SHARED(...) __attribute__((release_shared_capability(__VA_ARGS__))) /// function releases the given shared capability +# define TSA_SCOPED_LOCKABLE __attribute__((scoped_lockable)) /// object of a class has scoped lockable capability /// Macros for suppressing TSA warnings for specific reads/writes (instead of suppressing it for the whole function) /// They use a lambda function to apply function attribute to a single statement. This enable us to suppress warnings locally instead of diff --git a/src/Common/SharedLockGuard.h b/src/Common/SharedLockGuard.h index 69b1f9469b6..9b71528db1a 100644 --- a/src/Common/SharedLockGuard.h +++ b/src/Common/SharedLockGuard.h @@ -6,18 +6,19 @@ namespace DB { +/// Same as std::lock_guard but for shared mutex template -class __attribute__((scoped_lockable)) SharedLockGuard +class TSA_SCOPED_LOCKABLE SharedLockGuard { public: explicit SharedLockGuard(Mutex & mutex_) TSA_ACQUIRE_SHARED(mutex_) : mutex(mutex_) { - mutex_.lock(); + mutex_.lock_shared(); } ~SharedLockGuard() TSA_RELEASE() { - mutex.unlock(); + mutex.unlock_shared(); } private: diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b5f0c2701ad..d1d18509310 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -205,6 +205,7 @@ struct ContextSharedPart : boost::noncopyable mutable std::map auxiliary_zookeepers TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Map for auxiliary ZooKeeper clients. ConfigurationPtr auxiliary_zookeepers_config TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Stores auxiliary zookeepers configs + /// No lock required for interserver_io_host, interserver_io_port, interserver_scheme modified only during initialization String interserver_io_host; /// The host name by which this server is available for other servers. UInt16 interserver_io_port = 0; /// and port. String interserver_scheme; /// http or https @@ -227,14 +228,12 @@ struct ContextSharedPart : boost::noncopyable mutable std::unique_ptr embedded_dictionaries TSA_GUARDED_BY(embedded_dictionaries_mutex); /// Metrica's dictionaries. Have lazy initialization. mutable std::unique_ptr external_dictionaries_loader TSA_GUARDED_BY(external_dictionaries_mutex); - scope_guard models_repository_guard; - ExternalLoaderXMLConfigRepository * external_dictionaries_config_repository TSA_GUARDED_BY(external_dictionaries_mutex) = nullptr; scope_guard dictionaries_xmls TSA_GUARDED_BY(external_dictionaries_mutex); - mutable std::unique_ptr external_user_defined_executable_functions_loader; - ExternalLoaderXMLConfigRepository * user_defined_executable_functions_config_repository = nullptr; - scope_guard user_defined_executable_functions_xmls; + mutable std::unique_ptr external_user_defined_executable_functions_loader TSA_GUARDED_BY(external_user_defined_executable_functions_mutex); + ExternalLoaderXMLConfigRepository * user_defined_executable_functions_config_repository TSA_GUARDED_BY(external_user_defined_executable_functions_mutex) = nullptr; + scope_guard user_defined_executable_functions_xmls TSA_GUARDED_BY(external_user_defined_executable_functions_mutex); mutable OnceFlag user_defined_sql_objects_loader_initialized; mutable std::unique_ptr user_defined_sql_objects_loader; @@ -250,29 +249,30 @@ struct ContextSharedPart : boost::noncopyable mutable OnceFlag backups_worker_initialized; std::optional backups_worker; + /// No lock required for default_profile_name, system_profile_name, buffer_profile_name modified only during initialization String default_profile_name; /// Default profile name used for default values. String system_profile_name; /// Profile used by system processes String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying - std::unique_ptr access_control; + std::unique_ptr access_control TSA_GUARDED_BY(mutex); mutable OnceFlag resource_manager_initialized; mutable ResourceManagerPtr resource_manager; - mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. - mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. + mutable UncompressedCachePtr uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks. + mutable MarkCachePtr mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files. mutable OnceFlag load_marks_threadpool_initialized; mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. mutable OnceFlag prefetch_threadpool_initialized; mutable std::unique_ptr prefetch_threadpool; /// Threadpool for loading marks cache. - mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices. - mutable QueryCachePtr query_cache; /// Cache of query results. - mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices. - mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. + mutable UncompressedCachePtr index_uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks for MergeTree indices. + mutable QueryCachePtr query_cache TSA_GUARDED_BY(mutex); /// Cache of query results. + mutable MarkCachePtr index_mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files of MergeTree indices. + mutable MMappedFileCachePtr mmap_cache TSA_GUARDED_BY(mutex); /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. ProcessList process_list; /// Executing queries at the moment. SessionTracker session_tracker; GlobalOvercommitTracker global_overcommit_tracker; MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) ReplicatedFetchList replicated_fetch_list; - ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections. + ConfigurationPtr users_config TSA_GUARDED_BY(mutex); /// Config with the users, profiles and quotas sections. InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. OnceFlag buffer_flush_schedule_pool_initialized; @@ -304,67 +304,72 @@ struct ContextSharedPart : boost::noncopyable mutable ThrottlerPtr backups_server_throttler; /// A server-wide throttler for BACKUPs MultiVersion macros; /// Substitutions extracted from config. - std::unique_ptr ddl_worker; /// Process ddl commands from zk. + std::unique_ptr ddl_worker TSA_GUARDED_BY(mutex); /// Process ddl commands from zk. /// Rules for selecting the compression settings, depending on the size of the part. - mutable std::unique_ptr compression_codec_selector; + mutable std::unique_ptr compression_codec_selector TSA_GUARDED_BY(mutex); /// Storage disk chooser for MergeTree engines - mutable std::shared_ptr merge_tree_disk_selector; + mutable std::shared_ptr merge_tree_disk_selector TSA_GUARDED_BY(storage_policies_mutex); /// Storage policy chooser for MergeTree engines - mutable std::shared_ptr merge_tree_storage_policy_selector; + mutable std::shared_ptr merge_tree_storage_policy_selector TSA_GUARDED_BY(storage_policies_mutex); ServerSettings server_settings; - std::optional merge_tree_settings; /// Settings of MergeTree* engines. - std::optional replicated_merge_tree_settings; /// Settings of ReplicatedMergeTree* engines. + std::optional merge_tree_settings TSA_GUARDED_BY(mutex); /// Settings of MergeTree* engines. + std::optional replicated_merge_tree_settings TSA_GUARDED_BY(mutex); /// Settings of ReplicatedMergeTree* engines. std::atomic_size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default) std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default) + /// No lock required for format_schema_path modified only during initialization String format_schema_path; /// Path to a directory that contains schema files used by input formats. mutable OnceFlag action_locks_manager_initialized; ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers OnceFlag system_logs_initialized; - std::unique_ptr system_logs; /// Used to log queries and operations on parts - std::optional storage_s3_settings; /// Settings of S3 storage - std::vector warnings; /// Store warning messages about server configuration. + std::unique_ptr system_logs TSA_GUARDED_BY(mutex); /// Used to log queries and operations on parts + std::optional storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage + std::vector warnings TSA_GUARDED_BY(mutex); /// Store warning messages about server configuration. /// Background executors for *MergeTree tables - MergeMutateBackgroundExecutorPtr merge_mutate_executor; - OrdinaryBackgroundExecutorPtr moves_executor; - OrdinaryBackgroundExecutorPtr fetch_executor; - OrdinaryBackgroundExecutorPtr common_executor; + /// Has background executors for MergeTree tables been initialized? + mutable ContextSharedMutex background_executors_mutex; + bool are_background_executors_initialized TSA_GUARDED_BY(background_executors_mutex) = false; + MergeMutateBackgroundExecutorPtr merge_mutate_executor TSA_GUARDED_BY(background_executors_mutex); + OrdinaryBackgroundExecutorPtr moves_executor TSA_GUARDED_BY(background_executors_mutex); + OrdinaryBackgroundExecutorPtr fetch_executor TSA_GUARDED_BY(background_executors_mutex); + OrdinaryBackgroundExecutorPtr common_executor TSA_GUARDED_BY(background_executors_mutex); - RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml - HTTPHeaderFilter http_header_filter; /// Forbidden HTTP headers from config.xml + RemoteHostFilter remote_host_filter TSA_GUARDED_BY(mutex); /// Allowed URL from config.xml + HTTPHeaderFilter http_header_filter TSA_GUARDED_BY(mutex); /// Forbidden HTTP headers from config.xml + /// No lock required for trace_collector modified only during initialization std::optional trace_collector; /// Thread collecting traces from threads executing queries /// Clusters for distributed tables /// Initialized on demand (on distributed storages initialization) since Settings should be initialized - std::shared_ptr clusters; - ConfigurationPtr clusters_config; /// Stores updated configs - std::unique_ptr cluster_discovery; mutable std::mutex clusters_mutex; /// Guards clusters, clusters_config and cluster_discovery + std::shared_ptr clusters TSA_GUARDED_BY(clusters_mutex); + ConfigurationPtr clusters_config TSA_GUARDED_BY(clusters_mutex); /// Stores updated configs + std::unique_ptr cluster_discovery TSA_GUARDED_BY(clusters_mutex); + /// No lock required for async_insert_queue modified only during initialization std::shared_ptr async_insert_queue; + std::map server_ports; - bool shutdown_called = false; + std::atomic shutdown_called = false; - /// Has background executors for MergeTree tables been initialized? - bool are_background_executors_initialized = false; - - Stopwatch uptime_watch; + Stopwatch uptime_watch TSA_GUARDED_BY(mutex); + /// No lock required for application_type modified only during initialization Context::ApplicationType application_type = Context::ApplicationType::SERVER; /// vector of xdbc-bridge commands, they will be killed when Context will be destroyed - std::vector> bridge_commands; + std::vector> bridge_commands TSA_GUARDED_BY(mutex); + /// No lock required for config_reload_callback, start_servers_callback, stop_servers_callback modified only during initialization Context::ConfigReloadCallback config_reload_callback; - Context::StartStopServersCallback start_servers_callback; Context::StartStopServersCallback stop_servers_callback; - bool is_server_completely_started = false; + bool is_server_completely_started TSA_GUARDED_BY(mutex) = false; ContextSharedPart() : access_control(std::make_unique()) @@ -505,9 +510,9 @@ struct ContextSharedPart : boost::noncopyable */ void shutdown() TSA_NO_THREAD_SAFETY_ANALYSIS { - if (shutdown_called) + bool is_shutdown_called = shutdown_called.exchange(true); + if (is_shutdown_called) return; - shutdown_called = true; /// Need to flush the async insert queue before shutting down the database catalog async_insert_queue.reset(); @@ -557,7 +562,7 @@ struct ContextSharedPart : boost::noncopyable /// Cause it can call Context::getZooKeeper and resurrect it. { - auto lock = std::lock_guard(mutex); + std::lock_guard lock(mutex); delete_ddl_worker = std::move(ddl_worker); } @@ -574,7 +579,7 @@ struct ContextSharedPart : boost::noncopyable cache->cache->deactivateBackgroundOperations(); { - auto lock = std::lock_guard(mutex); + std::lock_guard lock(mutex); /** Compiled expressions stored in cache need to be destroyed before destruction of static objects. * Because CHJIT instance can be static object. @@ -598,7 +603,6 @@ struct ContextSharedPart : boost::noncopyable /// but at least they can be preserved for storage termination. dictionaries_xmls.reset(); user_defined_executable_functions_xmls.reset(); - models_repository_guard.reset(); delete_system_logs = std::move(system_logs); delete_embedded_dictionaries = std::move(embedded_dictionaries); @@ -648,7 +652,7 @@ struct ContextSharedPart : boost::noncopyable trace_collector.emplace(std::move(trace_log)); } - void addWarningMessage(const String & message) + void addWarningMessage(const String & message) TSA_REQUIRES(mutex) { /// A warning goes both: into server's log; stored to be placed in `system.warnings` table. log->warning(message); @@ -1087,11 +1091,13 @@ const Poco::Util::AbstractConfiguration & Context::getConfigRef() const AccessControl & Context::getAccessControl() { + SharedLockGuard lock(shared->mutex); return *shared->access_control; } const AccessControl & Context::getAccessControl() const { + SharedLockGuard lock(shared->mutex); return *shared->access_control; } @@ -2386,9 +2392,6 @@ Lemmatizers & Context::getLemmatizers() const BackupsWorker & Context::getBackupsWorker() const { callOnce(shared->backups_worker_initialized, [&] { - if (shared->backups_worker) - return; - const auto & config = getConfigRef(); const bool allow_concurrent_backups = config.getBool("backups.allow_concurrent_backups", true); const bool allow_concurrent_restores = config.getBool("backups.allow_concurrent_restores", true); @@ -2400,7 +2403,6 @@ BackupsWorker & Context::getBackupsWorker() const shared->backups_worker.emplace(getGlobalContext(), backup_threads, restore_threads, allow_concurrent_backups, allow_concurrent_restores); }); - SharedLockGuard lock(shared->mutex); return *shared->backups_worker; } @@ -2516,7 +2518,6 @@ ThreadPool & Context::getLoadMarksThreadpool() const { callOnce(shared->load_marks_threadpool_initialized, [&] { const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".load_marks_threadpool_pool_size", 50); auto queue_size = config.getUInt(".load_marks_threadpool_queue_size", 1000000); shared->load_marks_threadpool = std::make_unique( @@ -2573,7 +2574,7 @@ void Context::setIndexMarkCache(const String & cache_policy, size_t max_cache_si void Context::updateIndexMarkCacheConfiguration(const Poco::Util::AbstractConfiguration & config) { - std::lock_guard lock(mutex); + std::lock_guard lock(shared->mutex); if (!shared->index_mark_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Index mark cache was not created yet."); @@ -3005,19 +3006,28 @@ void Context::setSystemZooKeeperLogAfterInitializationIfNeeded() /// This method explicitly sets correct pointer to system log after its initialization. /// TODO get rid of this if possible + std::shared_ptr zookeeper_log; { - std::lock_guard lock(shared->zookeeper_mutex); - if (!shared->system_logs || !shared->system_logs->zookeeper_log) + SharedLockGuard lock(shared->mutex); + if (!shared->system_logs) return; + zookeeper_log = shared->system_logs->zookeeper_log; + } + + if (!zookeeper_log) + return; + + { + std::lock_guard lock(shared->zookeeper_mutex); if (shared->zookeeper) - shared->zookeeper->setZooKeeperLog(shared->system_logs->zookeeper_log); + shared->zookeeper->setZooKeeperLog(zookeeper_log); } { - std::lock_guard lockAuxiliary(shared->auxiliary_zookeepers_mutex); + std::lock_guard lock_auxiliary_zookeepers(shared->auxiliary_zookeepers_mutex); for (auto & zk : shared->auxiliary_zookeepers) - zk.second->setZooKeeperLog(shared->system_logs->zookeeper_log); + zk.second->setZooKeeperLog(zookeeper_log); } } @@ -3228,21 +3238,25 @@ String Context::getInterserverScheme() const void Context::setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config) { + std::lock_guard lock(shared->mutex); shared->remote_host_filter.setValuesFromConfig(config); } const RemoteHostFilter & Context::getRemoteHostFilter() const { + SharedLockGuard lock(shared->mutex); return shared->remote_host_filter; } void Context::setHTTPHeaderFilter(const Poco::Util::AbstractConfiguration & config) { + std::lock_guard lock(shared->mutex); shared->http_header_filter.setValuesFromConfig(config); } const HTTPHeaderFilter & Context::getHTTPHeaderFilter() const { + SharedLockGuard lock(shared->mutex); return shared->http_header_filter; } @@ -3342,7 +3356,7 @@ std::map Context::getClusters() const return clusters; } -std::shared_ptr Context::getClustersImpl(std::lock_guard & /* lock */) const +std::shared_ptr Context::getClustersImpl(std::lock_guard & /* lock */) const TSA_REQUIRES(shared->clusters_mutex) { if (!shared->clusters) { @@ -3404,7 +3418,7 @@ void Context::initializeSystemLogs() /// of any other table if it is stored on a disk with cache. callOnce(shared->system_logs_initialized, [&] { auto system_logs = std::make_unique(getGlobalContext(), getConfigRef()); - auto lock = getGlobalLock(); + std::lock_guard lock(shared->mutex); shared->system_logs = std::move(system_logs); }); } @@ -3415,7 +3429,7 @@ void Context::initializeTraceCollector() } /// Call after unexpected crash happen. -void Context::handleCrash() const +void Context::handleCrash() const TSA_NO_THREAD_SAFETY_ANALYSIS { shared->system_logs->handleCrash(); } @@ -3476,7 +3490,7 @@ std::shared_ptr Context::getPartLog(const String & part_database) const std::shared_ptr Context::getTraceLog() const { - SharedLockGuard lock(mutex); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; @@ -3723,58 +3737,60 @@ StoragePoliciesMap Context::getPoliciesMap() const return getStoragePolicySelector(lock)->getPoliciesMap(); } -DiskSelectorPtr Context::getDiskSelector(std::lock_guard & /* lock */) const +DiskSelectorPtr Context::getDiskSelector(std::lock_guard & /* lock */) const TSA_REQUIRES(shared->storage_policies_mutex) { if (!shared->merge_tree_disk_selector) { constexpr auto config_name = "storage_configuration.disks"; const auto & config = getConfigRef(); - auto disk_selector = std::make_shared(); disk_selector->initialize(config, config_name, shared_from_this()); shared->merge_tree_disk_selector = disk_selector; } + return shared->merge_tree_disk_selector; } -StoragePolicySelectorPtr Context::getStoragePolicySelector(std::lock_guard & lock) const +StoragePolicySelectorPtr Context::getStoragePolicySelector(std::lock_guard & lock) const TSA_REQUIRES(shared->storage_policies_mutex) { if (!shared->merge_tree_storage_policy_selector) { constexpr auto config_name = "storage_configuration.policies"; const auto & config = getConfigRef(); - shared->merge_tree_storage_policy_selector = std::make_shared(config, config_name, getDiskSelector(lock)); } + return shared->merge_tree_storage_policy_selector; } void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration & config) { - std::lock_guard lock(shared->storage_policies_mutex); - - if (shared->merge_tree_disk_selector) - shared->merge_tree_disk_selector - = shared->merge_tree_disk_selector->updateFromConfig(config, "storage_configuration.disks", shared_from_this()); - - if (shared->merge_tree_storage_policy_selector) { - try + std::lock_guard lock(shared->storage_policies_mutex); + if (shared->merge_tree_disk_selector) + shared->merge_tree_disk_selector + = shared->merge_tree_disk_selector->updateFromConfig(config, "storage_configuration.disks", shared_from_this()); + + if (shared->merge_tree_storage_policy_selector) { - shared->merge_tree_storage_policy_selector = shared->merge_tree_storage_policy_selector->updateFromConfig( - config, "storage_configuration.policies", shared->merge_tree_disk_selector); - } - catch (Exception & e) - { - LOG_ERROR( - shared->log, "An error has occurred while reloading storage policies, storage policies were not applied: {}", e.message()); + try + { + shared->merge_tree_storage_policy_selector = shared->merge_tree_storage_policy_selector->updateFromConfig( + config, "storage_configuration.policies", shared->merge_tree_disk_selector); + } + catch (Exception & e) + { + LOG_ERROR( + shared->log, "An error has occurred while reloading storage policies, storage policies were not applied: {}", e.message()); + } } } - if (shared->storage_s3_settings) { - shared->storage_s3_settings->loadFromConfig("s3", config, getSettingsRef()); + std::lock_guard lock(shared->mutex); + if (shared->storage_s3_settings) + shared->storage_s3_settings->loadFromConfig("s3", config, getSettingsRef()); } } @@ -4473,7 +4489,7 @@ MergeTreeTransactionPtr Context::getCurrentTransaction() const bool Context::isServerCompletelyStarted() const { - std::lock_guard lock(shared->mutex); + SharedLockGuard lock(shared->mutex); assert(getApplicationType() == ApplicationType::SERVER); return shared->is_server_completely_started; } @@ -4594,7 +4610,7 @@ void Context::setAsynchronousInsertQueue(const std::shared_ptrmutex); + std::lock_guard lock(shared->background_executors_mutex); if (shared->are_background_executors_initialized) return; @@ -4656,27 +4672,31 @@ void Context::initializeBackgroundExecutorsIfNeeded() bool Context::areBackgroundExecutorsInitialized() { - std::lock_guard lock(shared->mutex); + SharedLockGuard lock(shared->background_executors_mutex); return shared->are_background_executors_initialized; } MergeMutateBackgroundExecutorPtr Context::getMergeMutateExecutor() const { + SharedLockGuard lock(shared->background_executors_mutex); return shared->merge_mutate_executor; } OrdinaryBackgroundExecutorPtr Context::getMovesExecutor() const { + SharedLockGuard lock(shared->background_executors_mutex); return shared->moves_executor; } OrdinaryBackgroundExecutorPtr Context::getFetchesExecutor() const { + SharedLockGuard lock(shared->background_executors_mutex); return shared->fetch_executor; } OrdinaryBackgroundExecutorPtr Context::getCommonExecutor() const { + SharedLockGuard lock(shared->background_executors_mutex); return shared->common_executor; } From 38a3655c5e7ea0457328f2ca41c979809e32f206 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 23 Oct 2023 10:09:06 +0100 Subject: [PATCH 494/634] add fibonacci to aspell --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2c73c1032f9..b5d66f5e17b 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1455,6 +1455,7 @@ farmFingerprint farmHash fastops fcoverage +fibonacci fifo filesystem filesystemAvailable From 4368af3ba3f605fd0919e0d244e00c676c180c78 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 23 Oct 2023 09:12:08 +0000 Subject: [PATCH 495/634] Docs: Point to formula in community tap for homebrew installation --- docs/en/getting-started/install.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index d2e7ab30478..6485232f20a 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -380,7 +380,7 @@ build. ### macOS-only: Install with Homebrew -To install ClickHouse using the popular `brew` package manager, follow the instructions listed in the [ClickHouse Homebrew tap](https://github.com/ClickHouse/homebrew-clickhouse). +To install ClickHouse using [homebrew](https://brew.sh/), see [here](https://formulae.brew.sh/cask/clickhouse#default). ## Launch {#launch} From 9a33d5171912c28941309e74db200c0227f1fa72 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 23 Oct 2023 09:14:09 +0000 Subject: [PATCH 496/634] Fix link --- docs/en/getting-started/install.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 6485232f20a..52fa1689d9d 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -380,7 +380,7 @@ build. ### macOS-only: Install with Homebrew -To install ClickHouse using [homebrew](https://brew.sh/), see [here](https://formulae.brew.sh/cask/clickhouse#default). +To install ClickHouse using [homebrew](https://brew.sh/), see [here](https://formulae.brew.sh/cask/clickhouse). ## Launch {#launch} From fe95c0d0e41f7a7c2302b29ac39d613c474c4f05 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 14 Aug 2023 09:58:08 +0000 Subject: [PATCH 497/634] Improve CHECK TABLE system query Resubmit PR #52745 --- .../sql-reference/statements/check-table.md | 107 +++++++--- src/Common/FileChecker.cpp | 46 ++-- src/Common/FileChecker.h | 37 +++- src/Interpreters/InterpreterCheckQuery.cpp | 201 ++++++++++++++---- src/Storages/CheckResults.h | 2 - src/Storages/IStorage.cpp | 11 + src/Storages/IStorage.h | 41 +++- src/Storages/StorageLog.cpp | 9 +- src/Storages/StorageLog.h | 16 +- src/Storages/StorageMergeTree.cpp | 26 ++- src/Storages/StorageMergeTree.h | 29 ++- src/Storages/StorageProxy.h | 5 +- src/Storages/StorageReplicatedMergeTree.cpp | 37 ++-- src/Storages/StorageReplicatedMergeTree.h | 31 ++- src/Storages/StorageStripeLog.cpp | 10 +- src/Storages/StorageStripeLog.h | 17 +- .../0_stateless/00063_check_query.reference | 1 + .../queries/0_stateless/00063_check_query.sql | 3 + .../queries/0_stateless/00961_check_table.sql | 12 +- ...1042_check_query_and_last_granule_size.sql | 12 +- ...02235_check_table_sparse_serialization.sql | 3 +- .../02841_check_table_progress.reference | 2 + .../0_stateless/02841_check_table_progress.sh | 29 +++ 23 files changed, 547 insertions(+), 140 deletions(-) create mode 100644 tests/queries/0_stateless/02841_check_table_progress.reference create mode 100755 tests/queries/0_stateless/02841_check_table_progress.sh diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index 0209d59b018..db8c32249ef 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -5,19 +5,38 @@ sidebar_label: CHECK TABLE title: "CHECK TABLE Statement" --- -Checks if the data in the table is corrupted. +The `CHECK TABLE` query in ClickHouse is used to perform a validation check on a specific table or its partitions. It ensures the integrity of the data by verifying the checksums and other internal data structures. -``` sql -CHECK TABLE [db.]name [PARTITION partition_expr] +Particularly it compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. + +:::note +The `CHECK TABLE`` query may read all the data in the table and hold some resources, making it resource-intensive. +Consider the potential impact on performance and resource utilization before executing this query. +::: + +## Syntax + +The basic syntax of the query is as follows: + +```sql +CHECK TABLE table_name [PARTITION partition_expression] [FORMAT format] [SETTINGS check_query_single_value_result = (0|1) [, other_settings] ] ``` -The `CHECK TABLE` query compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. +- `table_name`: Specifies the name of the table that you want to check. +- `partition_expression`: (Optional) If you want to check a specific partition of the table, you can use this expression to specify the partition. +- `FORMAT format`: (Optional) Allows you to specify the output format of the result. +- `SETTINGS`: (Optional) Allows additional settings. + - **`check_query_single_value_result`**: (Optional) This setting allows you to toggle between a detailed result (`0`) or a summarized result (`1`). + - Other settings (e.g. `max_threads` can be applied as well). -The query response contains the `result` column with a single row. The row has a value of -[Boolean](../../sql-reference/data-types/boolean.md) type: -- 0 - The data in the table is corrupted. -- 1 - The data maintains integrity. +The query response depends on the value of contains `check_query_single_value_result` setting. +In case of `check_query_single_value_result = 1` only `result` column with a single row is returned. Value inside this row is `1` if the integrity check is passed and `0` if data is corrupted. + +With `check_query_single_value_result = 0` the query returns the following columns: + - `part_path`: Indicates the path to the data part or file name. + - `is_passed`: Returns 1 if the check for this part is successful, 0 otherwise. + - `message`: Any additional messages related to the check, such as errors or success messages. The `CHECK TABLE` query supports the following table engines: @@ -26,30 +45,15 @@ The `CHECK TABLE` query supports the following table engines: - [StripeLog](../../engines/table-engines/log-family/stripelog.md) - [MergeTree family](../../engines/table-engines/mergetree-family/mergetree.md) -Performed over the tables with another table engines causes an exception. +Performed over the tables with another table engines causes an `NOT_IMPLEMETED` exception. Engines from the `*Log` family do not provide automatic data recovery on failure. Use the `CHECK TABLE` query to track data loss in a timely manner. -## Checking the MergeTree Family Tables +## Examples -For `MergeTree` family engines, if [check_query_single_value_result](../../operations/settings/settings.md#check_query_single_value_result) = 0, the `CHECK TABLE` query shows a check status for every individual data part of a table on the local server. +By default `CHECK TABLE` query shows the general table check status: ```sql -SET check_query_single_value_result = 0; -CHECK TABLE test_table; -``` - -```text -┌─part_path─┬─is_passed─┬─message─┐ -│ all_1_4_1 │ 1 │ │ -│ all_1_4_2 │ 1 │ │ -└───────────┴───────────┴─────────┘ -``` - -If `check_query_single_value_result` = 1, the `CHECK TABLE` query shows the general table check status. - -```sql -SET check_query_single_value_result = 1; CHECK TABLE test_table; ``` @@ -59,11 +63,60 @@ CHECK TABLE test_table; └────────┘ ``` +If you want to see the check status for every individual data part you may use `check_query_single_value_result` setting. + +Also, to check a specific partition of the table, you can use the `PARTITION` keyword. + +```sql +CHECK TABLE t0 PARTITION ID '201003' +FORMAT PrettyCompactMonoBlock +SETTINGS check_query_single_value_result = 0 +``` + +Output: + +```text +┌─part_path────┬─is_passed─┬─message─┐ +│ 201003_7_7_0 │ 1 │ │ +│ 201003_3_3_0 │ 1 │ │ +└──────────────┴───────────┴─────────┘ +``` + +### Receiving a 'Corrupted' Result + +:::warning +Disclaimer: The procedure described here, including the manual manipulating or removing files directly from the data directory, is for experimental or development environments only. Do **not** attempt this on a production server, as it may lead to data loss or other unintended consequences. +::: + +Remove the existing checksum file: + +```bash +rm /var/lib/clickhouse-server/data/default/t0/201003_3_3_0/checksums.txt +``` + +```sql +CHECK TABLE t0 PARTITION ID '201003' +FORMAT PrettyCompactMonoBlock +SETTINGS check_query_single_value_result = 0 + + +Output: + +```text +┌─part_path────┬─is_passed─┬─message──────────────────────────────────┐ +│ 201003_7_7_0 │ 1 │ │ +│ 201003_3_3_0 │ 1 │ Checksums recounted and written to disk. │ +└──────────────┴───────────┴──────────────────────────────────────────┘ +``` + +If the checksums.txt file is missing, it can be restored. It will be recalculated and rewritten during the execution of the CHECK TABLE command for the specific partition, and the status will still be reported as 'success.'" + + ## If the Data Is Corrupted If the table is corrupted, you can copy the non-corrupted data to another table. To do this: 1. Create a new table with the same structure as damaged table. To do this execute the query `CREATE TABLE AS `. -2. Set the [max_threads](../../operations/settings/settings.md#settings-max_threads) value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. +2. Set the `max_threads` value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. 3. Execute the query `INSERT INTO SELECT * FROM `. This request copies the non-corrupted data from the damaged table to another table. Only the data before the corrupted part will be copied. 4. Restart the `clickhouse-client` to reset the `max_threads` value. diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 876bc4e641c..122ea83835d 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -82,33 +82,35 @@ size_t FileChecker::getTotalSize() const } -CheckResults FileChecker::check() const +FileChecker::DataValidationTasksPtr FileChecker::getDataValidationTasks() { - if (map.empty()) - return {}; + return std::make_unique(map); +} - CheckResults results; - - for (const auto & name_size : map) +CheckResult FileChecker::checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const +{ + String name; + size_t expected_size; + bool is_finished = check_data_tasks->next(name, expected_size); + if (is_finished) { - const String & name = name_size.first; - String path = parentPath(files_info_path) + name; - bool exists = fileReallyExists(path); - auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files. - - if (real_size != name_size.second) - { - String failure_message = exists - ? ("Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(name_size.second)) - : ("File " + path + " doesn't exist"); - results.emplace_back(name, false, failure_message); - break; - } - - results.emplace_back(name, true, ""); + has_nothing_to_do = true; + return {}; } - return results; + String path = parentPath(files_info_path) + name; + bool exists = fileReallyExists(path); + auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files. + + if (real_size != expected_size) + { + String failure_message = exists + ? ("Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(expected_size)) + : ("File " + path + " doesn't exist"); + return CheckResult(name, false, failure_message); + } + + return CheckResult(name, true, ""); } void FileChecker::repair() diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index bb0383e4b56..8ffc310b84d 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace Poco { class Logger; } @@ -28,7 +29,11 @@ public: bool empty() const { return map.empty(); } /// Check the files whose parameters are specified in sizes.json - CheckResults check() const; + /// See comment in IStorage::checkDataNext + struct DataValidationTasks; + using DataValidationTasksPtr = std::unique_ptr; + DataValidationTasksPtr getDataValidationTasks(); + CheckResult checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const; /// Truncate files that have excessive size to the expected size. /// Throw exception if the file size is less than expected. @@ -41,6 +46,36 @@ public: /// Returns total size of all files. size_t getTotalSize() const; + struct DataValidationTasks + { + DataValidationTasks(const std::map & map_) + : map(map_), it(map.begin()) + {} + + bool next(String & out_name, size_t & out_size) + { + std::lock_guard lock(mutex); + if (it == map.end()) + return true; + out_name = it->first; + out_size = it->second; + ++it; + return false; + } + + size_t size() const + { + std::lock_guard lock(mutex); + return std::distance(it, map.end()); + } + + const std::map & map; + + mutable std::mutex mutex; + using Iterator = std::map::const_iterator; + Iterator it; + }; + private: void load(); diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 333aed84873..bd530654dd2 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -8,72 +8,201 @@ #include #include #include +#include #include +#include + +#include +#include + namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { -NamesAndTypes getBlockStructure() +Block getSingleValueBlock(UInt8 value) { - return { + return Block{{ColumnUInt8::create(1, value), std::make_shared(), "result"}}; +} + +Block getHeaderForCheckResult() +{ + auto names_and_types = NamesAndTypes{ {"part_path", std::make_shared()}, {"is_passed", std::make_shared()}, {"message", std::make_shared()}, }; + + return Block({ + {names_and_types[0].type->createColumn(), names_and_types[0].type, names_and_types[0].name}, + {names_and_types[1].type->createColumn(), names_and_types[1].type, names_and_types[1].name}, + {names_and_types[2].type->createColumn(), names_and_types[2].type, names_and_types[2].name}, + }); } +Chunk getChunkFromCheckResult(const CheckResult & check_result) +{ + MutableColumns columns = getHeaderForCheckResult().cloneEmptyColumns(); + columns[0]->insert(check_result.fs_path); + columns[1]->insert(static_cast(check_result.success)); + columns[2]->insert(check_result.failure_message); + return Chunk(std::move(columns), 1); +} + +class TableCheckWorkerProcessor : public ISource +{ + +public: + TableCheckWorkerProcessor(IStorage::DataValidationTasksPtr check_data_tasks_, StoragePtr table_) + : ISource(getHeaderForCheckResult()) + , table(table_) + , check_data_tasks(check_data_tasks_) + { + } + + String getName() const override { return "TableCheckWorkerProcessor"; } + +protected: + + std::optional tryGenerate() override + { + bool has_nothing_to_do = false; + auto check_result = table->checkDataNext(check_data_tasks, has_nothing_to_do); + if (has_nothing_to_do) + return {}; + + /// We can omit manual `progess` call, ISource will may count it automatically by returned chunk + /// However, we want to report only rows in progress + progress(1, 0); + + if (!check_result.success) + { + LOG_WARNING(&Poco::Logger::get("InterpreterCheckQuery"), + "Check query for table {} failed, path {}, reason: {}", + table->getStorageID().getNameForLogs(), + check_result.fs_path, + check_result.failure_message); + } + + return getChunkFromCheckResult(check_result); + } + +private: + StoragePtr table; + IStorage::DataValidationTasksPtr check_data_tasks; +}; + +class TableCheckResultEmitter : public IAccumulatingTransform +{ +public: + TableCheckResultEmitter() : IAccumulatingTransform(getHeaderForCheckResult(), getSingleValueBlock(1).cloneEmpty()) {} + + String getName() const override { return "TableCheckResultEmitter"; } + + void consume(Chunk chunk) override + { + if (result_value == 0) + return; + + auto columns = chunk.getColumns(); + if (columns.size() != 3) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong number of columns: {}", columns.size()); + + const auto * col = checkAndGetColumn(columns[1].get()); + for (size_t i = 0; i < col->size(); ++i) + { + if (col->getElement(i) == 0) + { + result_value = 0; + return; + } + } + } + + Chunk generate() override + { + if (is_value_emitted.exchange(true)) + return {}; + auto block = getSingleValueBlock(result_value); + return Chunk(block.getColumns(), block.rows()); + } + +private: + std::atomic result_value{1}; + std::atomic_bool is_value_emitted{false}; +}; + } - -InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) +InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) + : WithContext(context_) + , query_ptr(query_ptr_) { } - BlockIO InterpreterCheckQuery::execute() { const auto & check = query_ptr->as(); - auto table_id = getContext()->resolveStorageID(check, Context::ResolveOrdinary); + const auto & context = getContext(); + auto table_id = context->resolveStorageID(check, Context::ResolveOrdinary); - getContext()->checkAccess(AccessType::SHOW_TABLES, table_id); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); - auto check_results = table->checkData(query_ptr, getContext()); + context->checkAccess(AccessType::SHOW_TABLES, table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); - Block block; - if (getContext()->getSettingsRef().check_query_single_value_result) - { - bool result = std::all_of(check_results.begin(), check_results.end(), [] (const CheckResult & res) { return res.success; }); - auto column = ColumnUInt8::create(); - column->insertValue(static_cast(result)); - block = Block{{std::move(column), std::make_shared(), "result"}}; - } - else - { - auto block_structure = getBlockStructure(); - auto path_column = block_structure[0].type->createColumn(); - auto is_passed_column = block_structure[1].type->createColumn(); - auto message_column = block_structure[2].type->createColumn(); + auto check_data_tasks = table->getCheckTaskList(query_ptr, context); - for (const auto & check_result : check_results) - { - path_column->insert(check_result.fs_path); - is_passed_column->insert(static_cast(check_result.success)); - message_column->insert(check_result.failure_message); - } - - block = Block({ - {std::move(path_column), block_structure[0].type, block_structure[0].name}, - {std::move(is_passed_column), block_structure[1].type, block_structure[1].name}, - {std::move(message_column), block_structure[2].type, block_structure[2].name}}); - } + const auto & settings = context->getSettingsRef(); BlockIO res; - res.pipeline = QueryPipeline(std::make_shared(std::move(block))); + { + auto processors = std::make_shared(); + std::vector worker_ports; + + size_t num_streams = std::max(1, settings.max_threads); + + for (size_t i = 0; i < num_streams; ++i) + { + auto worker_processor = std::make_shared(check_data_tasks, table); + if (i == 0) + worker_processor->addTotalRowsApprox(check_data_tasks->size()); + worker_ports.emplace_back(&worker_processor->getPort()); + processors->emplace_back(worker_processor); + } + + OutputPort * resize_outport; + { + auto resize_processor = std::make_shared(getHeaderForCheckResult(), worker_ports.size(), 1); + + auto & resize_inputs = resize_processor->getInputs(); + auto resize_inport_it = resize_inputs.begin(); + for (size_t i = 0; i < worker_ports.size(); ++i, ++resize_inport_it) + connect(*worker_ports[i], *resize_inport_it); + + resize_outport = &resize_processor->getOutputs().front(); + processors->emplace_back(resize_processor); + } + + if (settings.check_query_single_value_result) + { + auto emitter_processor = std::make_shared(); + auto * input_port = &emitter_processor->getInputPort(); + processors->emplace_back(emitter_processor); + + connect(*resize_outport, *input_port); + } + + res.pipeline = QueryPipeline(Pipe(std::move(processors))); + res.pipeline.setNumThreads(num_streams); + } return res; } diff --git a/src/Storages/CheckResults.h b/src/Storages/CheckResults.h index b342b014fa4..2e4652fea29 100644 --- a/src/Storages/CheckResults.h +++ b/src/Storages/CheckResults.h @@ -22,6 +22,4 @@ struct CheckResult {} }; -using CheckResults = std::vector; - } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index e98b38438b9..f3abf050840 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -276,6 +276,17 @@ bool IStorage::isStaticStorage() const return false; } +IStorage::DataValidationTasksPtr IStorage::getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); +} + +CheckResult IStorage::checkDataNext(DataValidationTasksPtr & /* check_task_list */, bool & has_nothing_to_do) +{ + has_nothing_to_do = true; + return {}; +} + void IStorage::adjustCreateQueryForBackup(ASTPtr &) const { } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 92920c50ba9..9ee1c933679 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -600,8 +600,45 @@ public: /// Provides a hint that the storage engine may evaluate the IN-condition by using an index. virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */, ContextPtr /* query_context */, const StorageMetadataPtr & /* metadata_snapshot */) const { return false; } - /// Checks validity of the data - virtual CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); } + + /** A list of tasks to check a validity of data. + * Each IStorage implementation may interpret this task in its own way. + * E.g. for some storages it to check data it need to check a list of files in filesystem, for others it can be a list of parts. + * Also it may hold resources (e.g. locks) required during check. + */ + struct DataValidationTasksBase + { + /// Number of entries left to check. + /// It decreases after each call to checkDataNext(). + virtual size_t size() const = 0; + virtual ~DataValidationTasksBase() = default; + }; + + using DataValidationTasksPtr = std::shared_ptr; + + virtual DataValidationTasksPtr getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */); + + /** Executes one task from the list. + * If no tasks left, sets has_nothing_to_do to true. + * Note: Function `checkDataNext` is accessing `check_task_list` thread-safely, + * and can be called simultaneously for the same `getCheckTaskList` result + * to process different tasks in parallel. + * Usage: + * + * auto check_task_list = storage.getCheckTaskList(query, context); + * size_t total_tasks = check_task_list->size(); + * while (true) + * { + * size_t tasks_left = check_task_list->size(); + * std::cout << "Checking data: " << (total_tasks - tasks_left) << " / " << total_tasks << " tasks done." << std::endl; + * bool has_nothing_to_do = false; + * auto result = storage.checkDataNext(check_task_list, has_nothing_to_do); + * if (has_nothing_to_do) + * break; + * doSomething(result); + * } + */ + virtual CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do); /// Checks that table could be dropped right now /// Otherwise - throws an exception with detailed information. diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index cf76f7a16ba..57e3fa3991c 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -874,15 +874,18 @@ SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetada return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -CheckResults StorageLog::checkData(const ASTPtr & /* query */, ContextPtr local_context) +IStorage::DataValidationTasksPtr StorageLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context) { ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); - - return file_checker.check(); + return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); } +CheckResult StorageLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) +{ + return file_checker.checkNextEntry(assert_cast(check_task_list.get())->file_checker_tasks, has_nothing_to_do); +} IStorage::ColumnSizeByName StorageLog::getColumnSizes() const { diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index f1d05ed39ac..95f95088aa2 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -59,7 +59,8 @@ public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - CheckResults checkData(const ASTPtr & query, ContextPtr local_context) override; + DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; + CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; @@ -142,6 +143,19 @@ private: std::atomic total_rows = 0; std::atomic total_bytes = 0; + struct DataValidationTasks : public IStorage::DataValidationTasksBase + { + DataValidationTasks(FileChecker::DataValidationTasksPtr file_checker_tasks_, ReadLock && lock_) + : file_checker_tasks(std::move(file_checker_tasks_)), lock(std::move(lock_)) + {} + + size_t size() const override { return file_checker_tasks->size(); } + + FileChecker::DataValidationTasksPtr file_checker_tasks; + /// Lock to prevent table modification while checking + ReadLock lock; + }; + FileChecker file_checker; const size_t max_compress_block_size; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 4897da37a41..b49639d0770 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2200,9 +2200,8 @@ void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type) background_moves_assignee.trigger(); } -CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context) +IStorage::DataValidationTasksPtr StorageMergeTree::getCheckTaskList(const ASTPtr & query, ContextPtr local_context) { - CheckResults results; DataPartsVector data_parts; if (const auto & check_query = query->as(); check_query.partition) { @@ -2212,7 +2211,14 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ else data_parts = getVisibleDataPartsVector(local_context); - for (auto & part : data_parts) + return std::make_unique(std::move(data_parts), local_context); +} + +CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) +{ + auto * data_validation_tasks = assert_cast(check_task_list.get()); + auto local_context = data_validation_tasks->context; + if (auto part = data_validation_tasks->next()) { /// If the checksums file is not present, calculate the checksums and write them to disk. static constexpr auto checksums_path = "checksums.txt"; @@ -2226,7 +2232,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ auto & part_mutable = const_cast(*part); part_mutable.writeChecksums(part->checksums, local_context->getWriteSettings()); - results.emplace_back(part->name, true, "Checksums recounted and written to disk."); + return CheckResult(part->name, true, "Checksums recounted and written to disk."); } catch (...) { @@ -2234,7 +2240,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ throw; tryLogCurrentException(log, __PRETTY_FUNCTION__); - results.emplace_back(part->name, false, "Check of part finished with error: '" + getCurrentExceptionMessage(false) + "'"); + return CheckResult(part->name, false, "Check of part finished with error: '" + getCurrentExceptionMessage(false) + "'"); } } else @@ -2242,18 +2248,22 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ try { checkDataPart(part, true); - results.emplace_back(part->name, true, ""); + return CheckResult(part->name, true, ""); } catch (...) { if (isRetryableException(std::current_exception())) throw; - results.emplace_back(part->name, false, getCurrentExceptionMessage(false)); + return CheckResult(part->name, false, getCurrentExceptionMessage(false)); } } } - return results; + else + { + has_nothing_to_do = true; + return {}; + } } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 89da9ab839e..19571ee4b03 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -108,7 +108,8 @@ public: void onActionLockRemove(StorageActionBlockType action_type) override; - CheckResults checkData(const ASTPtr & query, ContextPtr context) override; + DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; + CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; @@ -278,6 +279,32 @@ private: friend class MergePlainMergeTreeTask; friend class MutatePlainMergeTreeTask; + struct DataValidationTasks : public IStorage::DataValidationTasksBase + { + DataValidationTasks(DataPartsVector && parts_, ContextPtr context_) + : parts(std::move(parts_)), it(parts.begin()), context(std::move(context_)) + {} + + DataPartPtr next() + { + std::lock_guard lock(mutex); + if (it == parts.end()) + return nullptr; + return *(it++); + } + + size_t size() const override + { + std::lock_guard lock(mutex); + return std::distance(it, parts.end()); + } + + mutable std::mutex mutex; + DataPartsVector parts; + DataPartsVector::const_iterator it; + + ContextPtr context; + }; protected: std::map getAlterMutationCommandsForPart(const DataPartPtr & part) const override; diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index d4357ffd3ac..f092ce666cf 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -149,8 +149,11 @@ public: return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); } - CheckResults checkData(const ASTPtr & query, ContextPtr context) override { return getNested()->checkData(query, context); } + DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override { return getNested()->getCheckTaskList(query, context); } + CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override { return getNested()->checkDataNext(check_task_list, has_nothing_to_do); } + void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override { getNested()->checkTableCanBeDropped(query_context); } + bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); } Strings getDataPaths() const override { return getNested()->getDataPaths(); } StoragePolicyPtr getStoragePolicy() const override { return getNested()->getStoragePolicy(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 14e1a65ce92..0486e965dd9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8562,9 +8562,8 @@ void StorageReplicatedMergeTree::enqueuePartForCheck(const String & part_name, t part_check_thread.enqueuePart(part_name, delay_to_check_seconds); } -CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, ContextPtr local_context) +IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList(const ASTPtr & query, ContextPtr local_context) { - CheckResults results; DataPartsVector data_parts; if (const auto & check_query = query->as(); check_query.partition) { @@ -8574,24 +8573,30 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, Context else data_parts = getVisibleDataPartsVector(local_context); - { - auto part_check_lock = part_check_thread.pausePartsCheck(); + auto part_check_lock = part_check_thread.pausePartsCheck(); + return std::make_unique(std::move(data_parts), std::move(part_check_lock)); +} - for (auto & part : data_parts) +CheckResult StorageReplicatedMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) +{ + + if (auto part = assert_cast(check_task_list.get())->next()) + { + try { - try - { - results.push_back(part_check_thread.checkPartAndFix(part->name)); - } - catch (const Exception & ex) - { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); - } + return CheckResult(part_check_thread.checkPartAndFix(part->name)); + } + catch (const Exception & ex) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + return CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); } } - - return results; + else + { + has_nothing_to_do = true; + return {}; + } } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 794991d8e06..cd59659f3d5 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -230,7 +230,8 @@ public: /// Add a part to the queue of parts whose data you want to check in the background thread. void enqueuePartForCheck(const String & part_name, time_t delay_to_check_seconds = 0); - CheckResults checkData(const ASTPtr & query, ContextPtr context) override; + DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; + CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; /// Checks ability to use granularity bool canUseAdaptiveGranularity() const override; @@ -995,6 +996,34 @@ private: bool waitZeroCopyLockToDisappear(const ZeroCopyLock & lock, size_t milliseconds_to_wait) override; void startupImpl(bool from_attach_thread); + + struct DataValidationTasks : public IStorage::DataValidationTasksBase + { + explicit DataValidationTasks(DataPartsVector && parts_, std::unique_lock && parts_check_lock_) + : parts_check_lock(std::move(parts_check_lock_)), parts(std::move(parts_)), it(parts.begin()) + {} + + DataPartPtr next() + { + std::lock_guard lock(mutex); + if (it == parts.end()) + return nullptr; + return *(it++); + } + + size_t size() const override + { + std::lock_guard lock(mutex); + return std::distance(it, parts.end()); + } + + std::unique_lock parts_check_lock; + + mutable std::mutex mutex; + DataPartsVector parts; + DataPartsVector::const_iterator it; + }; + }; String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 83336cbd22e..535e4d5994b 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -403,16 +403,18 @@ SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const Storage return std::make_shared(*this, metadata_snapshot, std::move(lock)); } - -CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr local_context) +IStorage::DataValidationTasksPtr StorageStripeLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context) { ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); - - return file_checker.check(); + return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); } +CheckResult StorageStripeLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) +{ + return file_checker.checkNextEntry(assert_cast(check_task_list.get())->file_checker_tasks, has_nothing_to_do); +} void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index f889a1de71b..5d4e2fcbd3a 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -53,7 +53,8 @@ public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - CheckResults checkData(const ASTPtr & query, ContextPtr ocal_context) override; + DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; + CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } @@ -93,6 +94,20 @@ private: const DiskPtr disk; String table_path; + struct DataValidationTasks : public IStorage::DataValidationTasksBase + { + DataValidationTasks(FileChecker::DataValidationTasksPtr file_checker_tasks_, ReadLock && lock_) + : file_checker_tasks(std::move(file_checker_tasks_)), lock(std::move(lock_)) + {} + + size_t size() const override { return file_checker_tasks->size(); } + + FileChecker::DataValidationTasksPtr file_checker_tasks; + + /// Lock to prevent table modification while checking + ReadLock lock; + }; + String data_file_path; String index_file_path; FileChecker file_checker; diff --git a/tests/queries/0_stateless/00063_check_query.reference b/tests/queries/0_stateless/00063_check_query.reference index 6ed281c757a..e8183f05f5d 100644 --- a/tests/queries/0_stateless/00063_check_query.reference +++ b/tests/queries/0_stateless/00063_check_query.reference @@ -1,2 +1,3 @@ 1 1 +1 diff --git a/tests/queries/0_stateless/00063_check_query.sql b/tests/queries/0_stateless/00063_check_query.sql index e7362074a05..263cf94fb4a 100644 --- a/tests/queries/0_stateless/00063_check_query.sql +++ b/tests/queries/0_stateless/00063_check_query.sql @@ -8,6 +8,9 @@ INSERT INTO check_query_tiny_log VALUES (1, 'A'), (2, 'B'), (3, 'C'); CHECK TABLE check_query_tiny_log; +-- Settings and FORMAT are supported +CHECK TABLE check_query_tiny_log SETTINGS max_threads = 16; +CHECK TABLE check_query_tiny_log FORMAT Null SETTINGS max_threads = 8, check_query_single_value_result = 0; DROP TABLE IF EXISTS check_query_log; diff --git a/tests/queries/0_stateless/00961_check_table.sql b/tests/queries/0_stateless/00961_check_table.sql index 0e0b2c3b483..079acc8cdbb 100644 --- a/tests/queries/0_stateless/00961_check_table.sql +++ b/tests/queries/0_stateless/00961_check_table.sql @@ -3,29 +3,29 @@ DROP TABLE IF EXISTS mt_table; CREATE TABLE mt_table (d Date, key UInt64, data String) ENGINE = MergeTree() PARTITION BY toYYYYMM(d) ORDER BY key; -CHECK TABLE mt_table; +CHECK TABLE mt_table SETTINGS max_threads = 1; INSERT INTO mt_table VALUES (toDate('2019-01-02'), 1, 'Hello'), (toDate('2019-01-02'), 2, 'World'); -CHECK TABLE mt_table; +CHECK TABLE mt_table SETTINGS max_threads = 1; INSERT INTO mt_table VALUES (toDate('2019-01-02'), 3, 'quick'), (toDate('2019-01-02'), 4, 'brown'); SELECT '========'; -CHECK TABLE mt_table; +CHECK TABLE mt_table SETTINGS max_threads = 1; OPTIMIZE TABLE mt_table FINAL; SELECT '========'; -CHECK TABLE mt_table; +CHECK TABLE mt_table SETTINGS max_threads = 1; SELECT '========'; INSERT INTO mt_table VALUES (toDate('2019-02-03'), 5, '!'), (toDate('2019-02-03'), 6, '?'); -CHECK TABLE mt_table; +CHECK TABLE mt_table SETTINGS max_threads = 1; SELECT '========'; @@ -33,6 +33,6 @@ INSERT INTO mt_table VALUES (toDate('2019-02-03'), 7, 'jump'), (toDate('2019-02- OPTIMIZE TABLE mt_table FINAL; -CHECK TABLE mt_table PARTITION 201902; +CHECK TABLE mt_table PARTITION 201902 SETTINGS max_threads = 1; DROP TABLE IF EXISTS mt_table; diff --git a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql index b66aff8384d..eccb2d25878 100644 --- a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql +++ b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql @@ -7,11 +7,11 @@ CREATE TABLE check_query_test (SomeKey UInt64, SomeValue String) ENGINE = MergeT -- Rows in this table are short, so granularity will be 8192. INSERT INTO check_query_test SELECT number, toString(number) FROM system.numbers LIMIT 81920; -CHECK TABLE check_query_test; +CHECK TABLE check_query_test SETTINGS max_threads = 1; OPTIMIZE TABLE check_query_test; -CHECK TABLE check_query_test; +CHECK TABLE check_query_test SETTINGS max_threads = 1; DROP TABLE IF EXISTS check_query_test; @@ -21,18 +21,18 @@ CREATE TABLE check_query_test_non_adaptive (SomeKey UInt64, SomeValue String) EN INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 81920; -CHECK TABLE check_query_test_non_adaptive; +CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; OPTIMIZE TABLE check_query_test_non_adaptive; -CHECK TABLE check_query_test_non_adaptive; +CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 77; -CHECK TABLE check_query_test_non_adaptive; +CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; OPTIMIZE TABLE check_query_test_non_adaptive; -CHECK TABLE check_query_test_non_adaptive; +CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; DROP TABLE IF EXISTS check_query_test_non_adaptive; diff --git a/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql b/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql index 0ac97404c46..625be63e0c0 100644 --- a/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql +++ b/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql @@ -12,7 +12,6 @@ SELECT name, column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_02235' ORDER BY name, column; -SET check_query_single_value_result = 0; -CHECK TABLE t_sparse_02235; +CHECK TABLE t_sparse_02235 SETTINGS check_query_single_value_result = 0, max_threads = 1; DROP TABLE t_sparse_02235; diff --git a/tests/queries/0_stateless/02841_check_table_progress.reference b/tests/queries/0_stateless/02841_check_table_progress.reference new file mode 100644 index 00000000000..541dab48def --- /dev/null +++ b/tests/queries/0_stateless/02841_check_table_progress.reference @@ -0,0 +1,2 @@ +Ok +Ok diff --git a/tests/queries/0_stateless/02841_check_table_progress.sh b/tests/queries/0_stateless/02841_check_table_progress.sh new file mode 100755 index 00000000000..166386b999b --- /dev/null +++ b/tests/queries/0_stateless/02841_check_table_progress.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t0"; +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t0 (x UInt64, val String) ENGINE = MergeTree ORDER BY x PARTITION BY x % 100"; +${CLICKHOUSE_CLIENT} -q "INSERT INTO t0 SELECT sipHash64(number), randomPrintableASCII(1000) FROM numbers(1000)"; + + +# Check that we have at least 3 different values for read_rows +UNIQUE_VALUES=$( + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d @- <<< "CHECK TABLE t0" -v |& { + grep -F -e X-ClickHouse-Progress: -e X-ClickHouse-Summary: | grep -o '"read_rows"\s*:\s*"[0-9]*"' + } | uniq | wc -l +) + +[ "$UNIQUE_VALUES" -ge "3" ] && echo "Ok" || echo "Fail: got $UNIQUE_VALUES" + + +# Check that we have we have at least 100 total_rows_to_read (at least one check task per partition) +MAX_TOTAL_VALUE=$( + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d @- <<< "CHECK TABLE t0" -v |& { + grep -F -e X-ClickHouse-Progress: -e X-ClickHouse-Summary: | grep -o '"total_rows_to_read"\s*:\s*"[0-9]*"' | grep -o '[0-9]*' + } | sort -n | tail -1 +) + +[ "$MAX_TOTAL_VALUE" -ge "100" ] && echo "Ok" || echo "Fail: got $MAX_TOTAL_VALUE" From e84fc76292bedb817df0c821292bedc04c8ad640 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 14 Aug 2023 10:06:57 +0000 Subject: [PATCH 498/634] Deterministic result for CHECK TABLE in tests --- .../configs/config.d/storage_conf_l.xml | 52 +++++++ .../test_broken_part_during_merge/test.py | 5 +- tests/integration/test_check_table/test.py | 33 +++-- tests/integration/test_store_cleanup/test.py | 134 ++++++++++++++++++ .../01037_zookeeper_check_table_empty_pk.sql | 4 +- .../01112_check_table_with_index.sql | 2 +- .../0_stateless/01390_check_table_codec.sql | 4 +- .../01710_projection_part_check.sql | 6 +- ...97_source_part_is_intact_when_mutation.sql | 8 +- 9 files changed, 219 insertions(+), 29 deletions(-) create mode 100644 tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf_l.xml diff --git a/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf_l.xml b/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf_l.xml new file mode 100644 index 00000000000..7f866e9beed --- /dev/null +++ b/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf_l.xml @@ -0,0 +1,52 @@ + + + + + 0 + + + s3 + https://vdimir-test2.s3.amazonaws.com/ttt/ + AKIAZURMN3FVQCQT6Y5U + pTfhdJgl4HOSIgL+aIE/pnGTZ7IAXMMcYvGhiDnb + eu-central-1 + /var/lib/clickhouse/gcs/ + false + + + cache + s3 + /var/lib/clickhouse/s3_cache/ + 10Gi + + + + + + + default + 10000000 + +

+ s3_cache +
+ + 0.99 + + + + + + default + + + s3 + + + + + + + + true + diff --git a/tests/integration/test_broken_part_during_merge/test.py b/tests/integration/test_broken_part_during_merge/test.py index 26962236869..19c22201fb0 100644 --- a/tests/integration/test_broken_part_during_merge/test.py +++ b/tests/integration/test_broken_part_during_merge/test.py @@ -24,7 +24,7 @@ def test_merge_and_part_corruption(started_cluster): node1.query( """ CREATE TABLE replicated_mt(date Date, id UInt32, value Int32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt', '{replica}') ORDER BY id + ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt', '{replica}') ORDER BY id SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0; """.format( replica=node1.name @@ -59,7 +59,8 @@ def test_merge_and_part_corruption(started_cluster): # corrupt part after merge already assigned, but not started res_opt = p.apply_async(optimize_with_delay, (1,)) node1.query( - "CHECK TABLE replicated_mt", settings={"check_query_single_value_result": 0} + "CHECK TABLE replicated_mt", + settings={"check_query_single_value_result": 0, "max_threads": 1}, ) # start merge node1.query("SYSTEM START REPLICATION QUEUES replicated_mt") diff --git a/tests/integration/test_check_table/test.py b/tests/integration/test_check_table/test.py index 99a5846d4ee..69ae11bfc1e 100644 --- a/tests/integration/test_check_table/test.py +++ b/tests/integration/test_check_table/test.py @@ -78,7 +78,7 @@ def test_check_normal_table_corruption(started_cluster): assert ( node1.query( "CHECK TABLE non_replicated_mt PARTITION 201902", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ) == "201902_1_1_0\t1\t\n" ) @@ -88,7 +88,7 @@ def test_check_normal_table_corruption(started_cluster): assert ( node1.query( "CHECK TABLE non_replicated_mt", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ).strip() == "201902_1_1_0\t1\tChecksums recounted and written to disk." ) @@ -100,7 +100,7 @@ def test_check_normal_table_corruption(started_cluster): assert ( node1.query( "CHECK TABLE non_replicated_mt PARTITION 201902", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ).strip() == "201902_1_1_0\t1\tChecksums recounted and written to disk." ) @@ -111,12 +111,12 @@ def test_check_normal_table_corruption(started_cluster): assert node1.query( "CHECK TABLE non_replicated_mt", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ).strip().split("\t")[0:2] == ["201902_1_1_0", "0"] assert node1.query( "CHECK TABLE non_replicated_mt", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ).strip().split("\t")[0:2] == ["201902_1_1_0", "0"] node1.query( @@ -126,7 +126,7 @@ def test_check_normal_table_corruption(started_cluster): assert ( node1.query( "CHECK TABLE non_replicated_mt PARTITION 201901", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ) == "201901_2_2_0\t1\t\n" ) @@ -137,10 +137,11 @@ def test_check_normal_table_corruption(started_cluster): assert node1.query( "CHECK TABLE non_replicated_mt PARTITION 201901", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ).strip().split("\t")[0:2] == ["201901_2_2_0", "0"] + def test_check_replicated_table_simple(started_cluster): for node in [node1, node2]: node.query("DROP TABLE IF EXISTS replicated_mt") @@ -164,13 +165,15 @@ def test_check_replicated_table_simple(started_cluster): assert ( node1.query( - "CHECK TABLE replicated_mt", settings={"check_query_single_value_result": 0} + "CHECK TABLE replicated_mt", + settings={"check_query_single_value_result": 0, "max_threads": 1}, ) == "201902_0_0_0\t1\t\n" ) assert ( node2.query( - "CHECK TABLE replicated_mt", settings={"check_query_single_value_result": 0} + "CHECK TABLE replicated_mt", + settings={"check_query_single_value_result": 0, "max_threads": 1}, ) == "201902_0_0_0\t1\t\n" ) @@ -185,14 +188,14 @@ def test_check_replicated_table_simple(started_cluster): assert ( node1.query( "CHECK TABLE replicated_mt PARTITION 201901", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ) == "201901_0_0_0\t1\t\n" ) assert ( node2.query( "CHECK TABLE replicated_mt PARTITION 201901", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ) == "201901_0_0_0\t1\t\n" ) @@ -229,7 +232,7 @@ def test_check_replicated_table_corruption(started_cluster): corrupt_data_part_on_disk(node1, "replicated_mt_1", part_name) assert node1.query( "CHECK TABLE replicated_mt_1 PARTITION 201901", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ) == "{p}\t0\tPart {p} looks broken. Removing it and will try to fetch.\n".format( p=part_name ) @@ -237,14 +240,14 @@ def test_check_replicated_table_corruption(started_cluster): node1.query_with_retry("SYSTEM SYNC REPLICA replicated_mt_1") assert node1.query( "CHECK TABLE replicated_mt_1 PARTITION 201901", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ) == "{}\t1\t\n".format(part_name) assert node1.query("SELECT count() from replicated_mt_1") == "4\n" remove_part_from_disk(node2, "replicated_mt_1", part_name) assert node2.query( "CHECK TABLE replicated_mt_1 PARTITION 201901", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ) == "{p}\t0\tPart {p} looks broken. Removing it and will try to fetch.\n".format( p=part_name ) @@ -252,6 +255,6 @@ def test_check_replicated_table_corruption(started_cluster): node1.query("SYSTEM SYNC REPLICA replicated_mt_1") assert node1.query( "CHECK TABLE replicated_mt_1 PARTITION 201901", - settings={"check_query_single_value_result": 0}, + settings={"check_query_single_value_result": 0, "max_threads": 1}, ) == "{}\t1\t\n".format(part_name) assert node1.query("SELECT count() from replicated_mt_1") == "4\n" diff --git a/tests/integration/test_store_cleanup/test.py b/tests/integration/test_store_cleanup/test.py index 6c5a20a758a..ce12e12003e 100644 --- a/tests/integration/test_store_cleanup/test.py +++ b/tests/integration/test_store_cleanup/test.py @@ -21,6 +21,140 @@ def started_cluster(): cluster.shutdown() +<<<<<<< HEAD:tests/integration/test_store_cleanup/test.py +======= +def remove_broken_detached_part_impl(table, node, expect_broken_prefix): + assert ( + node.query( + f"SELECT COUNT() FROM system.parts WHERE table='{table}' AND active=1" + ) + == "4\n" + ) + + path_to_detached = path_to_data + f"data/default/{table}/detached/" + + result = node.exec_in_container(["ls", path_to_detached]) + assert result.strip() == "" + + corrupt_part_data_on_disk(node, table, "all_3_3_0") + break_part(node, table, "all_3_3_0") + node.query(f"ALTER TABLE {table} DETACH PART 'all_1_1_0'") + result = node.exec_in_container(["touch", f"{path_to_detached}trash"]) + + node.exec_in_container(["mkdir", f"{path_to_detached}../broken_all_fake"]) + node.exec_in_container( + ["touch", "-t", "1312031429.30", f"{path_to_detached}../broken_all_fake"] + ) + result = node.exec_in_container(["stat", f"{path_to_detached}../broken_all_fake"]) + print(result) + assert "Modify: 2013-12-03" in result + node.exec_in_container( + [ + "mv", + f"{path_to_detached}../broken_all_fake", + f"{path_to_detached}broken_all_fake", + ] + ) + + node.exec_in_container(["mkdir", f"{path_to_detached}../unexpected_all_42_1337_5"]) + node.exec_in_container( + [ + "touch", + "-t", + "1312031429.30", + f"{path_to_detached}../unexpected_all_42_1337_5", + ] + ) + result = node.exec_in_container( + ["stat", f"{path_to_detached}../unexpected_all_42_1337_5"] + ) + print(result) + assert "Modify: 2013-12-03" in result + node.exec_in_container( + [ + "mv", + f"{path_to_detached}../unexpected_all_42_1337_5", + f"{path_to_detached}unexpected_all_42_1337_5", + ] + ) + + result = node.query( + f"CHECK TABLE {table}", + settings={"check_query_single_value_result": 0, "max_threads": 1}, + ) + assert "all_3_3_0\t0" in result + + node.query(f"DETACH TABLE {table}") + node.query(f"ATTACH TABLE {table}") + + result = node.exec_in_container(["ls", path_to_detached]) + print(result) + assert f"{expect_broken_prefix}_all_3_3_0" in result + assert "all_1_1_0" in result + assert "trash" in result + assert "broken_all_fake" in result + assert "unexpected_all_42_1337_5" in result + + time.sleep(15) + assert node.contains_in_log( + "Removed broken detached part unexpected_all_42_1337_5 due to a timeout" + ) + + result = node.exec_in_container(["ls", path_to_detached]) + print(result) + assert f"{expect_broken_prefix}_all_3_3_0" not in result + assert "all_1_1_0" in result + assert "trash" in result + assert "broken_all_fake" in result + assert "unexpected_all_42_1337_5" not in result + + node.query(f"DROP TABLE {table} SYNC") + + +def test_remove_broken_detached_part_merge_tree(started_cluster): + node1.query( + """ + CREATE TABLE + mt(id UInt32, value Int32) + ENGINE = MergeTree() ORDER BY id + SETTINGS + merge_tree_enable_clear_old_broken_detached=1, + merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds=5; + """ + ) + + for i in range(4): + node1.query( + f"INSERT INTO mt SELECT number, number * number FROM numbers ({i * 100000}, 100000)" + ) + + remove_broken_detached_part_impl("mt", node1, "broken-on-start") + + +def test_remove_broken_detached_part_replicated_merge_tree(started_cluster): + node1.query( + f""" + CREATE TABLE + replicated_mt(date Date, id UInt32, value Int32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt', '{node1.name}') ORDER BY id + SETTINGS + merge_tree_enable_clear_old_broken_detached=1, + merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds=5, + cleanup_delay_period=1, + cleanup_delay_period_random_add=0, + cleanup_thread_preferred_points_per_iteration=0; + """ + ) + + for i in range(4): + node1.query( + f"INSERT INTO replicated_mt SELECT toDate('2019-10-01'), number, number * number FROM numbers ({i * 100000}, 100000)" + ) + + remove_broken_detached_part_impl("replicated_mt", node1, "broken") + + +>>>>>>> fed4cb07fc8 (Deterministic result for CHECK TABLE in tests):tests/integration/test_broken_detached_part_clean_up/test.py def test_store_cleanup(started_cluster): node1.query("CREATE DATABASE db UUID '10000000-1000-4000-8000-000000000001'") node1.query( diff --git a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql index b836f806170..6e539774e4c 100644 --- a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql +++ b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql @@ -10,7 +10,7 @@ CREATE TABLE mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = MergeT INSERT INTO mt_without_pk VALUES (1, 2); -CHECK TABLE mt_without_pk; +CHECK TABLE mt_without_pk SETTINGS max_threads = 1; DROP TABLE IF EXISTS mt_without_pk SYNC; @@ -20,6 +20,6 @@ CREATE TABLE replicated_mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGI INSERT INTO replicated_mt_without_pk VALUES (1, 2); -CHECK TABLE replicated_mt_without_pk; +CHECK TABLE replicated_mt_without_pk SETTINGS max_threads = 1; DROP TABLE IF EXISTS replicated_mt_without_pk SYNC; diff --git a/tests/queries/0_stateless/01112_check_table_with_index.sql b/tests/queries/0_stateless/01112_check_table_with_index.sql index e9613df7d1a..8b59466926f 100644 --- a/tests/queries/0_stateless/01112_check_table_with_index.sql +++ b/tests/queries/0_stateless/01112_check_table_with_index.sql @@ -10,6 +10,6 @@ CREATE TABLE check_table_with_indices ( INSERT INTO check_table_with_indices VALUES (0, 'test'), (1, 'test2'); -CHECK TABLE check_table_with_indices; +CHECK TABLE check_table_with_indices SETTINGS max_threads = 1; DROP TABLE check_table_with_indices; diff --git a/tests/queries/0_stateless/01390_check_table_codec.sql b/tests/queries/0_stateless/01390_check_table_codec.sql index 639d5bea6e4..83a18b4f140 100644 --- a/tests/queries/0_stateless/01390_check_table_codec.sql +++ b/tests/queries/0_stateless/01390_check_table_codec.sql @@ -4,12 +4,12 @@ DROP TABLE IF EXISTS check_codec; CREATE TABLE check_codec(a Int, b Int CODEC(Delta, ZSTD)) ENGINE = MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part = 0; INSERT INTO check_codec SELECT number, number * 2 FROM numbers(1000); -CHECK TABLE check_codec; +CHECK TABLE check_codec SETTINGS max_threads = 1; DROP TABLE check_codec; CREATE TABLE check_codec(a Int, b Int CODEC(Delta, ZSTD)) ENGINE = MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part = '10M'; INSERT INTO check_codec SELECT number, number * 2 FROM numbers(1000); -CHECK TABLE check_codec; +CHECK TABLE check_codec SETTINGS max_threads = 1; DROP TABLE check_codec; diff --git a/tests/queries/0_stateless/01710_projection_part_check.sql b/tests/queries/0_stateless/01710_projection_part_check.sql index b15d9d7525e..8f496ddbeb5 100644 --- a/tests/queries/0_stateless/01710_projection_part_check.sql +++ b/tests/queries/0_stateless/01710_projection_part_check.sql @@ -5,7 +5,7 @@ create table tp (x Int32, y Int32, projection p (select x, y order by x)) engine insert into tp select number, number from numbers(3); insert into tp select number, number from numbers(5); -check table tp settings check_query_single_value_result=0; +check table tp settings check_query_single_value_result=0, max_threads=1; drop table tp; @@ -13,7 +13,7 @@ create table tp (p Date, k UInt64, v1 UInt64, v2 Int64, projection p1 ( select p insert into tp (p, k, v1, v2) values ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -check table tp settings check_query_single_value_result=0; +check table tp settings check_query_single_value_result=0, max_threads=1; drop table tp; @@ -22,5 +22,5 @@ create table tp (x int, projection p (select sum(x))) engine = MergeTree order b insert into tp values (1), (2), (3), (4); select part_type from system.parts where database = currentDatabase() and table = 'tp'; select part_type from system.projection_parts where database = currentDatabase() and table = 'tp'; -check table tp settings check_query_single_value_result=0; +check table tp settings check_query_single_value_result=0, max_threads=1; drop table tp; diff --git a/tests/queries/0_stateless/02497_source_part_is_intact_when_mutation.sql b/tests/queries/0_stateless/02497_source_part_is_intact_when_mutation.sql index 39231c12881..960c6c168a1 100644 --- a/tests/queries/0_stateless/02497_source_part_is_intact_when_mutation.sql +++ b/tests/queries/0_stateless/02497_source_part_is_intact_when_mutation.sql @@ -14,7 +14,7 @@ INSERT INTO t_source_part_is_intact SELECT if (number % 11 = 0, number, 0) FROM numbers(2000); -CHECK TABLE t_source_part_is_intact; +CHECK TABLE t_source_part_is_intact SETTINGS max_threads = 1; SELECT 1, count() FROM t_source_part_is_intact; BEGIN TRANSACTION; @@ -22,18 +22,18 @@ BEGIN TRANSACTION; ALTER TABLE t_source_part_is_intact update u = 0 where u != 0; ROLLBACK; -CHECK TABLE t_source_part_is_intact; +CHECK TABLE t_source_part_is_intact SETTINGS max_threads = 1; BEGIN TRANSACTION; -- size of the file serialization.json is different in the new part ALTER TABLE t_source_part_is_intact update u = 1 WHERE 1; ROLLBACK; -CHECK TABLE t_source_part_is_intact; +CHECK TABLE t_source_part_is_intact SETTINGS max_threads = 1; DETACH TABLE t_source_part_is_intact; ATTACH TABLE t_source_part_is_intact; -CHECK TABLE t_source_part_is_intact; +CHECK TABLE t_source_part_is_intact SETTINGS max_threads = 1; DROP TABLE t_source_part_is_intact; From 9d840c65327ebfbcbaf18b090e51818a239e9148 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 14 Aug 2023 10:33:07 +0000 Subject: [PATCH 499/634] Use single thread for check table with check_query_single_value_result=0 --- docs/en/sql-reference/statements/check-table.md | 2 +- src/Interpreters/InterpreterCheckQuery.cpp | 7 ++++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index db8c32249ef..c24fb83016f 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -27,7 +27,7 @@ CHECK TABLE table_name [PARTITION partition_expression] [FORMAT format] [SETTING - `FORMAT format`: (Optional) Allows you to specify the output format of the result. - `SETTINGS`: (Optional) Allows additional settings. - **`check_query_single_value_result`**: (Optional) This setting allows you to toggle between a detailed result (`0`) or a summarized result (`1`). - - Other settings (e.g. `max_threads` can be applied as well). + - Other settings can be applied as well. If you don't require a deterministic order for the results, you can set max_threads to a value greater than one to speed up the query. The query response depends on the value of contains `check_query_single_value_result` setting. diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index bd530654dd2..03ff165017e 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -167,7 +168,11 @@ BlockIO InterpreterCheckQuery::execute() std::vector worker_ports; - size_t num_streams = std::max(1, settings.max_threads); + /// For verbose mode (`check_query_single_value_result = 0`) + /// if `max_threads` is not explicitly set, we will use single thread to have deterministic output order + const auto * settings_ast = typeid_cast(check.settings_ast.get()); + bool use_max_threads = settings.check_query_single_value_result || (settings_ast && settings_ast->changes.tryGet("max_threads") != nullptr); + size_t num_streams = use_max_threads && settings.max_threads > 1 ? settings.max_threads : 1; for (size_t i = 0; i < num_streams; ++i) { From a3e5f24d4f161bc638e35d75e0e30fa9b8e76a75 Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 23 Oct 2023 09:39:39 +0000 Subject: [PATCH 500/634] Fix flaky test --- .../0_stateless/02896_union_distinct_http_format.reference | 3 +++ tests/queries/0_stateless/02896_union_distinct_http_format.sh | 3 ++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02896_union_distinct_http_format.reference b/tests/queries/0_stateless/02896_union_distinct_http_format.reference index 15c34d719e3..3a68ab6dc9f 100644 --- a/tests/queries/0_stateless/02896_union_distinct_http_format.reference +++ b/tests/queries/0_stateless/02896_union_distinct_http_format.reference @@ -1,3 +1,6 @@ +┌─1─┐ +│ 1 │ +└───┘ ┌─a─┐ │ 1 │ │ 2 │ diff --git a/tests/queries/0_stateless/02896_union_distinct_http_format.sh b/tests/queries/0_stateless/02896_union_distinct_http_format.sh index 1501801325d..bb35800e39d 100755 --- a/tests/queries/0_stateless/02896_union_distinct_http_format.sh +++ b/tests/queries/0_stateless/02896_union_distinct_http_format.sh @@ -4,4 +4,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -curl -d@- -sS "${CLICKHOUSE_URL}" <<< 'SELECT 1 as a UNION DISTINCT SELECT 2 as a order by a FORMAT PrettyCompactMonoBlock' +curl -d@- -sS "${CLICKHOUSE_URL}" <<< 'SELECT 1 UNION DISTINCT SELECT 1 FORMAT PrettyCompactMonoBlock' +curl -d@- -sS "${CLICKHOUSE_URL}" <<< 'SELECT * FROM (SELECT 1 as a UNION DISTINCT SELECT 2 as a) ORDER BY a FORMAT PrettyCompactMonoBlock' From 5cf3c95ea42d860036a20e21b4efc0c4af84282c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 23 Oct 2023 11:49:52 +0200 Subject: [PATCH 501/634] Update RPNBuilder.cpp --- src/Storages/MergeTree/RPNBuilder.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 1edfac65da4..c292505bcd8 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -397,7 +397,7 @@ size_t RPNBuilderFunctionTreeNode::getArgumentsSize() const RPNBuilderTreeNode RPNBuilderFunctionTreeNode::getArgumentAt(size_t index) const { size_t total_arguments = getArgumentsSize(); - if (index >= total_arguments) + if (index >= total_arguments) /// Bug #52632 throw Exception(ErrorCodes::LOGICAL_ERROR, "RPNBuilderFunctionTreeNode has {} arguments, attempted to get argument at index {}", total_arguments, index); From 835be79d4b68b3cdbcdc37fb0ebef3018ac866d7 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 23 Oct 2023 07:06:41 -0300 Subject: [PATCH 502/634] Update docs/en/sql-reference/functions/array-functions.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 61536b854ed..6e460a64bcf 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1149,7 +1149,7 @@ Result: └─────┘ ``` -**Example with the fibonacci sequence** +**Example with the Fibonacci sequence** ```sql SELECT arrayFold( x, acc -> (acc.2, acc.2 + acc.1), range(number), (1::Int64, 0::Int64)).1 AS fibonacci From 8f0d7954ff11ce9a01991b934b6b4d7b6a2afeef Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 23 Oct 2023 10:12:30 +0000 Subject: [PATCH 503/634] IStorage::checkDataNext returns optional --- src/Common/FileChecker.cpp | 5 +---- src/Common/FileChecker.h | 2 +- src/Interpreters/InterpreterCheckQuery.cpp | 13 ++++++------- src/Storages/IStorage.cpp | 3 +-- src/Storages/IStorage.h | 11 +++++------ src/Storages/StorageLog.cpp | 4 ++-- src/Storages/StorageLog.h | 2 +- src/Storages/StorageMergeTree.cpp | 9 +++------ src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageProxy.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 9 +++------ src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/StorageStripeLog.cpp | 4 ++-- src/Storages/StorageStripeLog.h | 2 +- 14 files changed, 29 insertions(+), 41 deletions(-) diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 122ea83835d..049dee459a7 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -87,16 +87,13 @@ FileChecker::DataValidationTasksPtr FileChecker::getDataValidationTasks() return std::make_unique(map); } -CheckResult FileChecker::checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const +std::optional FileChecker::checkNextEntry(DataValidationTasksPtr & check_data_tasks) const { String name; size_t expected_size; bool is_finished = check_data_tasks->next(name, expected_size); if (is_finished) - { - has_nothing_to_do = true; return {}; - } String path = parentPath(files_info_path) + name; bool exists = fileReallyExists(path); diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 8ffc310b84d..41b151e51b0 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -33,7 +33,7 @@ public: struct DataValidationTasks; using DataValidationTasksPtr = std::unique_ptr; DataValidationTasksPtr getDataValidationTasks(); - CheckResult checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const; + std::optional checkNextEntry(DataValidationTasksPtr & check_data_tasks) const; /// Truncate files that have excessive size to the expected size. /// Throw exception if the file size is less than expected. diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 03ff165017e..43b58fafe40 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -75,25 +75,24 @@ protected: std::optional tryGenerate() override { - bool has_nothing_to_do = false; - auto check_result = table->checkDataNext(check_data_tasks, has_nothing_to_do); - if (has_nothing_to_do) + auto check_result = table->checkDataNext(check_data_tasks); + if (!check_result) return {}; /// We can omit manual `progess` call, ISource will may count it automatically by returned chunk /// However, we want to report only rows in progress progress(1, 0); - if (!check_result.success) + if (!check_result->success) { LOG_WARNING(&Poco::Logger::get("InterpreterCheckQuery"), "Check query for table {} failed, path {}, reason: {}", table->getStorageID().getNameForLogs(), - check_result.fs_path, - check_result.failure_message); + check_result->fs_path, + check_result->failure_message); } - return getChunkFromCheckResult(check_result); + return getChunkFromCheckResult(*check_result); } private: diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index f3abf050840..8ea37ca8f30 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -281,9 +281,8 @@ IStorage::DataValidationTasksPtr IStorage::getCheckTaskList(const ASTPtr & /* qu throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); } -CheckResult IStorage::checkDataNext(DataValidationTasksPtr & /* check_task_list */, bool & has_nothing_to_do) +std::optional IStorage::checkDataNext(DataValidationTasksPtr & /* check_task_list */) { - has_nothing_to_do = true; return {}; } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 9ee1c933679..803ab5e92ba 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -619,7 +619,7 @@ public: virtual DataValidationTasksPtr getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */); /** Executes one task from the list. - * If no tasks left, sets has_nothing_to_do to true. + * If no tasks left - returns nullopt. * Note: Function `checkDataNext` is accessing `check_task_list` thread-safely, * and can be called simultaneously for the same `getCheckTaskList` result * to process different tasks in parallel. @@ -631,14 +631,13 @@ public: * { * size_t tasks_left = check_task_list->size(); * std::cout << "Checking data: " << (total_tasks - tasks_left) << " / " << total_tasks << " tasks done." << std::endl; - * bool has_nothing_to_do = false; - * auto result = storage.checkDataNext(check_task_list, has_nothing_to_do); - * if (has_nothing_to_do) + * auto result = storage.checkDataNext(check_task_list); + * if (!result) * break; - * doSomething(result); + * doSomething(*result); * } */ - virtual CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do); + virtual std::optional checkDataNext(DataValidationTasksPtr & check_task_list); /// Checks that table could be dropped right now /// Otherwise - throws an exception with detailed information. diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 57e3fa3991c..6cd814a69d8 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -882,9 +882,9 @@ IStorage::DataValidationTasksPtr StorageLog::getCheckTaskList(const ASTPtr & /* return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); } -CheckResult StorageLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) +std::optional StorageLog::checkDataNext(DataValidationTasksPtr & check_task_list) { - return file_checker.checkNextEntry(assert_cast(check_task_list.get())->file_checker_tasks, has_nothing_to_do); + return file_checker.checkNextEntry(assert_cast(check_task_list.get())->file_checker_tasks); } IStorage::ColumnSizeByName StorageLog::getColumnSizes() const diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 95f95088aa2..ee5bcc009e7 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -60,7 +60,7 @@ public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + std::optional checkDataNext(DataValidationTasksPtr & check_task_list) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b49639d0770..07f6a9f2455 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2214,7 +2214,7 @@ IStorage::DataValidationTasksPtr StorageMergeTree::getCheckTaskList(const ASTPtr return std::make_unique(std::move(data_parts), local_context); } -CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) +std::optional StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list) { auto * data_validation_tasks = assert_cast(check_task_list.get()); auto local_context = data_validation_tasks->context; @@ -2259,11 +2259,8 @@ CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_ } } } - else - { - has_nothing_to_do = true; - return {}; - } + + return {}; } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 19571ee4b03..038970aea44 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -109,7 +109,7 @@ public: void onActionLockRemove(StorageActionBlockType action_type) override; DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + std::optional checkDataNext(DataValidationTasksPtr & check_task_list) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index f092ce666cf..fdeadfff95b 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -150,7 +150,7 @@ public: } DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override { return getNested()->getCheckTaskList(query, context); } - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override { return getNested()->checkDataNext(check_task_list, has_nothing_to_do); } + std::optional checkDataNext(DataValidationTasksPtr & check_task_list) override { return getNested()->checkDataNext(check_task_list); } void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override { getNested()->checkTableCanBeDropped(query_context); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0486e965dd9..1f221cd142f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8577,7 +8577,7 @@ IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList(co return std::make_unique(std::move(data_parts), std::move(part_check_lock)); } -CheckResult StorageReplicatedMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) +std::optional StorageReplicatedMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list) { if (auto part = assert_cast(check_task_list.get())->next()) @@ -8592,11 +8592,8 @@ CheckResult StorageReplicatedMergeTree::checkDataNext(DataValidationTasksPtr & c return CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); } } - else - { - has_nothing_to_do = true; - return {}; - } + + return {}; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index cd59659f3d5..2ef3606d75d 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -231,7 +231,7 @@ public: void enqueuePartForCheck(const String & part_name, time_t delay_to_check_seconds = 0); DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + std::optional checkDataNext(DataValidationTasksPtr & check_task_list) override; /// Checks ability to use granularity bool canUseAdaptiveGranularity() const override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 535e4d5994b..3be74077073 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -411,9 +411,9 @@ IStorage::DataValidationTasksPtr StorageStripeLog::getCheckTaskList(const ASTPtr return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); } -CheckResult StorageStripeLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) +std::optional StorageStripeLog::checkDataNext(DataValidationTasksPtr & check_task_list) { - return file_checker.checkNextEntry(assert_cast(check_task_list.get())->file_checker_tasks, has_nothing_to_do); + return file_checker.checkNextEntry(assert_cast(check_task_list.get())->file_checker_tasks); } void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 5d4e2fcbd3a..636de56e9d9 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -54,7 +54,7 @@ public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + std::optional checkDataNext(DataValidationTasksPtr & check_task_list) override; bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } From d254c4e4a089ca9e9ecf40423dfdc31392ed64f6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 23 Oct 2023 10:13:26 +0000 Subject: [PATCH 504/634] Do no check ast in InterpreterCheckQuery to use max_threads --- src/Interpreters/InterpreterCheckQuery.cpp | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 43b58fafe40..1bdb9578dce 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -166,13 +166,7 @@ BlockIO InterpreterCheckQuery::execute() auto processors = std::make_shared(); std::vector worker_ports; - - /// For verbose mode (`check_query_single_value_result = 0`) - /// if `max_threads` is not explicitly set, we will use single thread to have deterministic output order - const auto * settings_ast = typeid_cast(check.settings_ast.get()); - bool use_max_threads = settings.check_query_single_value_result || (settings_ast && settings_ast->changes.tryGet("max_threads") != nullptr); - size_t num_streams = use_max_threads && settings.max_threads > 1 ? settings.max_threads : 1; - + size_t num_streams = std::max(settings.max_threads, 1); for (size_t i = 0; i < num_streams; ++i) { auto worker_processor = std::make_shared(check_data_tasks, table); From 14c97fb4261aa918005006a29821021beb394108 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 23 Oct 2023 12:48:05 +0200 Subject: [PATCH 505/634] Update docs/en/engines/table-engines/mergetree-family/mergetree.md --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 0a4c78929ef..b90513acbad 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -73,7 +73,7 @@ A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID, ClickHouse uses the sorting key as a primary key if the primary key is not defined explicitly by the `PRIMARY KEY` clause. -Use the `ORDER BY tuple()` syntax, if you do not need sorting Or set `create_table_empty_primary_key_by_default` to `true` to use the `ORDER BY tuple()` syntax by default. See [Selecting the Primary Key](#selecting-the-primary-key). +Use the `ORDER BY tuple()` syntax, if you do not need sorting, or set `create_table_empty_primary_key_by_default` to `true` to use the `ORDER BY tuple()` syntax by default. See [Selecting the Primary Key](#selecting-the-primary-key). #### PARTITION BY From 3332f702753e4ad39aa968222d93bf3331012ddb Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 23 Oct 2023 10:36:09 +0000 Subject: [PATCH 506/634] upd --- src/Interpreters/InterpreterCheckQuery.cpp | 7 ++++--- tests/integration/test_check_table/test.py | 1 - 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 1bdb9578dce..de5e5c2bf2b 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -80,7 +80,7 @@ protected: return {}; /// We can omit manual `progess` call, ISource will may count it automatically by returned chunk - /// However, we want to report only rows in progress + /// However, we want to report only rows in progress, since bytes doesn't make sense here progress(1, 0); if (!check_result->success) @@ -184,9 +184,10 @@ BlockIO InterpreterCheckQuery::execute() auto resize_inport_it = resize_inputs.begin(); for (size_t i = 0; i < worker_ports.size(); ++i, ++resize_inport_it) connect(*worker_ports[i], *resize_inport_it); - - resize_outport = &resize_processor->getOutputs().front(); processors->emplace_back(resize_processor); + + assert(resize_processor->getOutputs().size() == 1); + resize_outport = &resize_processor->getOutputs().front(); } if (settings.check_query_single_value_result) diff --git a/tests/integration/test_check_table/test.py b/tests/integration/test_check_table/test.py index 69ae11bfc1e..d1fbe740310 100644 --- a/tests/integration/test_check_table/test.py +++ b/tests/integration/test_check_table/test.py @@ -141,7 +141,6 @@ def test_check_normal_table_corruption(started_cluster): ).strip().split("\t")[0:2] == ["201901_2_2_0", "0"] - def test_check_replicated_table_simple(started_cluster): for node in [node1, node2]: node.query("DROP TABLE IF EXISTS replicated_mt") From 8bb2378952792e3a08b67ca096e815fda5682b6f Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 23 Oct 2023 10:38:39 +0000 Subject: [PATCH 507/634] [remove before merging] Shuffle CHECK TABLE results --- src/Interpreters/InterpreterCheckQuery.cpp | 51 ++++++++++++++++++++++ 1 file changed, 51 insertions(+) diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index de5e5c2bf2b..1057f8a13a8 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include @@ -140,6 +141,44 @@ private: std::atomic_bool is_value_emitted{false}; }; + +class RandomShuffleTransform : public IAccumulatingTransform +{ +public: + RandomShuffleTransform(const Block & header) + : IAccumulatingTransform(header, header) + {} + + String getName() const override { return "TableCheckResultEmitter"; } + + void consume(Chunk chunk) override + { + chunks.emplace_back(std::move(chunk)); + } + + Chunk generate() override + { + if (!is_shuffled.exchange(true)) + { + std::random_device rd; + std::mt19937 gen(rd()); + std::shuffle(chunks.begin(), chunks.end(), gen); + } + + if (chunks.empty()) + return {}; + + auto chunk = std::move(chunks.back()); + chunks.pop_back(); + return chunk; + } + +private: + Chunks chunks; + std::atomic_bool is_shuffled{false}; +}; + + } InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) @@ -190,6 +229,18 @@ BlockIO InterpreterCheckQuery::execute() resize_outport = &resize_processor->getOutputs().front(); } + /// TODO: for tesing only, remove after CI passed once + if (num_streams > 1) + { + auto shuffle_processor = std::make_shared(resize_outport->getHeader()); + auto * input_port = &shuffle_processor->getInputPort(); + connect(*resize_outport, *input_port); + processors->emplace_back(shuffle_processor); + + assert(resize_processor->getOutputs().size() == 1); + resize_outport = &shuffle_processor->getOutputs().front(); + } + if (settings.check_query_single_value_result) { auto emitter_processor = std::make_shared(); From 5923e1b1167b1bf755654e3dcb46c13106ec164b Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 23 Oct 2023 19:31:44 +0800 Subject: [PATCH 508/634] Cache cast function in set during execution (#55712) * Cache cast function in set during execution Signed-off-by: Duc Canh Le * minor fix for performance test Signed-off-by: Duc Canh Le * Update src/Interpreters/castColumn.cpp Co-authored-by: Nikita Taranov * improvement Signed-off-by: Duc Canh Le * fix use-after-free Signed-off-by: Duc Canh Le --------- Signed-off-by: Duc Canh Le Co-authored-by: Nikita Taranov --- src/Interpreters/Set.cpp | 4 ++-- src/Interpreters/Set.h | 11 ++++++++--- src/Interpreters/castColumn.cpp | 27 ++++++++++++++++----------- src/Interpreters/castColumn.h | 28 +++++++++++++++++++++++++--- tests/performance/enum_in_set.xml | 18 ++++++++++++++++++ 5 files changed, 69 insertions(+), 19 deletions(-) create mode 100644 tests/performance/enum_in_set.xml diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index b8b61c7c11f..5df226cc296 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -324,11 +324,11 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co if (!transform_null_in && data_types[i]->canBeInsideNullable()) { - result = castColumnAccurateOrNull(column_to_cast, data_types[i]); + result = castColumnAccurateOrNull(column_to_cast, data_types[i], cast_cache.get()); } else { - result = castColumnAccurate(column_to_cast, data_types[i]); + result = castColumnAccurate(column_to_cast, data_types[i], cast_cache.get()); } materialized_columns.emplace_back() = result; diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 9ea46e117ef..08c472c3c95 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -9,6 +9,7 @@ #include #include +#include namespace DB @@ -33,9 +34,9 @@ public: /// This is needed for subsequent use for index. Set(const SizeLimits & limits_, size_t max_elements_to_fill_, bool transform_null_in_) : log(&Poco::Logger::get("Set")), - limits(limits_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_) - { - } + limits(limits_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_), + cast_cache(std::make_unique()) + {} /** Set can be created either from AST or from a stream of data (subquery result). */ @@ -142,6 +143,10 @@ private: */ mutable SharedMutex rwlock; + /// A cache for cast functions (if any) to avoid rebuilding cast functions + /// for every call to `execute` + mutable std::unique_ptr cast_cache; + template void insertFromBlockImpl( Method & method, diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index dc9882b84b0..44e669a21ab 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -7,24 +7,29 @@ namespace DB { template -static ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type) +static ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache = nullptr) { if (arg.type->equals(*type) && cast_type != CastType::accurateOrNull) return arg.column; + const auto from_name = arg.type->getName(); + const auto to_name = type->getName(); ColumnsWithTypeAndName arguments { arg, { - DataTypeString().createColumnConst(arg.column->size(), type->getName()), + DataTypeString().createColumnConst(arg.column->size(), to_name), std::make_shared(), "" } }; + auto get_cast_func = [&arguments] + { + FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); + return func_builder_cast->build(arguments); + }; - FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); - - auto func_cast = func_builder_cast->build(arguments); + FunctionBasePtr func_cast = cache ? cache->getOrSet(cast_type, from_name, to_name, std::move(get_cast_func)) : get_cast_func(); if constexpr (cast_type == CastType::accurateOrNull) { @@ -36,19 +41,19 @@ static ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr } } -ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type) +ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache) { - return castColumn(arg, type); + return castColumn(arg, type, cache); } -ColumnPtr castColumnAccurate(const ColumnWithTypeAndName & arg, const DataTypePtr & type) +ColumnPtr castColumnAccurate(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache) { - return castColumn(arg, type); + return castColumn(arg, type, cache); } -ColumnPtr castColumnAccurateOrNull(const ColumnWithTypeAndName & arg, const DataTypePtr & type) +ColumnPtr castColumnAccurateOrNull(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache) { - return castColumn(arg, type); + return castColumn(arg, type, cache); } } diff --git a/src/Interpreters/castColumn.h b/src/Interpreters/castColumn.h index fcbea0f4646..8d2c05025bb 100644 --- a/src/Interpreters/castColumn.h +++ b/src/Interpreters/castColumn.h @@ -1,12 +1,34 @@ #pragma once +#include #include +#include namespace DB { -ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type); -ColumnPtr castColumnAccurate(const ColumnWithTypeAndName & arg, const DataTypePtr & type); -ColumnPtr castColumnAccurateOrNull(const ColumnWithTypeAndName & arg, const DataTypePtr & type); +struct InternalCastFunctionCache +{ +private: + /// Maps -> cast functions + /// Doesn't own key, never refer to key after inserted + std::map, FunctionBasePtr> impl; + mutable std::mutex mutex; +public: + template + FunctionBasePtr getOrSet(CastType cast_type, const String & from, const String & to, Getter && getter) + { + std::lock_guard lock{mutex}; + auto key = std::forward_as_tuple(cast_type, from, to); + auto it = impl.find(key); + if (it == impl.end()) + it = impl.emplace(key, getter()).first; + return it->second; + } +}; + +ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache = nullptr); +ColumnPtr castColumnAccurate(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache = nullptr); +ColumnPtr castColumnAccurateOrNull(const ColumnWithTypeAndName & arg, const DataTypePtr & type, InternalCastFunctionCache * cache = nullptr); } diff --git a/tests/performance/enum_in_set.xml b/tests/performance/enum_in_set.xml new file mode 100644 index 00000000000..bfd9742a206 --- /dev/null +++ b/tests/performance/enum_in_set.xml @@ -0,0 +1,18 @@ + + + + CREATE TABLE iso_3166_1_alpha_2 + ( + `c` Enum8('LI' = -128, 'LT' = -127, 'LU' = -126, 'MO' = -125, 'MK' = -124, 'MG' = -123, 'MW' = -122, 'MY' = -121, 'MV' = -120, 'ML' = -119, 'MT' = -118, 'MH' = -117, 'MQ' = -116, 'MR' = -115, 'MU' = -114, 'YT' = -113, 'MX' = -112, 'FM' = -111, 'MD' = -110, 'MC' = -109, 'MN' = -108, 'ME' = -107, 'MS' = -106, 'MA' = -105, 'MZ' = -104, 'MM' = -103, 'NA' = -102, 'NR' = -101, 'NP' = -100, 'NL' = -99, 'NC' = -98, 'NZ' = -97, 'NI' = -96, 'NE' = -95, 'NG' = -94, 'NU' = -93, 'NF' = -92, 'MP' = -91, 'NO' = -90, 'OM' = -89, 'PK' = -88, 'PW' = -87, 'PS' = -86, 'PA' = -85, 'PG' = -84, 'PY' = -83, 'PE' = -82, 'PH' = -81, 'PN' = -80, 'PL' = -79, 'PT' = -78, 'PR' = -77, 'QA' = -76, 'RE' = -75, 'RO' = -74, 'RU' = -73, 'RW' = -72, 'BL' = -71, 'SH' = -70, 'KN' = -69, 'LC' = -68, 'MF' = -67, 'PM' = -66, 'VC' = -65, 'WS' = -64, 'SM' = -63, 'ST' = -62, 'SA' = -61, 'SN' = -60, 'RS' = -59, 'SC' = -58, 'SL' = -57, 'SG' = -56, 'SX' = -55, 'SK' = -54, 'SI' = -53, 'SB' = -52, 'SO' = -51, 'ZA' = -50, 'GS' = -49, 'SS' = -48, 'ES' = -47, 'LK' = -46, 'SD' = -45, 'SR' = -44, 'SJ' = -43, 'SZ' = -42, 'SE' = -41, 'CH' = -40, 'SY' = -39, 'TW' = -38, 'TJ' = -37, 'TZ' = -36, 'TH' = -35, 'TL' = -34, 'TG' = -33, 'TK' = -32, 'TO' = -31, 'TT' = -30, 'TN' = -29, 'TR' = -28, 'TM' = -27, 'TC' = -26, 'TV' = -25, 'UG' = -24, 'UA' = -23, 'AE' = -22, 'GB' = -21, 'UM' = -20, 'US' = -19, 'UY' = -18, 'UZ' = -17, 'VU' = -16, 'VE' = -15, 'VN' = -14, 'VG' = -13, 'VI' = -12, 'WF' = -11, 'EH' = -10, 'YE' = -9, 'ZM' = -8, 'ZW' = -7, 'OTHER' = 0, 'AF' = 1, 'AX' = 2, 'AL' = 3, 'DZ' = 4, 'AS' = 5, 'AD' = 6, 'AO' = 7, 'AI' = 8, 'AQ' = 9, 'AG' = 10, 'AR' = 11, 'AM' = 12, 'AW' = 13, 'AU' = 14, 'AT' = 15, 'AZ' = 16, 'BS' = 17, 'BH' = 18, 'BD' = 19, 'BB' = 20, 'BY' = 21, 'BE' = 22, 'BZ' = 23, 'BJ' = 24, 'BM' = 25, 'BT' = 26, 'BO' = 27, 'BQ' = 28, 'BA' = 29, 'BW' = 30, 'BV' = 31, 'BR' = 32, 'IO' = 33, 'BN' = 34, 'BG' = 35, 'BF' = 36, 'BI' = 37, 'CV' = 38, 'KH' = 39, 'CM' = 40, 'CA' = 41, 'KY' = 42, 'CF' = 43, 'TD' = 44, 'CL' = 45, 'CN' = 46, 'CX' = 47, 'CC' = 48, 'CO' = 49, 'KM' = 50, 'CD' = 51, 'CG' = 52, 'CK' = 53, 'CR' = 54, 'CI' = 55, 'HR' = 56, 'CU' = 57, 'CW' = 58, 'CY' = 59, 'CZ' = 60, 'DK' = 61, 'DJ' = 62, 'DM' = 63, 'DO' = 64, 'EC' = 65, 'EG' = 66, 'SV' = 67, 'GQ' = 68, 'ER' = 69, 'EE' = 70, 'ET' = 71, 'FK' = 72, 'FO' = 73, 'FJ' = 74, 'FI' = 75, 'FR' = 76, 'GF' = 77, 'PF' = 78, 'TF' = 79, 'GA' = 80, 'GM' = 81, 'GE' = 82, 'DE' = 83, 'GH' = 84, 'GI' = 85, 'GR' = 86, 'GL' = 87, 'GD' = 88, 'GP' = 89, 'GU' = 90, 'GT' = 91, 'GG' = 92, 'GN' = 93, 'GW' = 94, 'GY' = 95, 'HT' = 96, 'HM' = 97, 'VA' = 98, 'HN' = 99, 'HK' = 100, 'HU' = 101, 'IS' = 102, 'IN' = 103, 'ID' = 104, 'IR' = 105, 'IQ' = 106, 'IE' = 107, 'IM' = 108, 'IL' = 109, 'IT' = 110, 'JM' = 111, 'JP' = 112, 'JE' = 113, 'JO' = 114, 'KZ' = 115, 'KE' = 116, 'KI' = 117, 'KP' = 118, 'KR' = 119, 'KW' = 120, 'KG' = 121, 'LA' = 122, 'LV' = 123, 'LB' = 124, 'LS' = 125, 'LR' = 126, 'LY' = 127) + ) + ENGINE = MergeTree + ORDER BY tuple() + SETTINGS index_granularity = 8192 + + INSERT INTO iso_3166_1_alpha_2 SELECT (rand(number) % 256) - 128 FROM numbers(200000000) + OPTIMIZE TABLE iso_3166_1_alpha_2 FINAL + + SELECT count() FROM iso_3166_1_alpha_2 WHERE c NOT IN ('CU', 'BN', 'VI', 'US', 'AQ', 'AG', 'AR', 'AM', 'AW', 'AU', 'AT', 'AZ', 'BS', 'BH', 'BD', 'BB', 'BY', 'BE') FORMAT Null SETTINGS max_threads = 1 + + DROP TABLE IF EXISTS iso_3166_1_alpha_2 + From 99a14410eeb1ba68e2abecc084a887acaf97fad7 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 23 Oct 2023 13:36:27 +0200 Subject: [PATCH 509/634] Set storage.has_lightweight_delete_parts flag when a part has been loaded --- src/Storages/MergeTree/MergeTreeData.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9bb4d1b9418..549f4793099 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1386,6 +1386,9 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( if (to_state == DataPartState::Active) addPartContributionToDataVolume(res.part); + if (res.part->hasLightweightDelete()) + has_lightweight_delete_parts.store(true); + LOG_TRACE(log, "Finished loading {} part {} on disk {}", magic_enum::enum_name(to_state), part_name, part_disk_ptr->getName()); return res; } From 32bbee50a23fb0c69ade137bc344d23906ae4bc7 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 23 Oct 2023 13:44:43 +0200 Subject: [PATCH 510/634] Include information about supported versions in bug report issue template --- .github/ISSUE_TEMPLATE/85_bug-report.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/ISSUE_TEMPLATE/85_bug-report.md b/.github/ISSUE_TEMPLATE/85_bug-report.md index fde5917a8a7..461a14867b4 100644 --- a/.github/ISSUE_TEMPLATE/85_bug-report.md +++ b/.github/ISSUE_TEMPLATE/85_bug-report.md @@ -7,6 +7,8 @@ assignees: '' --- +> Please make sure that the version you're using is still supported. The rules described [here](https://clickhouse.com/docs/en/faq/operations/production#how-to-choose-between-clickhouse-releases). + > You have to provide the following information whenever possible. **Describe what's wrong** From 900844605f03a872871b52d702dfbd98db7f2066 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 23 Oct 2023 13:52:27 +0200 Subject: [PATCH 511/634] Optimise memory consumption during loading of hierarchical dictionaries (#55838) * impl * add comment * fix build --- src/Dictionaries/FlatDictionary.cpp | 2 +- src/Dictionaries/HashedArrayDictionary.cpp | 2 +- src/Dictionaries/HashedDictionary.cpp | 8 ++++---- src/Dictionaries/HierarchyDictionariesUtils.h | 7 ++++++- .../tests/gtest_hierarchy_dictionaries_utils.cpp | 4 ++-- 5 files changed, 14 insertions(+), 9 deletions(-) diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index b06137740da..1c43dd34705 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -288,7 +288,7 @@ DictionaryHierarchyParentToChildIndexPtr FlatDictionary::getHierarchicalIndex() const auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; const ContainerType & parent_keys = std::get>(hierarchical_attribute.container); - HashMap> parent_to_child; + DictionaryHierarchicalParentToChildIndex::ParentToChildIndex parent_to_child; parent_to_child.reserve(element_count); UInt64 child_keys_size = static_cast(parent_keys.size()); diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 45525f1468b..c717462beb2 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -328,7 +328,7 @@ DictionaryHierarchicalParentToChildIndexPtr HashedArrayDictionary> parent_to_child; + DictionaryHierarchicalParentToChildIndex::ParentToChildIndex parent_to_child; parent_to_child.reserve(index_to_key.size()); size_t parent_keys_container_size = parent_keys_container.size(); diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index d6ee6e369c4..004cfbb643d 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -4,13 +4,13 @@ #include #include -#include -#include -#include #include #include #include +#include +#include #include +#include #include @@ -600,7 +600,7 @@ DictionaryHierarchyParentToChildIndexPtr HashedDictionary> parent_to_child; + DictionaryHierarchicalParentToChildIndex::ParentToChildIndex parent_to_child; parent_to_child.reserve(size); for (const auto & map : child_key_to_parent_key_maps) diff --git a/src/Dictionaries/HierarchyDictionariesUtils.h b/src/Dictionaries/HierarchyDictionariesUtils.h index c7508ddd220..4a986153998 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.h +++ b/src/Dictionaries/HierarchyDictionariesUtils.h @@ -26,7 +26,12 @@ public: UInt32 end_index; }; - explicit DictionaryHierarchicalParentToChildIndex(const HashMap> & parent_to_children_map_) + /// By default we use initial_bytes=4096 in PodArray. + /// It might lead to really high memory consumption when arrays are almost empty but there are a lot of them. + using Array = PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; + using ParentToChildIndex = HashMap; + + explicit DictionaryHierarchicalParentToChildIndex(const ParentToChildIndex & parent_to_children_map_) { size_t parent_to_children_map_size = parent_to_children_map_.size(); diff --git a/src/Dictionaries/tests/gtest_hierarchy_dictionaries_utils.cpp b/src/Dictionaries/tests/gtest_hierarchy_dictionaries_utils.cpp index 10d335ebbbc..ea8b7accfa1 100644 --- a/src/Dictionaries/tests/gtest_hierarchy_dictionaries_utils.cpp +++ b/src/Dictionaries/tests/gtest_hierarchy_dictionaries_utils.cpp @@ -173,7 +173,7 @@ TEST(HierarchyDictionariesUtils, getIsInHierarchy) TEST(HierarchyDictionariesUtils, getDescendants) { { - HashMap> parent_to_child; + DictionaryHierarchicalParentToChildIndex::ParentToChildIndex parent_to_child; parent_to_child[0].emplace_back(1); parent_to_child[1].emplace_back(2); parent_to_child[1].emplace_back(3); @@ -221,7 +221,7 @@ TEST(HierarchyDictionariesUtils, getDescendants) } } { - HashMap> parent_to_child; + DictionaryHierarchicalParentToChildIndex::ParentToChildIndex parent_to_child; parent_to_child[1].emplace_back(2); parent_to_child[2].emplace_back(1); From 7583394c9ced40b20dd4263b63023ba6c0cd3590 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 23 Oct 2023 12:13:36 +0000 Subject: [PATCH 512/634] Support CHECK TABLE ... PART ... query --- src/Parsers/ASTCheckQuery.h | 1 + src/Parsers/ParserCheckQuery.cpp | 14 +++++++++++ src/Storages/StorageLog.cpp | 10 +++++++- src/Storages/StorageMergeTree.cpp | 8 +++++++ src/Storages/StorageReplicatedMergeTree.cpp | 8 +++++++ tests/integration/test_check_table/test.py | 23 +++++++++++++++++++ .../queries/0_stateless/00063_check_query.sql | 3 +++ .../0_stateless/00961_check_table.reference | 20 ++++++++++------ .../queries/0_stateless/00961_check_table.sql | 6 +++++ 9 files changed, 85 insertions(+), 8 deletions(-) diff --git a/src/Parsers/ASTCheckQuery.h b/src/Parsers/ASTCheckQuery.h index f29a0bd5406..eb72a99c757 100644 --- a/src/Parsers/ASTCheckQuery.h +++ b/src/Parsers/ASTCheckQuery.h @@ -10,6 +10,7 @@ namespace DB struct ASTCheckQuery : public ASTQueryWithTableAndOutput { ASTPtr partition; + String part_name; /** Get the text that identifies this element. */ String getID(char delim) const override { return "CheckQuery" + (delim + getDatabase()) + delim + getTable(); } diff --git a/src/Parsers/ParserCheckQuery.cpp b/src/Parsers/ParserCheckQuery.cpp index 505b95bda8e..f70c7cda8d0 100644 --- a/src/Parsers/ParserCheckQuery.cpp +++ b/src/Parsers/ParserCheckQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -13,9 +14,11 @@ bool ParserCheckQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_check_table("CHECK TABLE"); ParserKeyword s_partition("PARTITION"); + ParserKeyword s_part("PART"); ParserToken s_dot(TokenType::Dot); ParserPartition partition_parser; + ParserStringLiteral parser_string_literal; if (!s_check_table.ignore(pos, expected)) return false; @@ -30,6 +33,17 @@ bool ParserCheckQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!partition_parser.parse(pos, query->partition, expected)) return false; } + else if (s_part.ignore(pos, expected)) + { + ASTPtr ast_part_name; + if (!parser_string_literal.parse(pos, ast_part_name, expected)) + return false; + + const auto * ast_literal = ast_part_name->as(); + if (!ast_literal || ast_literal->value.getType() != Field::Types::String) + return false; + query->part_name = ast_literal->value.get(); + } if (query->database) query->children.push_back(query->database); diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 6cd814a69d8..10f62cc9432 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -7,6 +7,8 @@ #include +#include + #include #include #include @@ -57,6 +59,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_FILE_NAME; extern const int CANNOT_RESTORE_TABLE; + extern const int NOT_IMPLEMENTED; } /// NOTE: The lock `StorageLog::rwlock` is NOT kept locked while reading, @@ -874,11 +877,16 @@ SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetada return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -IStorage::DataValidationTasksPtr StorageLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context) +IStorage::DataValidationTasksPtr StorageLog::getCheckTaskList(const ASTPtr & query, ContextPtr local_context) { + const auto * check_query = query->as(); + if (check_query->partition || !check_query->part_name.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "CHECK PART/PARTITION are not supported for {}", getName()); + ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); + return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 07f6a9f2455..70695591eb9 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2208,6 +2208,14 @@ IStorage::DataValidationTasksPtr StorageMergeTree::getCheckTaskList(const ASTPtr String partition_id = getPartitionIDFromQuery(check_query.partition, local_context); data_parts = getVisibleDataPartsVectorInPartition(local_context, partition_id); } + else if (!check_query.part_name.empty()) + { + auto part = getPartIfExists(check_query.part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + if (!part) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No such data part '{}' to check in table '{}'", + check_query.part_name, getStorageID().getFullTableName()); + data_parts.emplace_back(std::move(part)); + } else data_parts = getVisibleDataPartsVector(local_context); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1f221cd142f..6ebfca51ea6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8570,6 +8570,14 @@ IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList(co String partition_id = getPartitionIDFromQuery(check_query.partition, local_context); data_parts = getVisibleDataPartsVectorInPartition(local_context, partition_id); } + else if (!check_query.part_name.empty()) + { + auto part = getPartIfExists(check_query.part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + if (!part) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No such data part '{}' to check in table '{}'", + check_query.part_name, getStorageID().getFullTableName()); + data_parts.emplace_back(std::move(part)); + } else data_parts = getVisibleDataPartsVector(local_context); diff --git a/tests/integration/test_check_table/test.py b/tests/integration/test_check_table/test.py index d1fbe740310..17e4ee92e26 100644 --- a/tests/integration/test_check_table/test.py +++ b/tests/integration/test_check_table/test.py @@ -1,6 +1,7 @@ import pytest from helpers.cluster import ClickHouseCluster +from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) @@ -199,6 +200,28 @@ def test_check_replicated_table_simple(started_cluster): == "201901_0_0_0\t1\t\n" ) + assert sorted( + node2.query( + "CHECK TABLE replicated_mt", + settings={"check_query_single_value_result": 0}, + ).split("\n") + ) == ["", "201901_0_0_0\t1\t", "201902_0_0_0\t1\t"] + + with pytest.raises(QueryRuntimeException) as exc: + node2.query( + "CHECK TABLE replicated_mt PART '201801_0_0_0'", + settings={"check_query_single_value_result": 0}, + ) + assert "NO_SUCH_DATA_PART" in str(exc.value) + + assert ( + node2.query( + "CHECK TABLE replicated_mt PART '201902_0_0_0'", + settings={"check_query_single_value_result": 0}, + ) + == "201902_0_0_0\t1\t\n" + ) + def test_check_replicated_table_corruption(started_cluster): for node in [node1, node2]: diff --git a/tests/queries/0_stateless/00063_check_query.sql b/tests/queries/0_stateless/00063_check_query.sql index 263cf94fb4a..90711943150 100644 --- a/tests/queries/0_stateless/00063_check_query.sql +++ b/tests/queries/0_stateless/00063_check_query.sql @@ -8,6 +8,9 @@ INSERT INTO check_query_tiny_log VALUES (1, 'A'), (2, 'B'), (3, 'C'); CHECK TABLE check_query_tiny_log; +CHECK TABLE check_query_tiny_log PARTITION tuple(); -- { serverError NOT_IMPLEMENTED } +CHECK TABLE check_query_tiny_log PART 'all_0_0_0'; -- { serverError NOT_IMPLEMENTED } + -- Settings and FORMAT are supported CHECK TABLE check_query_tiny_log SETTINGS max_threads = 16; CHECK TABLE check_query_tiny_log FORMAT Null SETTINGS max_threads = 8, check_query_single_value_result = 0; diff --git a/tests/queries/0_stateless/00961_check_table.reference b/tests/queries/0_stateless/00961_check_table.reference index d85c66db622..a0a054898b9 100644 --- a/tests/queries/0_stateless/00961_check_table.reference +++ b/tests/queries/0_stateless/00961_check_table.reference @@ -1,11 +1,17 @@ -201901_1_1_0 1 -======== -201901_1_1_0 1 +201801_1_1_0 1 201901_2_2_0 1 ======== -201901_1_2_1 1 +201801_1_1_0 1 +201901_2_2_0 1 +201901_3_3_0 1 ======== -201901_1_2_1 1 -201902_3_3_0 1 +201801_1_1_1 1 +201901_2_3_1 1 ======== -201902_3_4_1 1 +201801_1_1_1 1 +201901_2_3_1 1 +201902_4_4_0 1 +======== +201902_4_5_1 1 +======== +201801_1_1_0 1 diff --git a/tests/queries/0_stateless/00961_check_table.sql b/tests/queries/0_stateless/00961_check_table.sql index 079acc8cdbb..a6abe8103d5 100644 --- a/tests/queries/0_stateless/00961_check_table.sql +++ b/tests/queries/0_stateless/00961_check_table.sql @@ -5,6 +5,8 @@ CREATE TABLE mt_table (d Date, key UInt64, data String) ENGINE = MergeTree() PAR CHECK TABLE mt_table SETTINGS max_threads = 1; +INSERT INTO mt_table VALUES (toDate('2018-01-01'), 1, 'old'); + INSERT INTO mt_table VALUES (toDate('2019-01-02'), 1, 'Hello'), (toDate('2019-01-02'), 2, 'World'); CHECK TABLE mt_table SETTINGS max_threads = 1; @@ -35,4 +37,8 @@ OPTIMIZE TABLE mt_table FINAL; CHECK TABLE mt_table PARTITION 201902 SETTINGS max_threads = 1; +SELECT '========'; + +CHECK TABLE mt_table PART '201801_1_1_0'; + DROP TABLE IF EXISTS mt_table; From 2245a4fba8b814c43cddb4dd0bbc74eef94ed210 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 23 Oct 2023 12:16:00 +0000 Subject: [PATCH 513/634] Fix test_store_cleanup/test.py after rebase --- tests/integration/test_store_cleanup/test.py | 134 ------------------- 1 file changed, 134 deletions(-) diff --git a/tests/integration/test_store_cleanup/test.py b/tests/integration/test_store_cleanup/test.py index ce12e12003e..6c5a20a758a 100644 --- a/tests/integration/test_store_cleanup/test.py +++ b/tests/integration/test_store_cleanup/test.py @@ -21,140 +21,6 @@ def started_cluster(): cluster.shutdown() -<<<<<<< HEAD:tests/integration/test_store_cleanup/test.py -======= -def remove_broken_detached_part_impl(table, node, expect_broken_prefix): - assert ( - node.query( - f"SELECT COUNT() FROM system.parts WHERE table='{table}' AND active=1" - ) - == "4\n" - ) - - path_to_detached = path_to_data + f"data/default/{table}/detached/" - - result = node.exec_in_container(["ls", path_to_detached]) - assert result.strip() == "" - - corrupt_part_data_on_disk(node, table, "all_3_3_0") - break_part(node, table, "all_3_3_0") - node.query(f"ALTER TABLE {table} DETACH PART 'all_1_1_0'") - result = node.exec_in_container(["touch", f"{path_to_detached}trash"]) - - node.exec_in_container(["mkdir", f"{path_to_detached}../broken_all_fake"]) - node.exec_in_container( - ["touch", "-t", "1312031429.30", f"{path_to_detached}../broken_all_fake"] - ) - result = node.exec_in_container(["stat", f"{path_to_detached}../broken_all_fake"]) - print(result) - assert "Modify: 2013-12-03" in result - node.exec_in_container( - [ - "mv", - f"{path_to_detached}../broken_all_fake", - f"{path_to_detached}broken_all_fake", - ] - ) - - node.exec_in_container(["mkdir", f"{path_to_detached}../unexpected_all_42_1337_5"]) - node.exec_in_container( - [ - "touch", - "-t", - "1312031429.30", - f"{path_to_detached}../unexpected_all_42_1337_5", - ] - ) - result = node.exec_in_container( - ["stat", f"{path_to_detached}../unexpected_all_42_1337_5"] - ) - print(result) - assert "Modify: 2013-12-03" in result - node.exec_in_container( - [ - "mv", - f"{path_to_detached}../unexpected_all_42_1337_5", - f"{path_to_detached}unexpected_all_42_1337_5", - ] - ) - - result = node.query( - f"CHECK TABLE {table}", - settings={"check_query_single_value_result": 0, "max_threads": 1}, - ) - assert "all_3_3_0\t0" in result - - node.query(f"DETACH TABLE {table}") - node.query(f"ATTACH TABLE {table}") - - result = node.exec_in_container(["ls", path_to_detached]) - print(result) - assert f"{expect_broken_prefix}_all_3_3_0" in result - assert "all_1_1_0" in result - assert "trash" in result - assert "broken_all_fake" in result - assert "unexpected_all_42_1337_5" in result - - time.sleep(15) - assert node.contains_in_log( - "Removed broken detached part unexpected_all_42_1337_5 due to a timeout" - ) - - result = node.exec_in_container(["ls", path_to_detached]) - print(result) - assert f"{expect_broken_prefix}_all_3_3_0" not in result - assert "all_1_1_0" in result - assert "trash" in result - assert "broken_all_fake" in result - assert "unexpected_all_42_1337_5" not in result - - node.query(f"DROP TABLE {table} SYNC") - - -def test_remove_broken_detached_part_merge_tree(started_cluster): - node1.query( - """ - CREATE TABLE - mt(id UInt32, value Int32) - ENGINE = MergeTree() ORDER BY id - SETTINGS - merge_tree_enable_clear_old_broken_detached=1, - merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds=5; - """ - ) - - for i in range(4): - node1.query( - f"INSERT INTO mt SELECT number, number * number FROM numbers ({i * 100000}, 100000)" - ) - - remove_broken_detached_part_impl("mt", node1, "broken-on-start") - - -def test_remove_broken_detached_part_replicated_merge_tree(started_cluster): - node1.query( - f""" - CREATE TABLE - replicated_mt(date Date, id UInt32, value Int32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt', '{node1.name}') ORDER BY id - SETTINGS - merge_tree_enable_clear_old_broken_detached=1, - merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds=5, - cleanup_delay_period=1, - cleanup_delay_period_random_add=0, - cleanup_thread_preferred_points_per_iteration=0; - """ - ) - - for i in range(4): - node1.query( - f"INSERT INTO replicated_mt SELECT toDate('2019-10-01'), number, number * number FROM numbers ({i * 100000}, 100000)" - ) - - remove_broken_detached_part_impl("replicated_mt", node1, "broken") - - ->>>>>>> fed4cb07fc8 (Deterministic result for CHECK TABLE in tests):tests/integration/test_broken_detached_part_clean_up/test.py def test_store_cleanup(started_cluster): node1.query("CREATE DATABASE db UUID '10000000-1000-4000-8000-000000000001'") node1.query( From 1e068640c8f9912017612b9f40bacbb9029c4546 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 23 Oct 2023 12:21:30 +0000 Subject: [PATCH 514/634] Add CHECK TABLE PART to docs --- .../sql-reference/statements/check-table.md | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index c24fb83016f..25158f8180d 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -19,11 +19,12 @@ Consider the potential impact on performance and resource utilization before exe The basic syntax of the query is as follows: ```sql -CHECK TABLE table_name [PARTITION partition_expression] [FORMAT format] [SETTINGS check_query_single_value_result = (0|1) [, other_settings] ] +CHECK TABLE table_name [PARTITION partition_expression | PART part_name] [FORMAT format] [SETTINGS check_query_single_value_result = (0|1) [, other_settings]] ``` - `table_name`: Specifies the name of the table that you want to check. - `partition_expression`: (Optional) If you want to check a specific partition of the table, you can use this expression to specify the partition. +- `part_name`: (Optional) If you want to check a specific part in the table, you can add string literal to specify a part name. - `FORMAT format`: (Optional) Allows you to specify the output format of the result. - `SETTINGS`: (Optional) Allows additional settings. - **`check_query_single_value_result`**: (Optional) This setting allows you to toggle between a detailed result (`0`) or a summarized result (`1`). @@ -82,6 +83,32 @@ Output: └──────────────┴───────────┴─────────┘ ``` +Similarly, you can check a specific part of the table by using the `PART` keyword. + +```sql +CHECK TABLE t0 PART '201003_7_7_0' +FORMAT PrettyCompactMonoBlock +SETTINGS check_query_single_value_result = 0 +``` + +Output: + +```text +┌─part_path────┬─is_passed─┬─message─┐ +│ 201003_7_7_0 │ 1 │ │ +└──────────────┴───────────┴─────────┘ +``` + +Note that when part does not exist, the query returns an error: + +```sql +CHECK TABLE t0 PART '201003_111_222_0' +``` + +```text +DB::Exception: No such data part '201003_111_222_0' to check in table 'default.t0'. (NO_SUCH_DATA_PART) +``` + ### Receiving a 'Corrupted' Result :::warning From 643fef589d0b1fce9f5312409bebe5b0a0db0117 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 23 Oct 2023 12:25:34 +0000 Subject: [PATCH 515/634] Revert "[remove before merging] Shuffle CHECK TABLE results" This reverts commit 8bb2378952792e3a08b67ca096e815fda5682b6f. --- src/Interpreters/InterpreterCheckQuery.cpp | 51 ---------------------- 1 file changed, 51 deletions(-) diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 1057f8a13a8..de5e5c2bf2b 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include @@ -141,44 +140,6 @@ private: std::atomic_bool is_value_emitted{false}; }; - -class RandomShuffleTransform : public IAccumulatingTransform -{ -public: - RandomShuffleTransform(const Block & header) - : IAccumulatingTransform(header, header) - {} - - String getName() const override { return "TableCheckResultEmitter"; } - - void consume(Chunk chunk) override - { - chunks.emplace_back(std::move(chunk)); - } - - Chunk generate() override - { - if (!is_shuffled.exchange(true)) - { - std::random_device rd; - std::mt19937 gen(rd()); - std::shuffle(chunks.begin(), chunks.end(), gen); - } - - if (chunks.empty()) - return {}; - - auto chunk = std::move(chunks.back()); - chunks.pop_back(); - return chunk; - } - -private: - Chunks chunks; - std::atomic_bool is_shuffled{false}; -}; - - } InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) @@ -229,18 +190,6 @@ BlockIO InterpreterCheckQuery::execute() resize_outport = &resize_processor->getOutputs().front(); } - /// TODO: for tesing only, remove after CI passed once - if (num_streams > 1) - { - auto shuffle_processor = std::make_shared(resize_outport->getHeader()); - auto * input_port = &shuffle_processor->getInputPort(); - connect(*resize_outport, *input_port); - processors->emplace_back(shuffle_processor); - - assert(resize_processor->getOutputs().size() == 1); - resize_outport = &shuffle_processor->getOutputs().front(); - } - if (settings.check_query_single_value_result) { auto emitter_processor = std::make_shared(); From cf44a53760a88f97f2d92b0d4c6bd41e5310e316 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 23 Oct 2023 15:13:30 +0200 Subject: [PATCH 516/634] upd reference --- .github/ISSUE_TEMPLATE/85_bug-report.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/ISSUE_TEMPLATE/85_bug-report.md b/.github/ISSUE_TEMPLATE/85_bug-report.md index 461a14867b4..93b2342af70 100644 --- a/.github/ISSUE_TEMPLATE/85_bug-report.md +++ b/.github/ISSUE_TEMPLATE/85_bug-report.md @@ -7,7 +7,7 @@ assignees: '' --- -> Please make sure that the version you're using is still supported. The rules described [here](https://clickhouse.com/docs/en/faq/operations/production#how-to-choose-between-clickhouse-releases). +> Please make sure that the version you're using is still supported (you can find the list [here](https://github.com/ClickHouse/ClickHouse/blob/master/SECURITY.md#scope-and-supported-versions)). > You have to provide the following information whenever possible. From 38a4547c05d25b7ae2e0da267e5d17eb5c18b4a4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 23 Oct 2023 13:36:33 +0200 Subject: [PATCH 517/634] Do not write retriable errors for Replicated mutate/merge into error log Fixes: e3f892f84a1 ("fix gtest with MemoryWriteBuffer, do not mute exception in ReplicatedMergeMutateTaskBase") Signed-off-by: Azat Khuzhin --- .../MergeTree/ReplicatedMergeMutateTaskBase.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 6ad77119016..32783004772 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -69,11 +69,11 @@ bool ReplicatedMergeMutateTaskBase::executeStep() else tryLogCurrentException(log, __PRETTY_FUNCTION__); - /** This exception will be written to the queue element, and it can be looked up using `system.replication_queue` table. - * The thread that performs this action will sleep a few seconds after the exception. - * See `queue.processEntry` function. - */ - throw; + /// This exception will be written to the queue element, and it can be looked up using `system.replication_queue` table. + /// The thread that performs this action will sleep a few seconds after the exception. + /// See `queue.processEntry` function. + if (!retryable_error) + throw; } catch (...) { From bd26f7096a4a3325f7a363c4be919700cdf10ca3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 23 Oct 2023 13:45:14 +0200 Subject: [PATCH 518/634] Add a test for retrible errors (i.e. "No active replica has part {} or covering part") They should not appears in the error log, only with Information level. Signed-off-by: Azat Khuzhin --- ...03_rmt_retriable_merge_exception.reference | 1 + .../02903_rmt_retriable_merge_exception.sql | 32 +++++++++++++++++++ 2 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/02903_rmt_retriable_merge_exception.reference create mode 100644 tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sql diff --git a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.reference b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.reference new file mode 100644 index 00000000000..6f9488be24d --- /dev/null +++ b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.reference @@ -0,0 +1 @@ +Information 1 diff --git a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sql b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sql new file mode 100644 index 00000000000..6d8a640e9ed --- /dev/null +++ b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sql @@ -0,0 +1,32 @@ +-- Test that retriable errors during merges/mutations +-- (i.e. "No active replica has part X or covering part") +-- does not appears as errors (level=Error), only as info message (level=Information). + +drop table if exists rmt1; +drop table if exists rmt2; + +create table rmt1 (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', '1') order by key settings always_fetch_merged_part=1; +create table rmt2 (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', '2') order by key settings always_fetch_merged_part=0; + +insert into rmt1 values (1); +insert into rmt1 values (2); + +system stop pulling replication log rmt2; +optimize table rmt1 final settings alter_sync=0; + +select sleep(3) format Null; +system start pulling replication log rmt2; + +system flush logs; +with + (select uuid from system.tables where database = currentDatabase() and table = 'rmt1') as uuid_ +select + level, count() > 0 +from system.text_log +where + event_date >= yesterday() and event_time >= now() - 60 and + ( + (logger_name = 'MergeTreeBackgroundExecutor' and message like '%{' || uuid_::String || '::all_0_1_1}%No active replica has part all_0_1_1 or covering part%') or + (logger_name = uuid_::String || '::all_0_1_1 (MergeFromLogEntryTask)' and message like '%No active replica has part all_0_1_1 or covering part%') + ) +group by level; From 601276aa605e01bf674f4696e127a0f3d0b6b03f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 23 Oct 2023 15:23:23 +0200 Subject: [PATCH 519/634] Update src/Storages/MergeTree/RPNBuilder.cpp Co-authored-by: Igor Nikonov <954088+devcrafter@users.noreply.github.com> --- src/Storages/MergeTree/RPNBuilder.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index c292505bcd8..f2dcb77ee3b 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -396,7 +396,7 @@ size_t RPNBuilderFunctionTreeNode::getArgumentsSize() const RPNBuilderTreeNode RPNBuilderFunctionTreeNode::getArgumentAt(size_t index) const { - size_t total_arguments = getArgumentsSize(); + const size_t total_arguments = getArgumentsSize(); if (index >= total_arguments) /// Bug #52632 throw Exception(ErrorCodes::LOGICAL_ERROR, "RPNBuilderFunctionTreeNode has {} arguments, attempted to get argument at index {}", From bb856680e782477693fb1347655e3d8cc168dd94 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 23 Oct 2023 14:07:14 +0000 Subject: [PATCH 520/634] Incorporated review feedback --- .../functions/date-time-functions.md | 4 ++-- .../FunctionDateOrDateTimeAddInterval.h | 23 ++++--------------- 2 files changed, 7 insertions(+), 20 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index b7e63f14f10..f65354921e2 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -2393,7 +2393,7 @@ Accepts an additional, optional `precision` parameter after the `timezone` param ## addYears, addQuarters, addMonths, addWeeks, addDays, addHours, addMinutes, addSeconds, addMilliseconds, addMicroseconds, addNanoseconds -These functions add units of the interval specified by the function name to a date, a date with time or a date or date with time encoded as string. A date or date with time is returned. +These functions add units of the interval specified by the function name to a date, a date with time or a string-encoded date / date with time. A date or date with time is returned. Example: @@ -2416,7 +2416,7 @@ SELECT ## subtractYears, subtractQuarters, subtractMonths, subtractWeeks, subtractDays, subtractHours, subtractMinutes, subtractSeconds, subtractMilliseconds, subtractMicroseconds, subtractNanoseconds -These functions subtract units of the interval specified by the function name from a date, a date with time or a date or date with time encoded as string. A date or date with time is returned. +These functions subtract units of the interval specified by the function name from a date, a date with time or a string-encoded date / date with time. A date or date with time is returned. Example: diff --git a/src/Functions/FunctionDateOrDateTimeAddInterval.h b/src/Functions/FunctionDateOrDateTimeAddInterval.h index 2ba53172178..b8c0d27c42e 100644 --- a/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -528,26 +528,13 @@ private: { static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); - if constexpr (std::is_same_v) - { - auto & vec_to = col_to.getData(); + auto & vec_to = col_to.getData(); - size_t size = delta.size(); - vec_to.resize(size); + size_t size = delta.size(); + vec_to.resize(size); - for (size_t i = 0; i < size; ++i) - vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone, utc_time_zone, scale); - } - else - { - auto & vec_to = col_to.getData(); - - size_t size = delta.size(); - vec_to.resize(size); - - for (size_t i = 0; i < size; ++i) - vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone, utc_time_zone, scale); - } + for (size_t i = 0; i < size; ++i) + vec_to[i] = transform.execute(from, checkOverflow(delta.getData()[i]), time_zone, utc_time_zone, scale); } }; From 0cd79cb7c01720b2abfd79157c0feca5f696059f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 23 Oct 2023 16:11:16 +0200 Subject: [PATCH 521/634] Prevent excesive memory usage when deserializing AggregateFunctionTopKGenericData --- src/AggregateFunctions/AggregateFunctionTopK.cpp | 12 ++++++------ src/AggregateFunctions/AggregateFunctionTopK.h | 15 ++++++++++++++- ...2_topKGeneric_deserialization_memory.reference | 0 .../02902_topKGeneric_deserialization_memory.sql | 7 +++++++ 4 files changed, 27 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02902_topKGeneric_deserialization_memory.reference create mode 100644 tests/queries/0_stateless/02902_topKGeneric_deserialization_memory.sql diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index 8f6652223cc..f7b3524d1b9 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -8,9 +8,6 @@ #include -static inline constexpr UInt64 TOP_K_MAX_SIZE = 0xFFFFFF; - - namespace DB { @@ -134,9 +131,12 @@ AggregateFunctionPtr createAggregateFunctionTopK(const std::string & name, const threshold = applyVisitor(FieldVisitorConvertToNumber(), params[0]); - if (threshold > TOP_K_MAX_SIZE || load_factor > TOP_K_MAX_SIZE || threshold * load_factor > TOP_K_MAX_SIZE) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, - "Too large parameter(s) for aggregate function '{}' (maximum is {})", name, toString(TOP_K_MAX_SIZE)); + if (threshold > DB::TOP_K_MAX_SIZE || load_factor > DB::TOP_K_MAX_SIZE || threshold * load_factor > DB::TOP_K_MAX_SIZE) + throw Exception( + ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Too large parameter(s) for aggregate function '{}' (maximum is {})", + name, + toString(DB::TOP_K_MAX_SIZE)); if (threshold == 0) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Parameter 0 is illegal for aggregate function '{}'", name); diff --git a/src/AggregateFunctions/AggregateFunctionTopK.h b/src/AggregateFunctions/AggregateFunctionTopK.h index 89985c0ea6b..89c49b24530 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.h +++ b/src/AggregateFunctions/AggregateFunctionTopK.h @@ -20,6 +20,12 @@ namespace DB { struct Settings; +static inline constexpr UInt64 TOP_K_MAX_SIZE = 0xFFFFFF; + +namespace ErrorCodes +{ + extern const int ARGUMENT_OUT_OF_BOUND; +} template struct AggregateFunctionTopKData @@ -163,11 +169,18 @@ public: { auto & set = this->data(place).value; set.clear(); - set.resize(reserved); // Specialized here because there's no deserialiser for StringRef size_t size = 0; readVarUInt(size, buf); + if (unlikely(size > TOP_K_MAX_SIZE)) + throw Exception( + ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Too large size ({}) for aggregate function '{}' state (maximum is {})", + size, + getName(), + TOP_K_MAX_SIZE); + set.resize(size); for (size_t i = 0; i < size; ++i) { auto ref = readStringBinaryInto(*arena, buf); diff --git a/tests/queries/0_stateless/02902_topKGeneric_deserialization_memory.reference b/tests/queries/0_stateless/02902_topKGeneric_deserialization_memory.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02902_topKGeneric_deserialization_memory.sql b/tests/queries/0_stateless/02902_topKGeneric_deserialization_memory.sql new file mode 100644 index 00000000000..2427edb5416 --- /dev/null +++ b/tests/queries/0_stateless/02902_topKGeneric_deserialization_memory.sql @@ -0,0 +1,7 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/49706 +-- Using format Parquet for convenience so it errors out without output (but still deserializes the output) +-- Without the fix this would OOM the client when deserializing the state +SELECT + topKResampleState(1048576, 257, 65536, 10)(toString(number), number) +FROM numbers(3) +FORMAT Parquet; -- { clientError UNKNOWN_TYPE } From ef071ac3f87f879fc506bac919b0f2b789394e6c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 23 Oct 2023 16:50:23 +0200 Subject: [PATCH 522/634] Fast tests don't build formats --- .../0_stateless/02902_topKGeneric_deserialization_memory.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02902_topKGeneric_deserialization_memory.sql b/tests/queries/0_stateless/02902_topKGeneric_deserialization_memory.sql index 2427edb5416..3228810e0ba 100644 --- a/tests/queries/0_stateless/02902_topKGeneric_deserialization_memory.sql +++ b/tests/queries/0_stateless/02902_topKGeneric_deserialization_memory.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + -- https://github.com/ClickHouse/ClickHouse/issues/49706 -- Using format Parquet for convenience so it errors out without output (but still deserializes the output) -- Without the fix this would OOM the client when deserializing the state From b9b66e76ddd33f45359b661d19d6b79801d6f7cc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 23 Oct 2023 11:06:34 +0000 Subject: [PATCH 523/634] Switch accumulator and array arguments --- .../functions/array-functions.md | 11 ++++- src/Functions/array/arrayFold.cpp | 46 +++++++++---------- .../queries/0_stateless/02718_array_fold.sql | 33 ++++++------- 3 files changed, 49 insertions(+), 41 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 6e460a64bcf..40bfb65e4e8 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1081,6 +1081,10 @@ Result: └─────────────────────────────────────────────────────────────┘ ``` +**See also** + +- [arrayFold](#arrayFold) + ## arrayReduceInRanges Applies an aggregate function to array elements in given ranges and returns an array containing the result corresponding to each range. The function will return the same result as multiple `arrayReduce(agg_func, arraySlice(arr1, index, length), ...)`. @@ -1138,7 +1142,7 @@ arrayFold(lambda_function, arr1, arr2, ..., accumulator) Query: ``` sql -SELECT arrayFold( x,acc -> acc + x*2, [1, 2, 3, 4], toInt64(3)) AS res; +SELECT arrayFold( acc,x -> acc + x*2, [1, 2, 3, 4], toInt64(3)) AS res; ``` Result: @@ -1152,7 +1156,7 @@ Result: **Example with the Fibonacci sequence** ```sql -SELECT arrayFold( x, acc -> (acc.2, acc.2 + acc.1), range(number), (1::Int64, 0::Int64)).1 AS fibonacci +SELECT arrayFold( acc,x -> (acc.2, acc.2 + acc.1), range(number), (1::Int64, 0::Int64)).1 AS fibonacci FROM numbers(1,10); ┌─fibonacci─┐ @@ -1169,6 +1173,9 @@ FROM numbers(1,10); └───────────┘ ``` +**See also** + +- [arrayReduce](#arrayReduce) ## arrayReverse(arr) diff --git a/src/Functions/array/arrayFold.cpp b/src/Functions/array/arrayFold.cpp index 94ed5d59ca9..b5b650e7289 100644 --- a/src/Functions/array/arrayFold.cpp +++ b/src/Functions/array/arrayFold.cpp @@ -30,37 +30,37 @@ public: void getLambdaArgumentTypes(DataTypes & arguments) const override { if (arguments.size() < 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires as arguments a lambda function, at least one array and an accumulator argument", getName()); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName()); - DataTypes nested_types(arguments.size() - 1); - for (size_t i = 0; i < nested_types.size() - 1; ++i) + DataTypes accumulator_and_array_types(arguments.size() - 1); + accumulator_and_array_types[0] = arguments.back(); + for (size_t i = 1; i < accumulator_and_array_types.size(); ++i) { - const auto * array_type = checkAndGetDataType(&*arguments[i + 1]); + const auto * array_type = checkAndGetDataType(&*arguments[i]); if (!array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument {} of function {} must be array, found {} instead", i + 2, getName(), arguments[i + 1]->getName()); - nested_types[i] = recursiveRemoveLowCardinality(array_type->getNestedType()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument {} of function {} must be of type Array, found {} instead", i + 1, getName(), arguments[i]->getName()); + accumulator_and_array_types[i] = recursiveRemoveLowCardinality(array_type->getNestedType()); } - nested_types[nested_types.size() - 1] = arguments[arguments.size() - 1]; - const auto * function_type = checkAndGetDataType(arguments[0].get()); - if (!function_type || function_type->getArgumentTypes().size() != nested_types.size()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for this overload of {} must be a function with {} arguments, found {} instead.", - getName(), nested_types.size(), arguments[0]->getName()); + const auto * lambda_function_type = checkAndGetDataType(arguments[0].get()); + if (!lambda_function_type || lambda_function_type->getArgumentTypes().size() != accumulator_and_array_types.size()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument of function {} must be a lambda function with {} arguments, found {} instead.", + getName(), accumulator_and_array_types.size(), arguments[0]->getName()); - arguments[0] = std::make_shared(nested_types); + arguments[0] = std::make_shared(accumulator_and_array_types); } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() < 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least 2 arguments, passed: {}.", getName(), arguments.size()); + if (arguments.size() < 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName()); - const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); - if (!data_type_function) + const auto * lambda_function_type = checkAndGetDataType(arguments[0].type.get()); + if (!lambda_function_type) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function", getName()); auto accumulator_type = arguments.back().type; - auto lambda_type = data_type_function->getReturnType(); + auto lambda_type = lambda_function_type->getReturnType(); if (!accumulator_type->equals(*lambda_type)) throw Exception(ErrorCodes::TYPE_MISMATCH, "Return type of lambda function must be the same as the accumulator type, inferred return type of lambda: {}, inferred type of accumulator: {}", @@ -71,12 +71,12 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - const auto & lambda_with_type_and_name = arguments[0]; + const auto & lambda_function_with_type_and_name = arguments[0]; - if (!lambda_with_type_and_name.column) + if (!lambda_function_with_type_and_name.column) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function", getName()); - const auto * lambda_function = typeid_cast(lambda_with_type_and_name.column.get()); + const auto * lambda_function = typeid_cast(lambda_function_with_type_and_name.column.get()); if (!lambda_function) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function", getName()); @@ -85,6 +85,7 @@ public: const ColumnArray * column_first_array = nullptr; ColumnsWithTypeAndName arrays; arrays.reserve(arguments.size() - 1); + /// Validate input types and get input array columns in convenient form for (size_t i = 1; i < arguments.size() - 1; ++i) { @@ -131,8 +132,7 @@ public: if (rows_count == 0) return arguments.back().column->convertToFullColumnIfConst()->cloneEmpty(); - ColumnPtr current_column; - current_column = arguments.back().column->convertToFullColumnIfConst(); + ColumnPtr current_column = arguments.back().column->convertToFullColumnIfConst(); MutableColumnPtr result_data = arguments.back().column->convertToFullColumnIfConst()->cloneEmpty(); size_t max_array_size = 0; @@ -198,9 +198,9 @@ public: auto res_lambda = lambda_function->cloneResized(prev[1]->size()); auto * res_lambda_ptr = typeid_cast(res_lambda.get()); + res_lambda_ptr->appendArguments(std::vector({ColumnWithTypeAndName(std::move(prev[1]), arguments.back().type, arguments.back().name)})); for (size_t i = 0; i < array_count; i++) res_lambda_ptr->appendArguments(std::vector({ColumnWithTypeAndName(std::move(data_arrays[i][ind]), arrays[i].type, arrays[i].name)})); - res_lambda_ptr->appendArguments(std::vector({ColumnWithTypeAndName(std::move(prev[1]), arguments.back().type, arguments.back().name)})); current_column = IColumn::mutate(res_lambda_ptr->reduce().column); prev_size = current_column->size(); diff --git a/tests/queries/0_stateless/02718_array_fold.sql b/tests/queries/0_stateless/02718_array_fold.sql index 7f20602a371..0486a5ce2e3 100644 --- a/tests/queries/0_stateless/02718_array_fold.sql +++ b/tests/queries/0_stateless/02718_array_fold.sql @@ -1,23 +1,24 @@ SELECT 'Negative tests'; SELECT arrayFold(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT arrayFold(1); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT arrayFold(1, toUInt64(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayFold( x,acc -> x, emptyArrayString(), toInt8(0)); -- { serverError TYPE_MISMATCH } -SELECT arrayFold( x,acc -> x, 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayFold( x,y,acc -> x, [0, 1], 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayFold( x,acc -> x, [0, 1], [2, 3], toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayFold( x,y,acc -> x, [0, 1], [2, 3, 4], toUInt8(0)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } +SELECT arrayFold(1, toUInt64(0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT arrayFold(1, emptyArrayUInt64(), toUInt64(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayFold( acc,x -> x, emptyArrayString(), toInt8(0)); -- { serverError TYPE_MISMATCH } +SELECT arrayFold( acc,x -> x, 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayFold( acc,x,y -> x, [0, 1], 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayFold( acc,x -> x, [0, 1], [2, 3], toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayFold( acc,x,y -> x, [0, 1], [2, 3, 4], toUInt8(0)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } SELECT 'Const arrays'; -SELECT arrayFold( x,acc -> acc+x*2, [1, 2, 3, 4], toInt64(3)); -SELECT arrayFold( x,acc -> acc+x*2, emptyArrayInt64(), toInt64(3)); -SELECT arrayFold( x,y,acc -> acc+x*2+y*3, [1, 2, 3, 4], [5, 6, 7, 8], toInt64(3)); -SELECT arrayFold( x,acc -> arrayPushBack(acc, x), [1, 2, 3, 4], emptyArrayInt64()); -SELECT arrayFold( x,acc -> arrayPushFront(acc, x), [1, 2, 3, 4], emptyArrayInt64()); -SELECT arrayFold( x,acc -> (arrayPushFront(acc.1, x),arrayPushBack(acc.2, x)), [1, 2, 3, 4], (emptyArrayInt64(), emptyArrayInt64())); -SELECT arrayFold( x,acc -> x%2 ? (arrayPushBack(acc.1, x), acc.2): (acc.1, arrayPushBack(acc.2, x)), [1, 2, 3, 4, 5, 6], (emptyArrayInt64(), emptyArrayInt64())); +SELECT arrayFold( acc,x -> acc+x*2, [1, 2, 3, 4], toInt64(3)); +SELECT arrayFold( acc,x -> acc+x*2, emptyArrayInt64(), toInt64(3)); +SELECT arrayFold( acc,x,y -> acc+x*2+y*3, [1, 2, 3, 4], [5, 6, 7, 8], toInt64(3)); +SELECT arrayFold( acc,x -> arrayPushBack(acc, x), [1, 2, 3, 4], emptyArrayInt64()); +SELECT arrayFold( acc,x -> arrayPushFront(acc, x), [1, 2, 3, 4], emptyArrayInt64()); +SELECT arrayFold( acc,x -> (arrayPushFront(acc.1, x),arrayPushBack(acc.2, x)), [1, 2, 3, 4], (emptyArrayInt64(), emptyArrayInt64())); +SELECT arrayFold( acc,x -> x%2 ? (arrayPushBack(acc.1, x), acc.2): (acc.1, arrayPushBack(acc.2, x)), [1, 2, 3, 4, 5, 6], (emptyArrayInt64(), emptyArrayInt64())); SELECT 'Non-const arrays'; -SELECT arrayFold( x,acc -> acc+x, range(number), number) FROM system.numbers LIMIT 5; -SELECT arrayFold( x,acc -> arrayPushFront(acc,x), range(number), emptyArrayUInt64()) FROM system.numbers LIMIT 5; -SELECT arrayFold( x,acc -> x%2 ? arrayPushFront(acc,x) : arrayPushBack(acc,x), range(number), emptyArrayUInt64()) FROM system.numbers LIMIT 5; +SELECT arrayFold( acc,x -> acc+x, range(number), number) FROM system.numbers LIMIT 5; +SELECT arrayFold( acc,x -> arrayPushFront(acc,x), range(number), emptyArrayUInt64()) FROM system.numbers LIMIT 5; +SELECT arrayFold( acc,x -> x%2 ? arrayPushFront(acc,x) : arrayPushBack(acc,x), range(number), emptyArrayUInt64()) FROM system.numbers LIMIT 5; From 6167bf4fcabe880c37259be71e6e8a228700d9cf Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 23 Oct 2023 17:27:38 +0200 Subject: [PATCH 524/634] Update stress.py --- tests/ci/stress.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 9db9f25cbca..446350863aa 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -109,9 +109,11 @@ def compress_stress_logs(output_path: Path, files_prefix: str) -> None: def call_with_retry(query: str, timeout: int = 30, retry_count: int = 5) -> None: + logging.info("Running command: %s", str(query)) for i in range(retry_count): code = call(query, shell=True, stderr=STDOUT, timeout=timeout) if code != 0: + logging.info("Command returend %s, retrying", str(code)) time.sleep(i) else: break @@ -132,6 +134,7 @@ def prepare_for_hung_check(drop_databases: bool) -> bool: # However, it obstruct checking for hung queries. logging.info("Will terminate gdb (if any)") call_with_retry("kill -TERM $(pidof gdb)") + call_with_retry("tail --pid=$(pidof gdb) -f /dev/null") # Sometimes there is a message `Child process was stopped by signal 19` in logs after stopping gdb call_with_retry( "kill -CONT $(cat /var/run/clickhouse-server/clickhouse-server.pid)" From ccd4f221d4b00ba7787faa5402f55a4e284eace5 Mon Sep 17 00:00:00 2001 From: wxybear Date: Mon, 23 Oct 2023 23:46:34 +0800 Subject: [PATCH 525/634] feat: add queries for StorageNull with subcolumns --- .../02902_select_subcolumns_from_engine_null.reference | 0 .../0_stateless/02902_select_subcolumns_from_engine_null.sql | 0 2 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/queries/0_stateless/02902_select_subcolumns_from_engine_null.reference create mode 100644 tests/queries/0_stateless/02902_select_subcolumns_from_engine_null.sql diff --git a/tests/queries/0_stateless/02902_select_subcolumns_from_engine_null.reference b/tests/queries/0_stateless/02902_select_subcolumns_from_engine_null.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02902_select_subcolumns_from_engine_null.sql b/tests/queries/0_stateless/02902_select_subcolumns_from_engine_null.sql new file mode 100644 index 00000000000..e69de29bb2d From 015d041dc3d80ca56f74885dd72c9fca308e0a5b Mon Sep 17 00:00:00 2001 From: FFish Date: Mon, 23 Oct 2023 23:51:39 +0800 Subject: [PATCH 526/634] Update 02902_select_subcolumns_from_engine_null.sql --- .../02902_select_subcolumns_from_engine_null.sql | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/02902_select_subcolumns_from_engine_null.sql b/tests/queries/0_stateless/02902_select_subcolumns_from_engine_null.sql index e69de29bb2d..4cd6e68fdaf 100644 --- a/tests/queries/0_stateless/02902_select_subcolumns_from_engine_null.sql +++ b/tests/queries/0_stateless/02902_select_subcolumns_from_engine_null.sql @@ -0,0 +1,6 @@ +CREATE TABLE null_02902 (t Tuple(num Int64, str String)) ENGINE = Null; +SELECT t FROM null_02902; +SELECT tupleElement(t, 'num') FROM null_02902; +SELECT t.num, t.str FROM null_02902; + +DROP TABLE null_02902; From 959b8b64bdedc7c1cc626c669b5fc55023cd3043 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 23 Oct 2023 19:59:17 +0200 Subject: [PATCH 527/634] Added a setting to allow reading rows marked as deleted --- src/Core/Settings.h | 1 + .../QueryPlan/ReadFromMergeTree.cpp | 1 + .../02902_diable_apply_deleted_mask.reference | 16 ++++++++++++ .../02902_diable_apply_deleted_mask.sql | 25 +++++++++++++++++++ 4 files changed, 43 insertions(+) create mode 100644 tests/queries/0_stateless/02902_diable_apply_deleted_mask.reference create mode 100644 tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b58705614d8..bb28e246a17 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -551,6 +551,7 @@ class IColumn; M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \ + M(Bool, apply_deleted_mask, true, "Enables filtering out rows deleted with lightweight DELETE. If disabled, a query will be able to read those rows. This is useful for debugging and \"undelete\" scenarious", 0) \ M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 05f39f72880..80fcc317d61 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -115,6 +115,7 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings( .save_marks_in_cache = true, .checksum_on_read = settings.checksum_on_read, .read_in_order = query_info.input_order_info != nullptr, + .apply_deleted_mask = settings.apply_deleted_mask, .use_asynchronous_read_from_pool = settings.allow_asynchronous_read_from_io_pool_for_merge_tree && (settings.max_streams_to_max_threads_ratio > 1 || settings.max_streams_for_merge_tree_reading > 1), .enable_multiple_prewhere_read_steps = settings.enable_multiple_prewhere_read_steps, diff --git a/tests/queries/0_stateless/02902_diable_apply_deleted_mask.reference b/tests/queries/0_stateless/02902_diable_apply_deleted_mask.reference new file mode 100644 index 00000000000..bb99676a056 --- /dev/null +++ b/tests/queries/0_stateless/02902_diable_apply_deleted_mask.reference @@ -0,0 +1,16 @@ +Normal SELECT does not see deleted rows +1 1 1 +3 3 1 +With the setting enabled the deleted rows are visible +0 0 0 +1 1 1 +2 2 0 +3 3 1 +4 4 0 +With the setting enabled the deleted rows are visible but still can be filterd out +1 1 +3 3 +Read the data after OPTIMIZE, all deleted rwos should be physically removed now +1 1 1 +3 3 1 +5 5 1 diff --git a/tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql b/tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql new file mode 100644 index 00000000000..4122871bf39 --- /dev/null +++ b/tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql @@ -0,0 +1,25 @@ +DROP TABLE IF EXISTS test_apply_deleted_mask; + +CREATE TABLE test_apply_deleted_mask(id Int64, value String) ENGINE = MergeTree ORDER BY id; + +INSERT INTO test_apply_deleted_mask SELECT number, number::String FROM numbers(5); + +DELETE FROM test_apply_deleted_mask WHERE id % 2 = 0; + +SELECT 'Normal SELECT does not see deleted rows'; +SELECT *, _row_exists FROM test_apply_deleted_mask; + +SELECT 'With the setting enabled the deleted rows are visible'; +SELECT *, _row_exists FROM test_apply_deleted_mask SETTINGS apply_deleted_mask = 0; + +SELECT 'With the setting enabled the deleted rows are visible but still can be filterd out'; +SELECT * FROM test_apply_deleted_mask WHERE _row_exists SETTINGS apply_deleted_mask = 0; + +INSERT INTO test_apply_deleted_mask SELECT number, number::String FROM numbers(5, 1); + +OPTIMIZE TABLE test_apply_deleted_mask FINAL SETTINGS mutations_sync=2; + +SELECT 'Read the data after OPTIMIZE, all deleted rwos should be physically removed now'; +SELECT *, _row_exists FROM test_apply_deleted_mask SETTINGS apply_deleted_mask = 0; + +DROP TABLE test_apply_deleted_mask; \ No newline at end of file From 3320b12e592ad7186bea8e415c16d6232ca5a4df Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 23 Oct 2023 20:50:40 +0200 Subject: [PATCH 528/634] impl (#55787) --- .gitmodules | 2 +- contrib/datasketches-cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index f790e0f8d5a..904d2cec249 100644 --- a/.gitmodules +++ b/.gitmodules @@ -184,7 +184,7 @@ url = https://github.com/ClickHouse/nanodbc [submodule "contrib/datasketches-cpp"] path = contrib/datasketches-cpp - url = https://github.com/ClickHouse/datasketches-cpp + url = https://github.com/apache/datasketches-cpp [submodule "contrib/yaml-cpp"] path = contrib/yaml-cpp url = https://github.com/ClickHouse/yaml-cpp diff --git a/contrib/datasketches-cpp b/contrib/datasketches-cpp index 7abd49bb2e7..c3abaaefe5f 160000 --- a/contrib/datasketches-cpp +++ b/contrib/datasketches-cpp @@ -1 +1 @@ -Subproject commit 7abd49bb2e72bf9a5029993d31dcb1872da88292 +Subproject commit c3abaaefe5fa400eed99e082af07c1b61a7144db From db1cca592e6af878503fd80b4f6ee03f3e4c99e6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 23 Oct 2023 20:47:58 +0000 Subject: [PATCH 529/634] Test for Bug 43644 --- src/Interpreters/SystemLog.h | 2 +- .../0_stateless/02903_bug_43644.reference | 1 + tests/queries/0_stateless/02903_bug_43644.sql | 22 +++++++++++++++++++ 3 files changed, 24 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02903_bug_43644.reference create mode 100644 tests/queries/0_stateless/02903_bug_43644.sql diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 932afec3bad..b27a8a48651 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -139,7 +139,7 @@ private: /* Saving thread data */ const StorageID table_id; const String storage_def; - String create_query; + const String create_query; String old_create_query; bool is_prepared = false; diff --git a/tests/queries/0_stateless/02903_bug_43644.reference b/tests/queries/0_stateless/02903_bug_43644.reference new file mode 100644 index 00000000000..2993076a861 --- /dev/null +++ b/tests/queries/0_stateless/02903_bug_43644.reference @@ -0,0 +1 @@ +2022-11-24 12:00:00 diff --git a/tests/queries/0_stateless/02903_bug_43644.sql b/tests/queries/0_stateless/02903_bug_43644.sql new file mode 100644 index 00000000000..c86988f8346 --- /dev/null +++ b/tests/queries/0_stateless/02903_bug_43644.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab +( + `machine_id` UInt64, + `name` String, + `timestamp` DateTime +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(timestamp) +ORDER BY machine_id; + +insert into tab(machine_id, name, timestamp) +select 1, 'a_name', '2022-11-24 12:00:00'; + +SELECT + toStartOfInterval(timestamp, INTERVAL 300 SECOND) AS ts +FROM tab +WHERE ts > '2022-11-24 11:19:00' +GROUP BY ts; + +DROP TABLE tab; From e3a63ad5278e15d8022f2ae80dabb30d90a83642 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 24 Oct 2023 10:44:38 +0800 Subject: [PATCH 530/634] Update src/Core/Settings.h Co-authored-by: vdimir --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 560245d4644..15af651d4b5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -546,7 +546,7 @@ class IColumn; M(Bool, database_atomic_wait_for_drop_and_detach_synchronously, false, "When executing DROP or DETACH TABLE in Atomic database, wait for table data to be finally dropped or detached.", 0) \ M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ - M(Bool, rocksdb_enable_approximate_count, true, "If `optimize_trivial_count_query` is true, process trivial 'SELECT count() FROM rockdb_table' from metadata, the returned result is approximated.", 0) \ + M(Bool, optimize_trivial_approximate_count_query, true, "Use an approximate value for trivial count optimization of storages that support such estimations.", 0) \ M(Bool, optimize_count_from_files, true, "Optimize counting rows from files in supported input formats", 0) \ M(Bool, use_cache_for_count_from_files, true, "Use cache to count the number of rows in files", 0) \ M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ From e3aa799ca26f87978e09d83fdf3b3a6c7d2063d7 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 24 Oct 2023 03:18:33 +0000 Subject: [PATCH 531/634] disable approximate count by default + fix tests Signed-off-by: Duc Canh Le --- src/Core/Settings.h | 2 +- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- .../queries/0_stateless/02892_rocksdb_trivial_count.reference | 1 - tests/queries/0_stateless/02892_rocksdb_trivial_count.sql | 4 ++-- 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 15af651d4b5..1e4e0afe507 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -546,7 +546,7 @@ class IColumn; M(Bool, database_atomic_wait_for_drop_and_detach_synchronously, false, "When executing DROP or DETACH TABLE in Atomic database, wait for table data to be finally dropped or detached.", 0) \ M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ - M(Bool, optimize_trivial_approximate_count_query, true, "Use an approximate value for trivial count optimization of storages that support such estimations.", 0) \ + M(Bool, optimize_trivial_approximate_count_query, false, "Use an approximate value for trivial count optimization of storages that support such estimations.", 0) \ M(Bool, optimize_count_from_files, true, "Optimize counting rows from files in supported input formats", 0) \ M(Bool, use_cache_for_count_from_files, true, "Use cache to count the number of rows in files", 0) \ M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index b08fd71d5ec..5e8d54bcdf1 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -611,7 +611,7 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory) std::optional StorageEmbeddedRocksDB::totalRows(const Settings & settings) const { - if (settings.rocksdb_enable_approximate_count) + if (settings.optimize_trivial_approximate_count_query) { std::shared_lock lock(rocksdb_ptr_mx); if (!rocksdb_ptr) diff --git a/tests/queries/0_stateless/02892_rocksdb_trivial_count.reference b/tests/queries/0_stateless/02892_rocksdb_trivial_count.reference index 75d516255f1..9289ddcee34 100644 --- a/tests/queries/0_stateless/02892_rocksdb_trivial_count.reference +++ b/tests/queries/0_stateless/02892_rocksdb_trivial_count.reference @@ -1,2 +1 @@ 121 -121 diff --git a/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql b/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql index 4fc1a681b26..6ced27f94d4 100644 --- a/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql +++ b/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql @@ -1,4 +1,4 @@ CREATE TABLE dict (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY key; INSERT INTO dict SELECT number, toString(number) FROM numbers(121); -SELECT count() FROM dict SETTINGS rocksdb_enable_approximate_count = 0; -SELECT count() FROM dict SETTINGS optimize_trivial_count_query = 1, rocksdb_enable_approximate_count = 1, max_rows_to_read = 1; +SELECT count() FROM dict SETTINGS optimize_trivial_approximate_count_query = 0, max_rows_to_read = 1; -- { serverError TOO_MANY_ROWS } +SELECT count() FROM dict SETTINGS optimize_trivial_approximate_count_query = 1, max_rows_to_read = 1; From a02c49e16fb0b2ba10b89e9acb0e3ed53c46c546 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 24 Oct 2023 11:47:46 +0800 Subject: [PATCH 532/634] allow skip null values when serailize tuple to json objects --- src/Core/Settings.h | 1 + .../Serializations/SerializationTuple.cpp | 26 ++++++++++--- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 2 +- .../02902_json_skip_null_values.sql | 38 +++++++++++++++++++ 5 files changed, 62 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02902_json_skip_null_values.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b58705614d8..3a85687cf0e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -982,6 +982,7 @@ class IColumn; \ M(Bool, output_format_json_escape_forward_slashes, true, "Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.", 0) \ M(Bool, output_format_json_named_tuples_as_objects, true, "Serialize named tuple columns as JSON objects.", 0) \ + M(Bool, output_format_json_skip_null_value_in_named_tuples, false, "Skip key value pairs with null value when serialize named tuple columns as JSON objects. It is only valid when output_format_json_named_tuples_as_objects is true.", 0) \ M(Bool, output_format_json_array_of_rows, false, "Output a JSON array of all rows in JSONEachRow(Compact) format.", 0) \ M(Bool, output_format_json_validate_utf8, false, "Validate UTF-8 sequences in JSON output formats, doesn't impact formats JSON/JSONCompact/JSONColumnsWithMetadata, they always validate utf8", 0) \ \ diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 5c9487b97d4..979010b1c07 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -163,16 +163,23 @@ void SerializationTuple::serializeTextJSON(const IColumn & column, size_t row_nu && have_explicit_names) { writeChar('{', ostr); + + bool first = true; for (size_t i = 0; i < elems.size(); ++i) { - if (i != 0) - { + if (!first) writeChar(',', ostr); - } + + const auto & element_column = extractElementColumn(column, i); + if (settings.json.skip_null_value_in_named_tuples && element_column.isNullAt(row_num)) + continue; + writeJSONString(elems[i]->getElementName(), ostr, settings); writeChar(':', ostr); - elems[i]->serializeTextJSON(extractElementColumn(column, i), row_num, ostr, settings); + elems[i]->serializeTextJSON(element_column, row_num, ostr, settings); + first = false; } + writeChar('}', ostr); } else @@ -194,15 +201,24 @@ void SerializationTuple::serializeTextJSONPretty(const IColumn & column, size_t && have_explicit_names) { writeCString("{\n", ostr); + + bool first = true; for (size_t i = 0; i < elems.size(); ++i) { - if (i != 0) + if (!first) writeCString(",\n", ostr); + + const auto & element_column = extractElementColumn(column, i); + if (settings.json.skip_null_value_in_named_tuples && element_column.isNullAt(row_num)) + continue; + writeChar(' ', (indent + 1) * 4, ostr); writeJSONString(elems[i]->getElementName(), ostr, settings); writeCString(": ", ostr); elems[i]->serializeTextJSONPretty(extractElementColumn(column, i), row_num, ostr, settings, indent + 1); + first = false; } + writeChar('\n', ostr); writeChar(' ', indent * 4, ostr); writeChar('}', ostr); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 2713b7cb35f..6efab3aefd7 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -100,6 +100,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.json.array_of_rows = settings.output_format_json_array_of_rows; format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes; format_settings.json.write_named_tuples_as_objects = settings.output_format_json_named_tuples_as_objects; + format_settings.json.skip_null_value_in_named_tuples = settings.output_format_json_skip_null_value_in_named_tuples; format_settings.json.read_named_tuples_as_objects = settings.input_format_json_named_tuples_as_objects; format_settings.json.defaults_for_missing_elements_in_named_tuple = settings.input_format_json_defaults_for_missing_elements_in_named_tuple; format_settings.json.ignore_unknown_keys_in_named_tuple = settings.input_format_json_ignore_unknown_keys_in_named_tuple; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 655aaa81d35..88a87028c66 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -188,6 +188,7 @@ struct FormatSettings bool escape_forward_slashes = true; bool read_named_tuples_as_objects = false; bool write_named_tuples_as_objects = false; + bool skip_null_value_in_named_tuples = false; bool defaults_for_missing_elements_in_named_tuple = false; bool ignore_unknown_keys_in_named_tuple = false; bool serialize_as_strings = false; @@ -203,7 +204,6 @@ struct FormatSettings bool compact_allow_variable_number_of_columns = false; bool try_infer_objects_as_tuples = false; bool infer_incomplete_types_as_strings = true; - } json; struct diff --git a/tests/queries/0_stateless/02902_json_skip_null_values.sql b/tests/queries/0_stateless/02902_json_skip_null_values.sql new file mode 100644 index 00000000000..088cf7e5b02 --- /dev/null +++ b/tests/queries/0_stateless/02902_json_skip_null_values.sql @@ -0,0 +1,38 @@ +-- Tags: no-fasttest + +create table test_02902 engine File(JSONEachRow) + settings output_format_json_named_tuples_as_objects = 1, output_format_json_skip_null_value_in_named_tuples = 1 + as select cast((number::String, null, (number::String, null)), 'Tuple(a Nullable(String), b Nullable(Int64), c Tuple(x Nullable(String), y Nullable(Float64)))') as c + from numbers(3); + +select * from test_02902 format JSONEachRow; + +drop table test_02902; + +select + toJSONString(c) +from + ( + select + cast( + (number:: String, null, (number:: String, null)), + 'Tuple(a Nullable(String), b Nullable(Int64), c Tuple(x Nullable(String), y Nullable(Float64)))' + ) as c + from + numbers(3) + ) +settings output_format_json_named_tuples_as_objects = 1, output_format_json_skip_null_value_in_named_tuples = 0; + +select + toJSONString(c) +from + ( + select + cast( + (number:: String, null, (number:: String, null)), + 'Tuple(a Nullable(String), b Nullable(Int64), c Tuple(x Nullable(String), y Nullable(Float64)))' + ) as c + from + numbers(3) + ) +settings output_format_json_named_tuples_as_objects = 1, output_format_json_skip_null_value_in_named_tuples = 1; From af684a36048c56851a66b2dde25111357eba2a66 Mon Sep 17 00:00:00 2001 From: Srikanth Chekuri Date: Tue, 24 Oct 2023 09:33:27 +0530 Subject: [PATCH 533/634] Rename the file number since master already has it --- ...rror.reference => 02903_empty_order_by_throws_error.reference} | 0 ...er_by_throws_error.sh => 02903_empty_order_by_throws_error.sh} | 0 ...erence => 02904_empty_order_by_with_setting_enabled.reference} | 0 ...ng_enabled.sh => 02904_empty_order_by_with_setting_enabled.sh} | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02902_empty_order_by_throws_error.reference => 02903_empty_order_by_throws_error.reference} (100%) rename tests/queries/0_stateless/{02902_empty_order_by_throws_error.sh => 02903_empty_order_by_throws_error.sh} (100%) rename tests/queries/0_stateless/{02903_empty_order_by_with_setting_enabled.reference => 02904_empty_order_by_with_setting_enabled.reference} (100%) rename tests/queries/0_stateless/{02903_empty_order_by_with_setting_enabled.sh => 02904_empty_order_by_with_setting_enabled.sh} (100%) diff --git a/tests/queries/0_stateless/02902_empty_order_by_throws_error.reference b/tests/queries/0_stateless/02903_empty_order_by_throws_error.reference similarity index 100% rename from tests/queries/0_stateless/02902_empty_order_by_throws_error.reference rename to tests/queries/0_stateless/02903_empty_order_by_throws_error.reference diff --git a/tests/queries/0_stateless/02902_empty_order_by_throws_error.sh b/tests/queries/0_stateless/02903_empty_order_by_throws_error.sh similarity index 100% rename from tests/queries/0_stateless/02902_empty_order_by_throws_error.sh rename to tests/queries/0_stateless/02903_empty_order_by_throws_error.sh diff --git a/tests/queries/0_stateless/02903_empty_order_by_with_setting_enabled.reference b/tests/queries/0_stateless/02904_empty_order_by_with_setting_enabled.reference similarity index 100% rename from tests/queries/0_stateless/02903_empty_order_by_with_setting_enabled.reference rename to tests/queries/0_stateless/02904_empty_order_by_with_setting_enabled.reference diff --git a/tests/queries/0_stateless/02903_empty_order_by_with_setting_enabled.sh b/tests/queries/0_stateless/02904_empty_order_by_with_setting_enabled.sh similarity index 100% rename from tests/queries/0_stateless/02903_empty_order_by_with_setting_enabled.sh rename to tests/queries/0_stateless/02904_empty_order_by_with_setting_enabled.sh From 93c1e5a9e6c1a4d3231c92f9680b47152bcb28ba Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 24 Oct 2023 12:31:55 +0800 Subject: [PATCH 534/634] allow skip null values when serailize tuple to json objects --- src/DataTypes/Serializations/SerializationTuple.cpp | 12 ++++++------ .../02902_json_skip_null_values.reference | 12 ++++++++++++ .../0_stateless/02902_json_skip_null_values.sql | 3 ++- 3 files changed, 20 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02902_json_skip_null_values.reference diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 979010b1c07..cbbe97eb05c 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -167,13 +167,13 @@ void SerializationTuple::serializeTextJSON(const IColumn & column, size_t row_nu bool first = true; for (size_t i = 0; i < elems.size(); ++i) { - if (!first) - writeChar(',', ostr); - const auto & element_column = extractElementColumn(column, i); if (settings.json.skip_null_value_in_named_tuples && element_column.isNullAt(row_num)) continue; + if (!first) + writeChar(',', ostr); + writeJSONString(elems[i]->getElementName(), ostr, settings); writeChar(':', ostr); elems[i]->serializeTextJSON(element_column, row_num, ostr, settings); @@ -205,13 +205,13 @@ void SerializationTuple::serializeTextJSONPretty(const IColumn & column, size_t bool first = true; for (size_t i = 0; i < elems.size(); ++i) { - if (!first) - writeCString(",\n", ostr); - const auto & element_column = extractElementColumn(column, i); if (settings.json.skip_null_value_in_named_tuples && element_column.isNullAt(row_num)) continue; + if (!first) + writeCString(",\n", ostr); + writeChar(' ', (indent + 1) * 4, ostr); writeJSONString(elems[i]->getElementName(), ostr, settings); writeCString(": ", ostr); diff --git a/tests/queries/0_stateless/02902_json_skip_null_values.reference b/tests/queries/0_stateless/02902_json_skip_null_values.reference new file mode 100644 index 00000000000..776b1983416 --- /dev/null +++ b/tests/queries/0_stateless/02902_json_skip_null_values.reference @@ -0,0 +1,12 @@ +{"c":{"a":"0","c":{"x":"0"}}} +{"c":{"a":"1","c":{"x":"1"}}} +{"c":{"a":"2","c":{"x":"2"}}} +{"c":{"a":"0","b":null,"c":{"x":"0","y":null}}} +{"c":{"a":"1","b":null,"c":{"x":"1","y":null}}} +{"c":{"a":"2","b":null,"c":{"x":"2","y":null}}} +{"a":"0","b":null,"c":{"x":"0","y":null}} +{"a":"1","b":null,"c":{"x":"1","y":null}} +{"a":"2","b":null,"c":{"x":"2","y":null}} +{"a":"0","c":{"x":"0"}} +{"a":"1","c":{"x":"1"}} +{"a":"2","c":{"x":"2"}} diff --git a/tests/queries/0_stateless/02902_json_skip_null_values.sql b/tests/queries/0_stateless/02902_json_skip_null_values.sql index 088cf7e5b02..24b162216bb 100644 --- a/tests/queries/0_stateless/02902_json_skip_null_values.sql +++ b/tests/queries/0_stateless/02902_json_skip_null_values.sql @@ -5,7 +5,8 @@ create table test_02902 engine File(JSONEachRow) as select cast((number::String, null, (number::String, null)), 'Tuple(a Nullable(String), b Nullable(Int64), c Tuple(x Nullable(String), y Nullable(Float64)))') as c from numbers(3); -select * from test_02902 format JSONEachRow; +select * from test_02902 format JSONEachRow settings output_format_json_named_tuples_as_objects = 1, output_format_json_skip_null_value_in_named_tuples = 1; +select * from test_02902 format JSONEachRow settings output_format_json_named_tuples_as_objects = 1, output_format_json_skip_null_value_in_named_tuples = 0; drop table test_02902; From 5291cb7790cb89784683e91b5bcaabe89111b2aa Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 24 Oct 2023 07:07:47 +0000 Subject: [PATCH 535/634] add document Signed-off-by: Duc Canh Le --- docs/en/operations/settings/settings.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ccbf54843e4..853d89ecf35 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2403,6 +2403,17 @@ See also: - [optimize_functions_to_subcolumns](#optimize-functions-to-subcolumns) +## optimize_trivial_approximate_count_query {#optimize_trivial_approximate_count_query} + +Use an approximate value for trivial count optimization of storages that support such estimation, for example, EmbeddedRocksDB. + +Possible values: + + - 0 — Optimization disabled. + - 1 — Optimization enabled. + +Default value: `0`. + ## optimize_count_from_files {#optimize_count_from_files} Enables or disables the optimization of counting number of rows from files in different input formats. It applies to table functions/engines `file`/`s3`/`url`/`hdfs`/`azureBlobStorage`. From 9c186d18a83fad8be972ade99d00dcb20de666ae Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 24 Oct 2023 16:13:53 +0800 Subject: [PATCH 536/634] retrigger ci --- src/Formats/FormatSettings.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 88a87028c66..2b005e39e62 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -204,6 +204,7 @@ struct FormatSettings bool compact_allow_variable_number_of_columns = false; bool try_infer_objects_as_tuples = false; bool infer_incomplete_types_as_strings = true; + } json; struct From 2a894335b71df44564e1267f33237307b0dccbde Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 24 Oct 2023 16:16:05 +0800 Subject: [PATCH 537/634] commit again --- src/IO/readFloatText.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/readFloatText.h b/src/IO/readFloatText.h index e3d549c8e17..ad14563c859 100644 --- a/src/IO/readFloatText.h +++ b/src/IO/readFloatText.h @@ -149,6 +149,7 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf) static constexpr bool throw_exception = std::is_same_v; static constexpr int MAX_LENGTH = 316; + ReadBufferFromMemory * buf_from_memory = dynamic_cast(&buf); /// Fast path (avoid copying) if the buffer have at least MAX_LENGTH bytes or buf is ReadBufferFromMemory From 40a01902e9ad7d916f46cc9d38f0495ced7fb7e7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 23 Oct 2023 22:03:41 +0200 Subject: [PATCH 538/634] Convert 02903_rmt_retriable_merge_exception to an sh test with proper retries CI: https://s3.amazonaws.com/clickhouse-test-reports/55944/bd26f7096a4a3325f7a363c4be919700cdf10ca3/stateless_tests_flaky_check__asan_.html Signed-off-by: Azat Khuzhin --- .../02903_rmt_retriable_merge_exception.sh | 62 +++++++++++++++++++ .../02903_rmt_retriable_merge_exception.sql | 32 ---------- 2 files changed, 62 insertions(+), 32 deletions(-) create mode 100755 tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh delete mode 100644 tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sql diff --git a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh new file mode 100755 index 00000000000..4ab84fc732c --- /dev/null +++ b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh @@ -0,0 +1,62 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# Test that retriable errors during merges/mutations +# (i.e. "No active replica has part X or covering part") +# does not appears as errors (level=Error), only as info message (level=Information). + +$CLICKHOUSE_CLIENT -nm -q " + drop table if exists rmt1; + drop table if exists rmt2; + + create table rmt1 (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', '1') order by key settings always_fetch_merged_part=1; + create table rmt2 (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', '2') order by key settings always_fetch_merged_part=0; + + insert into rmt1 values (1); + insert into rmt1 values (2); + + system stop pulling replication log rmt2; + optimize table rmt1 final settings alter_sync=0; +" + +# wait while there be at least one 'No active replica has part all_0_1_1 or covering part' in logs +for _ in {0..1000}; do + no_active_repilica_messages=$($CLICKHOUSE_CLIENT -nm -q " + system flush logs; + with + (select uuid from system.tables where database = currentDatabase() and table = 'rmt1') as uuid_ + select count() + from system.text_log + where + event_date >= yesterday() and event_time >= now() - 600 and + ( + (logger_name = 'MergeTreeBackgroundExecutor' and message like '%{' || uuid_::String || '::all_0_1_1}%No active replica has part all_0_1_1 or covering part%') or + (logger_name = uuid_::String || '::all_0_1_1 (MergeFromLogEntryTask)' and message like '%No active replica has part all_0_1_1 or covering part%') + ); + ") + if [[ $no_active_repilica_messages -gt 0 ]]; then + break + fi + # too frequent "system flush logs" causes troubles + sleep 1 +done + +$CLICKHOUSE_CLIENT -nm -q " + system start pulling replication log rmt2; + system flush logs; + with + (select uuid from system.tables where database = currentDatabase() and table = 'rmt1') as uuid_ + select + level, count() > 0 + from system.text_log + where + event_date >= yesterday() and event_time >= now() - 600 and + ( + (logger_name = 'MergeTreeBackgroundExecutor' and message like '%{' || uuid_::String || '::all_0_1_1}%No active replica has part all_0_1_1 or covering part%') or + (logger_name = uuid_::String || '::all_0_1_1 (MergeFromLogEntryTask)' and message like '%No active replica has part all_0_1_1 or covering part%') + ) + group by level; +" diff --git a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sql b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sql deleted file mode 100644 index 6d8a640e9ed..00000000000 --- a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sql +++ /dev/null @@ -1,32 +0,0 @@ --- Test that retriable errors during merges/mutations --- (i.e. "No active replica has part X or covering part") --- does not appears as errors (level=Error), only as info message (level=Information). - -drop table if exists rmt1; -drop table if exists rmt2; - -create table rmt1 (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', '1') order by key settings always_fetch_merged_part=1; -create table rmt2 (key Int) engine=ReplicatedMergeTree('/clickhouse/{database}', '2') order by key settings always_fetch_merged_part=0; - -insert into rmt1 values (1); -insert into rmt1 values (2); - -system stop pulling replication log rmt2; -optimize table rmt1 final settings alter_sync=0; - -select sleep(3) format Null; -system start pulling replication log rmt2; - -system flush logs; -with - (select uuid from system.tables where database = currentDatabase() and table = 'rmt1') as uuid_ -select - level, count() > 0 -from system.text_log -where - event_date >= yesterday() and event_time >= now() - 60 and - ( - (logger_name = 'MergeTreeBackgroundExecutor' and message like '%{' || uuid_::String || '::all_0_1_1}%No active replica has part all_0_1_1 or covering part%') or - (logger_name = uuid_::String || '::all_0_1_1 (MergeFromLogEntryTask)' and message like '%No active replica has part all_0_1_1 or covering part%') - ) -group by level; From da408df4a7296835897d7cef80d63f252df79b75 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 23 Oct 2023 22:16:15 +0200 Subject: [PATCH 539/634] Proper ignoring of retriable errors during merge/mutate The exception cannot be simply suppressed, since sanity checks in the WriteBuffer dtor relies on the std::uncaught_exceptions(), and if the exception will be suppressed, then if the buffer was not finalized, it may abort (even though it is only in debug/sanitizers build). So instead, IExecutableTask::printExecutionException() had been introduced to distinguish when exception should be printed and when not. --- src/Storages/MergeTree/IExecutableTask.h | 9 +++++++++ src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp | 3 ++- src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp | 6 ++++-- src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h | 3 +++ 4 files changed, 18 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/IExecutableTask.h b/src/Storages/MergeTree/IExecutableTask.h index 738056e0ea0..ee453e45071 100644 --- a/src/Storages/MergeTree/IExecutableTask.h +++ b/src/Storages/MergeTree/IExecutableTask.h @@ -30,7 +30,16 @@ class IExecutableTask { public: using TaskResultCallback = std::function; + virtual bool executeStep() = 0; + + /// Sometimes exceptions from the executeStep() had been already printed to + /// the log, but with different level (see + /// ReplicatedMergeMutateTaskBase::executeStep()), but the exception should + /// be throw, since there are some sanity assertions based on the + /// std::uncaught_exceptions() (i.e. WriteBuffer::~WriteBuffer()) + virtual bool printExecutionException() const { return true; } + virtual void onCompleted() = 0; virtual StorageID getStorageID() const = 0; virtual String getQueryId() const = 0; diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 2df0a6c1c1f..1a7a0b5b2c1 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -281,7 +281,8 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) } catch (...) { - printExceptionWithRespectToAbort(log, query_id); + if (item->task->printExecutionException()) + printExceptionWithRespectToAbort(log, query_id); /// Release the task with exception context. /// An exception context is needed to proper delete write buffers without finalization release_task(std::move(item)); diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 32783004772..94c069d789b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -72,8 +72,7 @@ bool ReplicatedMergeMutateTaskBase::executeStep() /// This exception will be written to the queue element, and it can be looked up using `system.replication_queue` table. /// The thread that performs this action will sleep a few seconds after the exception. /// See `queue.processEntry` function. - if (!retryable_error) - throw; + throw; } catch (...) { @@ -121,6 +120,9 @@ bool ReplicatedMergeMutateTaskBase::executeStep() } } + if (retryable_error) + print_exception = false; + if (saved_exception) std::rethrow_exception(saved_exception); diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h index ba514f11f20..18fcacecc9e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h @@ -37,6 +37,8 @@ public: String getQueryId() const override { return getStorageID().getShortName() + "::" + selected_entry->log_entry->new_part_name; } bool executeStep() override; + bool printExecutionException() const override { return print_exception; } + protected: using PartLogWriter = std::function; @@ -91,6 +93,7 @@ private: PartLogWriter part_log_writer{}; State state{State::NEED_PREPARE}; IExecutableTask::TaskResultCallback task_result_callback; + bool print_exception = true; }; } From 68c3f41b71cbd38df935b4d1c61f312b8c37f111 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 24 Oct 2023 08:56:09 +0000 Subject: [PATCH 540/634] Fix performance tests --- tests/performance/array_fold.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/performance/array_fold.xml b/tests/performance/array_fold.xml index fae8bd164a7..32bd45beb1e 100644 --- a/tests/performance/array_fold.xml +++ b/tests/performance/array_fold.xml @@ -1,5 +1,5 @@ - SELECT arrayFold((x, acc) -> acc + x, range(number % 100), toUInt64(0)) from numbers(100000) Format Null - SELECT arrayFold((x, acc) -> acc + 1, range(number % 100), toUInt64(0)) from numbers(100000) Format Null - SELECT arrayFold((x, acc) -> acc + x, range(number), toUInt64(0)) from numbers(10000) Format Null + SELECT arrayFold((acc, x) -> acc + x, range(number % 100), toUInt64(0)) from numbers(100000) Format Null + SELECT arrayFold((acc, x) -> acc + 1, range(number % 100), toUInt64(0)) from numbers(100000) Format Null + SELECT arrayFold((acc, x) -> acc + x, range(number), toUInt64(0)) from numbers(10000) Format Null From 70adb4ca1798a803c581280f6c78cccfb98a71be Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 24 Oct 2023 11:40:55 +0200 Subject: [PATCH 541/634] Fix SHOW DATABASES LIMIT --- src/Interpreters/InterpreterShowTablesQuery.cpp | 6 +++--- .../0_stateless/02902_show_databases_limit.reference | 0 tests/queries/0_stateless/02902_show_databases_limit.sql | 1 + 3 files changed, 4 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02902_show_databases_limit.reference create mode 100644 tests/queries/0_stateless/02902_show_databases_limit.sql diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 97bd8e7a8b7..ea832668cd0 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -48,12 +48,12 @@ String InterpreterShowTablesQuery::getRewrittenQuery() << DB::quote << query.like; } - if (query.limit_length) - rewritten_query << " LIMIT " << query.limit_length; - /// (*) rewritten_query << " ORDER BY name"; + if (query.limit_length) + rewritten_query << " LIMIT " << query.limit_length; + return rewritten_query.str(); } diff --git a/tests/queries/0_stateless/02902_show_databases_limit.reference b/tests/queries/0_stateless/02902_show_databases_limit.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02902_show_databases_limit.sql b/tests/queries/0_stateless/02902_show_databases_limit.sql new file mode 100644 index 00000000000..e13ae5a7e34 --- /dev/null +++ b/tests/queries/0_stateless/02902_show_databases_limit.sql @@ -0,0 +1 @@ +SHOW DATABASES LIMIT 0; From 38a2ba5769ebc780bad146d44732b358a2df4ccb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 24 Oct 2023 09:53:20 +0000 Subject: [PATCH 542/634] Cosmetics --- docs/en/sql-reference/statements/show.md | 2 +- src/Interpreters/InterpreterShowTablesQuery.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 21bd674bd24..5f48220ea98 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -473,7 +473,7 @@ Shows all [users](../../guides/sre/user-management/index.md#user-account-managem ``` sql SHOW ACCESS ``` -## SHOW CLUSTER(s) +## SHOW CLUSTER(S) Returns a list of clusters. All available clusters are listed in the [system.clusters](../../operations/system-tables/clusters.md) table. diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index ea832668cd0..71ad9d8d8e5 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -158,7 +158,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery() else if (query.where_expression) rewritten_query << " AND (" << query.where_expression << ")"; - /// (*) + /// (*) rewritten_query << " ORDER BY name "; if (query.limit_length) From 666f854b2771631f18d1325b9f03bde776a1146b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 24 Oct 2023 09:58:42 +0000 Subject: [PATCH 543/634] Bump protobuf to 3.19.3 --- contrib/google-protobuf | 2 +- contrib/google-protobuf-cmake/CMakeLists.txt | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/contrib/google-protobuf b/contrib/google-protobuf index c47efe2d8f6..f99a7c85928 160000 --- a/contrib/google-protobuf +++ b/contrib/google-protobuf @@ -1 +1 @@ -Subproject commit c47efe2d8f6a60022b49ecd6cc23660687c8598f +Subproject commit f99a7c859281aba9dfd0e26b4925968710e800e0 diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index 8afb86b25dd..32657b62c17 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -94,6 +94,7 @@ set(libprotobuf_files ${protobuf_source_dir}/src/google/protobuf/map_field.cc ${protobuf_source_dir}/src/google/protobuf/message.cc ${protobuf_source_dir}/src/google/protobuf/reflection_ops.cc + ${protobuf_source_dir}/src/google/protobuf/repeated_ptr_field.cc ${protobuf_source_dir}/src/google/protobuf/service.cc ${protobuf_source_dir}/src/google/protobuf/source_context.pb.cc ${protobuf_source_dir}/src/google/protobuf/struct.pb.cc From a962161832a51ac486ccf167c18f2dd0f775b3eb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 24 Oct 2023 10:22:58 +0000 Subject: [PATCH 544/634] Bump to 3.19.6 --- contrib/google-protobuf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/google-protobuf b/contrib/google-protobuf index f99a7c85928..9650fe3fa9b 160000 --- a/contrib/google-protobuf +++ b/contrib/google-protobuf @@ -1 +1 @@ -Subproject commit f99a7c859281aba9dfd0e26b4925968710e800e0 +Subproject commit 9650fe3fa9b326bbe0ecfc52883bf5a0e030f8bb From 72f2288ee4bb8fcddd1d2dc00bc079f55cd8ad00 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 24 Oct 2023 12:44:18 +0200 Subject: [PATCH 545/634] Add optimize_trivial_approximate_count_query to stress tests --- tests/ci/stress.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 8cc40839706..5d5f03fce73 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -65,6 +65,9 @@ def get_options(i: int, upgrade_check: bool) -> str: f"partial_result_update_duration_ms={random.randint(10, 1000)}" ) + if random.random() < 0.1: + client_options.append("optimize_trivial_approximate_count_query=1") + if client_options: options.append(" --client-option " + " ".join(client_options)) From f7c1e8f69ed9cb6540568f478cfa34661e429853 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 24 Oct 2023 12:45:11 +0200 Subject: [PATCH 546/634] Set tag use-rocksdb for 02892_rocksdb_trivial_count --- tests/queries/0_stateless/02892_rocksdb_trivial_count.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql b/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql index 6ced27f94d4..0cdf2d1b2b2 100644 --- a/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql +++ b/tests/queries/0_stateless/02892_rocksdb_trivial_count.sql @@ -1,3 +1,5 @@ +-- Tags: use-rocksdb + CREATE TABLE dict (key UInt64, value String) ENGINE = EmbeddedRocksDB PRIMARY KEY key; INSERT INTO dict SELECT number, toString(number) FROM numbers(121); SELECT count() FROM dict SETTINGS optimize_trivial_approximate_count_query = 0, max_rows_to_read = 1; -- { serverError TOO_MANY_ROWS } From fa76a88c4f74d3bb8a81f28937fe4f8954433ab0 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 12 Sep 2023 15:34:21 +0200 Subject: [PATCH 547/634] Revert "Revert "Fix issues in accurateCastOrNull"" This reverts commit 71d37d64ab37ee24716868fa83c0421f5168a307. --- src/Functions/FunctionsConversion.h | 60 +++++++++++++++---- .../01556_accurate_cast_or_null.reference | 5 ++ .../01556_accurate_cast_or_null.sql | 7 +++ ...03_cast_nullable_to_custom_types.reference | 4 +- 4 files changed, 63 insertions(+), 13 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index cf60eea547b..617b11a1a9d 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -89,6 +89,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NOT_IMPLEMENTED; extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; + extern const int CANNOT_PARSE_BOOL; } @@ -1683,7 +1684,19 @@ struct ConvertImplGenericFromString const auto & val = col_from_string->getDataAt(i); ReadBufferFromMemory read_buffer(val.data, val.size); - serialization_from.deserializeWholeText(column_to, read_buffer, format_settings); + try + { + serialization_from.deserializeWholeText(column_to, read_buffer, format_settings); + } + catch (const Exception & e) + { + if (e.code() == ErrorCodes::CANNOT_PARSE_BOOL && typeid_cast(&column_to)) + { + column_to.insertDefault(); + continue; + } + throw; + } if (!read_buffer.eof()) { @@ -4177,15 +4190,21 @@ private: { if constexpr (std::is_same_v) { - ret = [cast_ipv4_ipv6_default_on_conversion_error_value, input_format_ipv4_default_on_conversion_error_value, requested_result_is_nullable]( - ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t) - -> ColumnPtr + ret = [cast_ipv4_ipv6_default_on_conversion_error_value, + input_format_ipv4_default_on_conversion_error_value, + requested_result_is_nullable]( + ColumnsWithTypeAndName & arguments, + const DataTypePtr & result_type, + const ColumnNullable * column_nullable, + size_t) -> ColumnPtr { if (!WhichDataType(result_type).isIPv4()) throw Exception(ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName()); const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; - if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv4_default_on_conversion_error_value || requested_result_is_nullable) + if (requested_result_is_nullable) + return convertToIPv4(arguments[0].column, null_map); + else if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv4_default_on_conversion_error_value) return convertToIPv4(arguments[0].column, null_map); else return convertToIPv4(arguments[0].column, null_map); @@ -4196,16 +4215,22 @@ private: if constexpr (std::is_same_v) { - ret = [cast_ipv4_ipv6_default_on_conversion_error_value, input_format_ipv6_default_on_conversion_error_value, requested_result_is_nullable]( - ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t) - -> ColumnPtr + ret = [cast_ipv4_ipv6_default_on_conversion_error_value, + input_format_ipv6_default_on_conversion_error_value, + requested_result_is_nullable]( + ColumnsWithTypeAndName & arguments, + const DataTypePtr & result_type, + const ColumnNullable * column_nullable, + size_t) -> ColumnPtr { if (!WhichDataType(result_type).isIPv6()) throw Exception( ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv6", result_type->getName()); const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; - if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv6_default_on_conversion_error_value || requested_result_is_nullable) + if (requested_result_is_nullable) + return convertToIPv6(arguments[0].column, null_map); + else if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv6_default_on_conversion_error_value) return convertToIPv6(arguments[0].column, null_map); else return convertToIPv6(arguments[0].column, null_map); @@ -4216,7 +4241,18 @@ private: if (to_type->getCustomSerialization() && to_type->getCustomName()) { - ret = &ConvertImplGenericFromString::execute; + ret = [requested_result_is_nullable]( + ColumnsWithTypeAndName & arguments, + const DataTypePtr & result_type, + const ColumnNullable * column_nullable, + size_t input_rows_count) -> ColumnPtr + { + auto wrapped_result_type = result_type; + if (requested_result_is_nullable) + wrapped_result_type = makeNullable(result_type); + return ConvertImplGenericFromString::execute( + arguments, wrapped_result_type, column_nullable, input_rows_count); + }; return true; } } @@ -4231,7 +4267,9 @@ private: ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName()); const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; - if (cast_ipv4_ipv6_default_on_conversion_error_value || requested_result_is_nullable) + if (requested_result_is_nullable) + return convertIPv6ToIPv4(arguments[0].column, null_map); + else if (cast_ipv4_ipv6_default_on_conversion_error_value) return convertIPv6ToIPv4(arguments[0].column, null_map); else return convertIPv6ToIPv4(arguments[0].column, null_map); diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference index 31a9c37421e..296b0b553f4 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference @@ -42,3 +42,8 @@ 2023-05-30 2149-06-06 1970-01-20 +Tests for issue #38585 +\N +\N +\N +\N diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.sql b/tests/queries/0_stateless/01556_accurate_cast_or_null.sql index f00f6ef837f..5d580a8f32f 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.sql +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.sql @@ -49,3 +49,10 @@ SELECT accurateCastOrNull('1xxx', 'Date'); SELECT accurateCastOrNull('2023-05-30', 'Date'); SELECT accurateCastOrNull('2180-01-01', 'Date'); SELECT accurateCastOrNull(19, 'Date'); + +SELECT 'Tests for issue #38585'; +select accurateCastOrNull('test', 'Bool'); +select accurateCast('test', 'Nullable(Bool)'); + +select accurateCastOrNull('test', 'IPv4'); +select accurateCastOrNull('test', 'IPv6'); diff --git a/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.reference b/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.reference index 159ee805f26..08ade20ed1d 100644 --- a/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.reference +++ b/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.reference @@ -17,7 +17,7 @@ true \N 0.0.0.0 \N -0.0.0.0 +\N \N \N \N @@ -25,7 +25,7 @@ true \N :: \N -:: +\N \N \N \N From f29a618d7e5cc1258361580861130a91591e5423 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 13 Sep 2023 17:35:45 +0200 Subject: [PATCH 548/634] Add stronger exception guarantees to bool serialization before this change the output column may or may not contain an invalid value when an exception was thrown. We now make sure that, in case of error, we either don't make changes or we revert them. --- src/DataTypes/Serializations/SerializationBool.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index 41b5bf806e5..005fe1095e1 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -166,13 +166,13 @@ void deserializeImpl( buf.rollbackToCheckpoint(); if (checkString(settings.bool_false_representation, buf) && check_end_of_value(buf)) { - col->insert(false); buf.dropCheckpoint(); if (buf.hasUnreadData()) throw Exception( ErrorCodes::CANNOT_PARSE_BOOL, "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " "bool_true_representation or bool_false_representation contains some delimiters of input format"); + col->insert(false); return; } @@ -181,15 +181,19 @@ void deserializeImpl( { buf.dropCheckpoint(); if (buf.hasUnreadData()) + { + col->popBack(1); throw Exception( ErrorCodes::CANNOT_PARSE_BOOL, "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " "bool_true_representation or bool_false_representation contains some delimiters of input format"); + } return; } buf.makeContinuousMemoryFromCheckpointToPos(); buf.rollbackToCheckpoint(); + col->popBack(1); throw Exception( ErrorCodes::CANNOT_PARSE_BOOL, "Cannot parse boolean value here: '{}', should be '{}' or '{}' controlled by setting bool_true_representation and " From f417de4820c937aae06caaa781afdf5a95185f86 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 14 Sep 2023 13:54:28 +0200 Subject: [PATCH 549/634] Add tests for issue found fuzzing --- .../02303_cast_nullable_to_custom_types.reference | 5 +++++ .../0_stateless/02303_cast_nullable_to_custom_types.sql | 4 ++++ 2 files changed, 9 insertions(+) diff --git a/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.reference b/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.reference index 08ade20ed1d..c847b553212 100644 --- a/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.reference +++ b/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.reference @@ -30,3 +30,8 @@ true \N \N \0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0 +fuzzer issue +\N +\N +\N +\N diff --git a/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.sql b/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.sql index 1cc5140f339..f21243c2942 100644 --- a/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.sql +++ b/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.sql @@ -22,3 +22,7 @@ select toIPv6(number % 2 ? '0000:0000:0000:0000:0000:0000:0000:0000' : NULL) fro select toIPv6OrDefault(number % 2 ? '' : NULL) from numbers(2); select toIPv6OrNull(number % 2 ? '' : NULL) from numbers(2); select IPv6StringToNum(number % 2 ? '0000:0000:0000:0000:0000:0000:0000:0000' : NULL) from numbers(2); + +select 'fuzzer issue'; +SELECT CAST(if(number % 2, 'truetrue', NULL), 'Nullable(Bool)') FROM numbers(2); +SELECT CAST(if(number % 2, 'falsefalse', NULL), 'Nullable(Bool)') FROM numbers(2); From 36da379b9371ea57fd5677aeb540a22f2cef0bab Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 28 Sep 2023 15:42:33 +0200 Subject: [PATCH 550/634] Revert "Add stronger exception guarantees to bool serialization" This reverts commit 62d06a0a4cd39344b41b90300b772299f9719924. --- src/DataTypes/Serializations/SerializationBool.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index 005fe1095e1..41b5bf806e5 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -166,13 +166,13 @@ void deserializeImpl( buf.rollbackToCheckpoint(); if (checkString(settings.bool_false_representation, buf) && check_end_of_value(buf)) { + col->insert(false); buf.dropCheckpoint(); if (buf.hasUnreadData()) throw Exception( ErrorCodes::CANNOT_PARSE_BOOL, "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " "bool_true_representation or bool_false_representation contains some delimiters of input format"); - col->insert(false); return; } @@ -181,19 +181,15 @@ void deserializeImpl( { buf.dropCheckpoint(); if (buf.hasUnreadData()) - { - col->popBack(1); throw Exception( ErrorCodes::CANNOT_PARSE_BOOL, "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " "bool_true_representation or bool_false_representation contains some delimiters of input format"); - } return; } buf.makeContinuousMemoryFromCheckpointToPos(); buf.rollbackToCheckpoint(); - col->popBack(1); throw Exception( ErrorCodes::CANNOT_PARSE_BOOL, "Cannot parse boolean value here: '{}', should be '{}' or '{}' controlled by setting bool_true_representation and " From 22ae8731f2949f5901bc108b66ca75b87ede6ead Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 29 Sep 2023 11:16:56 +0200 Subject: [PATCH 551/634] Handle the case when bool serialization generates incorrect values on failure Fix issue found by fuzzer --- src/Functions/FunctionsConversion.h | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 617b11a1a9d..7d2386b0529 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1690,9 +1690,15 @@ struct ConvertImplGenericFromString } catch (const Exception & e) { - if (e.code() == ErrorCodes::CANNOT_PARSE_BOOL && typeid_cast(&column_to)) + auto * nullable_column = typeid_cast(&column_to); + if (e.code() == ErrorCodes::CANNOT_PARSE_BOOL && nullable_column) { - column_to.insertDefault(); + auto & col_nullmap = nullable_column->getNullMapData(); + if (col_nullmap.size() != nullable_column->size()) + col_nullmap.push_back(0); + if (nullable_column->size() == column_from.size()) + nullable_column->popBack(1); + nullable_column->insertDefault(); continue; } throw; From 471fe28165bc93f5a37db3b514c4fec3c1205f44 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 20 Oct 2023 16:00:33 +0200 Subject: [PATCH 552/634] Improve tests --- .../01556_accurate_cast_or_null.reference | 15 +++++++++++- .../01556_accurate_cast_or_null.sql | 18 ++++++++++++-- .../0_stateless/01601_accurate_cast.reference | 12 ++++++++++ .../0_stateless/01601_accurate_cast.sql | 24 +++++++++++++++++++ .../02026_accurate_cast_or_default.reference | 21 ++++++++++++++++ .../02026_accurate_cast_or_default.sql | 24 +++++++++++++++++++ 6 files changed, 111 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference index 296b0b553f4..a2ccd5af868 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference @@ -42,8 +42,21 @@ 2023-05-30 2149-06-06 1970-01-20 -Tests for issue #38585 \N \N \N +true +false +true +false +true +false +\N +\N +\N +192.0.2.1 +\N +\N +::ffff:192.0.2.1 +2001:db8::1 \N diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.sql b/tests/queries/0_stateless/01556_accurate_cast_or_null.sql index 5d580a8f32f..2fb7b1177e6 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.sql +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.sql @@ -50,9 +50,23 @@ SELECT accurateCastOrNull('2023-05-30', 'Date'); SELECT accurateCastOrNull('2180-01-01', 'Date'); SELECT accurateCastOrNull(19, 'Date'); -SELECT 'Tests for issue #38585'; select accurateCastOrNull('test', 'Bool'); -select accurateCast('test', 'Nullable(Bool)'); +select accurateCastOrNull('truex', 'Bool'); +select accurateCastOrNull('xfalse', 'Bool'); +select accurateCastOrNull('true', 'Bool'); +select accurateCastOrNull('false', 'Bool'); +select accurateCastOrNull('1', 'Bool'); +select accurateCastOrNull('0', 'Bool'); +select accurateCastOrNull(1, 'Bool'); +select accurateCastOrNull(0, 'Bool'); select accurateCastOrNull('test', 'IPv4'); +select accurateCastOrNull('2001:db8::1', 'IPv4'); +select accurateCastOrNull('::ffff:192.0.2.1', 'IPv4'); +select accurateCastOrNull('192.0.2.1', 'IPv4'); +select accurateCastOrNull('192.0.2.1x', 'IPv4'); + select accurateCastOrNull('test', 'IPv6'); +select accurateCastOrNull('192.0.2.1', 'IPv6'); +select accurateCastOrNull('2001:db8::1', 'IPv6'); +select accurateCastOrNull('2001:db8::1x', 'IPv6'); diff --git a/tests/queries/0_stateless/01601_accurate_cast.reference b/tests/queries/0_stateless/01601_accurate_cast.reference index 3c6dceb1f16..dbf9666f4cd 100644 --- a/tests/queries/0_stateless/01601_accurate_cast.reference +++ b/tests/queries/0_stateless/01601_accurate_cast.reference @@ -10,3 +10,15 @@ 1970-01-01 00:00:19 2023-05-30 1970-01-20 +\N +true +false +true +false +true +false +\N +192.0.2.1 +\N +::ffff:192.0.2.1 +2001:db8::1 diff --git a/tests/queries/0_stateless/01601_accurate_cast.sql b/tests/queries/0_stateless/01601_accurate_cast.sql index 2108e42df05..d2ecede2402 100644 --- a/tests/queries/0_stateless/01601_accurate_cast.sql +++ b/tests/queries/0_stateless/01601_accurate_cast.sql @@ -34,3 +34,27 @@ SELECT accurateCast(0xFFFFFFFF + 1, 'Date'); -- { serverError CANNOT_CONVERT_T SELECT accurateCast('1xxx', 'Date'); -- { serverError CANNOT_PARSE_DATE } SELECT accurateCast('2023-05-30', 'Date'); SELECT accurateCast(19, 'Date'); + +select accurateCast('test', 'Nullable(Bool)'); +select accurateCast('test', 'Bool'); -- { serverError CANNOT_PARSE_BOOL } +select accurateCast('truex', 'Bool'); -- { serverError CANNOT_PARSE_BOOL } +select accurateCast('xfalse', 'Bool'); -- { serverError CANNOT_PARSE_BOOL } +select accurateCast('true', 'Bool'); +select accurateCast('false', 'Bool'); +select accurateCast('1', 'Bool'); +select accurateCast('0', 'Bool'); +select accurateCast(1, 'Bool'); +select accurateCast(0, 'Bool'); + +select accurateCast('test', 'Nullable(IPv4)'); +select accurateCast('test', 'IPv4'); -- { serverError CANNOT_PARSE_IPV4 } +select accurateCast('2001:db8::1', 'IPv4'); -- { serverError CANNOT_PARSE_IPV4 } +select accurateCast('::ffff:192.0.2.1', 'IPv4'); -- { serverError CANNOT_PARSE_IPV4 } +select accurateCast('192.0.2.1', 'IPv4'); +select accurateCast('192.0.2.1x', 'IPv4'); -- { serverError CANNOT_PARSE_IPV4 } + +select accurateCast('test', 'Nullable(IPv6)'); +select accurateCast('test', 'IPv6'); -- { serverError CANNOT_PARSE_IPV6 } +select accurateCast('192.0.2.1', 'IPv6'); +select accurateCast('2001:db8::1', 'IPv6'); +select accurateCast('2001:db8::1x', 'IPv6'); -- { serverError CANNOT_PARSE_IPV6 } diff --git a/tests/queries/0_stateless/02026_accurate_cast_or_default.reference b/tests/queries/0_stateless/02026_accurate_cast_or_default.reference index 67be2da9975..c31e00dd8f4 100644 --- a/tests/queries/0_stateless/02026_accurate_cast_or_default.reference +++ b/tests/queries/0_stateless/02026_accurate_cast_or_default.reference @@ -30,3 +30,24 @@ 0 5 127 127 0 5 +\N +false +false +false +true +false +true +false +true +false +\N +0.0.0.0 +0.0.0.0 +0.0.0.0 +192.0.2.1 +0.0.0.0 +\N +:: +::ffff:192.0.2.1 +2001:db8::1 +:: diff --git a/tests/queries/0_stateless/02026_accurate_cast_or_default.sql b/tests/queries/0_stateless/02026_accurate_cast_or_default.sql index 1c35055749e..d493914c956 100644 --- a/tests/queries/0_stateless/02026_accurate_cast_or_default.sql +++ b/tests/queries/0_stateless/02026_accurate_cast_or_default.sql @@ -34,3 +34,27 @@ SELECT accurateCastOrDefault(nan, 'UInt64'), accurateCastOrDefault(nan, 'UInt64' SELECT accurateCastOrDefault(nan, 'UInt256'), accurateCastOrDefault(nan, 'UInt256', toUInt256(5)); SELECT accurateCastOrDefault(number + 127, 'Int8') AS x, accurateCastOrDefault(number + 127, 'Int8', toInt8(5)) AS x_with_default FROM numbers (2) ORDER BY number; + +select accurateCastOrDefault('test', 'Nullable(Bool)'); +select accurateCastOrDefault('test', 'Bool'); +select accurateCastOrDefault('truex', 'Bool'); +select accurateCastOrDefault('xfalse', 'Bool'); +select accurateCastOrDefault('true', 'Bool'); +select accurateCastOrDefault('false', 'Bool'); +select accurateCastOrDefault('1', 'Bool'); +select accurateCastOrDefault('0', 'Bool'); +select accurateCastOrDefault(1, 'Bool'); +select accurateCastOrDefault(0, 'Bool'); + +select accurateCastOrDefault('test', 'Nullable(IPv4)'); +select accurateCastOrDefault('test', 'IPv4'); +select accurateCastOrDefault('2001:db8::1', 'IPv4'); +select accurateCastOrDefault('::ffff:192.0.2.1', 'IPv4'); +select accurateCastOrDefault('192.0.2.1', 'IPv4'); +select accurateCastOrDefault('192.0.2.1x', 'IPv4'); + +select accurateCastOrDefault('test', 'Nullable(IPv6)'); +select accurateCastOrDefault('test', 'IPv6'); +select accurateCastOrDefault('192.0.2.1', 'IPv6'); +select accurateCastOrDefault('2001:db8::1', 'IPv6'); +select accurateCastOrDefault('2001:db8::1x', 'IPv6'); From 6b7b3a641f9549f2a24727d6228cd18e7413efff Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Mon, 23 Oct 2023 13:01:53 +0200 Subject: [PATCH 553/634] Fix more fuzzying issues --- src/Functions/FunctionsConversion.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 7d2386b0529..69c5aa48155 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1695,8 +1695,8 @@ struct ConvertImplGenericFromString { auto & col_nullmap = nullable_column->getNullMapData(); if (col_nullmap.size() != nullable_column->size()) - col_nullmap.push_back(0); - if (nullable_column->size() == column_from.size()) + col_nullmap.resize_fill(nullable_column->size()); + if (nullable_column->size() == (i + 1)) nullable_column->popBack(1); nullable_column->insertDefault(); continue; From 4cf17dc50a1d11e9bbf6392eb44f6a134f98bd0c Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Mon, 23 Oct 2023 13:02:09 +0200 Subject: [PATCH 554/634] Add other fuzzer tests --- .../02303_cast_nullable_to_custom_types.reference | 8 ++++++++ .../0_stateless/02303_cast_nullable_to_custom_types.sql | 2 ++ 2 files changed, 10 insertions(+) diff --git a/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.reference b/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.reference index c847b553212..717484d4670 100644 --- a/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.reference +++ b/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.reference @@ -35,3 +35,11 @@ fuzzer issue \N \N \N +\N +\N +\N +\N +\N +\N +\N +\N diff --git a/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.sql b/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.sql index f21243c2942..b56ebc2b09d 100644 --- a/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.sql +++ b/tests/queries/0_stateless/02303_cast_nullable_to_custom_types.sql @@ -26,3 +26,5 @@ select IPv6StringToNum(number % 2 ? '0000:0000:0000:0000:0000:0000:0000:0000' : select 'fuzzer issue'; SELECT CAST(if(number % 2, 'truetrue', NULL), 'Nullable(Bool)') FROM numbers(2); SELECT CAST(if(number % 2, 'falsefalse', NULL), 'Nullable(Bool)') FROM numbers(2); +SELECT accurateCastOrNull(if(number % 2, NULL, 'truex'), 'Bool') FROM numbers(4); +SELECT accurateCastOrNull(if(number % 2, 'truex', NULL), 'Bool') FROM numbers(4); From 837637caa6174ec84ecb34ddbfb74f9209bf7e7b Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 24 Oct 2023 13:45:01 +0200 Subject: [PATCH 555/634] Fix typo Co-authored-by: SmitaRKulkarni --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bb28e246a17..25f9040411c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -551,7 +551,7 @@ class IColumn; M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \ - M(Bool, apply_deleted_mask, true, "Enables filtering out rows deleted with lightweight DELETE. If disabled, a query will be able to read those rows. This is useful for debugging and \"undelete\" scenarious", 0) \ + M(Bool, apply_deleted_mask, true, "Enables filtering out rows deleted with lightweight DELETE. If disabled, a query will be able to read those rows. This is useful for debugging and \"undelete\" scenarios", 0) \ M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ From 247722d44ea0c08382b001cb425f11075dab4533 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 24 Oct 2023 13:45:19 +0200 Subject: [PATCH 556/634] Update tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql Co-authored-by: SmitaRKulkarni --- tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql b/tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql index 4122871bf39..84a7a6f9cac 100644 --- a/tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql +++ b/tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql @@ -9,7 +9,7 @@ DELETE FROM test_apply_deleted_mask WHERE id % 2 = 0; SELECT 'Normal SELECT does not see deleted rows'; SELECT *, _row_exists FROM test_apply_deleted_mask; -SELECT 'With the setting enabled the deleted rows are visible'; +SELECT 'With the setting disabled the deleted rows are visible'; SELECT *, _row_exists FROM test_apply_deleted_mask SETTINGS apply_deleted_mask = 0; SELECT 'With the setting enabled the deleted rows are visible but still can be filterd out'; From ab8fe6857177cb2151dec23b995bf174723738ab Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 24 Oct 2023 13:45:27 +0200 Subject: [PATCH 557/634] Update tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql Co-authored-by: SmitaRKulkarni --- tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql b/tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql index 84a7a6f9cac..81d5714f00b 100644 --- a/tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql +++ b/tests/queries/0_stateless/02902_diable_apply_deleted_mask.sql @@ -12,7 +12,7 @@ SELECT *, _row_exists FROM test_apply_deleted_mask; SELECT 'With the setting disabled the deleted rows are visible'; SELECT *, _row_exists FROM test_apply_deleted_mask SETTINGS apply_deleted_mask = 0; -SELECT 'With the setting enabled the deleted rows are visible but still can be filterd out'; +SELECT 'With the setting disabled the deleted rows are visible but still can be filterd out'; SELECT * FROM test_apply_deleted_mask WHERE _row_exists SETTINGS apply_deleted_mask = 0; INSERT INTO test_apply_deleted_mask SELECT number, number::String FROM numbers(5, 1); From ff7e9e16d54e2e70ebd8e7612bb0c56b2bf8512a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 24 Oct 2023 13:00:44 +0200 Subject: [PATCH 558/634] Fix possible performance test error CI [1]: 2023-10-24 12:18:27 ++ rg -c array_fill.xml changed-test-definitions.txt 2023-10-24 12:18:27 + '[' '' -gt 0 ']' 2023-10-24 12:18:27 right/scripts/compare.sh: line 288: [: : integer expression expected [1]: https://s3.amazonaws.com/clickhouse-test-reports/55944/da408df4a7296835897d7cef80d63f252df79b75/performance_comparison_[2_4]/compare.log v1: use --include-zero v2: revert to grep v3: check exit code Signed-off-by: Azat Khuzhin --- docker/test/performance-comparison/compare.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 24df1b9c6b1..7d6de732489 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -285,7 +285,7 @@ function run_tests # Use awk because bash doesn't support floating point arithmetic. profile_seconds=$(awk "BEGIN { print ($profile_seconds_left > 0 ? 10 : 0) }") - if [ "$(rg -c $(basename $test) changed-test-definitions.txt)" -gt 0 ] + if rg --quiet "$(basename $test)" changed-test-definitions.txt then # Run all queries from changed test files to ensure that all new queries will be tested. max_queries=0 From 2ec9433b4e8f8f8b9ffad6e4f50e9c207b7b59ad Mon Sep 17 00:00:00 2001 From: MeenaRenganathan22 Date: Tue, 24 Oct 2023 06:10:11 -0700 Subject: [PATCH 559/634] S390x by default uses openssl. So, the error message is different from boringSSL. Update the tets to match the error message --- .../0_stateless/02550_client_connections_credentials.sh | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02550_client_connections_credentials.sh b/tests/queries/0_stateless/02550_client_connections_credentials.sh index 3776216751e..6b1ca1843ee 100755 --- a/tests/queries/0_stateless/02550_client_connections_credentials.sh +++ b/tests/queries/0_stateless/02550_client_connections_credentials.sh @@ -81,7 +81,13 @@ echo 'port' $CLICKHOUSE_CLIENT --config $CONFIG --connection test_port -q 'select tcpPort()' |& grep -F -o 'Connection refused (localhost:0).' $CLICKHOUSE_CLIENT --config $CONFIG --connection test_port --port $TEST_PORT -q 'select tcpPort()' echo 'secure' -$CLICKHOUSE_CLIENT --config $CONFIG --connection test_secure -q 'select tcpPort()' |& grep -c -F -o -e OPENSSL_internal:WRONG_VERSION_NUMBER -e 'tcp_secure protocol is disabled because poco library was built without NetSSL support.' + +if [ `uname -m` == 's390x' ]; then + $CLICKHOUSE_CLIENT --config $CONFIG --connection test_secure -q 'select tcpPort()' |& grep -c -F -o -e 'SSL routines::wrong version number' -e 'tcp_secure protocol is disabled because poco library was built without NetSSL support.' +else + $CLICKHOUSE_CLIENT --config $CONFIG --connection test_secure -q 'select tcpPort()' |& grep -c -F -o -e OPENSSL_internal:WRONG_VERSION_NUMBER -e 'tcp_secure protocol is disabled because poco library was built without NetSSL support.' +fi + echo 'database' $CLICKHOUSE_CLIENT --config $CONFIG --connection test_database -q 'select currentDatabase()' echo 'user' From 40062ca210ffbb433729f6c26cbc8080d641998b Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 24 Oct 2023 15:30:32 +0200 Subject: [PATCH 560/634] Fix test --- .../0_stateless/02902_diable_apply_deleted_mask.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02902_diable_apply_deleted_mask.reference b/tests/queries/0_stateless/02902_diable_apply_deleted_mask.reference index bb99676a056..9563c41e550 100644 --- a/tests/queries/0_stateless/02902_diable_apply_deleted_mask.reference +++ b/tests/queries/0_stateless/02902_diable_apply_deleted_mask.reference @@ -1,13 +1,13 @@ Normal SELECT does not see deleted rows 1 1 1 3 3 1 -With the setting enabled the deleted rows are visible +With the setting disabled the deleted rows are visible 0 0 0 1 1 1 2 2 0 3 3 1 4 4 0 -With the setting enabled the deleted rows are visible but still can be filterd out +With the setting disabled the deleted rows are visible but still can be filterd out 1 1 3 3 Read the data after OPTIMIZE, all deleted rwos should be physically removed now From fdabea61585fc49e81631d6f9b1b11c836b5619c Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 4 Oct 2023 15:50:27 +0200 Subject: [PATCH 561/634] clickhouse-client: don't show "0 rows in set" if it is zero and if exception was thrown --- src/Client/ClientBase.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a350654cdda..c80600434f0 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1950,9 +1950,10 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin if (is_interactive) { - std::cout << std::endl - << processed_rows << " row" << (processed_rows == 1 ? "" : "s") - << " in set. Elapsed: " << progress_indication.elapsedSeconds() << " sec. "; + std::cout << std::endl; + if (!server_exception) + std::cout << processed_rows << " row" << (processed_rows == 1 ? "" : "s") << " in set. "; + std::cout << "Elapsed: " << progress_indication.elapsedSeconds() << " sec. "; progress_indication.writeFinalProgress(); std::cout << std::endl << std::endl; } From 7e8dac6064ee1413e378b03f65714071300fc062 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 13 Oct 2023 09:39:57 +0200 Subject: [PATCH 562/634] Fix logic --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index c80600434f0..450922e1590 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1951,7 +1951,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin if (is_interactive) { std::cout << std::endl; - if (!server_exception) + if (!server_exception || processed_rows != 0) std::cout << processed_rows << " row" << (processed_rows == 1 ? "" : "s") << " in set. "; std::cout << "Elapsed: " << progress_indication.elapsedSeconds() << " sec. "; progress_indication.writeFinalProgress(); From 1380dc942ef35dfb443ceb46eabbf0e77c294cd4 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 4 Oct 2023 15:11:15 +0200 Subject: [PATCH 563/634] Add formatQuery function --- .../functions/string-functions.md | 44 +++++++ src/Functions/formatQuery.cpp | 110 ++++++++++++++++++ .../0_stateless/02882_formatQuery.reference | 9 ++ .../queries/0_stateless/02882_formatQuery.sql | 10 ++ 4 files changed, 173 insertions(+) create mode 100644 src/Functions/formatQuery.cpp create mode 100644 tests/queries/0_stateless/02882_formatQuery.reference create mode 100644 tests/queries/0_stateless/02882_formatQuery.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 47e16b67643..069c7a1379e 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1415,3 +1415,47 @@ Result: │ foo │ └────────────────────────────┘ ``` + +## formatQuery + +Returns a formatted version of the given SQL query. + +**Syntax** + +```sql +formatQuery(query) +``` + +**Arguments** + +- `query` - The SQL query to be formatted. [String](../../sql-reference/data-types/string.md) + +**Returned value** + +- The formatted query. [String](../../sql-reference/data-types/string.md). + +**Example** + +```sql +SELECT formatQuery('select 1;'); +``` + +Result: + +```result +┌─formatQuery('select 1;')─┐ +│ SELECT 1 │ +└──────────────────────────┘ +``` + +```sql +SELECT formatQuery('select 1;') == formatQuery('SeLecT 1'); +``` + +Result: + +```result +┌─equals(formatQuery('select 1;'), formatQuery('SeLecT 1'))─┐ +│ 1 │ +└───────────────────────────────────────────────────────────┘ +``` diff --git a/src/Functions/formatQuery.cpp b/src/Functions/formatQuery.cpp new file mode 100644 index 00000000000..6b5abca4a0e --- /dev/null +++ b/src/Functions/formatQuery.cpp @@ -0,0 +1,110 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +class FunctionFormatQuery : public IFunction +{ +public: + static constexpr auto name = "formatQuery"; + static FunctionPtr create(ContextPtr context) + { + const auto & settings = context->getSettings(); + return std::make_shared(settings.max_query_size, settings.max_parser_depth); + } + + FunctionFormatQuery(size_t max_query_size_, size_t max_parser_depth_) + : max_query_size(max_query_size_), max_parser_depth(max_parser_depth_) + { + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", arguments[0]->getName(), getName()); + + return arguments[0]; + } + + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const ColumnPtr column = arguments[0].column; + if (const ColumnString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnString::create(); + formatVector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); + return col_res; + } + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); + } + +private: + void formatQueryImpl(const char * begin, const char * end, ColumnString::Chars & output) const + { + ParserQuery parser{end}; + auto ast = parseQuery(parser, begin, end, {}, max_query_size, max_parser_depth); + WriteBufferFromVector buf(output, AppendModeTag{}); + formatAST(*ast, buf, /* hilite */ false); + buf.finalize(); + } + void formatVector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) const + { + const size_t size = offsets.size(); + res_offsets.resize(size); + res_data.reserve(data.size()); + + size_t prev_in_offset = 0; + for (size_t i = 0; i < size; ++i) + { + const auto * begin = reinterpret_cast(&data[prev_in_offset]); + const char * end = begin + offsets[i] - 1; + formatQueryImpl(begin, end, res_data); + res_offsets[i] = res_data.size() + 1; + prev_in_offset = offsets[i]; + } + } + size_t max_query_size; + size_t max_parser_depth; +}; + + +REGISTER_FUNCTION(formatQuery) +{ + factory.registerFunction(FunctionDocumentation{ + .description = "Returns a formatted version of the given SQL query.\n[example:simple]\n[example:compare]", + .syntax = "formatQuery(query)", + .arguments = {{"query", "The SQL query to be formatted. [String](../../sql-reference/data-types/string.md)"}}, + .returned_value = "The formatted query. [String](../../sql-reference/data-types/string.md).", + .examples{ + {"simple", "SELECT formatQuery('select 1;')", "SELECT 1"}, + {"compare", "SELECT formatQuery('select 1;') == formatQuery('SeLecT 1')", "1"}}, + .categories{"String"}}); +} +} diff --git a/tests/queries/0_stateless/02882_formatQuery.reference b/tests/queries/0_stateless/02882_formatQuery.reference new file mode 100644 index 00000000000..ce0f08bb15a --- /dev/null +++ b/tests/queries/0_stateless/02882_formatQuery.reference @@ -0,0 +1,9 @@ +SELECT 1 +SELECT 1 +SELECT 1 +SELECT 1 +1 +INSERT INTO tab FORMAT Values +CREATE TABLE default.no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +EXPLAIN SYNTAX\nSELECT\n CAST(1, \'INT\'),\n CEIL(1),\n CEILING(1),\n CHAR(49),\n CHAR_LENGTH(\'1\'),\n CHARACTER_LENGTH(\'1\'),\n COALESCE(1),\n CONCAT(\'1\', \'1\'),\n CORR(1, 1),\n COS(1),\n COUNT(1),\n COVAR_POP(1, 1),\n COVAR_SAMP(1, 1),\n DATABASE(),\n SCHEMA(),\n dateDiff(\'DAY\', toDate(\'2020-10-24\'), toDate(\'2019-10-24\')),\n EXP(1),\n FLATTEN([[1]]),\n FLOOR(1),\n FQDN(),\n GREATEST(1),\n IF(1, 1, 1),\n IFNULL(1, 1),\n LCASE(\'A\'),\n LEAST(1),\n LENGTH(\'1\'),\n LN(1),\n LOCATE(\'1\', \'1\'),\n LOG(1),\n LOG10(1),\n LOG2(1),\n LOWER(\'A\'),\n MAX(1),\n MID(\'123\', 1, 1),\n MIN(1),\n MOD(1, 1),\n NOT 1,\n NOW(),\n NOW64(),\n NULLIF(1, 1),\n PI(),\n position(\'123\', \'2\'),\n POW(1, 1),\n POWER(1, 1),\n RAND(),\n REPLACE(\'1\', \'1\', \'2\'),\n REVERSE(\'123\'),\n ROUND(1),\n SIN(1),\n SQRT(1),\n STDDEV_POP(1),\n STDDEV_SAMP(1),\n SUBSTR(\'123\', 2),\n substring(\'123\', 2),\n SUM(1),\n TAN(1),\n TANH(1),\n TRUNC(1),\n TRUNCATE(1),\n UCASE(\'A\'),\n UPPER(\'A\'),\n USER(),\n VAR_POP(1),\n VAR_SAMP(1),\n WEEK(toDate(\'2020-10-24\')),\n YEARWEEK(toDate(\'2020-10-24\'))\nFORMAT TSVRaw +1 diff --git a/tests/queries/0_stateless/02882_formatQuery.sql b/tests/queries/0_stateless/02882_formatQuery.sql new file mode 100644 index 00000000000..369523c3eb8 --- /dev/null +++ b/tests/queries/0_stateless/02882_formatQuery.sql @@ -0,0 +1,10 @@ +SELECT formatQuery('select 1;'); +SELECT formatQuery('select 1'); +SELECT formatQuery('SELECT 1;'); +SELECT formatQuery('SELECT 1'); +SELECT formatQuery('select 1;') == formatQuery('SeLecT 1'); +SELECT formatQuery('INSERT INTO tab VALUES (\'\') (\'test\')'); +SELECT formatQuery('CREATE TABLE default.no_prop_table(`some_column` UInt64) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192'); +SELECT formatQuery('EXPLAIN SYNTAX SELECT CAST(1 AS INT), CEIL(1), CEILING(1), CHAR(49), CHAR_LENGTH(\'1\'), CHARACTER_LENGTH(\'1\'), COALESCE(1), CONCAT(\'1\', \'1\'), CORR(1, 1), COS(1), COUNT(1), COVAR_POP(1, 1), COVAR_SAMP(1, 1), DATABASE(), SCHEMA(), DATEDIFF(\'DAY\', toDate(\'2020-10-24\'), toDate(\'2019-10-24\')), EXP(1), FLATTEN([[1]]), FLOOR(1), FQDN(), GREATEST(1), IF(1, 1, 1), IFNULL(1, 1), LCASE(\'A\'), LEAST(1), LENGTH(\'1\'), LN(1), LOCATE(\'1\', \'1\'), LOG(1), LOG10(1), LOG2(1), LOWER(\'A\'), MAX(1), MID(\'123\', 1, 1), MIN(1), MOD(1, 1), NOT(1), NOW(), NOW64(), NULLIF(1, 1), PI(), POSITION(\'123\', \'2\'), POW(1, 1), POWER(1, 1), RAND(), REPLACE(\'1\', \'1\', \'2\'), REVERSE(\'123\'), ROUND(1), SIN(1), SQRT(1), STDDEV_POP(1), STDDEV_SAMP(1), SUBSTR(\'123\', 2), SUBSTRING(\'123\', 2), SUM(1), TAN(1), TANH(1), TRUNC(1), TRUNCATE(1), UCASE(\'A\'), UPPER(\'A\'), USER(), VAR_POP(1), VAR_SAMP(1), WEEK(toDate(\'2020-10-24\')), YEARWEEK(toDate(\'2020-10-24\')) format TSVRaw;'); +SELECT normalizedQueryHash(formatQuery('select 1')) = normalizedQueryHash(formatQuery('SELECT 1')); +SELECT formatQuery('SEECTwrong'); -- { serverError SYNTAX_ERROR } From 6be7fd0fa12e3002313d6926f7e598a79ee9a557 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 6 Oct 2023 11:12:45 +0200 Subject: [PATCH 564/634] Ignore formatQuery word during spell check --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index b5d66f5e17b..7c4e02ca2e2 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1476,6 +1476,7 @@ fmtlib formatDateTime formatDateTimeInJoda formatDateTimeInJodaSyntax +formatQuery formatReadableDecimalSize formatReadableQuantity formatReadableSize From 457e7ba7bd77a16399c5ba96c664e9baa14bcc1c Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 6 Oct 2023 11:17:50 +0200 Subject: [PATCH 565/634] formatQuery: improve doc --- docs/en/sql-reference/functions/string-functions.md | 8 ++++---- src/Functions/formatQuery.cpp | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 069c7a1379e..2e47995350c 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1449,13 +1449,13 @@ Result: ``` ```sql -SELECT formatQuery('select 1;') == formatQuery('SeLecT 1'); +SELECT formatQuery('SeLecT 1'); ``` Result: ```result -┌─equals(formatQuery('select 1;'), formatQuery('SeLecT 1'))─┐ -│ 1 │ -└───────────────────────────────────────────────────────────┘ +┌─formatQuery('SeLecT 1')──┐ +│ SELECT 1 │ +└──────────────────────────┘ ``` diff --git a/src/Functions/formatQuery.cpp b/src/Functions/formatQuery.cpp index 6b5abca4a0e..b078bd5387b 100644 --- a/src/Functions/formatQuery.cpp +++ b/src/Functions/formatQuery.cpp @@ -98,13 +98,13 @@ private: REGISTER_FUNCTION(formatQuery) { factory.registerFunction(FunctionDocumentation{ - .description = "Returns a formatted version of the given SQL query.\n[example:simple]\n[example:compare]", + .description = "Returns a formatted version of the given SQL query.\n[example:simple]\n[example:camelcase]", .syntax = "formatQuery(query)", .arguments = {{"query", "The SQL query to be formatted. [String](../../sql-reference/data-types/string.md)"}}, .returned_value = "The formatted query. [String](../../sql-reference/data-types/string.md).", .examples{ {"simple", "SELECT formatQuery('select 1;')", "SELECT 1"}, - {"compare", "SELECT formatQuery('select 1;') == formatQuery('SeLecT 1')", "1"}}, + {"camelcase", "SELECT formatQuery('SeLecT 1')", "SELECT 1"}}, .categories{"String"}}); } } From 66c4a3bdbce488ae2d43230d5a26ba1e7e8e49a7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 24 Oct 2023 12:35:32 +0200 Subject: [PATCH 566/634] tests: more robust 02903_rmt_retriable_merge_exception - separate uuid extraction - add preliminary exit - disable for ordinary database - less number of attempts - add optimize_throw_if_noop and missing sync replica, to fix: 2023.10.24 15:18:35.925533 [ 640 ] {da7418c6-3d51-45bc-a0d0-4970bb0cdd51} test_3kgjgry1.rmt1 (d18afb81-3a4b-4c02-b281-5575dce2f440): Cannot select parts for optimization: Entry for part all_1_1_0 hasn't been read from the replication log yet (in partition all) - fix in case of ZooKeeper retries 2023.10.24 11:50:24.792511 [ 1437 ] {c39fd15b-e2e6-4291-9912-39fda75ebcd5} test_qxkzmigq.rmt1 (1c086c74-9ebe-495c-bbd2-87ab2d8ec43d): Renaming temporary part tmp_insert_all_1_1_0 to all_1_1_0 with tid (1, 1, 00000000-0000-0000-0000-000000000000). 2023.10.24 11:50:24.797320 [ 1437 ] {c39fd15b-e2e6-4291-9912-39fda75ebcd5} test_qxkzmigq.rmt1 (1c086c74-9ebe-495c-bbd2-87ab2d8ec43d) (Replicated OutputStream): ZooKeeperWithFaultInjection call FAILED: seed=17644626169032325693 func=tryMulti path=/clickhouse/zero_copy code=Session expired message=Fault injection before operation 2023.10.24 11:50:24.797536 [ 1437 ] {c39fd15b-e2e6-4291-9912-39fda75ebcd5} test_qxkzmigq.rmt1 (1c086c74-9ebe-495c-bbd2-87ab2d8ec43d): Undoing transaction. Rollbacking parts state to temporary and removing from working set: all_1_1_0. ... 2023.10.24 11:50:25.000349 [ 1437 ] {c39fd15b-e2e6-4291-9912-39fda75ebcd5} test_qxkzmigq.rmt1 (1c086c74-9ebe-495c-bbd2-87ab2d8ec43d): Renaming temporary part tmp_insert_all_1_1_0 to all_2_2_0 with tid (1, 1, 00000000-0000-0000-0000-000000000000). 2023.10.24 11:50:25.007477 [ 760 ] {} test_qxkzmigq.rmt1 (ReplicatedMergeTreeQueue): Insert entry queue-0000000000 to queue with type GET_PART with virtual parts [all_2_2_0] CI: - https://s3.amazonaws.com/clickhouse-test-reports/55944/da408df4a7296835897d7cef80d63f252df79b75/stateless_tests__tsan__s3_storage__[2_5].html - https://s3.amazonaws.com/clickhouse-test-reports/55944/da408df4a7296835897d7cef80d63f252df79b75/stateless_tests_flaky_check__asan_.html - https://s3.amazonaws.com/clickhouse-test-reports/55944/02fdd0513f7d413ce4ac39a70566855327ebfade/stateless_tests__aarch64_.html Signed-off-by: Azat Khuzhin --- .../02903_rmt_retriable_merge_exception.sh | 32 ++++++++++++------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh index 4ab84fc732c..074a3a6725e 100755 --- a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh +++ b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: no-ordinary-database +# Tag no-ordinary-database: requires UUID CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -18,23 +20,32 @@ $CLICKHOUSE_CLIENT -nm -q " insert into rmt1 values (1); insert into rmt1 values (2); + system sync replica rmt1; system stop pulling replication log rmt2; - optimize table rmt1 final settings alter_sync=0; -" + optimize table rmt1 final settings alter_sync=0, optimize_throw_if_noop=1; +" || exit 1 + +table_uuid=$($CLICKHOUSE_CLIENT -q "select uuid from system.tables where database = currentDatabase() and table = 'rmt1'") +if [[ -z $table_uuid ]]; then + echo "Table does not have UUID" >&2 + exit 1 +fi + +# NOTE: that part name can be different from all_0_1_1, in case of ZooKeeper retries +part_name='%' # wait while there be at least one 'No active replica has part all_0_1_1 or covering part' in logs -for _ in {0..1000}; do +for _ in {0..50}; do no_active_repilica_messages=$($CLICKHOUSE_CLIENT -nm -q " system flush logs; - with - (select uuid from system.tables where database = currentDatabase() and table = 'rmt1') as uuid_ + select count() from system.text_log where event_date >= yesterday() and event_time >= now() - 600 and ( - (logger_name = 'MergeTreeBackgroundExecutor' and message like '%{' || uuid_::String || '::all_0_1_1}%No active replica has part all_0_1_1 or covering part%') or - (logger_name = uuid_::String || '::all_0_1_1 (MergeFromLogEntryTask)' and message like '%No active replica has part all_0_1_1 or covering part%') + (logger_name = 'MergeTreeBackgroundExecutor' and message like '%{$table_uuid::$part_name}%No active replica has part $part_name or covering part%') or + (logger_name like '$table_uuid::$part_name (MergeFromLogEntryTask)' and message like '%No active replica has part $part_name or covering part%') ); ") if [[ $no_active_repilica_messages -gt 0 ]]; then @@ -47,16 +58,15 @@ done $CLICKHOUSE_CLIENT -nm -q " system start pulling replication log rmt2; system flush logs; - with - (select uuid from system.tables where database = currentDatabase() and table = 'rmt1') as uuid_ + select level, count() > 0 from system.text_log where event_date >= yesterday() and event_time >= now() - 600 and ( - (logger_name = 'MergeTreeBackgroundExecutor' and message like '%{' || uuid_::String || '::all_0_1_1}%No active replica has part all_0_1_1 or covering part%') or - (logger_name = uuid_::String || '::all_0_1_1 (MergeFromLogEntryTask)' and message like '%No active replica has part all_0_1_1 or covering part%') + (logger_name = 'MergeTreeBackgroundExecutor' and message like '%{$table_uuid::$part_name}%No active replica has part $part_name or covering part%') or + (logger_name like '$table_uuid::$part_name (MergeFromLogEntryTask)' and message like '%No active replica has part $part_name or covering part%') ) group by level; " From 3e0b6eb77bfa393d3ec265df31820ebb49f8fdf3 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 24 Oct 2023 16:10:43 +0200 Subject: [PATCH 567/634] formatQuery: validate args using helper function --- src/Functions/formatQuery.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Functions/formatQuery.cpp b/src/Functions/formatQuery.cpp index b078bd5387b..05a0aed985a 100644 --- a/src/Functions/formatQuery.cpp +++ b/src/Functions/formatQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -36,13 +37,11 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isString(arguments[0])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", arguments[0]->getName(), getName()); - - return arguments[0]; + FunctionArgumentDescriptors mandatory_args{{"query", &isString, nullptr, "String"}}; + validateFunctionArgumentTypes(*this, arguments, mandatory_args); + return arguments[0].type; } bool useDefaultImplementationForConstants() const override { return true; } From 2d2eaceb7782575c656cbd73a7c874e58e19dcea Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 24 Oct 2023 16:13:15 +0200 Subject: [PATCH 568/634] formatQuery: move doc --- .../functions/other-functions.md | 44 +++++++++++++++++++ .../functions/string-functions.md | 44 ------------------- src/Functions/formatQuery.cpp | 6 +-- 3 files changed, 46 insertions(+), 48 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 22a362cd39c..eb8b63434f9 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2794,3 +2794,47 @@ message Root uint32 column2 = 2; } ``` + +## formatQuery + +Returns a formatted version of the given SQL query. + +**Syntax** + +```sql +formatQuery(query) +``` + +**Arguments** + +- `query` - The SQL query to be formatted. [String](../../sql-reference/data-types/string.md) + +**Returned value** + +- The formatted query. [String](../../sql-reference/data-types/string.md). + +**Example** + +```sql +SELECT formatQuery('select 1;'); +``` + +Result: + +```result +┌─formatQuery('select 1;')─┐ +│ SELECT 1 │ +└──────────────────────────┘ +``` + +```sql +SELECT formatQuery('SeLecT 1'); +``` + +Result: + +```result +┌─formatQuery('SeLecT 1')──┐ +│ SELECT 1 │ +└──────────────────────────┘ +``` diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 2e47995350c..47e16b67643 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1415,47 +1415,3 @@ Result: │ foo │ └────────────────────────────┘ ``` - -## formatQuery - -Returns a formatted version of the given SQL query. - -**Syntax** - -```sql -formatQuery(query) -``` - -**Arguments** - -- `query` - The SQL query to be formatted. [String](../../sql-reference/data-types/string.md) - -**Returned value** - -- The formatted query. [String](../../sql-reference/data-types/string.md). - -**Example** - -```sql -SELECT formatQuery('select 1;'); -``` - -Result: - -```result -┌─formatQuery('select 1;')─┐ -│ SELECT 1 │ -└──────────────────────────┘ -``` - -```sql -SELECT formatQuery('SeLecT 1'); -``` - -Result: - -```result -┌─formatQuery('SeLecT 1')──┐ -│ SELECT 1 │ -└──────────────────────────┘ -``` diff --git a/src/Functions/formatQuery.cpp b/src/Functions/formatQuery.cpp index 05a0aed985a..7c574b701d6 100644 --- a/src/Functions/formatQuery.cpp +++ b/src/Functions/formatQuery.cpp @@ -101,9 +101,7 @@ REGISTER_FUNCTION(formatQuery) .syntax = "formatQuery(query)", .arguments = {{"query", "The SQL query to be formatted. [String](../../sql-reference/data-types/string.md)"}}, .returned_value = "The formatted query. [String](../../sql-reference/data-types/string.md).", - .examples{ - {"simple", "SELECT formatQuery('select 1;')", "SELECT 1"}, - {"camelcase", "SELECT formatQuery('SeLecT 1')", "SELECT 1"}}, - .categories{"String"}}); + .examples{{"simple", "SELECT formatQuery('select 1;')", "SELECT 1"}, {"camelcase", "SELECT formatQuery('SeLecT 1')", "SELECT 1"}}, + .categories{"Other"}}); } } From 010cc6918acb81bfaa9b2839fa022f14c05e549b Mon Sep 17 00:00:00 2001 From: xuzifu666 <1206332514@qq.com> Date: Tue, 24 Oct 2023 23:02:54 +0800 Subject: [PATCH 569/634] Add get_direct_children_number command to keeper-client (#55898) --- .../utilities/clickhouse-keeper-client.md | 1 + programs/keeper-client/Commands.cpp | 21 +++++++++++++++++++ programs/keeper-client/Commands.h | 14 +++++++++++++ programs/keeper-client/KeeperClient.cpp | 1 + tests/integration/helpers/keeper_utils.py | 3 +++ tests/integration/test_keeper_client/test.py | 12 +++++++++++ 6 files changed, 52 insertions(+) diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md index 67aa4babb3f..d06d88598a7 100644 --- a/docs/en/operations/utilities/clickhouse-keeper-client.md +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -55,6 +55,7 @@ keeper foo bar - `rmr ` -- Recursively deletes path. Confirmation required - `flwc ` -- Executes four-letter-word command - `help` -- Prints this message +- `get_direct_children_number [path]` -- Get numbers of direct children nodes under a specific path - `get_all_children_number [path]` -- Get all numbers of children nodes under a specific path - `get_stat [path]` -- Returns the node's stat (default `.`) - `find_super_nodes [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 543ea8f4449..ab9252dd62e 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -475,6 +475,27 @@ void FourLetterWordCommand::execute(const ASTKeeperQuery * query, KeeperClient * std::cout << client->executeFourLetterCommand(query->args[0].safeGet()) << "\n"; } +bool GetDirectChildrenNumberCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String path; + if (!parseKeeperPath(pos, expected, path)) + path = "."; + + node->args.push_back(std::move(path)); + + return true; +} + +void GetDirectChildrenNumberCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + auto path = client->getAbsolutePath(query->args[0].safeGet()); + + Coordination::Stat stat; + client->zookeeper->get(path, &stat); + + std::cout << stat.numChildren << "\n"; +} + bool GetAllChildrenNumberCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { String path; diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index 45a7c85266d..c6dd731fb3b 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -238,6 +238,20 @@ class FourLetterWordCommand : public IKeeperClientCommand String getHelpMessage() const override { return "{} -- Executes four-letter-word command"; } }; +class GetDirectChildrenNumberCommand : public IKeeperClientCommand +{ + String getName() const override { return "get_direct_children_number"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override + { + return "{} [path] -- Get numbers of direct children nodes under a specific path"; + } +}; + class GetAllChildrenNumberCommand : public IKeeperClientCommand { String getName() const override { return "get_all_children_number"; } diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index f96975f5ab1..7ed4499efbd 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -207,6 +207,7 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) std::make_shared(), std::make_shared(), std::make_shared(), + std::make_shared(), std::make_shared(), }); diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 478bb5ebf72..79d498b909f 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -143,6 +143,9 @@ class KeeperClient(object): def find_super_nodes(self, threshold: int, timeout: float = 60.0) -> str: return self.execute_query(f"find_super_nodes {threshold}", timeout) + def get_direct_children_number(self, path: str, timeout: float = 60.0) -> str: + return self.execute_query(f"get_direct_children_number {path}", timeout) + def get_all_children_number(self, path: str, timeout: float = 60.0) -> str: return self.execute_query(f"get_all_children_number {path}", timeout) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 92b5b95dc50..9d7a46001e7 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -218,6 +218,18 @@ def test_quoted_argument_parsing(client: KeeperClient): assert client.get(node_path) == "value4 with some whitespace" +def get_direct_children_number(client: KeeperClient): + client.touch("/get_direct_children_number") + client.touch("/get_direct_children_number/1") + client.touch("/get_direct_children_number/1/1") + client.touch("/get_direct_children_number/1/2") + client.touch("/get_direct_children_number/2") + client.touch("/get_direct_children_number/2/1") + client.touch("/get_direct_children_number/2/2") + + assert client.get_direct_children_number("/get_direct_children_number") == "2" + + def test_get_all_children_number(client: KeeperClient): client.touch("/test_get_all_children_number") client.touch("/test_get_all_children_number/1") From f94c33586c8fe322220365634506b1f919ba3974 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 24 Oct 2023 15:14:20 +0000 Subject: [PATCH 570/634] Fix issue #42966 --- .../functions/type-conversion-functions.md | 6 +- src/DataTypes/IDataType.h | 13 +++++ src/Functions/FunctionSnowflake.h | 55 ++++++++++++------- .../01942_dateTimeToSnowflake.reference | 12 ++-- .../0_stateless/01942_dateTimeToSnowflake.sql | 48 +++++++++++----- .../0_stateless/01942_snowflakeToDateTime.sql | 12 ++-- 6 files changed, 97 insertions(+), 49 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 0df72b5818c..e037c00e2eb 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1987,7 +1987,7 @@ Extracts time from [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as **Syntax** ``` sql -snowflakeToDateTime(value [, time_zone]) +snowflakeToDateTime(value[, time_zone]) ``` **Parameters** @@ -2023,7 +2023,7 @@ Extracts time from [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as **Syntax** ``` sql -snowflakeToDateTime64(value [, time_zone]) +snowflakeToDateTime64(value[, time_zone]) ``` **Parameters** @@ -2033,7 +2033,7 @@ snowflakeToDateTime64(value [, time_zone]) **Returned value** -- Input value converted to the [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) data type. +- Input value converted to the [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) data type with millisecond precision (scale = 3). **Example** diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index fb0a015821d..d74c4ea8cec 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -452,10 +452,23 @@ inline bool isObject(const T & data_type) { return WhichDataType(data_type).isOb template inline bool isUInt8(const T & data_type) { return WhichDataType(data_type).isUInt8(); } template +inline bool isUInt16(const T & data_type) { return WhichDataType(data_type).isUInt16(); } +template +inline bool isUInt32(const T & data_type) { return WhichDataType(data_type).isUInt32(); } +template inline bool isUInt64(const T & data_type) { return WhichDataType(data_type).isUInt64(); } template inline bool isUnsignedInteger(const T & data_type) { return WhichDataType(data_type).isUInt(); } +template +inline bool isInt8(const T & data_type) { return WhichDataType(data_type).isInt8(); } +template +inline bool isInt16(const T & data_type) { return WhichDataType(data_type).isInt16(); } +template +inline bool isInt32(const T & data_type) { return WhichDataType(data_type).isInt32(); } +template +inline bool isInt64(const T & data_type) { return WhichDataType(data_type).isInt64(); } + template inline bool isInteger(const T & data_type) { diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index b57e21e9a50..42b71c789f0 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -4,10 +4,12 @@ #include #include #include +#include #include #include #include #include +#include #include #include @@ -40,14 +42,15 @@ public: String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } - bool isVariadic() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isDateTime(arguments[0].type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The only argument for function {} must be DateTime", name); + FunctionArgumentDescriptors args{ + {"value", &isDateTime, nullptr, "DateTime"} + }; + validateFunctionArgumentTypes(*this, arguments, args); return std::make_shared(); } @@ -88,13 +91,15 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.empty() || arguments.size() > 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", name); + FunctionArgumentDescriptors mandatory_args{ + {"value", &isInt64, nullptr, "Int64"} + }; + FunctionArgumentDescriptors optional_args{ + {"time_zone", &isString, nullptr, "String"} + }; + validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); - if (!typeid_cast(arguments[0].type.get())) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The first argument for function {} must be Int64", name); - - std::string timezone; + String timezone; if (arguments.size() == 2) timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, allow_nonconst_timezone_arguments); @@ -141,14 +146,15 @@ public: String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } - bool isVariadic() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isDateTime64(arguments[0].type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The only argument for function {} must be DateTime64", name); + FunctionArgumentDescriptors args{ + {"value", &isDateTime64, nullptr, "DateTime64"} + }; + validateFunctionArgumentTypes(*this, arguments, args); return std::make_shared(); } @@ -156,14 +162,21 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const auto & src = arguments[0]; - const auto & src_column = *src.column; + const auto & src_column = *src.column; auto res_column = ColumnInt64::create(input_rows_count); auto & res_data = res_column->getData(); const auto & src_data = typeid_cast &>(src_column).getData(); + + /// timestamps in snowflake-ids are millisecond-based, convert input to milliseconds + UInt32 src_scale = getDecimalScale(*arguments[0].type); + Int64 multiplier_msec = DecimalUtils::scaleMultiplier(3); + Int64 multiplier_src = DecimalUtils::scaleMultiplier(src_scale); + auto factor = multiplier_msec / static_cast(multiplier_src); + for (size_t i = 0; i < input_rows_count; ++i) - res_data[i] = (src_data[i] - snowflake_epoch) << time_shift; + res_data[i] = static_cast(src_data[i] * factor - snowflake_epoch) << time_shift; return res_column; } @@ -190,13 +203,15 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.empty() || arguments.size() > 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", name); + FunctionArgumentDescriptors mandatory_args{ + {"value", &isInt64, nullptr, "Int64"} + }; + FunctionArgumentDescriptors optional_args{ + {"time_zone", &isString, nullptr, "String"} + }; + validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); - if (!typeid_cast(arguments[0].type.get())) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The first argument for function {} must be Int64", name); - - std::string timezone; + String timezone; if (arguments.size() == 2) timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, allow_nonconst_timezone_arguments); diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference b/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference index dfca3a10eeb..4f51ec55e89 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference @@ -1,6 +1,6 @@ -const column -2021-08-15 18:57:56 1426860702823350272 -2021-08-15 18:57:56.492 1426860704886947840 -non-const column -2021-08-15 18:57:56 1426860702823350272 -2021-08-15 18:57:56.492 1426860704886947840 +-- const / non-const inputs +2021-08-15 18:57:56 1426860702823350272 1426860702823350272 +2021-08-15 18:57:56.492 1426860704886947840 1426860704886947840 +-- different DateTime64 scales +1426981498778550272 1426981500456271872 1426981500833759232 1426981500842147840 1426981500842147840 +1 1 1 1 0 diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql index 047d8be7be5..62371e901a2 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql @@ -1,23 +1,43 @@ -- Error cases -SELECT dateTimeToSnowflake(); -- {serverError 42} -SELECT dateTime64ToSnowflake(); -- {serverError 42} +SELECT dateTimeToSnowflake(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT dateTime64ToSnowflake(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} -SELECT dateTimeToSnowflake('abc'); -- {serverError 43} -SELECT dateTime64ToSnowflake('abc'); -- {serverError 43} +SELECT dateTimeToSnowflake('abc'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT dateTime64ToSnowflake('abc'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -SELECT dateTimeToSnowflake('abc', 123); -- {serverError 42} -SELECT dateTime64ToSnowflake('abc', 123); -- {serverError 42} +SELECT dateTimeToSnowflake('abc', 123); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT dateTime64ToSnowflake('abc', 123); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} + +SELECT '-- const / non-const inputs'; -SELECT 'const column'; WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt -SELECT dt, dateTimeToSnowflake(dt); +SELECT dt, dateTimeToSnowflake(dt), materialize(dateTimeToSnowflake(dt)); WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS dt64 -SELECT dt64, dateTime64ToSnowflake(dt64); +SELECT dt64, dateTime64ToSnowflake(dt64), materialize(dateTime64ToSnowflake(dt64)); -SELECT 'non-const column'; -WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS x -SELECT materialize(x) as dt, dateTimeToSnowflake(dt);; +SELECT '-- different DateTime64 scales'; -WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS x -SELECT materialize(x) as dt64, dateTime64ToSnowflake(dt64); +WITH toDateTime64('2021-08-15 18:57:56.492', 0, 'UTC') AS dt64_0, + toDateTime64('2021-08-15 18:57:56.492', 1, 'UTC') AS dt64_1, + toDateTime64('2021-08-15 18:57:56.492', 2, 'UTC') AS dt64_2, + toDateTime64('2021-08-15 18:57:56.492', 3, 'UTC') AS dt64_3, + toDateTime64('2021-08-15 18:57:56.492', 4, 'UTC') AS dt64_4 +SELECT dateTime64ToSnowflake(dt64_0), + dateTime64ToSnowflake(dt64_1), + dateTime64ToSnowflake(dt64_2), + dateTime64ToSnowflake(dt64_3), + dateTime64ToSnowflake(dt64_4); + +-- DateTime64-to-Snowflake-to-DateTime64 is idempotent *if* the scale is <=3 (millisecond +-- precision) +WITH now64(0) AS dt64_0, + now64(1) AS dt64_1, + now64(2) AS dt64_2, + now64(3) AS dt64_3, + now64(4) AS dt64_4 +SELECT snowflakeToDateTime64(dateTime64ToSnowflake(dt64_0)) == dt64_0, + snowflakeToDateTime64(dateTime64ToSnowflake(dt64_1)) == dt64_1, + snowflakeToDateTime64(dateTime64ToSnowflake(dt64_2)) == dt64_2, + snowflakeToDateTime64(dateTime64ToSnowflake(dt64_3)) == dt64_3, + snowflakeToDateTime64(dateTime64ToSnowflake(dt64_4)) == dt64_4; diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql index 0092eca848c..f1a50dd370d 100644 --- a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql @@ -1,12 +1,12 @@ -- -- Error cases -SELECT snowflakeToDateTime(); -- {serverError 42} -SELECT snowflakeToDateTime64(); -- {serverError 42} +SELECT snowflakeToDateTime(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT snowflakeToDateTime64(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} -SELECT snowflakeToDateTime('abc'); -- {serverError 43} -SELECT snowflakeToDateTime64('abc'); -- {serverError 43} +SELECT snowflakeToDateTime('abc'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeToDateTime64('abc'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} -SELECT snowflakeToDateTime('abc', 123); -- {serverError 43} -SELECT snowflakeToDateTime64('abc', 123); -- {serverError 43} +SELECT snowflakeToDateTime('abc', 123); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeToDateTime64('abc', 123); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} SELECT 'const column'; WITH From 0e176fa0edfdbf0343b2d69dac44b5b0f840a88d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 24 Oct 2023 19:39:25 +0300 Subject: [PATCH 571/634] Fixed code review issues --- base/base/defines.h | 1 + src/Common/SharedLockGuard.h | 19 ++++------- .../{MutexUtils.h => SharedMutexHelper.h} | 34 ++++++++++++++++++- src/Interpreters/Context.cpp | 2 +- src/Interpreters/Context.h | 2 +- 5 files changed, 43 insertions(+), 15 deletions(-) rename src/Common/{MutexUtils.h => SharedMutexHelper.h} (64%) diff --git a/base/base/defines.h b/base/base/defines.h index 2019200c120..d852f6b9f63 100644 --- a/base/base/defines.h +++ b/base/base/defines.h @@ -178,6 +178,7 @@ # define TSA_ACQUIRE_SHARED(...) # define TSA_TRY_ACQUIRE_SHARED(...) # define TSA_RELEASE_SHARED(...) +# define TSA_SCOPED_LOCKABLE # define TSA_SUPPRESS_WARNING_FOR_READ(x) (x) # define TSA_SUPPRESS_WARNING_FOR_WRITE(x) (x) diff --git a/src/Common/SharedLockGuard.h b/src/Common/SharedLockGuard.h index 9b71528db1a..9186dd27596 100644 --- a/src/Common/SharedLockGuard.h +++ b/src/Common/SharedLockGuard.h @@ -1,30 +1,25 @@ #pragma once -#include #include namespace DB { -/// Same as std::lock_guard but for shared mutex +/** SharedLockGuard provide RAII-style locking mechanism for acquiring shared ownership of the implementation + * of the SharedLockable concept (for example std::shared_mutex) supplied as the constructor argument. + * On construction it acquires shared ownership using `lock_shared` method. + * On desruction shared ownership is released using `unlock_shared` method. + */ template class TSA_SCOPED_LOCKABLE SharedLockGuard { public: - explicit SharedLockGuard(Mutex & mutex_) TSA_ACQUIRE_SHARED(mutex_) : mutex(mutex_) - { - mutex_.lock_shared(); - } + explicit SharedLockGuard(Mutex & mutex_) TSA_ACQUIRE_SHARED(mutex_) : mutex(mutex_) { mutex_.lock_shared(); } - ~SharedLockGuard() TSA_RELEASE() - { - mutex.unlock_shared(); - } + ~SharedLockGuard() TSA_RELEASE() { mutex.unlock_shared(); } private: Mutex & mutex; }; } - - diff --git a/src/Common/MutexUtils.h b/src/Common/SharedMutexHelper.h similarity index 64% rename from src/Common/MutexUtils.h rename to src/Common/SharedMutexHelper.h index e50dc3a9bab..8dddaab6c78 100644 --- a/src/Common/MutexUtils.h +++ b/src/Common/SharedMutexHelper.h @@ -7,6 +7,38 @@ namespace DB { +/** SharedMutexHelper class allows to inject specific logic when underlying shared mutex is acquired + * and released. + * + * Example: + * + * class ProfileSharedMutex : public SharedMutexHelper + * { + * public: + * size_t getLockCount() const { return lock_count; } + * + * size_t getSharedLockCount() const { return shared_lock_count; } + * + * private: + * using Base = SharedMutexHelper; + * friend class SharedMutexHelper; + * + * void lockImpl() + * { + * ++lock_count; + * Base::lockImpl(); + * } + * + * void lockSharedImpl() + * { + * ++shared_lock_count; + * Base::lockSharedImpl(); + * } + * + * std::atomic lock_count = 0; + * std::atomic shared_lock_count = 0; + * }; + */ template class TSA_CAPABILITY("SharedMutexHelper") SharedMutexHelper { @@ -51,7 +83,7 @@ protected: void tryLockImpl() TSA_NO_THREAD_SAFETY_ANALYSIS { - mutex.TryLock(); + mutex.try_lock(); } void unlockImpl() TSA_NO_THREAD_SAFETY_ANALYSIS diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d1d18509310..cec24f20cd1 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3596,7 +3596,7 @@ std::shared_ptr Context::getFilesystemCacheLog() const std::shared_ptr Context::getS3QueueLog() const { - auto lock = getGlobalSharedLock(); + SharedLockGuard lock(shared->mutex); if (!shared->system_logs) return {}; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index fef34f4f2ac..ae04151e47e 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include #include From f8348681aadcda50b7d1623e7cd7678088b58d8c Mon Sep 17 00:00:00 2001 From: AN Date: Tue, 24 Oct 2023 19:47:27 +0300 Subject: [PATCH 572/634] Update architecture.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Russian language fixes (hyphens in compounds, tautologies etc.), term unification («column» → «столбец», а было то «столбец», то «колонка», «строка/столбец» is traditional translation for ‘row/column‘ in translations (see translations of Date) --- docs/ru/development/architecture.md | 62 ++++++++++++++--------------- 1 file changed, 31 insertions(+), 31 deletions(-) diff --git a/docs/ru/development/architecture.md b/docs/ru/development/architecture.md index 1a6fbade06f..9c4a503a276 100644 --- a/docs/ru/development/architecture.md +++ b/docs/ru/development/architecture.md @@ -6,21 +6,21 @@ sidebar_label: "Обзор архитектуры ClickHouse" # Обзор архитектуры ClickHouse {#overview-of-clickhouse-architecture} -ClickHouse - полноценная колоночная СУБД. Данные хранятся в колонках, а в процессе обработки - в массивах (векторах или фрагментах (chunk’ах) колонок). По возможности операции выполняются на массивах, а не на индивидуальных значениях. Это называется “векторизованное выполнения запросов” (vectorized query execution), и помогает снизить стоимость фактической обработки данных. +ClickHouse — полноценная столбцовая СУБД. Данные хранятся в столбцах, а в процессе обработки — в массивах (векторах или фрагментах столбцов — chunks). По возможности операции выполняются на массивах, а не на индивидуальных значениях. Это называется “векторизованное выполнения запросов” (vectorized query execution), и помогает снизить стоимость фактической обработки данных. -> Эта идея не нова. Такой подход использовался в `APL` (A programming language, 1957) и его потомках: `A +` (диалект `APL`), `J` (1990), `K` (1993) и `Q` (язык программирования Kx Systems, 2003). Программирование на массивах (Array programming) используется в научных вычислительных системах. Эта идея не является чем-то новым и для реляционных баз данных: например, она используется в системе `VectorWise` (так же известной как Actian Vector Analytic Database от Actian Corporation). +> Эта идея не нова. Такой подход использовался в языке `APL` (A programming language, 1957) и его потомках: `A +` (диалект `APL`), `J` (1990), `K` (1993) и `Q` (язык программирования Kx Systems, 2003). Программирование на массивах (array programming) используется в научных вычислительных системах. Эта идея не является чем-то новым и для реляционных баз данных: например, она используется в системе `VectorWise` (так же известной как Actian Vector Analytic Database от Actian Corporation). -Существует два различных подхода для увеличения скорости обработки запросов: выполнение векторизованного запроса и генерация кода во время выполнения (runtime code generation). В последнем случае код генерируется на лету для каждого типа запроса, удаляя все косвенные и динамические обращения. Ни один из этих подходов не имеет явного преимущества. Генерация кода во время выполнения выигрывает, если объединяет большое число операций, таким образом полностью используя вычислительные блоки и конвейер CPU. Выполнение векторизованного запроса может быть менее практично потому, что задействует временные векторы данных, которые должны быть записаны и прочитаны из кэша. Если временные данные не помещаются в L2 кэш, будут проблемы. С другой стороны выполнение векторизованного запроса упрощает использование SIMD инструкций CPU. [Научная работа](http://15721.courses.cs.cmu.edu/spring2016/papers/p5-sompolski.pdf) наших друзей показывает преимущества сочетания обоих подходов. ClickHouse использует выполнение векторизованного запроса и имеет ограниченную начальную поддержку генерации кода во время выполнения. +Существует два различных подхода для увеличения скорости обработки запросов: выполнение векторизованного запроса и генерация кода во время выполнения (runtime code generation). В последнем случае код генерируется на лету для каждого типа запроса, и удаляются все косвенные и динамические обращения. Ни один из этих подходов не имеет явного преимущества. Генерация кода во время выполнения выигрывает, если объединяет большое число операций, таким образом полностью используя вычислительные блоки и конвейер CPU. Выполнение векторизованного запроса может быть менее практично потому, что задействует временные векторы данных, которые должны быть записаны и прочитаны из кэша. Если временные данные не помещаются в L2-кэш, будут проблемы. С другой стороны выполнение векторизованного запроса упрощает использование SIMD-инструкций CPU. [Научная работа](http://15721.courses.cs.cmu.edu/spring2016/papers/p5-sompolski.pdf) наших друзей показывает преимущества сочетания обоих подходов. ClickHouse использует выполнение векторизованного запроса и имеет ограниченную начальную поддержку генерации кода во время выполнения. -## Колонки {#columns} +## Столбцы {#columns} -Для представления столбцов в памяти (фактически, фрагментов столбцов) используется интерфейс `IColumn`. Интерфейс предоставляет вспомогательные методы для реализации различных реляционных операторов. Почти все операции иммутабельные: они не изменяют оригинальных колонок, а создают новую с измененными значениями. Например, метод `IColumn :: filter` принимает фильтр - набор байт. Он используется для реляционных операторов `WHERE` и `HAVING`. Другой пример: метод `IColumn :: permute` используется для поддержки `ORDER BY`, метод `IColumn :: cut` - `LIMIT` и т. д. +Для представления столбцов в памяти (фактически, фрагментов столбцов) используется интерфейс `IColumn`. Интерфейс предоставляет вспомогательные методы для реализации различных реляционных операторов. Почти все операции не изменяют данные (immutable): они не изменяют содержимое столбцов, а создают новые с изменёнными значениями. Например, метод `IColumn :: filter` принимает фильтр — набор байтов. Он используется для реляционных операторов `WHERE` и `HAVING`. Другой пример: метод `IColumn :: permute` используется для поддержки `ORDER BY`, метод `IColumn :: cut` — `LIMIT` и т. д. -Различные реализации `IColumn` (`ColumnUInt8`, `ColumnString` и т. д.) отвечают за распределение данных колонки в памяти. Для колонок целочисленного типа это один смежный массив, такой как `std :: vector`. Для колонок типа `String` и `Array` это два вектора: один для всех элементов массивов, располагающихся смежно, второй для хранения смещения до начала каждого массива. Также существует реализация `ColumnConst`, в которой хранится только одно значение в памяти, но выглядит как колонка. +Различные реализации `IColumn` (`ColumnUInt8`, `ColumnString` и т. д.) отвечают за распределение данных столбца в памяти. Для столбцов целочисленного типа — это один смежный массив, такой как `std :: vector`. Для столбцов типа `String` и `Array` — это два вектора: один для всех элементов массивов, располагающихся смежно, второй для хранения смещения до начала каждого массива. Также существует реализация `ColumnConst`, в которой хранится только одно значение в памяти, но выглядит как столбец. ## Поля {#field} -Тем не менее, можно работать и с индивидуальными значениями. Для представления индивидуальных значений используется `Поле` (`Field`). `Field` - размеченное объединение `UInt64`, `Int64`, `Float64`, `String` и `Array`. `IColumn` имеет метод `оператор []` для получения значения по индексу n как `Field`, а также метод insert для добавления `Field` в конец колонки. Эти методы не очень эффективны, так как требуют временных объектов `Field`, представляющих индивидуальное значение. Есть более эффективные методы, такие как `insertFrom`, `insertRangeFrom` и т.д. +Тем не менее, можно работать и с индивидуальными значениями. Для представления индивидуальных значений используется `Поле` (`Field`). `Field` — размеченное объединение `UInt64`, `Int64`, `Float64`, `String` и `Array`. `IColumn` имеет метод `оператор []` для получения значения по индексу n как `Field`, а также метод insert для добавления `Field` в конец колонки. Эти методы не очень эффективны, так как требуют временных объектов `Field`, представляющих индивидуальное значение. Есть более эффективные методы, такие как `insertFrom`, `insertRangeFrom` и т.д. `Field` не несет в себе достаточно информации о конкретном типе данных в таблице. Например, `UInt8`, `UInt16`, `UInt32` и `UInt64` в `Field` представлены как `UInt64`. @@ -28,12 +28,12 @@ ClickHouse - полноценная колоночная СУБД. Данные `IColumn` предоставляет методы для общих реляционных преобразований данных, но они не отвечают всем потребностям. Например, `ColumnUInt64` не имеет метода для вычисления суммы двух столбцов, а `ColumnString` не имеет метода для запуска поиска по подстроке. Эти бесчисленные процедуры реализованы вне `IColumn`. -Различные функции на колонках могут быть реализованы обобщенным, неэффективным путем, используя `IColumn` методы для извлечения значений `Field`, или специальным путем, используя знания о внутреннем распределение данных в памяти в конкретной реализации `IColumn`. Для этого функции приводятся к конкретному типу `IColumn` и работают напрямую с его внутренним представлением. Например, в `ColumnUInt64` есть метод `getData`, который возвращает ссылку на внутренний массив, чтение и заполнение которого, выполняется отдельной процедурой напрямую. Фактически, мы имеем "дырявые абстракции", обеспечивающие эффективные специализации различных процедур. +Различные функции на столбцах могут быть реализованы обобщённым, неэффективным путем, используя `IColumn`-методы для извлечения значений `Field`, или специальным путем, используя знания о внутреннем распределение данных в памяти в конкретной реализации `IColumn`. Для этого функции приводятся к конкретному типу `IColumn` и работают напрямую с его внутренним представлением. Например, в `ColumnUInt64` есть метод `getData`, который возвращает ссылку на внутренний массив, чтение и заполнение которого, выполняется отдельной процедурой напрямую. Фактически, мы имеем “дырявые абстракции”, обеспечивающие эффективные специализации различных процедур. ## Типы данных (Data Types) {#data_types} -`IDataType` отвечает за сериализацию и десериализацию - чтение и запись фрагментов колонок или индивидуальных значений в бинарном или текстовом формате. -`IDataType` точно соответствует типам данных в таблицах - `DataTypeUInt32`, `DataTypeDateTime`, `DataTypeString` и т. д. +`IDataType` отвечает за сериализацию и десериализацию — чтение и запись фрагментов столбцов или индивидуальных значений в бинарном или текстовом формате. +`IDataType` точно соответствует типам данных в таблицах — `DataTypeUInt32`, `DataTypeDateTime`, `DataTypeString` и т. д. `IDataType` и `IColumn` слабо связаны друг с другом. Различные типы данных могут быть представлены в памяти с помощью одной реализации `IColumn`. Например, и `DataTypeUInt32`, и `DataTypeDateTime` в памяти представлены как `ColumnUInt32` или `ColumnConstUInt32`. В добавок к этому, один тип данных может быть представлен различными реализациями `IColumn`. Например, `DataTypeUInt8` может быть представлен как `ColumnUInt8` и `ColumnConstUInt8`. @@ -43,11 +43,11 @@ ClickHouse - полноценная колоночная СУБД. Данные ## Блоки (Block) {#block} -`Block` это контейнер, который представляет фрагмент (chunk) таблицы в памяти. Это набор троек - `(IColumn, IDataType, имя колонки)`. В процессе выполнения запроса, данные обрабатываются `Block`-ами. Если у нас есть `Block`, значит у нас есть данные (в объекте `IColumn`), информация о типе (в `IDataType`), которая говорит нам, как работать с колонкой, и имя колонки (оригинальное имя колонки таблицы или служебное имя, присвоенное для получения промежуточных результатов вычислений). +`Block` — это контейнер, который представляет фрагмент (chunk) таблицы в памяти. Это набор троек — `(IColumn, IDataType, имя столбца)`. В процессе выполнения запроса, данные обрабатываются блоками (`Block`). Если есть `Block`, значит у нас есть данные (в объекте `IColumn`), информация о типе (в `IDataType`), которая говорит, как работать со столбцов, и имя столбца (оригинальное имя столбца таблицы или служебное имя, присвоенное для получения промежуточных результатов вычислений). -При вычислении некоторой функции на колонках в блоке мы добавляем еще одну колонку с результатами в блок, не трогая колонки аргументов функции, потому что операции иммутабельные. Позже ненужные колонки могут быть удалены из блока, но не модифицированы. Это удобно для устранения общих подвыражений. +При вычислении некоторой функции на столбцах в блоке добавляется ещё один столбец с результатами в блок, не трогая колонки аргументов функции, потому что операции иммутабельные. Позже ненужные столбцы могут быть удалены из блока, но не модифицированы. Это удобно для устранения общих подвыражений. -Блоки создаются для всех обработанных фрагментов данных. Напоминаем, что одни и те же типы вычислений, имена колонок и типы переиспользуются в разных блоках и только данные колонок изменяются. Лучше разделить данные и заголовок блока потому, что в блоках маленького размера мы имеем большой оверхэд по временным строкам при копировании умных указателей (`shared_ptrs`) и имен колонок. +Блоки создаются для всех обработанных фрагментов данных. Напоминаем, что одни и те же типы вычислений, имена столбцов и типы переиспользуются в разных блоках и только данные колонок изменяются. Лучше разделить данные и заголовок блока потому, что в блоках маленького размера мы имеем большой оверхэд по временным строкам при копировании умных указателей (`shared_ptrs`) и имен столбцов. ## Потоки блоков (Block Streams) {#block-streams} @@ -73,13 +73,13 @@ ClickHouse - полноценная колоночная СУБД. Данные ## I/O {#io} -Для байт-ориентированных ввода/вывода существуют абстрактные классы `ReadBuffer` и `WriteBuffer`. Они используются вместо C++ `iostream`. Не волнуйтесь: каждый зрелый проект C++ использует что-то другое вместо `iostream` по уважительным причинам. +Для байт-ориентированного ввода-вывода существуют абстрактные классы `ReadBuffer` и `WriteBuffer`. Они используются вместо `iostream`. Не волнуйтесь: каждый зрелый проект C++ использует что-то другое вместо `iostream` по уважительным причинам. `ReadBuffer` и `WriteBuffer` это просто непрерывный буфер и курсор, указывающий на позицию в этом буфере. Реализации могут как владеть так и не владеть памятью буфера. Существует виртуальный метод заполнения буфера следующими данными (для `ReadBuffer`) или сброса буфера куда-нибудь (например `WriteBuffer`). Виртуальные методы редко вызываются. Реализации `ReadBuffer`/`WriteBuffer` используются для работы с файлами и файловыми дескрипторами, а также сетевыми сокетами, для реализации сжатия (`CompressedWriteBuffer` инициализируется вместе с другим `WriteBuffer` и осуществляет сжатие данных перед записью в него), и для других целей – названия `ConcatReadBuffer`, `LimitReadBuffer`, и `HashingWriteBuffer` говорят сами за себя. -Буферы чтения/записи имеют дело только с байтами. В заголовочных файлах `ReadHelpers` и `WriteHelpers` объявлены некоторые функции, чтобы помочь с форматированием ввода/вывода. Например, есть помощники для записи числа в десятичном формате. +Буферы чтения-записи имеют дело только с байтами. В заголовочных файлах `ReadHelpers` и `WriteHelpers` объявлены некоторые функции, чтобы помочь с форматированием ввода-вывода. Например, есть помощники для записи числа в десятичном формате. Давайте посмотрим, что происходит, когда вы хотите вывести результат в `JSON` формате в стандартный вывод (stdout). У вас есть результирующий набор данных, готовый к извлечению из `IBlockInputStream`. Вы создаете `WriteBufferFromFileDescriptor(STDOUT_FILENO)` чтобы записать байты в stdout. Вы создаете `JSONRowOutputStream`, инициализируете с этим `WriteBuffer`'ом, чтобы записать строки `JSON` в stdout. Кроме того вы создаете `BlockOutputStreamFromRowOutputStream`, реализуя `IBlockOutputStream`. Затем вызывается `copyData` для передачи данных из `IBlockInputStream` в `IBlockOutputStream` и все работает. Внутренний `JSONRowOutputStream` будет писать в формате `JSON` различные разделители и вызвать `IDataType::serializeTextJSON` метод со ссылкой на `IColumn` и номер строки в качестве аргументов. Следовательно, `IDataType::serializeTextJSON` вызовет метод из `WriteHelpers.h`: например, `writeText` для числовых типов и `writeJSONString` для `DataTypeString`. @@ -93,7 +93,7 @@ ClickHouse - полноценная колоночная СУБД. Данные Но есть и заметные исключения: -- AST запрос, передающийся в метод `read`, может использоваться движком таблицы для получения информации о возможности использования индекса и считывания меньшего количества данных из таблицы. +- AST-запрос, передающийся в метод `read`, может использоваться движком таблицы для получения информации о возможности использования индекса и считывания меньшего количества данных из таблицы. - Иногда движок таблиц может сам обрабатывать данные до определенного этапа. Например, `StorageDistributed` можно отправить запрос на удаленные серверы, попросить их обработать данные до этапа, когда данные с разных удаленных серверов могут быть объединены, и вернуть эти предварительно обработанные данные. Затем интерпретатор запросов завершает обработку данных. Метод `read` может возвращать несколько объектов `IBlockInputStream`, позволяя осуществлять параллельную обработку данных. Эти несколько блочных входных потоков могут считываться из таблицы параллельно. Затем вы можете обернуть эти потоки различными преобразованиями (такими как вычисление выражений или фильтрация), которые могут быть вычислены независимо, и создать `UnionBlockInputStream` поверх них, чтобы читать из нескольких потоков параллельно. @@ -104,11 +104,11 @@ ClickHouse - полноценная колоночная СУБД. Данные > В качестве результата выполнения метода `read`, `IStorage` возвращает `QueryProcessingStage` – информацию о том, какие части запроса были обработаны внутри хранилища. -## Парсеры (Parsers) {#parsers} +## Разборщики (Parsers) {#parsers} -Написанный от руки парсер, анализирующий запрос, работает по методу рекурсивного спуска. Например, `ParserSelectQuery` просто рекурсивно вызывает нижестоящие парсеры для различных частей запроса. Парсеры создают абстрактное синтаксическое дерево (`AST`). `AST` представлен узлами, которые являются экземплярами `IAST`. +Написанный от руки разборщик, анализирующий запрос, работает по методу рекурсивного спуска. Например, `ParserSelectQuery` просто рекурсивно вызывает нижестоящие разборщики для различных частей запроса. Разборщики создают абстрактное синтаксическое дерево (`AST`). `AST` представлен узлами, которые являются экземплярами `IAST`. -> Генераторы парсеров не используются по историческим причинам. +> Генераторы разборщиков не используются по историческим причинам. ## Интерпретаторы {#interpreters} @@ -134,7 +134,7 @@ ClickHouse имеет сильную типизацию, поэтому нет ## Агрегатные функции {#aggregate-functions} -Агрегатные функции - это функции с состоянием (stateful). Они накапливают переданные значения в некотором состоянии и позволяют получать результаты из этого состояния. Работа с ними осуществляется с помощью интерфейса `IAggregateFunction`. Состояния могут быть как простыми (состояние для `AggregateFunctionCount` это всего лишь одна переменная типа `UInt64`) так и довольно сложными (состояние `AggregateFunctionUniqCombined` представляет собой комбинацию линейного массива, хэш-таблицы и вероятностной структуры данных `HyperLogLog`). +Агрегатные функции — это функции с состоянием (stateful). Они накапливают переданные значения в некотором состоянии и позволяют получать результаты из этого состояния. Работа с ними осуществляется с помощью интерфейса `IAggregateFunction`. Состояния могут быть как простыми (состояние для `AggregateFunctionCount` это всего лишь одна переменная типа `UInt64`) так и довольно сложными (состояние `AggregateFunctionUniqCombined` представляет собой комбинацию линейного массива, хэш-таблицы и вероятностной структуры данных `HyperLogLog`). Состояния распределяются в `Arena` (пул памяти) для работы с несколькими состояниями при выполнении запроса `GROUP BY` высокой кардинальности (большим числом уникальных данных). Состояния могут иметь нетривиальный конструктор и деструктор: например, сложные агрегатные состояния могут сами аллоцировать дополнительную память. Потому к созданию и уничтожению состояний, правильной передаче владения и порядку уничтожения следует уделять больше внимание. @@ -146,18 +146,18 @@ ClickHouse имеет сильную типизацию, поэтому нет Сервер предоставляет несколько различных интерфейсов. -- HTTP интерфейс для любых сторонних клиентов. -- TCP интерфейс для родного ClickHouse клиента и межсерверной взаимодействия при выполнении распределенных запросов. +- HTTP-интерфейс для любых сторонних клиентов. +- TCP-интерфейс для родного ClickHouse-клиента и межсерверной взаимодействия при выполнении распределенных запросов. - Интерфейс для передачи данных при репликации. -Внутри простой многопоточный сервер без корутин (coroutines), файберов (fibers) и т.д. Поскольку сервер не предназначен для обработки большого количества простых запросов, а ориентирован на обработку сложных запросов относительно низкой интенсивности, каждый из потоков может обрабатывать огромное количество аналитических запросов. +Внутри простой многопоточный сервер без сопрограмм (coroutines), фиберов (fibers) и т. д. Поскольку сервер не предназначен для обработки большого количества простых запросов, а ориентирован на обработку сложных запросов относительно низкой интенсивности, каждый из потоков может обрабатывать огромное количество аналитических запросов. -Сервер инициализирует класс `Context`, где хранит необходимое для выполнения запроса окружение: список доступных баз данных, пользователей и прав доступа, настройки, кластеры, список процессов, журнал запросов и т.д. Это окружение используется интерпретаторами. +Сервер инициализирует класс `Context`, где хранит необходимое для выполнения запроса окружение: список доступных баз данных, пользователей и прав доступа, настройки, кластеры, список процессов, журнал запросов и т. д. Это окружение используется интерпретаторами. -Мы поддерживаем полную обратную и прямую совместимость для TCP интерфейса: старые клиенты могут общаться с новыми серверами, а новые клиенты могут общаться со старыми серверами. Но мы не хотим поддерживать его вечно и прекращаем поддержку старых версий примерно через год. +Мы поддерживаем полную обратную и прямую совместимость для TCP-интерфейса: старые клиенты могут общаться с новыми серверами, а новые клиенты могут общаться со старыми серверами. Но мы не хотим поддерживать его вечно и прекращаем поддержку старых версий примерно через год. :::note Примечание -Для всех сторонних приложений мы рекомендуем использовать HTTP интерфейс, потому что он прост и удобен в использовании. TCP интерфейс тесно связан с внутренними структурами данных: он использует внутренний формат для передачи блоков данных и использует специальное кадрирование для сжатых данных. Мы не выпустили библиотеку C для этого протокола, потому что потребовала бы линковки большей части кодовой базы ClickHouse, что непрактично. +Для всех сторонних приложений мы рекомендуем использовать HTTP-интерфейс, потому что он прост и удобен в использовании. TCP-интерфейс тесно связан с внутренними структурами данных: он использует внутренний формат для передачи блоков данных и использует специальное кадрирование для сжатых данных. Мы не выпустили библиотеку C для этого протокола, потому что потребовала бы линковки большей части кодовой базы ClickHouse, что непрактично. ::: ## Выполнение распределенных запросов (Distributed Query Execution) {#distributed-query-execution} @@ -169,15 +169,15 @@ ClickHouse имеет сильную типизацию, поэтому нет ## Merge Tree {#merge-tree} -`MergeTree` - это семейство движков хранения, поддерживающих индексацию по первичному ключу. Первичный ключ может быть произвольным набором (кортежем) столбцов или выражений. Данные в таблице `MergeTree` хранятся "частями" (“parts”). Каждая часть хранит данные отсортированные по первичному ключу (данные упорядочены лексикографически). Все столбцы таблицы хранятся в отдельных файлах `column.bin` в этих частях. Файлы состоят из сжатых блоков. Каждый блок обычно содержит от 64 КБ до 1 МБ несжатых данных, в зависимости от среднего значения размера данных. Блоки состоят из значений столбцов, расположенных последовательно один за другим. Значения столбцов находятся в одинаковом порядке для каждого столбца (порядок определяется первичным ключом), поэтому, когда вы выполняете итерацию по многим столбцам, вы получаете значения для соответствующих строк. +`MergeTree` — это семейство движков хранения, поддерживающих индексацию по первичному ключу. Первичный ключ может быть произвольным набором (кортежем) столбцов или выражений. Данные в таблице `MergeTree` хранятся "частями" (“parts”). Каждая часть хранит данные отсортированные по первичному ключу (данные упорядочены лексикографически). Все столбцы таблицы хранятся в отдельных файлах `column.bin` в этих частях. Файлы состоят из сжатых блоков. Каждый блок обычно содержит от 64 КБ до 1 МБ несжатых данных, в зависимости от среднего значения размера данных. Блоки состоят из значений столбцов, расположенных последовательно один за другим. Значения столбцов находятся в одинаковом порядке для каждого столбца (порядок определяется первичным ключом), поэтому, когда вы выполняете итерацию по многим столбцам, вы получаете значения для соответствующих строк. -Сам первичный ключ является "разреженным" ("sparse"). Он не относится к каждой отдельной строке, а только к некоторым диапазонам данных. Отдельный файл «primary.idx» имеет значение первичного ключа для каждой N-й строки, где N называется гранулярностью индекса ("index_granularity", обычно N = 8192). Также для каждого столбца у нас есть файлы `column.mrk` с "метками" ("marks"), которые обозначают смещение для каждой N-й строки в файле данных. Каждая метка представляет собой пару: смещение начала сжатого блока от начала файла и смещение к началу данных в распакованном блоке. Обычно сжатые блоки выравниваются по меткам, а смещение в распакованном блоке равно нулю. Данные для `primary.idx` всегда находятся в памяти, а данные для файлов `column.mrk` кэшируются. +Сам первичный ключ является “разреженным” (sparse). Он не относится к каждой отдельной строке, а только к некоторым диапазонам данных. Отдельный файл «primary.idx» имеет значение первичного ключа для каждой N-й строки, где N называется гранулярностью индекса (index_granularity, обычно N = 8192). Также для каждого столбца у нас есть файлы `column.mrk` с "метками" ("marks"), которые обозначают смещение для каждой N-й строки в файле данных. Каждая метка представляет собой пару: смещение начала сжатого блока от начала файла и смещение к началу данных в распакованном блоке. Обычно сжатые блоки выравниваются по меткам, а смещение в распакованном блоке равно нулю. Данные для `primary.idx` всегда находятся в памяти, а данные для файлов `column.mrk` кэшируются. Когда мы собираемся читать что-то из части данных `MergeTree`, мы смотрим содержимое `primary.idx` и определяем диапазоны, которые могут содержать запрошенные данные, затем просматриваем содержимое `column.mrk` и вычисляем смещение, чтобы начать чтение этих диапазонов. Из-за разреженности могут быть прочитаны лишние данные. ClickHouse не подходит для простых точечных запросов высокой интенсивности, потому что весь диапазон строк размером `index_granularity` должен быть прочитан для каждого ключа, а сжатый блок должен быть полностью распакован для каждого столбца. Мы сделали индекс разреженным, потому что мы должны иметь возможность поддерживать триллионы строк на один сервер без существенных расходов памяти на индексацию. Кроме того, поскольку первичный ключ является разреженным, он не уникален: он не может проверить наличие ключа в таблице во время INSERT. Вы можете иметь множество строк с одним и тем же ключом в таблице. При выполнении `INSERT` для группы данных в `MergeTree`, элементы группы сортируются по первичному ключу и образует новую “часть”. Фоновые потоки периодически выбирают некоторые части и объединяют их в одну отсортированную часть, чтобы сохранить относительно небольшое количество частей. Вот почему он называется `MergeTree`. Конечно, объединение приводит к повышению интенсивности записи. Все части иммутабельные: они только создаются и удаляются, но не изменяются. Когда выполняется `SELECT`, он содержит снимок таблицы (набор частей). После объединения старые части также сохраняются в течение некоторого времени, чтобы упростить восстановление после сбоя, поэтому, если мы видим, что какая-то объединенная часть, вероятно, повреждена, мы можем заменить ее исходными частями. -`MergeTree` не является деревом LSM (Log-structured merge-tree — журнально-структурированное дерево со слиянием), потому что оно не содержит «memtable» и «log»: вставленные данные записываются непосредственно в файловую систему. Это делает его пригодным только для вставки данных в пакетах, а не по отдельным строкам и не очень часто - примерно раз в секунду это нормально, а тысячу раз в секунду - нет. Мы сделали это для простоты и потому, что мы уже вставляем данные в пакеты в наших приложениях. +`MergeTree` не является LSM (Log-structured merge-tree — журнально-структурированным деревом со слиянием), потому что оно не содержит «memtable» и «log»: вставленные данные записываются непосредственно в файловую систему. Это делает его пригодным только для вставки данных в пакетах, а не по отдельным строкам и не очень часто — примерно раз в секунду это нормально, а тысячу раз в секунду - нет. Мы сделали это для простоты и потому, что мы уже вставляем данные в пакеты в наших приложениях. > Таблицы `MergeTree` могут иметь только один (первичный) индекс: вторичных индексов нет. Было бы неплохо разрешить несколько физических представлениям в одной логической таблице, например, хранить данные в более чем одном физическом порядке или даже разрешить представления с предварительно агрегированными данными вместе с исходными данными. @@ -189,7 +189,7 @@ ClickHouse имеет сильную типизацию, поэтому нет Репликация реализована в движке таблицы `ReplicatedMergeTree`. Путь в `ZooKeeper` указывается в качестве параметра движка. Все таблицы с одинаковым путем в `ZooKeeper` становятся репликами друг друга: они синхронизируют свои данные и поддерживают согласованность. Реплики можно добавлять и удалять динамически, просто создавая или удаляя таблицу. -Репликация использует асинхронную multi-master схему. Вы можете вставить данные в любую реплику, которая имеет открытую сессию в `ZooKeeper`, и данные реплицируются на все другие реплики асинхронно. Поскольку ClickHouse не поддерживает UPDATE, репликация исключает конфликты (conflict-free replication). Поскольку подтверждение вставок кворумом не реализовано, только что вставленные данные могут быть потеряны в случае сбоя одного узла. +Репликация использует асинхронную multi-master-схему. Вы можете вставить данные в любую реплику, которая имеет открытую сессию в `ZooKeeper`, и данные реплицируются на все другие реплики асинхронно. Поскольку ClickHouse не поддерживает UPDATE, репликация исключает конфликты (conflict-free replication). Поскольку подтверждение вставок кворумом не реализовано, только что вставленные данные могут быть потеряны в случае сбоя одного узла. Метаданные для репликации хранятся в `ZooKeeper`. Существует журнал репликации, в котором перечислены действия, которые необходимо выполнить. Среди этих действий: получить часть (get the part); объединить части (merge parts); удалить партицию (drop a partition) и так далее. Каждая реплика копирует журнал репликации в свою очередь, а затем выполняет действия из очереди. Например, при вставке в журнале создается действие «получить часть» (get the part), и каждая реплика загружает эту часть. Слияния координируются между репликами, чтобы получить идентичные до байта результаты. Все части объединяются одинаково на всех репликах. Одна из реплик-лидеров инициирует новое слияние кусков первой и записывает действия «слияния частей» в журнал. Несколько реплик (или все) могут быть лидерами одновременно. Реплике можно запретить быть лидером с помощью `merge_tree` настройки `replicated_can_become_leader`. @@ -198,7 +198,7 @@ ClickHouse имеет сильную типизацию, поэтому нет Кроме того, каждая реплика сохраняет свое состояние в `ZooKeeper` в виде набора частей и его контрольных сумм. Когда состояние в локальной файловой системе расходится с эталонным состоянием в `ZooKeeper`, реплика восстанавливает свою согласованность путем загрузки отсутствующих и поврежденных частей из других реплик. Когда в локальной файловой системе есть неожиданные или испорченные данные, ClickHouse не удаляет их, а перемещает в отдельный каталог и забывает об этом. :::note Примечание -Кластер ClickHouse состоит из независимых шардов, а каждый шард состоит из реплик. Кластер **не является эластичным** (not elastic), поэтому после добавления нового шарда данные не будут автоматически распределены между ними. Вместо этого нужно изменить настройки, чтобы выровнять нагрузку на кластер. Эта реализация дает вам больший контроль, и вполне приемлема для относительно небольших кластеров, таких как десятки узлов. Но для кластеров с сотнями узлов, которые мы используем в эксплуатации, такой подход становится существенным недостатком. Движки таблиц, которые охватывают весь кластер с динамически реплицируемыми областями, которые могут быть автоматически разделены и сбалансированы между кластерами, еще предстоит реализовать. +Кластер ClickHouse состоит из независимых сегментов (shards), а каждый сегмент состоит из реплик. Кластер **не является эластичным** (not elastic), поэтому после добавления нового сегмента данные не будут автоматически распределены между ними. Вместо этого нужно изменить настройки, чтобы выровнять нагрузку на кластер. Эта реализация дает вам больший контроль, и вполне приемлема для относительно небольших кластеров, таких как десятки узлов. Но для кластеров с сотнями узлов, которые мы используем в эксплуатации, такой подход становится существенным недостатком. Движки таблиц, которые охватывают весь кластер с динамически реплицируемыми областями, которые могут быть автоматически разделены и сбалансированы между кластерами, еще предстоит реализовать. ::: {## [Original article](https://clickhouse.com/docs/ru/development/architecture/) ##} From 465962df7f87c241e2da20a3317d1b028d0bbf07 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Wed, 25 Oct 2023 03:08:17 +0800 Subject: [PATCH 573/634] Support orc filter push down (file + stripe + rowgroup level) (#55330) * support orc filter push down * update orc lib version * replace setqueryinfo with setkeycondition * fix issue https://github.com/ClickHouse/ClickHouse/issues/53536 * refactor source with key condition * fix building error * remove std::cout * update orc * update orc version * fix bugs * improve code * upgrade orc lib * fix code style * change as requested * add performance tests for orc filter push down * add performance tests for orc filter push down * fix all bugs * fix default as null issue * add uts for null as default issues * upgrade orc lib * fix failed orc lib uts and fix typo * fix failed uts * fix failed uts * fix ast fuzzer tests * fix bug of uint64 overflow in https://s3.amazonaws.com/clickhouse-test-reports/55330/de22fdcaea2e12c96f300e95f59beba84401712d/fuzzer_astfuzzerubsan/report.html * fix asan fatal caused by reused column vector batch in native orc input format. refer to https://s3.amazonaws.com/clickhouse-test-reports/55330/be39d23af2d7e27f5ec7f168947cf75aeaabf674/stateless_tests__asan__[4_4].htm * fix wrong performance tests * disable 02892_orc_filter_pushdown on aarch64. https://s3.amazonaws.com/clickhouse-test-reports/55330/be39d23af2d7e27f5ec7f168947cf75aeaabf674/stateless_tests__aarch64_.html * add some comments * add some comments * inline range::equals and range::less * fix data race of key condition * trigger ci --- contrib/orc | 2 +- src/Core/Range.cpp | 15 +- src/Core/Range.h | 3 + src/Core/Settings.h | 2 + src/Formats/FormatFactory.cpp | 2 + src/Formats/FormatSettings.h | 2 + src/Interpreters/Set.h | 4 +- src/Processors/Formats/IInputFormat.cpp | 2 +- src/Processors/Formats/IInputFormat.h | 13 +- .../Impl/NativeORCBlockInputFormat.cpp | 498 +++++++++++++++++- .../Formats/Impl/NativeORCBlockInputFormat.h | 7 +- .../Formats/Impl/ORCBlockOutputFormat.cpp | 13 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 20 +- .../Formats/Impl/ParquetBlockInputFormat.h | 5 - .../QueryPlan/ReadFromPreparedSource.cpp | 24 +- .../QueryPlan/ReadFromPreparedSource.h | 14 +- src/Processors/SourceWithKeyCondition.h | 62 +++ src/Storages/HDFS/StorageHDFS.cpp | 1 - src/Storages/Hive/StorageHive.cpp | 1 - src/Storages/IStorage.cpp | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 1 - src/Storages/MergeTree/KeyCondition.h | 6 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 3 +- src/Storages/NATS/StorageNATS.cpp | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 2 +- src/Storages/S3Queue/S3QueueSource.cpp | 19 +- src/Storages/S3Queue/S3QueueSource.h | 2 + src/Storages/StorageAzureBlob.cpp | 1 - src/Storages/StorageFile.cpp | 22 +- src/Storages/StorageS3.cpp | 18 +- src/Storages/StorageS3.h | 21 +- src/Storages/StorageURL.cpp | 8 +- src/Storages/StorageURL.h | 14 +- tests/performance/orc_filter_push_down.xml | 26 + .../02892_orc_filter_pushdown.reference | 227 ++++++++ .../0_stateless/02892_orc_filter_pushdown.sql | 229 ++++++++ 36 files changed, 1204 insertions(+), 89 deletions(-) create mode 100644 src/Processors/SourceWithKeyCondition.h create mode 100644 tests/performance/orc_filter_push_down.xml create mode 100644 tests/queries/0_stateless/02892_orc_filter_pushdown.reference create mode 100644 tests/queries/0_stateless/02892_orc_filter_pushdown.sql diff --git a/contrib/orc b/contrib/orc index a20d1d9d7ad..f31c271110a 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit a20d1d9d7ad4a4be7b7ba97588e16ca8b9abb2b6 +Subproject commit f31c271110a2f0dac908a152f11708193ae209ee diff --git a/src/Core/Range.cpp b/src/Core/Range.cpp index 293c80e70ab..de88313b9f3 100644 --- a/src/Core/Range.cpp +++ b/src/Core/Range.cpp @@ -89,17 +89,14 @@ void Range::shrinkToIncludedIfPossible() } } -namespace +bool Range::equals(const Field & lhs, const Field & rhs) { - inline bool equals(const Field & lhs, const Field & rhs) - { - return applyVisitor(FieldVisitorAccurateEquals(), lhs, rhs); - } + return applyVisitor(FieldVisitorAccurateEquals(), lhs, rhs); +} - inline bool less(const Field & lhs, const Field & rhs) - { - return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); - } +bool Range::less(const Field & lhs, const Field & rhs) +{ + return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); } bool Range::empty() const diff --git a/src/Core/Range.h b/src/Core/Range.h index 89113e960be..9680107cd51 100644 --- a/src/Core/Range.h +++ b/src/Core/Range.h @@ -59,6 +59,9 @@ public: static Range createRightBounded(const FieldRef & right_point, bool right_included, bool with_null = false); static Range createLeftBounded(const FieldRef & left_point, bool left_included, bool with_null = false); + static ALWAYS_INLINE bool equals(const Field & lhs, const Field & rhs); + static ALWAYS_INLINE bool less(const Field & lhs, const Field & rhs); + /** Optimize the range. If it has an open boundary and the Field type is "loose" * - then convert it to closed, narrowing by one. * That is, for example, turn (0,2) into [1]. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6d25d5ec436..610e216e1ff 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -900,6 +900,7 @@ class IColumn; M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \ M(Bool, input_format_orc_allow_missing_columns, false, "Allow missing columns while reading ORC input formats", 0) \ M(Bool, input_format_orc_use_fast_decoder, true, "Use a faster ORC decoder implementation.", 0) \ + M(Bool, input_format_orc_filter_push_down, true, "When reading ORC files, skip whole stripes or row groups based on the WHERE/PREWHERE expressions, min/max statistics or bloom filter in the ORC metadata.", 0) \ M(Bool, input_format_parquet_allow_missing_columns, false, "Allow missing columns while reading Parquet input formats", 0) \ M(UInt64, input_format_parquet_local_file_min_bytes_for_seek, 8192, "Min bytes required for local read (file) to do seek, instead of read with ignore in Parquet input format", 0) \ M(Bool, input_format_arrow_allow_missing_columns, false, "Allow missing columns while reading Arrow input formats", 0) \ @@ -1051,6 +1052,7 @@ class IColumn; \ M(Bool, output_format_orc_string_as_string, false, "Use ORC String type instead of Binary for String columns", 0) \ M(ORCCompression, output_format_orc_compression_method, "lz4", "Compression method for ORC output format. Supported codecs: lz4, snappy, zlib, zstd, none (uncompressed)", 0) \ + M(UInt64, output_format_orc_row_index_stride, 10'000, "Target row index stride in ORC output format", 0) \ \ M(CapnProtoEnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::CapnProtoEnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0) \ \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 6efab3aefd7..920a3cb1bb5 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -194,7 +194,9 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.orc.case_insensitive_column_matching = settings.input_format_orc_case_insensitive_column_matching; format_settings.orc.output_string_as_string = settings.output_format_orc_string_as_string; format_settings.orc.output_compression_method = settings.output_format_orc_compression_method; + format_settings.orc.output_row_index_stride = settings.output_format_orc_row_index_stride; format_settings.orc.use_fast_decoder = settings.input_format_orc_use_fast_decoder; + format_settings.orc.filter_push_down = settings.input_format_orc_filter_push_down; format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields; format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode; format_settings.capn_proto.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 2b005e39e62..77fd8e1fcbd 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -364,6 +364,8 @@ struct FormatSettings bool output_string_as_string = false; ORCCompression output_compression_method = ORCCompression::NONE; bool use_fast_decoder = true; + bool filter_push_down = true; + UInt64 output_row_index_stride = 10'000; } orc; /// For capnProto format we should determine how to diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 08c472c3c95..11e0ddbfae2 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -198,7 +198,7 @@ using FunctionPtr = std::shared_ptr; */ struct FieldValue { - FieldValue(MutableColumnPtr && column_) : column(std::move(column_)) {} + explicit FieldValue(MutableColumnPtr && column_) : column(std::move(column_)) {} void update(const Field & x); bool isNormal() const { return !value.isPositiveInfinity() && !value.isNegativeInfinity(); } @@ -230,6 +230,8 @@ public: size_t size() const { return ordered_set.at(0)->size(); } + const Columns & getOrderedSet() const { return ordered_set; } + bool hasMonotonicFunctionsChain() const; BoolMask checkInRange(const std::vector & key_ranges, const DataTypes & data_types, bool single_point = false) const; diff --git a/src/Processors/Formats/IInputFormat.cpp b/src/Processors/Formats/IInputFormat.cpp index a87db5a0d4d..031b396679c 100644 --- a/src/Processors/Formats/IInputFormat.cpp +++ b/src/Processors/Formats/IInputFormat.cpp @@ -6,7 +6,7 @@ namespace DB { IInputFormat::IInputFormat(Block header, ReadBuffer * in_) - : ISource(std::move(header)), in(in_) + : SourceWithKeyCondition(std::move(header)), in(in_) { column_mapping = std::make_shared(); } diff --git a/src/Processors/Formats/IInputFormat.h b/src/Processors/Formats/IInputFormat.h index c7b8d97d145..5afc24c9298 100644 --- a/src/Processors/Formats/IInputFormat.h +++ b/src/Processors/Formats/IInputFormat.h @@ -1,10 +1,11 @@ #pragma once -#include -#include +#include #include #include -#include +#include +#include +#include namespace DB @@ -16,7 +17,7 @@ using ColumnMappingPtr = std::shared_ptr; /** Input format is a source, that reads data from ReadBuffer. */ -class IInputFormat : public ISource +class IInputFormat : public SourceWithKeyCondition { protected: @@ -26,10 +27,6 @@ public: /// ReadBuffer can be nullptr for random-access formats. IInputFormat(Block header, ReadBuffer * in_); - /// If the format is used by a SELECT query, this method may be called. - /// The format may use it for filter pushdown. - virtual void setQueryInfo(const SelectQueryInfo &, ContextPtr) {} - /** In some usecase (hello Kafka) we need to read a lot of tiny streams in exactly the same format. * The recreating of parser for each small stream takes too long, so we introduce a method * resetParser() which allow to reset the state of parser to continue reading of diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index ed6f92c669e..b346ef3d232 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -14,6 +14,7 @@ # include # include # include +# include # include # include # include @@ -28,7 +29,9 @@ # include # include # include +# include # include +# include # include "ArrowBufferedStreams.h" @@ -97,6 +100,15 @@ std::unique_ptr asORCInputStreamLoadIntoMemory(ReadBuffer & in return std::make_unique(std::move(file_data), file_size); } +static const orc::Type * getORCTypeByName(const orc::Type & schema, const String & name, bool case_insensitive_column_matching) +{ + for (uint64_t i = 0; i != schema.getSubtypeCount(); ++i) + if (boost::equals(schema.getFieldName(i), name) + || (case_insensitive_column_matching && boost::iequals(schema.getFieldName(i), name))) + return schema.getSubtype(i); + return nullptr; +} + static DataTypePtr parseORCType(const orc::Type * orc_type, bool skip_columns_with_unsupported_types, bool & skipped) { assert(orc_type != nullptr); @@ -199,6 +211,473 @@ static DataTypePtr parseORCType(const orc::Type * orc_type, bool skip_columns_wi } } +static std::optional convertORCTypeToPredicateType(const orc::Type & orc_type) +{ + switch (orc_type.getKind()) + { + case orc::BOOLEAN: + return orc::PredicateDataType::BOOLEAN; + case orc::BYTE: + case orc::SHORT: + case orc::INT: + case orc::LONG: + return orc::PredicateDataType::LONG; + case orc::FLOAT: + case orc::DOUBLE: + return orc::PredicateDataType::FLOAT; + case orc::VARCHAR: + case orc::CHAR: + case orc::STRING: + return orc::PredicateDataType::STRING; + case orc::DATE: + return orc::PredicateDataType::DATE; + case orc::TIMESTAMP: + return orc::PredicateDataType::TIMESTAMP; + case orc::DECIMAL: + return orc::PredicateDataType::DECIMAL; + default: + return {}; + } +} + +static String getColumnNameFromKeyCondition(const KeyCondition & key_condition, size_t indice) +{ + const auto & key_columns = key_condition.getKeyColumns(); + for (const auto & [name, i] : key_columns) + { + if (i == indice) + return name; + } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't get column from KeyCondition with indice {}", indice); +} + +static std::optional +convertFieldToORCLiteral(const orc::Type & orc_type, const Field & field, DataTypePtr type_hint = nullptr) +{ + try + { + /// We always fallback to return null if possible CH type hint not consistent with ORC type + switch (orc_type.getKind()) + { + case orc::BOOLEAN: { + /// May throw exception + auto val = field.get(); + return orc::Literal(val != 0); + } + case orc::BYTE: + case orc::SHORT: + case orc::INT: + case orc::LONG: { + /// May throw exception + auto val = field.get(); + return orc::Literal(val); + } + case orc::FLOAT: + case orc::DOUBLE: { + Float64 val; + if (field.tryGet(val)) + return orc::Literal(val); + break; + } + case orc::VARCHAR: + case orc::CHAR: + case orc::STRING: { + String str; + if (field.tryGet(str)) + return orc::Literal(str.data(), str.size()); + break; + } + case orc::DATE: { + Int64 val; + if (field.tryGet(val)) + return orc::Literal(orc::PredicateDataType::DATE, val); + break; + } + case orc::TIMESTAMP: { + if (type_hint && isDateTime64(type_hint)) + { + const auto * datetime64_type = typeid_cast(type_hint.get()); + if (datetime64_type->getScale() != 9) + return std::nullopt; + } + + DecimalField ts; + if (field.tryGet(ts)) + { + Int64 secs = (ts.getValue() / ts.getScaleMultiplier()).convertTo(); + Int32 nanos = (ts.getValue() - (ts.getValue() / ts.getScaleMultiplier()) * ts.getScaleMultiplier()).convertTo(); + return orc::Literal(secs, nanos); + } + break; + } + case orc::DECIMAL: { + auto precision = orc_type.getPrecision(); + if (precision == 0) + precision = 38; + + if (precision <= DecimalUtils::max_precision) + { + DecimalField val; + if (field.tryGet(val)) + { + Int64 right = val.getValue().convertTo(); + return orc::Literal( + orc::Int128(right), static_cast(orc_type.getPrecision()), static_cast(orc_type.getScale())); + } + } + else if (precision <= DecimalUtils::max_precision) + { + DecimalField val; + if (field.tryGet(val)) + { + Int64 right = val.getValue().convertTo(); + return orc::Literal( + orc::Int128(right), static_cast(orc_type.getPrecision()), static_cast(orc_type.getScale())); + } + } + else if (precision <= DecimalUtils::max_precision) + { + DecimalField val; + if (field.tryGet(val)) + { + Int64 high = val.getValue().value.items[1]; + UInt64 low = static_cast(val.getValue().value.items[0]); + return orc::Literal( + orc::Int128(high, low), static_cast(orc_type.getPrecision()), static_cast(orc_type.getScale())); + } + } + break; + } + default: + break; + } + return std::nullopt; + } + catch (Exception &) + { + return std::nullopt; + } +} + +/// Attention: evaluateRPNElement is only invoked in buildORCSearchArgumentImpl. +/// So it is guaranteed that: +/// 1. elem has no monotonic_functions_chains. +/// 2. if elem function is FUNCTION_IN_RANGE/FUNCTION_NOT_IN_RANGE, `set_index` is not null and `set_index->getOrderedSet().size()` is 1. +/// 3. elem function should be FUNCTION_IN_RANGE/FUNCTION_NOT_IN_RANGE/FUNCTION_IN_SET/FUNCTION_NOT_IN_SET/FUNCTION_IS_NULL/FUNCTION_IS_NOT_NULL +static bool evaluateRPNElement(const Field & field, const KeyCondition::RPNElement & elem) +{ + Range key_range(field); + switch (elem.function) + { + case KeyCondition::RPNElement::FUNCTION_IN_RANGE: + case KeyCondition::RPNElement::FUNCTION_NOT_IN_RANGE: { + /// Rows with null values should never output when filters like ">=", ">", "<=", "<", '=' are applied + if (field.isNull()) + return false; + + bool res = elem.range.intersectsRange(key_range); + if (elem.function == KeyCondition::RPNElement::FUNCTION_NOT_IN_RANGE) + res = !res; + return res; + } + case KeyCondition::RPNElement::FUNCTION_IN_SET: + case KeyCondition::RPNElement::FUNCTION_NOT_IN_SET: { + const auto & set_index = elem.set_index; + const auto & ordered_set = set_index->getOrderedSet(); + const auto & set_column = ordered_set[0]; + + bool res = false; + for (size_t i = 0; i < set_column->size(); ++i) + { + if (Range::equals(field, (*set_column)[i])) + { + res = true; + break; + } + } + + if (elem.function == KeyCondition::RPNElement::FUNCTION_NOT_IN_SET) + res = !res; + return res; + } + case KeyCondition::RPNElement::FUNCTION_IS_NULL: + case KeyCondition::RPNElement::FUNCTION_IS_NOT_NULL: { + if (field.isNull()) + return elem.function == KeyCondition::RPNElement::FUNCTION_IS_NULL; + else + return elem.function == KeyCondition::RPNElement::FUNCTION_IS_NOT_NULL; + } + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected RPNElement Function {}", elem.toString()); + } +} + +static void buildORCSearchArgumentImpl( + const KeyCondition & key_condition, + const Block & header, + const orc::Type & schema, + KeyCondition::RPN & rpn_stack, + orc::SearchArgumentBuilder & builder, + const FormatSettings & format_settings) +{ + if (rpn_stack.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty rpn stack in buildORCSearchArgumentImpl"); + + const auto & curr = rpn_stack.back(); + switch (curr.function) + { + case KeyCondition::RPNElement::FUNCTION_IN_RANGE: + case KeyCondition::RPNElement::FUNCTION_NOT_IN_RANGE: + case KeyCondition::RPNElement::FUNCTION_IN_SET: + case KeyCondition::RPNElement::FUNCTION_NOT_IN_SET: + case KeyCondition::RPNElement::FUNCTION_IS_NULL: + case KeyCondition::RPNElement::FUNCTION_IS_NOT_NULL: { + const bool need_wrap_not = curr.function == KeyCondition::RPNElement::FUNCTION_IS_NOT_NULL + || curr.function == KeyCondition::RPNElement::FUNCTION_NOT_IN_RANGE + || curr.function == KeyCondition::RPNElement::FUNCTION_NOT_IN_SET; + const bool contains_is_null = curr.function == KeyCondition::RPNElement::FUNCTION_IS_NULL + || curr.function == KeyCondition::RPNElement::FUNCTION_IS_NOT_NULL; + const bool contains_in_set = curr.function == KeyCondition::RPNElement::FUNCTION_IN_SET + || curr.function == KeyCondition::RPNElement::FUNCTION_NOT_IN_SET; + const bool contains_in_range = curr.function == KeyCondition::RPNElement::FUNCTION_IN_RANGE + || curr.function == KeyCondition::RPNElement::FUNCTION_NOT_IN_RANGE; + + SCOPE_EXIT({rpn_stack.pop_back();}); + + + /// Key filter expressions like "func(col) > 100" are not supported for ORC filter push down + if (!curr.monotonic_functions_chain.empty()) + { + builder.literal(orc::TruthValue::YES_NO_NULL); + break; + } + + /// key filter expressions like "(a, b, c) in " or "(func(a), b) in " are not supported for ORC filter push down + /// Only expressions like "a in " are supported currently, maybe we can improve it later. + auto set_index = curr.set_index; + if (contains_in_set) + { + if (!set_index || set_index->getOrderedSet().size() != 1 || set_index->hasMonotonicFunctionsChain()) + { + builder.literal(orc::TruthValue::YES_NO_NULL); + break; + } + } + + String column_name = getColumnNameFromKeyCondition(key_condition, curr.key_column); + const auto * orc_type = getORCTypeByName(schema, column_name, format_settings.orc.case_insensitive_column_matching); + if (!orc_type) + { + builder.literal(orc::TruthValue::YES_NO_NULL); + break; + } + + /// Make sure key column in header has exactly the same type with key column in ORC file schema + /// Counter-example 1: + /// Column a has type "Nullable(Int64)" in ORC file, but in header column a has type "Int64", which is allowed in CH. + /// For queries with where condition like "a is null", if a column contains null value, pushing or not pushing down filters + /// would result in different outputs. + /// Counter-example 2: + /// Column a has type "Nullable(Int64)" in ORC file, but in header column a has type "Nullable(UInt64)". + /// For queries with where condition like "a > 10", if a column contains negative values such as "-1", pushing or not pushing + /// down filters would result in different outputs. + bool skipped = false; + auto expect_type = makeNullableRecursively(parseORCType(orc_type, true, skipped)); + const ColumnWithTypeAndName * column = header.findByName(column_name, format_settings.orc.case_insensitive_column_matching); + if (!expect_type || !column) + { + builder.literal(orc::TruthValue::YES_NO_NULL); + break; + } + + auto nested_type = removeNullable(recursiveRemoveLowCardinality(column->type)); + auto expect_nested_type = removeNullable(expect_type); + if (!nested_type->equals(*expect_nested_type)) + { + builder.literal(orc::TruthValue::YES_NO_NULL); + break; + } + + /// If null_as_default is true, the only difference is nullable, and the evaluations of current RPNElement based on default and null field + /// have the same result, we still should push down current filter. + if (format_settings.null_as_default && !column->type->isNullable() && !column->type->isLowCardinalityNullable()) + { + bool match_if_null = evaluateRPNElement({}, curr); + bool match_if_default = evaluateRPNElement(column->type->getDefault(), curr); + if (match_if_default != match_if_null) + { + builder.literal(orc::TruthValue::YES_NO_NULL); + break; + } + } + + auto predicate_type = convertORCTypeToPredicateType(*orc_type); + if (!predicate_type.has_value()) + { + builder.literal(orc::TruthValue::YES_NO_NULL); + break; + } + + if (need_wrap_not) + builder.startNot(); + + if (contains_is_null) + { + builder.isNull(orc_type->getColumnId(), *predicate_type); + } + else if (contains_in_range) + { + const auto & range = curr.range; + bool has_left_bound = !range.left.isNegativeInfinity(); + bool has_right_bound = !range.right.isPositiveInfinity(); + if (!has_left_bound && !has_right_bound) + { + /// Transform whole range orc::TruthValue::YES_NULL + builder.literal(orc::TruthValue::YES_NULL); + } + else if (has_left_bound && has_right_bound && range.left_included && range.right_included && range.left == range.right) + { + /// Transform range with the same left bound and right bound to equal, which could utilize bloom filters in ORC + auto literal = convertFieldToORCLiteral(*orc_type, range.left); + if (literal.has_value()) + builder.equals(orc_type->getColumnId(), *predicate_type, *literal); + else + builder.literal(orc::TruthValue::YES_NO_NULL); + } + else + { + std::optional left_literal; + if (has_left_bound) + left_literal = convertFieldToORCLiteral(*orc_type, range.left); + + std::optional right_literal; + if (has_right_bound) + right_literal = convertFieldToORCLiteral(*orc_type, range.right); + + if (has_left_bound && has_right_bound) + builder.startAnd(); + + if (has_left_bound) + { + if (left_literal.has_value()) + { + /// >= is transformed to not < and > is transformed to not <= + builder.startNot(); + if (range.left_included) + builder.lessThan(orc_type->getColumnId(), *predicate_type, *left_literal); + else + builder.lessThanEquals(orc_type->getColumnId(), *predicate_type, *left_literal); + builder.end(); + } + else + builder.literal(orc::TruthValue::YES_NO_NULL); + } + + if (has_right_bound) + { + if (right_literal.has_value()) + { + if (range.right_included) + builder.lessThanEquals(orc_type->getColumnId(), *predicate_type, *right_literal); + else + builder.lessThan(orc_type->getColumnId(), *predicate_type, *right_literal); + } + else + builder.literal(orc::TruthValue::YES_NO_NULL); + } + + if (has_left_bound && has_right_bound) + builder.end(); + } + } + else if (contains_in_set) + { + /// Build literals from MergeTreeSetIndex + const auto & ordered_set = set_index->getOrderedSet(); + const auto & set_column = ordered_set[0]; + + bool fail = false; + std::vector literals; + literals.reserve(set_column->size()); + for (size_t i = 0; i < set_column->size(); ++i) + { + auto literal = convertFieldToORCLiteral(*orc_type, (*set_column)[i]); + if (!literal.has_value()) + { + fail = true; + break; + } + + literals.emplace_back(*literal); + } + + /// set has zero element + if (literals.empty()) + builder.literal(orc::TruthValue::YES); + else if (fail) + builder.literal(orc::TruthValue::YES_NO_NULL); + else + builder.in(orc_type->getColumnId(), *predicate_type, literals); + } + + if (need_wrap_not) + builder.end(); + + break; + } + case KeyCondition::RPNElement::FUNCTION_UNKNOWN: { + builder.literal(orc::TruthValue::YES_NO_NULL); + rpn_stack.pop_back(); + break; + } + case KeyCondition::RPNElement::FUNCTION_NOT: { + builder.startNot(); + rpn_stack.pop_back(); + buildORCSearchArgumentImpl(key_condition, header, schema, rpn_stack, builder, format_settings); + builder.end(); + break; + } + case KeyCondition::RPNElement::FUNCTION_AND: { + builder.startAnd(); + rpn_stack.pop_back(); + buildORCSearchArgumentImpl(key_condition, header, schema, rpn_stack, builder, format_settings); + buildORCSearchArgumentImpl(key_condition, header, schema, rpn_stack, builder, format_settings); + builder.end(); + break; + } + case KeyCondition::RPNElement::FUNCTION_OR: { + builder.startOr(); + rpn_stack.pop_back(); + buildORCSearchArgumentImpl(key_condition, header, schema, rpn_stack, builder, format_settings); + buildORCSearchArgumentImpl(key_condition, header, schema, rpn_stack, builder, format_settings); + builder.end(); + break; + } + case KeyCondition::RPNElement::ALWAYS_FALSE: { + builder.literal(orc::TruthValue::NO); + rpn_stack.pop_back(); + break; + } + case KeyCondition::RPNElement::ALWAYS_TRUE: { + builder.literal(orc::TruthValue::YES); + rpn_stack.pop_back(); + break; + } + } +} + +std::unique_ptr +buildORCSearchArgument(const KeyCondition & key_condition, const Block & header, const orc::Type & schema, const FormatSettings & format_settings) +{ + auto rpn_stack = key_condition.getRPN(); + if (rpn_stack.empty()) + return nullptr; + + auto builder = orc::SearchArgumentFactory::newBuilder(); + buildORCSearchArgumentImpl(key_condition, header, schema, rpn_stack, *builder, format_settings); + return builder->build(); +} + static void getFileReaderAndSchema( ReadBuffer & in, @@ -257,6 +736,11 @@ void NativeORCBlockInputFormat::prepareFileReader() if (getPort().getHeader().has(name, ignore_case) || nested_table_names.contains(ignore_case ? boost::to_lower_copy(name) : name)) include_indices.push_back(static_cast(i)); } + + if (format_settings.orc.filter_push_down && key_condition && !sarg) + { + sarg = buildORCSearchArgument(*key_condition, getPort().getHeader(), file_reader->getType(), format_settings); + } } bool NativeORCBlockInputFormat::prepareStripeReader() @@ -278,11 +762,12 @@ bool NativeORCBlockInputFormat::prepareStripeReader() orc::RowReaderOptions row_reader_options; row_reader_options.include(include_indices); row_reader_options.range(current_stripe_info->getOffset(), current_stripe_info->getLength()); + if (format_settings.orc.filter_push_down && sarg) + { + row_reader_options.searchArgument(sarg); + } + stripe_reader = file_reader->createRowReader(row_reader_options); - - if (!batch) - batch = stripe_reader->createRowBatch(format_settings.orc.row_batch_size); - return true; } @@ -314,6 +799,9 @@ Chunk NativeORCBlockInputFormat::generate() if (is_stopped) return {}; + /// TODO: figure out why reuse batch would cause asan fatals in https://s3.amazonaws.com/clickhouse-test-reports/55330/be39d23af2d7e27f5ec7f168947cf75aeaabf674/stateless_tests__asan__[4_4].html + /// Not sure if it is a false positive case. Notice that reusing batch will speed up reading ORC by 1.15x. + auto batch = stripe_reader->createRowBatch(format_settings.orc.row_batch_size); while (true) { bool ok = stripe_reader->next(*batch); @@ -341,7 +829,7 @@ void NativeORCBlockInputFormat::resetParser() file_reader.reset(); stripe_reader.reset(); include_indices.clear(); - batch.reset(); + sarg.reset(); block_missing_values.clear(); } diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h index 3326999f0aa..6ea7a063e0d 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h @@ -6,6 +6,8 @@ # include # include # include +# include +# include # include namespace DB @@ -42,6 +44,8 @@ std::unique_ptr asORCInputStream(ReadBuffer & in, const Format // Reads the whole file into a memory buffer, owned by the returned RandomAccessFile. std::unique_ptr asORCInputStreamLoadIntoMemory(ReadBuffer & in, std::atomic & is_cancelled); +std::unique_ptr buildORCSearchArgument( + const KeyCondition & key_condition, const Block & header, const orc::Type & schema, const FormatSettings & format_settings); class ORCColumnToCHColumn; class NativeORCBlockInputFormat : public IInputFormat @@ -69,7 +73,8 @@ private: std::unique_ptr file_reader; std::unique_ptr stripe_reader; std::unique_ptr orc_column_to_ch_column; - std::unique_ptr batch; + + std::shared_ptr sarg; // indices of columns to read from ORC file std::list include_indices; diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 827752d9db0..1e36c100667 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -426,14 +426,14 @@ void ORCBlockOutputFormat::writeColumn( const auto * timestamp_type = assert_cast(type.get()); UInt32 scale = timestamp_type->getScale(); writeDateTimes( - orc_column, - column, null_bytemap, - [scale](UInt64 value){ return value / std::pow(10, scale); }, - [scale](UInt64 value){ return (value % UInt64(std::pow(10, scale))) * std::pow(10, 9 - scale); }); + orc_column, + column, + null_bytemap, + [scale](Int64 value) { return value / Int64(std::pow(10, scale)); }, + [scale](Int64 value) { return (value % Int64(std::pow(10, scale))) * Int64(std::pow(10, 9 - scale)); }); break; } - case TypeIndex::Decimal32: - { + case TypeIndex::Decimal32: { writeDecimals( orc_column, column, @@ -608,6 +608,7 @@ void ORCBlockOutputFormat::prepareWriter() const Block & header = getPort(PortKind::Main).getHeader(); schema = orc::createStructType(); options.setCompression(getORCCompression(format_settings.orc.output_compression_method)); + options.setRowIndexStride(format_settings.orc.output_row_index_stride); size_t columns_count = header.columns(); for (size_t i = 0; i != columns_count; ++i) schema->addStructField(header.safeGetByPosition(i).name, getORCType(recursiveRemoveLowCardinality(data_types[i]))); diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index c947eda42c7..3cfeb80afd5 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -387,16 +387,6 @@ ParquetBlockInputFormat::~ParquetBlockInputFormat() pool->wait(); } -void ParquetBlockInputFormat::setQueryInfo(const SelectQueryInfo & query_info, ContextPtr context) -{ - /// When analyzer is enabled, query_info.filter_asts is missing sets and maybe some type casts, - /// so don't use it. I'm not sure how to support analyzer here: https://github.com/ClickHouse/ClickHouse/issues/53536 - if (format_settings.parquet.filter_push_down && !context->getSettingsRef().allow_experimental_analyzer) - key_condition.emplace(query_info, context, getPort().getHeader().getNames(), - std::make_shared(std::make_shared( - getPort().getHeader().getColumnsWithTypeAndName()))); -} - void ParquetBlockInputFormat::initializeIfNeeded() { if (std::exchange(is_initialized, true)) @@ -428,10 +418,12 @@ void ParquetBlockInputFormat::initializeIfNeeded() if (skip_row_groups.contains(row_group)) continue; - if (key_condition.has_value() && - !key_condition->checkInHyperrectangle( - getHyperrectangleForRowGroup(*metadata, row_group, getPort().getHeader(), format_settings), - getPort().getHeader().getDataTypes()).can_be_true) + if (format_settings.parquet.filter_push_down && key_condition + && !key_condition + ->checkInHyperrectangle( + getHyperrectangleForRowGroup(*metadata, row_group, getPort().getHeader(), format_settings), + getPort().getHeader().getDataTypes()) + .can_be_true) continue; if (row_group_batches.empty() || row_group_batches.back().total_bytes_compressed >= min_bytes_for_seek) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index c102dbee0f4..7fdf03a0606 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -56,8 +56,6 @@ public: ~ParquetBlockInputFormat() override; - void setQueryInfo(const SelectQueryInfo & query_info, ContextPtr context) override; - void resetParser() override; String getName() const override { return "ParquetBlockInputFormat"; } @@ -255,9 +253,6 @@ private: std::shared_ptr metadata; /// Indices of columns to read from Parquet file. std::vector column_indices; - /// Pushed-down filter that we'll use to skip row groups. - std::optional key_condition; - // Window of active row groups: // diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index a24c4dbe4d0..798073f94d3 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -1,11 +1,13 @@ +#include #include +#include #include namespace DB { ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, ContextPtr context_, Context::QualifiedProjectionName qualified_projection_name_) - : ISourceStep(DataStream{.header = pipe_.getHeader()}) + : SourceStepWithFilter(DataStream{.header = pipe_.getHeader()}) , pipe(std::move(pipe_)) , context(std::move(context_)) , qualified_projection_name(std::move(qualified_projection_name_)) @@ -23,4 +25,24 @@ void ReadFromPreparedSource::initializePipeline(QueryPipelineBuilder & pipeline, pipeline.init(std::move(pipe)); } +void ReadFromStorageStep::applyFilters() +{ + if (!context) + return; + + std::shared_ptr key_condition; + if (!context->getSettingsRef().allow_experimental_analyzer) + { + for (const auto & processor : pipe.getProcessors()) + if (auto * source = dynamic_cast(processor.get())) + source->setKeyCondition(query_info, context); + } + else + { + for (const auto & processor : pipe.getProcessors()) + if (auto * source = dynamic_cast(processor.get())) + source->setKeyCondition(filter_nodes.nodes, context); + } +} + } diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.h b/src/Processors/QueryPlan/ReadFromPreparedSource.h index 2606f501009..16e790273ea 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.h +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.h @@ -2,13 +2,15 @@ #include #include +#include #include +#include namespace DB { /// Create source from prepared pipe. -class ReadFromPreparedSource : public ISourceStep +class ReadFromPreparedSource : public SourceStepWithFilter { public: explicit ReadFromPreparedSource( @@ -27,19 +29,21 @@ protected: class ReadFromStorageStep : public ReadFromPreparedSource { public: - ReadFromStorageStep(Pipe pipe_, String storage_name, std::shared_ptr storage_limits_) - : ReadFromPreparedSource(std::move(pipe_)), storage_limits(std::move(storage_limits_)) + ReadFromStorageStep(Pipe pipe_, String storage_name, const SelectQueryInfo & query_info_, ContextPtr context_) + : ReadFromPreparedSource(std::move(pipe_), std::move(context_)), query_info(query_info_) { setStepDescription(storage_name); for (const auto & processor : pipe.getProcessors()) - processor->setStorageLimits(storage_limits); + processor->setStorageLimits(query_info.storage_limits); } String getName() const override { return "ReadFromStorage"; } + void applyFilters() override; + private: - std::shared_ptr storage_limits; + SelectQueryInfo query_info; }; } diff --git a/src/Processors/SourceWithKeyCondition.h b/src/Processors/SourceWithKeyCondition.h new file mode 100644 index 00000000000..d4b2d01c520 --- /dev/null +++ b/src/Processors/SourceWithKeyCondition.h @@ -0,0 +1,62 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +/// Source with KeyCondition to push down filters. +class SourceWithKeyCondition : public ISource +{ +protected: + /// Represents pushed down filters in source + std::shared_ptr key_condition; + + void setKeyConditionImpl(const SelectQueryInfo & query_info, ContextPtr context, const Block & keys) + { + if (!context->getSettingsRef().allow_experimental_analyzer) + { + key_condition = std::make_shared( + query_info, + context, + keys.getNames(), + std::make_shared(std::make_shared(keys.getColumnsWithTypeAndName()))); + } + } + + void setKeyConditionImpl(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context, const Block & keys) + { + if (context->getSettingsRef().allow_experimental_analyzer) + { + std::unordered_map node_name_to_input_column; + for (const auto & column : keys.getColumnsWithTypeAndName()) + node_name_to_input_column.insert({column.name, column}); + + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_column, context); + key_condition = std::make_shared( + filter_actions_dag, + context, + keys.getNames(), + std::make_shared(std::make_shared(keys.getColumnsWithTypeAndName())), + NameSet{}); + } + } + +public: + using Base = ISource; + using Base::Base; + + /// Set key_condition directly. It is used for filter push down in source. + virtual void setKeyCondition(const std::shared_ptr & key_condition_) { key_condition = key_condition_; } + + /// Set key_condition created by query_info and context. It is used for filter push down when allow_experimental_analyzer is false. + virtual void setKeyCondition(const SelectQueryInfo & /*query_info*/, ContextPtr /*context*/) { } + + /// Set key_condition created by nodes and context. It is used for filter push down when allow_experimental_analyzer is true. + virtual void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & /*nodes*/, ContextPtr /*context*/) { } +}; +} diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 99c3e6e29cf..2e0e09c4b18 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -602,7 +602,6 @@ bool HDFSSource::initialize() max_parsing_threads = 1; input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, std::nullopt, max_parsing_threads); - input_format->setQueryInfo(query_info, getContext()); if (need_only_count) input_format->needOnlyCount(); diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 28fa010b6d2..1587354452e 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -280,7 +280,6 @@ public: auto input_format = FormatFactory::instance().getInput( format, *read_buf, to_read_block, getContext(), max_block_size, updateFormatSettings(current_file), /* max_parsing_threads */ 1); - input_format->setQueryInfo(query_info, getContext()); Pipe pipe(input_format); if (columns_description.hasDefaults()) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index e98b38438b9..6ca72bdbc87 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -168,7 +168,7 @@ void IStorage::readFromPipe( } else { - auto read_step = std::make_unique(std::move(pipe), storage_name, query_info.storage_limits); + auto read_step = std::make_unique(std::move(pipe), storage_name, query_info, context); query_plan.addStep(std::move(read_step)); } } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 2d643454ecd..666d9d3815d 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -690,7 +690,6 @@ static ActionsDAGPtr cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs return res; } - /** Calculate expressions, that depend only on constants. * For index to work when something like "WHERE Date = toDate(now())" is written. */ diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 93b07b1b8fd..6bc044c4dd4 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -160,7 +160,6 @@ public: bool matchesExactContinuousRange() const; -private: /// The expression is stored as Reverse Polish Notation. struct RPNElement { @@ -207,11 +206,12 @@ private: using RPN = std::vector; using ColumnIndices = std::map; - -public: using AtomMap = std::unordered_map; static const AtomMap atom_map; + const RPN & getRPN() const { return rpn; } + const ColumnIndices & getKeyColumns() const { return key_columns; } + private: BoolMask checkInRange( size_t used_key_size, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index aabf4d379c4..419950c3037 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -474,7 +474,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( auto step = std::make_unique( std::move(pipe), fmt::format("MergeTree(with {} projection {})", query_info.projection->desc->type, query_info.projection->desc->name), - query_info.storage_limits); + query_info, + context); plan->addStep(std::move(step)); plan->addInterpreterContext(query_info.projection->context); return plan; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index a3478069356..a011d1d2c2f 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -346,7 +346,7 @@ void StorageNATS::read( } else { - auto read_step = std::make_unique(std::move(pipe), getName(), query_info.storage_limits); + auto read_step = std::make_unique(std::move(pipe), getName(), query_info, local_context); query_plan.addStep(std::move(read_step)); query_plan.addInterpreterContext(modified_context); } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index ec552dd1032..f78424359bd 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -757,7 +757,7 @@ void StorageRabbitMQ::read( } else { - auto read_step = std::make_unique(std::move(pipe), getName(), query_info.storage_limits); + auto read_step = std::make_unique(std::move(pipe), getName(), query_info, local_context); query_plan.addStep(std::move(read_step)); query_plan.addInterpreterContext(modified_context); } diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 6ea222df71f..5d957d885f5 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -99,11 +99,6 @@ StorageS3QueueSource::StorageS3QueueSource( , remove_file_func(remove_file_func_) , log(&Poco::Logger::get("StorageS3QueueSource")) { - reader = std::move(internal_source->reader); - if (reader) - { - reader_future = std::move(internal_source->reader_future); - } } StorageS3QueueSource::~StorageS3QueueSource() @@ -116,8 +111,22 @@ String StorageS3QueueSource::getName() const return name; } +void StorageS3QueueSource::lazyInitialize() +{ + if (initialized) + return; + + internal_source->lazyInitialize(); + reader = std::move(internal_source->reader); + if (reader) + reader_future = std::move(internal_source->reader_future); + initialized = true; +} + Chunk StorageS3QueueSource::generate() { + lazyInitialize(); + while (true) { if (!reader) diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 7c8eb3eeb74..8af5256899a 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -93,8 +93,10 @@ private: using ReaderHolder = StorageS3Source::ReaderHolder; ReaderHolder reader; std::future reader_future; + std::atomic initialized{false}; size_t processed_rows_from_file = 0; + void lazyInitialize(); void applyActionAfterProcessing(const String & path); void appendLogElement(const std::string & filename, S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); }; diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index a4a686b2691..d0115bf84df 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1137,7 +1137,6 @@ StorageAzureBlobSource::ReaderHolder StorageAzureBlobSource::createReader() format, *read_buf, sample_block, getContext(), max_block_size, format_settings, max_parsing_threads, std::nullopt, /* is_remote_fs */ true, compression_method); - input_format->setQueryInfo(query_info, getContext()); if (need_only_count) input_format->needOnlyCount(); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 856c1f21d27..58c9a973575 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -34,7 +34,7 @@ #include #include #include -#include +#include #include #include #include @@ -925,7 +925,7 @@ static std::chrono::seconds getLockTimeout(ContextPtr context) using StorageFilePtr = std::shared_ptr; -class StorageFileSource : public ISource +class StorageFileSource : public SourceWithKeyCondition { public: class FilesIterator @@ -1000,7 +1000,7 @@ public: FilesIteratorPtr files_iterator_, std::unique_ptr read_buf_, bool need_only_count_) - : ISource(info.source_header, false) + : SourceWithKeyCondition(info.source_header, false) , storage(std::move(storage_)) , storage_snapshot(storage_snapshot_) , files_iterator(std::move(files_iterator_)) @@ -1087,6 +1087,17 @@ public: return storage->getName(); } + void setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) override + { + setKeyConditionImpl(query_info_, context_, block_for_format); + } + + void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override + { + setKeyConditionImpl(nodes, context_, block_for_format); + } + + bool tryGetCountFromCache(const struct stat & file_stat) { if (!context->getSettingsRef().use_cache_for_count_from_files) @@ -1240,7 +1251,10 @@ public: input_format = FormatFactory::instance().getInput( storage->format_name, *read_buf, block_for_format, context, max_block_size, storage->format_settings, max_parsing_threads, std::nullopt, /*is_remote_fs*/ false, CompressionMethod::None, need_only_count); - input_format->setQueryInfo(query_info, context); + + if (key_condition) + input_format->setKeyCondition(key_condition); + if (need_only_count) input_format->needOnlyCount(); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 4f54a938ada..a33e5884bf5 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -498,7 +498,7 @@ size_t StorageS3Source::KeysIterator::estimatedKeysCount() StorageS3Source::ReadTaskIterator::ReadTaskIterator( const DB::ReadTaskCallback & callback_, - const size_t max_threads_count) + size_t max_threads_count) : callback(callback_) { ThreadPool pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, max_threads_count); @@ -546,7 +546,7 @@ StorageS3Source::StorageS3Source( const size_t max_parsing_threads_, bool need_only_count_, std::optional query_info_) - : ISource(info.source_header, false) + : SourceWithKeyCondition(info.source_header, false) , WithContext(context_) , name(std::move(name_)) , bucket(bucket_) @@ -569,9 +569,17 @@ StorageS3Source::StorageS3Source( , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3Reader")) { +} + +void StorageS3Source::lazyInitialize() +{ + if (initialized) + return; + reader = createReader(); if (reader) reader_future = createReaderAsync(); + initialized = true; } StorageS3Source::ReaderHolder StorageS3Source::createReader() @@ -620,8 +628,8 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() compression_method, need_only_count); - if (query_info.has_value()) - input_format->setQueryInfo(query_info.value(), getContext()); + if (key_condition) + input_format->setKeyCondition(key_condition); if (need_only_count) input_format->needOnlyCount(); @@ -736,6 +744,8 @@ String StorageS3Source::getName() const Chunk StorageS3Source::generate() { + lazyInitialize(); + while (true) { if (isCancelled() || !reader) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index adcfa6ebdb9..3330ac6c210 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -11,7 +11,7 @@ #include #include -#include +#include #include #include #include @@ -36,7 +36,7 @@ namespace DB class PullingPipelineExecutor; class NamedCollection; -class StorageS3Source : public ISource, WithContext +class StorageS3Source : public SourceWithKeyCondition, WithContext { public: @@ -120,7 +120,7 @@ public: class ReadTaskIterator : public IIterator { public: - explicit ReadTaskIterator(const ReadTaskCallback & callback_, const size_t max_threads_count); + explicit ReadTaskIterator(const ReadTaskCallback & callback_, size_t max_threads_count); KeyWithInfoPtr next() override; size_t estimatedKeysCount() override; @@ -154,6 +154,16 @@ public: String getName() const override; + void setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) override + { + setKeyConditionImpl(query_info_, context_, sample_block); + } + + void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override + { + setKeyConditionImpl(nodes, context_, sample_block); + } + Chunk generate() override; private: @@ -245,9 +255,14 @@ private: ThreadPool create_reader_pool; ThreadPoolCallbackRunner create_reader_scheduler; std::future reader_future; + std::atomic initialized{false}; size_t total_rows_in_file = 0; + /// Notice: we should initialize reader and future_reader lazily in generate to make sure key_condition + /// is set before createReader is invoked for key_condition is read in createReader. + void lazyInitialize(); + /// Recreate ReadBuffer and Pipeline for each file. ReaderHolder createReader(); std::future createReaderAsync(); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 58f01312399..f2afa2380ab 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -272,12 +272,12 @@ StorageURLSource::StorageURLSource( const ConnectionTimeouts & timeouts, CompressionMethod compression_method, size_t max_parsing_threads, - const SelectQueryInfo & query_info, + const SelectQueryInfo &, const HTTPHeaderEntries & headers_, const URIParams & params, bool glob_url, bool need_only_count_) - : ISource(info.source_header, false), WithContext(context_) + : SourceWithKeyCondition(info.source_header, false), WithContext(context_) , name(std::move(name_)) , columns_description(info.columns_description) , requested_columns(info.requested_columns) @@ -358,7 +358,9 @@ StorageURLSource::StorageURLSource( /* is_remote_ fs */ true, compression_method, need_only_count); - input_format->setQueryInfo(query_info, getContext()); + + if (key_condition) + input_format->setKeyCondition(key_condition); if (need_only_count) input_format->needOnlyCount(); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 6c5c50af326..6306d16742e 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -142,7 +142,7 @@ private: }; -class StorageURLSource : public ISource, WithContext +class StorageURLSource : public SourceWithKeyCondition, WithContext { using URIParams = std::vector>; @@ -184,6 +184,16 @@ public: String getName() const override { return name; } + void setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) override + { + setKeyConditionImpl(query_info_, context_, block_for_format); + } + + void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override + { + setKeyConditionImpl(nodes, context_, block_for_format); + } + Chunk generate() override; static void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri); diff --git a/tests/performance/orc_filter_push_down.xml b/tests/performance/orc_filter_push_down.xml new file mode 100644 index 00000000000..9f49c20a075 --- /dev/null +++ b/tests/performance/orc_filter_push_down.xml @@ -0,0 +1,26 @@ + + + 1 + 10000 + + + + create table test_orc_fpd + ( + a Nullable(Int64), + b Nullable(String) + ) Engine=File(ORC) + + + + insert into test_orc_fpd select number as a, cast(number as String) as b from numbers(10000000) + + + DROP TABLE IF EXISTS test_orc_fpd + + + select a % 10, length(b) % 10, count(1) from test_orc_fpd where a > 9000000 group by a % 10, length(b) % 10 + + + select a % 10, length(b) % 10, count(1) from test_orc_fpd where a in (9000000, 1000) group by a % 10, length(b) % 10 + \ No newline at end of file diff --git a/tests/queries/0_stateless/02892_orc_filter_pushdown.reference b/tests/queries/0_stateless/02892_orc_filter_pushdown.reference new file mode 100644 index 00000000000..9059b403a34 --- /dev/null +++ b/tests/queries/0_stateless/02892_orc_filter_pushdown.reference @@ -0,0 +1,227 @@ +number Nullable(Int64) +u8 Nullable(Int8) +u16 Nullable(Int16) +u32 Nullable(Int32) +u64 Nullable(Int64) +i8 Nullable(Int8) +i16 Nullable(Int16) +i32 Nullable(Int32) +i64 Nullable(Int64) +date32 Nullable(Date32) +dt64_ms Nullable(DateTime64(9)) +dt64_us Nullable(DateTime64(9)) +dt64_ns Nullable(DateTime64(9)) +dt64_s Nullable(DateTime64(9)) +dt64_cs Nullable(DateTime64(9)) +f32 Nullable(Float32) +f64 Nullable(Float64) +s Nullable(String) +fs Nullable(String) +d32 Nullable(Decimal(9, 3)) +d64 Nullable(Decimal(18, 10)) +d128 Nullable(Decimal(38, 20)) +-- Go over all types individually +-- { echoOn } +select count(), sum(number) from file('02892.orc') where indexHint(u8 in (10, 15, 250)); +800 4229600 +select count(1), min(u8), max(u8) from file('02892.orc') where u8 in (10, 15, 250); +66 10 15 +select count(), sum(number) from file('02892.orc') where indexHint(i8 between -3 and 2); +1000 4999500 +select count(1), min(i8), max(i8) from file('02892.orc') where i8 between -3 and 2; +208 -3 2 +select count(), sum(number) from file('02892.orc') where indexHint(u16 between 4000 and 61000 or u16 == 42); +1800 6479100 +select count(1), min(u16), max(u16) from file('02892.orc') where u16 between 4000 and 61000 or u16 == 42; +1002 42 5000 +select count(), sum(number) from file('02892.orc') where indexHint(i16 between -150 and 250); +500 2474750 +select count(1), min(i16), max(i16) from file('02892.orc') where i16 between -150 and 250; +401 -150 250 +select count(), sum(number) from file('02892.orc') where indexHint(u32 in (42, 4294966296)); +200 999900 +select count(1), min(u32), max(u32) from file('02892.orc') where u32 in (42, 4294966296); +1 42 42 +select count(), sum(number) from file('02892.orc') where indexHint(i32 between -150 and 250); +500 2474750 +select count(1), min(i32), max(i32) from file('02892.orc') where i32 between -150 and 250; +401 -150 250 +select count(), sum(number) from file('02892.orc') where indexHint(u64 in (42, 18446744073709550616)); +100 494950 +select count(1), min(u64), max(u64) from file('02892.orc') where u64 in (42, 18446744073709550616); +1 42 42 +select count(), sum(number) from file('02892.orc') where indexHint(i64 between -150 and 250); +500 2474750 +select count(1), min(i64), max(i64) from file('02892.orc') where i64 between -150 and 250; +401 -150 250 +select count(), sum(number) from file('02892.orc') where indexHint(date32 between '1992-01-01' and '2023-08-02'); +2100 5563950 +select count(1), min(date32), max(date32) from file('02892.orc') where date32 between '1992-01-01' and '2023-08-02'; +1994 1992-01-04 2023-08-02 +select count(), sum(number) from file('02892.orc') where indexHint(dt64_ms between '2000-01-01' and '2005-01-01'); +300 1184850 +select count(1), min(dt64_ms), max(dt64_ms) from file('02892.orc') where dt64_ms between '2000-01-01' and '2005-01-01'; +158 2000-01-04 15:33:20.000000000 2004-12-25 18:40:00.000000000 +select count(), sum(number) from file('02892.orc') where indexHint(dt64_us between toDateTime64(900000000, 2) and '2005-01-01'); +400 1599800 +select count(1), min(dt64_us), max(dt64_us) from file('02892.orc') where (dt64_us between toDateTime64(900000000, 2) and '2005-01-01'); +205 1998-07-09 16:00:00.000000000 2004-12-25 18:40:00.000000000 +select count(), sum(number) from file('02892.orc') where indexHint(dt64_ns between '2000-01-01' and '2005-01-01'); +300 1184850 +select count(1), min(dt64_ns), max(dt64_ns) from file('02892.orc') where (dt64_ns between '2000-01-01' and '2005-01-01'); +158 2000-01-04 15:33:20.000000000 2004-12-25 18:40:00.000000000 +select count(), sum(number) from file('02892.orc') where indexHint(dt64_s between toDateTime64('-2.01e8'::Decimal64(0), 0) and toDateTime64(1.5e8::Decimal64(0), 0)); +500 2524750 +select count(1), min(dt64_s), max(dt64_s) from file('02892.orc') where (dt64_s between toDateTime64('-2.01e8'::Decimal64(0), 0) and toDateTime64(1.5e8::Decimal64(0), 0)); +352 1963-08-19 14:40:00.000000000 1974-10-03 02:40:00.000000000 +select count(), sum(number) from file('02892.orc') where indexHint(dt64_cs between toDateTime64('-2.01e8'::Decimal64(1), 1) and toDateTime64(1.5e8::Decimal64(2), 2)); +500 2524750 +select count(1), min(dt64_cs), max(dt64_cs) from file('02892.orc') where (dt64_cs between toDateTime64('-2.01e8'::Decimal64(1), 1) and toDateTime64(1.5e8::Decimal64(2), 2)); +352 1963-08-19 14:40:00.000000000 1974-10-03 02:40:00.000000000 +select count(), sum(number) from file('02892.orc') where indexHint(f32 between -0.11::Float32 and 0.06::Float32); +300 1514850 +select count(1), min(f32), max(f32) from file('02892.orc') where (f32 between -0.11::Float32 and 0.06::Float32); +171 -0.11 0.06 +select count(), sum(number) from file('02892.orc') where indexHint(f64 between -0.11 and 0.06); +300 1514850 +select count(1), min(f64), max(f64) from file('02892.orc') where (f64 between -0.11 and 0.06); +171 -0.11 0.06 +select count(), sum(number) from file('02892.orc') where indexHint(s between '-9' and '1!!!'); +300 1594850 +select count(1), min(s), max(s) from file('02892.orc') where (s between '-9' and '1!!!'); +113 -9 1 +select count(), sum(number) from file('02892.orc') where indexHint(fs between '-9' and '1!!!'); +300 1594850 +select count(1), min(fs), max(fs) from file('02892.orc') where (fs between '-9' and '1!!!'); +113 -9\0\0\0\0\0\0\0 1\0\0\0\0\0\0\0\0 +select count(), sum(number) from file('02892.orc') where indexHint(d32 between '-0.011'::Decimal32(3) and 0.006::Decimal32(3)); +200 999900 +select count(1), min(d32), max(d32) from file('02892.orc') where (d32 between '-0.011'::Decimal32(3) and 0.006::Decimal32(3)); +23 -0.011 0.006 +select count(), sum(number) from file('02892.orc') where indexHint(d64 between '-0.0000011'::Decimal64(7) and 0.0000006::Decimal64(9)); +200 999900 +select count(1), min(d64), max(d64) from file('02892.orc') where (d64 between '-0.0000011'::Decimal64(7) and 0.0000006::Decimal64(9)); +21 -0.000001053 0.000000567 +select count(), sum(number) from file('02892.orc') where indexHint(d128 between '-0.00000000000011'::Decimal128(20) and 0.00000000000006::Decimal128(20)); +200 999900 +select count(1), min(d128), max(128) from file('02892.orc') where (d128 between '-0.00000000000011'::Decimal128(20) and 0.00000000000006::Decimal128(20)); +21 -0.0000000000001053 128 +-- Some random other cases. +select count(), sum(number) from file('02892.orc') where indexHint(0); +0 \N +select count(), min(number), max(number) from file('02892.orc') where indexHint(0); +0 \N \N +select count(), sum(number) from file('02892.orc') where indexHint(s like '99%' or u64 == 2000); +300 1204850 +select count(), min(s), max(s) from file('02892.orc') where (s like '99%' or u64 == 2000); +12 2000 999 +select count(), sum(number) from file('02892.orc') where indexHint(s like 'z%'); +0 \N +select count(), min(s), max(s) from file('02892.orc') where (s like 'z%'); +0 \N \N +select count(), sum(number) from file('02892.orc') where indexHint(u8 == 10 or 1 == 1); +10000 49995000 +select count(), min(u8), max(u8) from file('02892.orc') where (u8 == 10 or 1 == 1); +10000 -128 127 +select count(), sum(number) from file('02892.orc') where indexHint(u8 < 0); +5300 26042350 +select count(), min(u8), max(u8) from file('02892.orc') where (u8 < 0); +5001 -128 -1 +-- { echoOn } +select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null is NULL); +500 244750 +select count(), min(sometimes_null), max(sometimes_null) from file('02892.orc') where (sometimes_null is NULL); +5 \N \N +select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null_lc is NULL); +500 244750 +select count(), min(sometimes_null_lc), max(sometimes_null_lc) from file('02892.orc') where (sometimes_null_lc is NULL); +5 \N \N +select count(), sum(number) from file('02892.orc') where indexHint(mostly_null is not NULL); +300 104850 +select count(), min(mostly_null), max(mostly_null) from file('02892.orc') where (mostly_null is not NULL); +3 0 690 +select count(), sum(number) from file('02892.orc') where indexHint(mostly_null_lc is not NULL); +300 104850 +select count(), min(mostly_null_lc), max(mostly_null_lc) from file('02892.orc') where (mostly_null_lc is not NULL); +3 0 690 +select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null > 850); +200 179900 +select count(), min(sometimes_null), max(sometimes_null) from file('02892.orc') where (sometimes_null > 850); +148 851 999 +select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null_lc > 850); +200 179900 +select count(), min(sometimes_null_lc), max(sometimes_null_lc) from file('02892.orc') where (sometimes_null_lc > 850); +148 851 999 +select count(), sum(number) from file('02892.orc') where indexHint(never_null > 850); +200 179900 +select count(), min(never_null), max(never_null) from file('02892.orc') where (never_null > 850); +149 851 999 +select count(), sum(number) from file('02892.orc') where indexHint(never_null_lc > 850); +200 179900 +select count(), min(never_null_lc), max(never_null_lc) from file('02892.orc') where (never_null_lc > 850); +149 851 999 +select count(), sum(number) from file('02892.orc') where indexHint(never_null < 150); +200 19900 +select count(), min(never_null), max(never_null) from file('02892.orc') where (never_null < 150); +150 0 149 +select count(), sum(number) from file('02892.orc') where indexHint(never_null_lc < 150); +200 19900 +select count(), min(never_null_lc), max(never_null_lc) from file('02892.orc') where (never_null_lc < 150); +150 0 149 +select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null < 150); +200 19900 +select count(), min(sometimes_null), max(sometimes_null) from file('02892.orc') where (sometimes_null < 150); +149 1 149 +select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null_lc < 150); +200 19900 +select count(), min(sometimes_null_lc), max(sometimes_null_lc) from file('02892.orc') where (sometimes_null_lc < 150); +149 1 149 +-- { echoOn } +select count(), sum(number) from file('02892.orc') where indexHint(positive_or_null < 50); -- quirk with infinities +0 \N +select count(), min(positive_or_null), max(positive_or_null) from file('02892.orc') where (positive_or_null < 50); +0 \N \N +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, positive_or_null UInt64') where indexHint(positive_or_null < 50); +1000 499500 +select count(), min(positive_or_null), max(positive_or_null) from file('02892.orc', ORC, 'number UInt64, positive_or_null UInt64') where (positive_or_null < 50); +5 0 0 +select count(), sum(number) from file('02892.orc') where indexHint(negative_or_null > -50); +0 \N +select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc') where (negative_or_null > -50); +0 \N \N +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null > -50); +1000 499500 +select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null > -50); +5 0 0 +select count(), sum(number) from file('02892.orc') where indexHint(string_or_null == ''); -- quirk with infinities +0 \N +select count(), min(string_or_null), max(string_or_null) from file('02892.orc') where (string_or_null == ''); +0 \N \N +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, string_or_null String') where indexHint(string_or_null == ''); +1000 499500 +select count(), min(string_or_null), max(string_or_null) from file('02892.orc', ORC, 'number UInt64, string_or_null String') where (string_or_null == ''); +5 +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, nEgAtIvE_oR_nUlL Int64') where indexHint(nEgAtIvE_oR_nUlL > -50) settings input_format_orc_case_insensitive_column_matching = 1; +1000 499500 +select count(), min(nEgAtIvE_oR_nUlL), max(nEgAtIvE_oR_nUlL) from file('02892.orc', ORC, 'number UInt64, nEgAtIvE_oR_nUlL Int64') where (nEgAtIvE_oR_nUlL > -50) settings input_format_orc_case_insensitive_column_matching = 1; +5 0 0 +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null < -500); +600 419700 +select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null < -500); +596 -1099 -501 +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null is null); +1000 499500 +select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null is null); +0 0 0 +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null in (0, -1, -10, -100, -1000)); +1000 499500 +select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null in (0, -1, -10, -100, -1000)); +6 -1000 0 +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, string_or_null LowCardinality(String)') where indexHint(string_or_null like 'I am%'); +1000 499500 +select count(), min(string_or_null), max(string_or_null) from file('02892.orc', ORC, 'number UInt64, string_or_null LowCardinality(String)') where (string_or_null like 'I am%'); +995 I am a string I am a string +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, string_or_null LowCardinality(Nullable(String))') where indexHint(string_or_null like 'I am%'); +1000 499500 +select count(), min(string_or_null), max(string_or_null) from file('02892.orc', ORC, 'number UInt64, string_or_null LowCardinality(Nullable(String))') where (string_or_null like 'I am%'); +995 I am a string I am a string diff --git a/tests/queries/0_stateless/02892_orc_filter_pushdown.sql b/tests/queries/0_stateless/02892_orc_filter_pushdown.sql new file mode 100644 index 00000000000..d319252f592 --- /dev/null +++ b/tests/queries/0_stateless/02892_orc_filter_pushdown.sql @@ -0,0 +1,229 @@ +-- Tags: no-fasttest, no-parallel, no-cpu-aarch64 + +set output_format_orc_string_as_string = 1; +set output_format_orc_row_index_stride = 100; +set input_format_orc_row_batch_size = 100; +set input_format_orc_filter_push_down = 1; +set input_format_null_as_default = 1; + +set engine_file_truncate_on_insert = 1; +set optimize_or_like_chain = 0; +set max_block_size = 100000; +set max_insert_threads = 1; + +SET session_timezone = 'UTC'; + +-- Analyzer breaks the queries with IN and some queries with BETWEEN. +set allow_experimental_analyzer=0; + + +-- Try all the types. +insert into function file('02892.orc') + -- Use negative numbers to test sign extension for signed types and lack of sign extension for + -- unsigned types. + with 5000 - number as n +select + number, + intDiv(n, 11)::UInt8 as u8, + n::UInt16 u16, + n::UInt32 as u32, + n::UInt64 as u64, + intDiv(n, 11)::Int8 as i8, + n::Int16 i16, + n::Int32 as i32, + n::Int64 as i64, + + toDate32(n*500000) as date32, + toDateTime64(n*1e6, 3) as dt64_ms, + toDateTime64(n*1e6, 6) as dt64_us, + toDateTime64(n*1e6, 9) as dt64_ns, + toDateTime64(n*1e6, 0) as dt64_s, + toDateTime64(n*1e6, 2) as dt64_cs, + (n/1000)::Float32 as f32, + (n/1000)::Float64 as f64, + n::String as s, + n::String::FixedString(9) as fs, + n::Decimal32(3)/1234 as d32, + n::Decimal64(10)/12345678 as d64, + n::Decimal128(20)/123456789012345 as d128 + from numbers(10000); + +desc file('02892.orc'); + + +-- Go over all types individually +-- { echoOn } +select count(), sum(number) from file('02892.orc') where indexHint(u8 in (10, 15, 250)); +select count(1), min(u8), max(u8) from file('02892.orc') where u8 in (10, 15, 250); + +select count(), sum(number) from file('02892.orc') where indexHint(i8 between -3 and 2); +select count(1), min(i8), max(i8) from file('02892.orc') where i8 between -3 and 2; + +select count(), sum(number) from file('02892.orc') where indexHint(u16 between 4000 and 61000 or u16 == 42); +select count(1), min(u16), max(u16) from file('02892.orc') where u16 between 4000 and 61000 or u16 == 42; + +select count(), sum(number) from file('02892.orc') where indexHint(i16 between -150 and 250); +select count(1), min(i16), max(i16) from file('02892.orc') where i16 between -150 and 250; + +select count(), sum(number) from file('02892.orc') where indexHint(u32 in (42, 4294966296)); +select count(1), min(u32), max(u32) from file('02892.orc') where u32 in (42, 4294966296); + +select count(), sum(number) from file('02892.orc') where indexHint(i32 between -150 and 250); +select count(1), min(i32), max(i32) from file('02892.orc') where i32 between -150 and 250; + +select count(), sum(number) from file('02892.orc') where indexHint(u64 in (42, 18446744073709550616)); +select count(1), min(u64), max(u64) from file('02892.orc') where u64 in (42, 18446744073709550616); + +select count(), sum(number) from file('02892.orc') where indexHint(i64 between -150 and 250); +select count(1), min(i64), max(i64) from file('02892.orc') where i64 between -150 and 250; + +select count(), sum(number) from file('02892.orc') where indexHint(date32 between '1992-01-01' and '2023-08-02'); +select count(1), min(date32), max(date32) from file('02892.orc') where date32 between '1992-01-01' and '2023-08-02'; + +select count(), sum(number) from file('02892.orc') where indexHint(dt64_ms between '2000-01-01' and '2005-01-01'); +select count(1), min(dt64_ms), max(dt64_ms) from file('02892.orc') where dt64_ms between '2000-01-01' and '2005-01-01'; + +select count(), sum(number) from file('02892.orc') where indexHint(dt64_us between toDateTime64(900000000, 2) and '2005-01-01'); +select count(1), min(dt64_us), max(dt64_us) from file('02892.orc') where (dt64_us between toDateTime64(900000000, 2) and '2005-01-01'); + +select count(), sum(number) from file('02892.orc') where indexHint(dt64_ns between '2000-01-01' and '2005-01-01'); +select count(1), min(dt64_ns), max(dt64_ns) from file('02892.orc') where (dt64_ns between '2000-01-01' and '2005-01-01'); + +select count(), sum(number) from file('02892.orc') where indexHint(dt64_s between toDateTime64('-2.01e8'::Decimal64(0), 0) and toDateTime64(1.5e8::Decimal64(0), 0)); +select count(1), min(dt64_s), max(dt64_s) from file('02892.orc') where (dt64_s between toDateTime64('-2.01e8'::Decimal64(0), 0) and toDateTime64(1.5e8::Decimal64(0), 0)); + +select count(), sum(number) from file('02892.orc') where indexHint(dt64_cs between toDateTime64('-2.01e8'::Decimal64(1), 1) and toDateTime64(1.5e8::Decimal64(2), 2)); +select count(1), min(dt64_cs), max(dt64_cs) from file('02892.orc') where (dt64_cs between toDateTime64('-2.01e8'::Decimal64(1), 1) and toDateTime64(1.5e8::Decimal64(2), 2)); + +select count(), sum(number) from file('02892.orc') where indexHint(f32 between -0.11::Float32 and 0.06::Float32); +select count(1), min(f32), max(f32) from file('02892.orc') where (f32 between -0.11::Float32 and 0.06::Float32); + +select count(), sum(number) from file('02892.orc') where indexHint(f64 between -0.11 and 0.06); +select count(1), min(f64), max(f64) from file('02892.orc') where (f64 between -0.11 and 0.06); + +select count(), sum(number) from file('02892.orc') where indexHint(s between '-9' and '1!!!'); +select count(1), min(s), max(s) from file('02892.orc') where (s between '-9' and '1!!!'); + +select count(), sum(number) from file('02892.orc') where indexHint(fs between '-9' and '1!!!'); +select count(1), min(fs), max(fs) from file('02892.orc') where (fs between '-9' and '1!!!'); + +select count(), sum(number) from file('02892.orc') where indexHint(d32 between '-0.011'::Decimal32(3) and 0.006::Decimal32(3)); +select count(1), min(d32), max(d32) from file('02892.orc') where (d32 between '-0.011'::Decimal32(3) and 0.006::Decimal32(3)); + +select count(), sum(number) from file('02892.orc') where indexHint(d64 between '-0.0000011'::Decimal64(7) and 0.0000006::Decimal64(9)); +select count(1), min(d64), max(d64) from file('02892.orc') where (d64 between '-0.0000011'::Decimal64(7) and 0.0000006::Decimal64(9)); + +select count(), sum(number) from file('02892.orc') where indexHint(d128 between '-0.00000000000011'::Decimal128(20) and 0.00000000000006::Decimal128(20)); +select count(1), min(d128), max(128) from file('02892.orc') where (d128 between '-0.00000000000011'::Decimal128(20) and 0.00000000000006::Decimal128(20)); + +-- Some random other cases. +select count(), sum(number) from file('02892.orc') where indexHint(0); +select count(), min(number), max(number) from file('02892.orc') where indexHint(0); + +select count(), sum(number) from file('02892.orc') where indexHint(s like '99%' or u64 == 2000); +select count(), min(s), max(s) from file('02892.orc') where (s like '99%' or u64 == 2000); + +select count(), sum(number) from file('02892.orc') where indexHint(s like 'z%'); +select count(), min(s), max(s) from file('02892.orc') where (s like 'z%'); + +select count(), sum(number) from file('02892.orc') where indexHint(u8 == 10 or 1 == 1); +select count(), min(u8), max(u8) from file('02892.orc') where (u8 == 10 or 1 == 1); + +select count(), sum(number) from file('02892.orc') where indexHint(u8 < 0); +select count(), min(u8), max(u8) from file('02892.orc') where (u8 < 0); +-- { echoOff } + +-- Nullable and LowCardinality. +insert into function file('02892.orc') select + number, + if(number%234 == 0, NULL, number) as sometimes_null, + toNullable(number) as never_null, + if(number%345 == 0, number::String, NULL) as mostly_null, + toLowCardinality(if(number%234 == 0, NULL, number)) as sometimes_null_lc, + toLowCardinality(toNullable(number)) as never_null_lc, + toLowCardinality(if(number%345 == 0, number::String, NULL)) as mostly_null_lc + from numbers(1000); + +-- { echoOn } +select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null is NULL); +select count(), min(sometimes_null), max(sometimes_null) from file('02892.orc') where (sometimes_null is NULL); + +select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null_lc is NULL); +select count(), min(sometimes_null_lc), max(sometimes_null_lc) from file('02892.orc') where (sometimes_null_lc is NULL); + +select count(), sum(number) from file('02892.orc') where indexHint(mostly_null is not NULL); +select count(), min(mostly_null), max(mostly_null) from file('02892.orc') where (mostly_null is not NULL); + +select count(), sum(number) from file('02892.orc') where indexHint(mostly_null_lc is not NULL); +select count(), min(mostly_null_lc), max(mostly_null_lc) from file('02892.orc') where (mostly_null_lc is not NULL); + +select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null > 850); +select count(), min(sometimes_null), max(sometimes_null) from file('02892.orc') where (sometimes_null > 850); + +select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null_lc > 850); +select count(), min(sometimes_null_lc), max(sometimes_null_lc) from file('02892.orc') where (sometimes_null_lc > 850); + +select count(), sum(number) from file('02892.orc') where indexHint(never_null > 850); +select count(), min(never_null), max(never_null) from file('02892.orc') where (never_null > 850); + +select count(), sum(number) from file('02892.orc') where indexHint(never_null_lc > 850); +select count(), min(never_null_lc), max(never_null_lc) from file('02892.orc') where (never_null_lc > 850); + +select count(), sum(number) from file('02892.orc') where indexHint(never_null < 150); +select count(), min(never_null), max(never_null) from file('02892.orc') where (never_null < 150); + +select count(), sum(number) from file('02892.orc') where indexHint(never_null_lc < 150); +select count(), min(never_null_lc), max(never_null_lc) from file('02892.orc') where (never_null_lc < 150); + +select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null < 150); +select count(), min(sometimes_null), max(sometimes_null) from file('02892.orc') where (sometimes_null < 150); + +select count(), sum(number) from file('02892.orc') where indexHint(sometimes_null_lc < 150); +select count(), min(sometimes_null_lc), max(sometimes_null_lc) from file('02892.orc') where (sometimes_null_lc < 150); +-- { echoOff } + +-- Settings that affect the table schema or contents. +insert into function file('02892.orc') select + number, + if(number%234 == 0, NULL, number + 100) as positive_or_null, + if(number%234 == 0, NULL, -number - 100) as negative_or_null, + if(number%234 == 0, NULL, 'I am a string') as string_or_null + from numbers(1000); + +-- { echoOn } +select count(), sum(number) from file('02892.orc') where indexHint(positive_or_null < 50); -- quirk with infinities +select count(), min(positive_or_null), max(positive_or_null) from file('02892.orc') where (positive_or_null < 50); + +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, positive_or_null UInt64') where indexHint(positive_or_null < 50); +select count(), min(positive_or_null), max(positive_or_null) from file('02892.orc', ORC, 'number UInt64, positive_or_null UInt64') where (positive_or_null < 50); + +select count(), sum(number) from file('02892.orc') where indexHint(negative_or_null > -50); +select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc') where (negative_or_null > -50); + +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null > -50); +select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null > -50); + +select count(), sum(number) from file('02892.orc') where indexHint(string_or_null == ''); -- quirk with infinities +select count(), min(string_or_null), max(string_or_null) from file('02892.orc') where (string_or_null == ''); + +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, string_or_null String') where indexHint(string_or_null == ''); +select count(), min(string_or_null), max(string_or_null) from file('02892.orc', ORC, 'number UInt64, string_or_null String') where (string_or_null == ''); + +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, nEgAtIvE_oR_nUlL Int64') where indexHint(nEgAtIvE_oR_nUlL > -50) settings input_format_orc_case_insensitive_column_matching = 1; +select count(), min(nEgAtIvE_oR_nUlL), max(nEgAtIvE_oR_nUlL) from file('02892.orc', ORC, 'number UInt64, nEgAtIvE_oR_nUlL Int64') where (nEgAtIvE_oR_nUlL > -50) settings input_format_orc_case_insensitive_column_matching = 1; + +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null < -500); +select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null < -500); + +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null is null); +select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null is null); + +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null in (0, -1, -10, -100, -1000)); +select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null in (0, -1, -10, -100, -1000)); + +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, string_or_null LowCardinality(String)') where indexHint(string_or_null like 'I am%'); +select count(), min(string_or_null), max(string_or_null) from file('02892.orc', ORC, 'number UInt64, string_or_null LowCardinality(String)') where (string_or_null like 'I am%'); + +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, string_or_null LowCardinality(Nullable(String))') where indexHint(string_or_null like 'I am%'); +select count(), min(string_or_null), max(string_or_null) from file('02892.orc', ORC, 'number UInt64, string_or_null LowCardinality(Nullable(String))') where (string_or_null like 'I am%'); +-- { echoOff } From d378e10a4daeff2716727aa9f05d6c367fe92bf1 Mon Sep 17 00:00:00 2001 From: MeenaRenganathan22 Date: Tue, 24 Oct 2023 14:09:46 -0700 Subject: [PATCH 574/634] Fixed the sytle check --- .../queries/0_stateless/02550_client_connections_credentials.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02550_client_connections_credentials.sh b/tests/queries/0_stateless/02550_client_connections_credentials.sh index 6b1ca1843ee..cc023cad930 100755 --- a/tests/queries/0_stateless/02550_client_connections_credentials.sh +++ b/tests/queries/0_stateless/02550_client_connections_credentials.sh @@ -82,7 +82,7 @@ $CLICKHOUSE_CLIENT --config $CONFIG --connection test_port -q 'select tcpPort()' $CLICKHOUSE_CLIENT --config $CONFIG --connection test_port --port $TEST_PORT -q 'select tcpPort()' echo 'secure' -if [ `uname -m` == 's390x' ]; then +if [ "`uname -m`" == 's390x' ]; then $CLICKHOUSE_CLIENT --config $CONFIG --connection test_secure -q 'select tcpPort()' |& grep -c -F -o -e 'SSL routines::wrong version number' -e 'tcp_secure protocol is disabled because poco library was built without NetSSL support.' else $CLICKHOUSE_CLIENT --config $CONFIG --connection test_secure -q 'select tcpPort()' |& grep -c -F -o -e OPENSSL_internal:WRONG_VERSION_NUMBER -e 'tcp_secure protocol is disabled because poco library was built without NetSSL support.' From ca766e3033d271184875f3b0a966b596644ccf27 Mon Sep 17 00:00:00 2001 From: jetgm Date: Wed, 25 Oct 2023 11:08:25 +0800 Subject: [PATCH 575/634] update doc and change order by --- docs/en/sql-reference/statements/show.md | 12 ++++++------ src/Interpreters/InterpreterShowTablesQuery.cpp | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 0567a300738..5bf77ef06a1 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -674,9 +674,9 @@ SHOW MERGES; Result: ```text -┌─table──────┬─database─┬─────elapsed─┬─merge_type─┐ -│ your_table │ default │ 0.039891507 │ Regular │ -└────────────┴──────────┴─────────────┴────────────┘ +┌─table──────┬─database─┬─estimate_complete─┬─────elapsed─┬─progress─┬─is_mutation─┬─size─────┬─mem───────┐ +│ your_table │ default │ 0.14 │ 0.365592338 │ 0.73 │ 0 │ 5.40 MiB │ 10.25 MiB │ +└────────────┴──────────┴───────────────────┴─────────────┴──────────┴─────────────┴────────────┴─────────┘ ``` @@ -689,9 +689,9 @@ SHOW MERGES LIKE 'your_t%' LIMIT 1; Result: ```text -┌─table──────┬─database─┬─────elapsed─┬─merge_type─┐ -│ your_table │ default │ 0.049192524 │ Regular │ -└────────────┴──────────┴─────────────┴────────────┘ +┌─table──────┬─database─┬─estimate_complete─┬─────elapsed─┬─progress─┬─is_mutation─┬─size─────┬─mem───────┐ +│ your_table │ default │ 0.05 │ 1.727629065 │ 0.97 │ 0 │ 5.40 MiB │ 10.25 MiB │ +└────────────┴──────────┴───────────────────┴─────────────┴──────────┴─────────────┴────────────┴─────────┘ ``` diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index e5a286a15e1..60887e470c6 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -134,7 +134,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery() } /// (*) - rewritten_query << " ORDER BY elapsed desc, database, table"; + rewritten_query << " ORDER BY elapsed desc"; if (query.limit_length) rewritten_query << " LIMIT " << query.limit_length; From 46d34a412f9feead9ac446a73a32d939e53a0f89 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 25 Oct 2023 07:56:10 +0000 Subject: [PATCH 576/634] Fix style --- src/Functions/FunctionSnowflake.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index 42b71c789f0..26c8138633b 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -19,7 +19,6 @@ namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } From 9838e0ce3a739031d4c343131f32d5f91afd460b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 25 Oct 2023 08:01:57 +0000 Subject: [PATCH 577/634] Update docs --- .../functions/type-conversion-functions.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index e037c00e2eb..ed2575d1110 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1982,7 +1982,7 @@ Result: ## snowflakeToDateTime -Extracts time from [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as [DateTime](/docs/en/sql-reference/data-types/datetime.md) format. +Extracts the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) in [DateTime](/docs/en/sql-reference/data-types/datetime.md) format. **Syntax** @@ -1997,7 +1997,7 @@ snowflakeToDateTime(value[, time_zone]) **Returned value** -- Input value converted to the [DateTime](/docs/en/sql-reference/data-types/datetime.md) data type. +- The timestamp component of `value` as a [DateTime](/docs/en/sql-reference/data-types/datetime.md) value. **Example** @@ -2018,7 +2018,7 @@ Result: ## snowflakeToDateTime64 -Extracts time from [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) format. +Extracts the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) in [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) format. **Syntax** @@ -2033,7 +2033,7 @@ snowflakeToDateTime64(value[, time_zone]) **Returned value** -- Input value converted to the [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) data type with millisecond precision (scale = 3). +- The timestamp component of `value` as a [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) with scale = 3, i.e. millisecond precison. **Example** @@ -2054,7 +2054,7 @@ Result: ## dateTimeToSnowflake -Converts [DateTime](/docs/en/sql-reference/data-types/datetime.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. +Converts a [DateTime](/docs/en/sql-reference/data-types/datetime.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. **Syntax** @@ -2064,7 +2064,7 @@ dateTimeToSnowflake(value) **Parameters** -- `value` — Date and time. [DateTime](/docs/en/sql-reference/data-types/datetime.md). +- `value` — Date with time. [DateTime](/docs/en/sql-reference/data-types/datetime.md). **Returned value** @@ -2088,7 +2088,7 @@ Result: ## dateTime64ToSnowflake -Convert [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. +Convert a [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. **Syntax** @@ -2098,7 +2098,7 @@ dateTime64ToSnowflake(value) **Parameters** -- `value` — Date and time. [DateTime64](/docs/en/sql-reference/data-types/datetime64.md). +- `value` — Date with time. [DateTime64](/docs/en/sql-reference/data-types/datetime64.md). **Returned value** From a95a2a5d663d2273ac0642409b5abf9e44b0b583 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 25 Oct 2023 10:10:04 +0200 Subject: [PATCH 578/634] formatQueryOneLine: create new function format queries on one line --- .../functions/other-functions.md | 72 +++++++++++++++++++ src/Functions/formatQuery.cpp | 43 +++++++++-- .../0_stateless/02882_formatQuery.reference | 10 +++ .../queries/0_stateless/02882_formatQuery.sql | 11 +++ 4 files changed, 132 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index eb8b63434f9..960f83e229a 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2838,3 +2838,75 @@ Result: │ SELECT 1 │ └──────────────────────────┘ ``` + +```sql +SELECT formatQuery('select a, b FRom tab WHERE a > 3 and b < 3'); +``` + +Result: + +```result +┌─formatQuery('select a, b FRom tab WHERE a > 3 and b < 3')─┐ +│ SELECT + a, + b +FROM tab +WHERE (a > 3) AND (b < 3) │ +└───────────────────────────────────────────────────────────────┘ +``` + +## formatQueryOneLine + +Returns a formatted version of the given SQL query on a single line. + +**Syntax** + +```sql +formatQueryOneLine(query) +``` + +**Arguments** + +- `query` - The SQL query to be formatted. [String](../../sql-reference/data-types/string.md) + +**Returned value** + +- The formatted query. [String](../../sql-reference/data-types/string.md). + +**Example** + +```sql +SELECT formatQueryOneLine('select 1;'); +``` + +Result: + +```result +┌─formatQueryOneLine('select 1;')─┐ +│ SELECT 1 │ +└─────────────────────────────────┘ +``` + +```sql +SELECT formatQueryOneLine('SeLecT 1'); +``` + +Result: + +```result +┌─formatQueryOneLine('SeLecT 1')──┐ +│ SELECT 1 │ +└─────────────────────────────────┘ +``` + +```sql +SELECT formatQueryOneLine('select a, b FRom tab WHERE a > 3 and b < 3'); +``` + +Result: + +```result +┌─formatQueryOneLine('select a, b FRom tab WHERE a > 3 and b < 3')─┐ +│ SELECT a, b FROM tab WHERE (a > 3) AND (b < 3) │ +└──────────────────────────────────────────────────────────────────────┘ +``` diff --git a/src/Functions/formatQuery.cpp b/src/Functions/formatQuery.cpp index 7c574b701d6..6fdeb9ca752 100644 --- a/src/Functions/formatQuery.cpp +++ b/src/Functions/formatQuery.cpp @@ -16,10 +16,11 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +template class FunctionFormatQuery : public IFunction { public: - static constexpr auto name = "formatQuery"; + static constexpr auto name = Name::name; static FunctionPtr create(ContextPtr context) { const auto & settings = context->getSettings(); @@ -66,7 +67,7 @@ private: ParserQuery parser{end}; auto ast = parseQuery(parser, begin, end, {}, max_query_size, max_parser_depth); WriteBufferFromVector buf(output, AppendModeTag{}); - formatAST(*ast, buf, /* hilite */ false); + formatAST(*ast, buf, /* hilite */ false, /* one_line */ one_line); buf.finalize(); } void formatVector( @@ -93,15 +94,49 @@ private: size_t max_parser_depth; }; +struct NameFormatQuery +{ + static constexpr auto name = "formatQuery"; +}; + +struct NameFormatQueryOneLine +{ + static constexpr auto name = "formatQueryOneLine"; +}; REGISTER_FUNCTION(formatQuery) { - factory.registerFunction(FunctionDocumentation{ + factory.registerFunction>(FunctionDocumentation{ .description = "Returns a formatted version of the given SQL query.\n[example:simple]\n[example:camelcase]", .syntax = "formatQuery(query)", .arguments = {{"query", "The SQL query to be formatted. [String](../../sql-reference/data-types/string.md)"}}, .returned_value = "The formatted query. [String](../../sql-reference/data-types/string.md).", - .examples{{"simple", "SELECT formatQuery('select 1;')", "SELECT 1"}, {"camelcase", "SELECT formatQuery('SeLecT 1')", "SELECT 1"}}, + .examples{ + {"simple", "SELECT formatQuery('select 1;')", "SELECT 1"}, + {"camelcase", "SELECT formatQuery('SeLecT 1')", "SELECT 1"}, + {"multiline", + "SELECT formatQuery('select a, b FRom tab WHERE a > 3 and b < 3');", + "SELECT\n" + " a,\n" + " b\n" + "FROM tab\n" + "WHERE (a > 3) AND (b < 3)"}}, + .categories{"Other"}}); +} + +REGISTER_FUNCTION(formatQueryOneLine) +{ + factory.registerFunction>(FunctionDocumentation{ + .description = "Returns a formatted version of the given SQL query on a single line.\n[example:simple]\n[example:camelcase]", + .syntax = "formatQueryOneLine(query)", + .arguments = {{"query", "The SQL query to be formatted. [String](../../sql-reference/data-types/string.md)"}}, + .returned_value = "The formatted query. [String](../../sql-reference/data-types/string.md).", + .examples{ + {"simple", "SELECT formatQueryOneLine('select 1;')", "SELECT 1"}, + {"camelcase", "SELECT formatQueryOneLine('SeLecT 1')", "SELECT 1"}, + {"multiline", + "SELECT formatQuery('select a, b FRom tab WHERE a > 3 and b < 3');", + "SELECT a, b FROM tab WHERE (a > 3) AND (b < 3)"}}, .categories{"Other"}}); } } diff --git a/tests/queries/0_stateless/02882_formatQuery.reference b/tests/queries/0_stateless/02882_formatQuery.reference index ce0f08bb15a..a7bd2dbe822 100644 --- a/tests/queries/0_stateless/02882_formatQuery.reference +++ b/tests/queries/0_stateless/02882_formatQuery.reference @@ -7,3 +7,13 @@ INSERT INTO tab FORMAT Values CREATE TABLE default.no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 EXPLAIN SYNTAX\nSELECT\n CAST(1, \'INT\'),\n CEIL(1),\n CEILING(1),\n CHAR(49),\n CHAR_LENGTH(\'1\'),\n CHARACTER_LENGTH(\'1\'),\n COALESCE(1),\n CONCAT(\'1\', \'1\'),\n CORR(1, 1),\n COS(1),\n COUNT(1),\n COVAR_POP(1, 1),\n COVAR_SAMP(1, 1),\n DATABASE(),\n SCHEMA(),\n dateDiff(\'DAY\', toDate(\'2020-10-24\'), toDate(\'2019-10-24\')),\n EXP(1),\n FLATTEN([[1]]),\n FLOOR(1),\n FQDN(),\n GREATEST(1),\n IF(1, 1, 1),\n IFNULL(1, 1),\n LCASE(\'A\'),\n LEAST(1),\n LENGTH(\'1\'),\n LN(1),\n LOCATE(\'1\', \'1\'),\n LOG(1),\n LOG10(1),\n LOG2(1),\n LOWER(\'A\'),\n MAX(1),\n MID(\'123\', 1, 1),\n MIN(1),\n MOD(1, 1),\n NOT 1,\n NOW(),\n NOW64(),\n NULLIF(1, 1),\n PI(),\n position(\'123\', \'2\'),\n POW(1, 1),\n POWER(1, 1),\n RAND(),\n REPLACE(\'1\', \'1\', \'2\'),\n REVERSE(\'123\'),\n ROUND(1),\n SIN(1),\n SQRT(1),\n STDDEV_POP(1),\n STDDEV_SAMP(1),\n SUBSTR(\'123\', 2),\n substring(\'123\', 2),\n SUM(1),\n TAN(1),\n TANH(1),\n TRUNC(1),\n TRUNCATE(1),\n UCASE(\'A\'),\n UPPER(\'A\'),\n USER(),\n VAR_POP(1),\n VAR_SAMP(1),\n WEEK(toDate(\'2020-10-24\')),\n YEARWEEK(toDate(\'2020-10-24\'))\nFORMAT TSVRaw 1 +formatQueryOneLine +SELECT 1 +SELECT 1 +SELECT 1 +SELECT 1 +1 +INSERT INTO tab FORMAT Values +CREATE TABLE default.no_prop_table (`some_column` UInt64) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192 +EXPLAIN SYNTAX SELECT CAST(1, \'INT\'), CEIL(1), CEILING(1), CHAR(49), CHAR_LENGTH(\'1\'), CHARACTER_LENGTH(\'1\'), COALESCE(1), CONCAT(\'1\', \'1\'), CORR(1, 1), COS(1), COUNT(1), COVAR_POP(1, 1), COVAR_SAMP(1, 1), DATABASE(), SCHEMA(), dateDiff(\'DAY\', toDate(\'2020-10-24\'), toDate(\'2019-10-24\')), EXP(1), FLATTEN([[1]]), FLOOR(1), FQDN(), GREATEST(1), IF(1, 1, 1), IFNULL(1, 1), LCASE(\'A\'), LEAST(1), LENGTH(\'1\'), LN(1), LOCATE(\'1\', \'1\'), LOG(1), LOG10(1), LOG2(1), LOWER(\'A\'), MAX(1), MID(\'123\', 1, 1), MIN(1), MOD(1, 1), NOT 1, NOW(), NOW64(), NULLIF(1, 1), PI(), position(\'123\', \'2\'), POW(1, 1), POWER(1, 1), RAND(), REPLACE(\'1\', \'1\', \'2\'), REVERSE(\'123\'), ROUND(1), SIN(1), SQRT(1), STDDEV_POP(1), STDDEV_SAMP(1), SUBSTR(\'123\', 2), substring(\'123\', 2), SUM(1), TAN(1), TANH(1), TRUNC(1), TRUNCATE(1), UCASE(\'A\'), UPPER(\'A\'), USER(), VAR_POP(1), VAR_SAMP(1), WEEK(toDate(\'2020-10-24\')), YEARWEEK(toDate(\'2020-10-24\')) FORMAT TSVRaw +1 diff --git a/tests/queries/0_stateless/02882_formatQuery.sql b/tests/queries/0_stateless/02882_formatQuery.sql index 369523c3eb8..9edb07cb0e3 100644 --- a/tests/queries/0_stateless/02882_formatQuery.sql +++ b/tests/queries/0_stateless/02882_formatQuery.sql @@ -8,3 +8,14 @@ SELECT formatQuery('CREATE TABLE default.no_prop_table(`some_column` UInt64) ENG SELECT formatQuery('EXPLAIN SYNTAX SELECT CAST(1 AS INT), CEIL(1), CEILING(1), CHAR(49), CHAR_LENGTH(\'1\'), CHARACTER_LENGTH(\'1\'), COALESCE(1), CONCAT(\'1\', \'1\'), CORR(1, 1), COS(1), COUNT(1), COVAR_POP(1, 1), COVAR_SAMP(1, 1), DATABASE(), SCHEMA(), DATEDIFF(\'DAY\', toDate(\'2020-10-24\'), toDate(\'2019-10-24\')), EXP(1), FLATTEN([[1]]), FLOOR(1), FQDN(), GREATEST(1), IF(1, 1, 1), IFNULL(1, 1), LCASE(\'A\'), LEAST(1), LENGTH(\'1\'), LN(1), LOCATE(\'1\', \'1\'), LOG(1), LOG10(1), LOG2(1), LOWER(\'A\'), MAX(1), MID(\'123\', 1, 1), MIN(1), MOD(1, 1), NOT(1), NOW(), NOW64(), NULLIF(1, 1), PI(), POSITION(\'123\', \'2\'), POW(1, 1), POWER(1, 1), RAND(), REPLACE(\'1\', \'1\', \'2\'), REVERSE(\'123\'), ROUND(1), SIN(1), SQRT(1), STDDEV_POP(1), STDDEV_SAMP(1), SUBSTR(\'123\', 2), SUBSTRING(\'123\', 2), SUM(1), TAN(1), TANH(1), TRUNC(1), TRUNCATE(1), UCASE(\'A\'), UPPER(\'A\'), USER(), VAR_POP(1), VAR_SAMP(1), WEEK(toDate(\'2020-10-24\')), YEARWEEK(toDate(\'2020-10-24\')) format TSVRaw;'); SELECT normalizedQueryHash(formatQuery('select 1')) = normalizedQueryHash(formatQuery('SELECT 1')); SELECT formatQuery('SEECTwrong'); -- { serverError SYNTAX_ERROR } +SELECT 'formatQueryOneLine'; +SELECT formatQueryOneLine('select 1;'); +SELECT formatQueryOneLine('select 1'); +SELECT formatQueryOneLine('SELECT 1;'); +SELECT formatQueryOneLine('SELECT 1'); +SELECT formatQueryOneLine('select 1;') == formatQueryOneLine('SeLecT 1'); +SELECT formatQueryOneLine('INSERT INTO tab VALUES (\'\') (\'test\')'); +SELECT formatQueryOneLine('CREATE TABLE default.no_prop_table(`some_column` UInt64) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192'); +SELECT formatQueryOneLine('EXPLAIN SYNTAX SELECT CAST(1 AS INT), CEIL(1), CEILING(1), CHAR(49), CHAR_LENGTH(\'1\'), CHARACTER_LENGTH(\'1\'), COALESCE(1), CONCAT(\'1\', \'1\'), CORR(1, 1), COS(1), COUNT(1), COVAR_POP(1, 1), COVAR_SAMP(1, 1), DATABASE(), SCHEMA(), DATEDIFF(\'DAY\', toDate(\'2020-10-24\'), toDate(\'2019-10-24\')), EXP(1), FLATTEN([[1]]), FLOOR(1), FQDN(), GREATEST(1), IF(1, 1, 1), IFNULL(1, 1), LCASE(\'A\'), LEAST(1), LENGTH(\'1\'), LN(1), LOCATE(\'1\', \'1\'), LOG(1), LOG10(1), LOG2(1), LOWER(\'A\'), MAX(1), MID(\'123\', 1, 1), MIN(1), MOD(1, 1), NOT(1), NOW(), NOW64(), NULLIF(1, 1), PI(), POSITION(\'123\', \'2\'), POW(1, 1), POWER(1, 1), RAND(), REPLACE(\'1\', \'1\', \'2\'), REVERSE(\'123\'), ROUND(1), SIN(1), SQRT(1), STDDEV_POP(1), STDDEV_SAMP(1), SUBSTR(\'123\', 2), SUBSTRING(\'123\', 2), SUM(1), TAN(1), TANH(1), TRUNC(1), TRUNCATE(1), UCASE(\'A\'), UPPER(\'A\'), USER(), VAR_POP(1), VAR_SAMP(1), WEEK(toDate(\'2020-10-24\')), YEARWEEK(toDate(\'2020-10-24\')) format TSVRaw;'); +SELECT normalizedQueryHash(formatQueryOneLine('select 1')) = normalizedQueryHash(formatQueryOneLine('SELECT 1')); +SELECT formatQueryOneLine('SEECTwrong'); -- { serverError SYNTAX_ERROR } From 3f30342bc754631c4647a329f87df2a547dd9eb0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 25 Oct 2023 08:05:31 +0000 Subject: [PATCH 579/634] Add isInt() convenience function --- src/DataTypes/IDataType.h | 2 ++ tests/queries/0_stateless/01942_dateTimeToSnowflake.sql | 3 +-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index d74c4ea8cec..7f5cd88fa2b 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -468,6 +468,8 @@ template inline bool isInt32(const T & data_type) { return WhichDataType(data_type).isInt32(); } template inline bool isInt64(const T & data_type) { return WhichDataType(data_type).isInt64(); } +template +inline bool isInt(const T & data_type) { return WhichDataType(data_type).isInt(); } template inline bool isInteger(const T & data_type) diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql index 62371e901a2..3789f130b99 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql @@ -29,8 +29,7 @@ SELECT dateTime64ToSnowflake(dt64_0), dateTime64ToSnowflake(dt64_3), dateTime64ToSnowflake(dt64_4); --- DateTime64-to-Snowflake-to-DateTime64 is idempotent *if* the scale is <=3 (millisecond --- precision) +-- DateTime64-to-Snowflake-to-DateTime64 is idempotent *if* the scale is <=3 (millisecond precision) WITH now64(0) AS dt64_0, now64(1) AS dt64_1, now64(2) AS dt64_2, From 0bda99e880a842b2b761e53a3c5b97e8c83e3cf4 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 25 Oct 2023 11:07:34 +0200 Subject: [PATCH 580/634] Fix code style --- src/Functions/formatQuery.cpp | 1 - utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/formatQuery.cpp b/src/Functions/formatQuery.cpp index 6fdeb9ca752..bd230292607 100644 --- a/src/Functions/formatQuery.cpp +++ b/src/Functions/formatQuery.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; } template diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 7c4e02ca2e2..edafd4e7a55 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1477,6 +1477,7 @@ formatDateTime formatDateTimeInJoda formatDateTimeInJodaSyntax formatQuery +formatQueryOneLine formatReadableDecimalSize formatReadableQuantity formatReadableSize From 04272fa481353c6c197a79c13656af9ed210e163 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 25 Oct 2023 09:26:16 +0000 Subject: [PATCH 581/634] Convert unnecessary std::scoped_lock to std::lock_guard --- src/Access/ExternalAuthenticators.cpp | 14 +++++++------- src/Access/GSSAcceptor.cpp | 14 +++++++------- src/Access/LDAPAccessStorage.cpp | 20 ++++++++++---------- src/Access/LDAPClient.cpp | 8 ++++---- src/Common/SharedLockGuard.h | 6 ++++-- src/Common/tests/gtest_async_loader.cpp | 2 +- src/Server/waitServersToFinish.cpp | 2 +- 7 files changed, 34 insertions(+), 32 deletions(-) diff --git a/src/Access/ExternalAuthenticators.cpp b/src/Access/ExternalAuthenticators.cpp index 9439a17dabd..981a7aafc6f 100644 --- a/src/Access/ExternalAuthenticators.cpp +++ b/src/Access/ExternalAuthenticators.cpp @@ -272,13 +272,13 @@ void ExternalAuthenticators::resetImpl() void ExternalAuthenticators::reset() { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); resetImpl(); } void ExternalAuthenticators::setConfiguration(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log) { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); resetImpl(); Poco::Util::AbstractConfiguration::Keys all_keys; @@ -390,7 +390,7 @@ bool ExternalAuthenticators::checkLDAPCredentials(const String & server, const B UInt128 params_hash = 0; { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); // Retrieve the server parameters. const auto pit = ldap_client_params_blueprint.find(server); @@ -460,7 +460,7 @@ bool ExternalAuthenticators::checkLDAPCredentials(const String & server, const B // Update the cache, but only if this is the latest check and the server is still configured in a compatible way. if (result) { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); // If the server was removed from the config while we were checking the password, we discard the current result. const auto pit = ldap_client_params_blueprint.find(server); @@ -507,7 +507,7 @@ bool ExternalAuthenticators::checkLDAPCredentials(const String & server, const B bool ExternalAuthenticators::checkKerberosCredentials(const String & realm, const GSSAcceptorContext & credentials) const { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); if (!kerberos_params.has_value()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Kerberos is not enabled"); @@ -526,7 +526,7 @@ bool ExternalAuthenticators::checkKerberosCredentials(const String & realm, cons GSSAcceptorContext::Params ExternalAuthenticators::getKerberosParams() const { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); if (!kerberos_params.has_value()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Kerberos is not enabled"); @@ -536,7 +536,7 @@ GSSAcceptorContext::Params ExternalAuthenticators::getKerberosParams() const HTTPAuthClientParams ExternalAuthenticators::getHTTPAuthenticationParams(const String& server) const { - std::scoped_lock lock{mutex}; + std::lock_guard lock{mutex}; const auto it = http_auth_servers.find(server); if (it == http_auth_servers.end()) diff --git a/src/Access/GSSAcceptor.cpp b/src/Access/GSSAcceptor.cpp index 7170028e4e6..02946f0d74d 100644 --- a/src/Access/GSSAcceptor.cpp +++ b/src/Access/GSSAcceptor.cpp @@ -90,7 +90,7 @@ String bufferToString(const gss_buffer_desc & buf) String extractSpecificStatusMessages(OM_uint32 status_code, int status_type, const gss_OID & mech_type) { - std::scoped_lock lock(gss_global_mutex); + std::lock_guard lock(gss_global_mutex); String messages; OM_uint32 message_context = 0; @@ -135,7 +135,7 @@ String extractSpecificStatusMessages(OM_uint32 status_code, int status_type, con String extractStatusMessages(OM_uint32 major_status_code, OM_uint32 minor_status_code, const gss_OID & mech_type) { - std::scoped_lock lock(gss_global_mutex); + std::lock_guard lock(gss_global_mutex); const auto gss_messages = extractSpecificStatusMessages(major_status_code, GSS_C_GSS_CODE, mech_type); const auto mech_messages = extractSpecificStatusMessages(minor_status_code, GSS_C_MECH_CODE, mech_type); @@ -158,7 +158,7 @@ String extractStatusMessages(OM_uint32 major_status_code, OM_uint32 minor_status std::pair extractNameAndRealm(const gss_name_t & name) { - std::scoped_lock lock(gss_global_mutex); + std::lock_guard lock(gss_global_mutex); gss_buffer_desc name_buf; name_buf.length = 0; @@ -186,7 +186,7 @@ std::pair extractNameAndRealm(const gss_name_t & name) bool equalMechanisms(const String & left_str, const gss_OID & right_oid) { - std::scoped_lock lock(gss_global_mutex); + std::lock_guard lock(gss_global_mutex); gss_buffer_desc left_buf; left_buf.length = left_str.size(); @@ -232,7 +232,7 @@ void GSSAcceptorContext::reset() void GSSAcceptorContext::resetHandles() noexcept { - std::scoped_lock lock(gss_global_mutex); + std::lock_guard lock(gss_global_mutex); if (acceptor_credentials_handle != GSS_C_NO_CREDENTIAL) { @@ -258,7 +258,7 @@ void GSSAcceptorContext::resetHandles() noexcept void GSSAcceptorContext::initHandles() { - std::scoped_lock lock(gss_global_mutex); + std::lock_guard lock(gss_global_mutex); resetHandles(); @@ -330,7 +330,7 @@ void GSSAcceptorContext::initHandles() String GSSAcceptorContext::processToken(const String & input_token, Poco::Logger * log) { - std::scoped_lock lock(gss_global_mutex); + std::lock_guard lock(gss_global_mutex); String output_token; diff --git a/src/Access/LDAPAccessStorage.cpp b/src/Access/LDAPAccessStorage.cpp index 9be28b763e0..c271cd39845 100644 --- a/src/Access/LDAPAccessStorage.cpp +++ b/src/Access/LDAPAccessStorage.cpp @@ -36,14 +36,14 @@ LDAPAccessStorage::LDAPAccessStorage(const String & storage_name_, AccessControl String LDAPAccessStorage::getLDAPServerName() const { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); return ldap_server_name; } void LDAPAccessStorage::setConfiguration(const Poco::Util::AbstractConfiguration & config, const String & prefix) { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); // TODO: switch to passing config as a ConfigurationView and remove this extra prefix once a version of Poco with proper implementation is available. const String prefix_str = (prefix.empty() ? "" : prefix + "."); @@ -102,7 +102,7 @@ void LDAPAccessStorage::setConfiguration(const Poco::Util::AbstractConfiguration void LDAPAccessStorage::processRoleChange(const UUID & id, const AccessEntityPtr & entity) { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); const auto role = typeid_cast>(entity); const auto it = granted_role_names.find(id); @@ -371,7 +371,7 @@ const char * LDAPAccessStorage::getStorageType() const String LDAPAccessStorage::getStorageParamsJSON() const { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); Poco::JSON::Object params_json; params_json.set("server", ldap_server_name); @@ -417,35 +417,35 @@ String LDAPAccessStorage::getStorageParamsJSON() const std::optional LDAPAccessStorage::findImpl(AccessEntityType type, const String & name) const { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); return memory_storage.find(type, name); } std::vector LDAPAccessStorage::findAllImpl(AccessEntityType type) const { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); return memory_storage.findAll(type); } bool LDAPAccessStorage::exists(const UUID & id) const { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); return memory_storage.exists(id); } AccessEntityPtr LDAPAccessStorage::readImpl(const UUID & id, bool throw_if_not_exists) const { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); return memory_storage.read(id, throw_if_not_exists); } std::optional> LDAPAccessStorage::readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); return memory_storage.readNameWithType(id, throw_if_not_exists); } @@ -458,7 +458,7 @@ std::optional LDAPAccessStorage::authenticateImpl( bool /* allow_no_password */, bool /* allow_plaintext_password */) const { - std::scoped_lock lock(mutex); + std::lock_guard lock(mutex); auto id = memory_storage.find(credentials.getUserName()); UserPtr user = id ? memory_storage.read(*id) : nullptr; diff --git a/src/Access/LDAPClient.cpp b/src/Access/LDAPClient.cpp index 023f305f5cd..7926298061d 100644 --- a/src/Access/LDAPClient.cpp +++ b/src/Access/LDAPClient.cpp @@ -172,7 +172,7 @@ namespace void LDAPClient::handleError(int result_code, String text) { - std::scoped_lock lock(ldap_global_mutex); + std::lock_guard lock(ldap_global_mutex); if (result_code != LDAP_SUCCESS) { @@ -212,7 +212,7 @@ void LDAPClient::handleError(int result_code, String text) bool LDAPClient::openConnection() { - std::scoped_lock lock(ldap_global_mutex); + std::lock_guard lock(ldap_global_mutex); closeConnection(); @@ -390,7 +390,7 @@ bool LDAPClient::openConnection() void LDAPClient::closeConnection() noexcept { - std::scoped_lock lock(ldap_global_mutex); + std::lock_guard lock(ldap_global_mutex); if (!handle) return; @@ -404,7 +404,7 @@ void LDAPClient::closeConnection() noexcept LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params) { - std::scoped_lock lock(ldap_global_mutex); + std::lock_guard lock(ldap_global_mutex); SearchResults result; diff --git a/src/Common/SharedLockGuard.h b/src/Common/SharedLockGuard.h index 9186dd27596..93d2f42e907 100644 --- a/src/Common/SharedLockGuard.h +++ b/src/Common/SharedLockGuard.h @@ -6,9 +6,11 @@ namespace DB { /** SharedLockGuard provide RAII-style locking mechanism for acquiring shared ownership of the implementation - * of the SharedLockable concept (for example std::shared_mutex) supplied as the constructor argument. + * of the SharedLockable concept (for example std::shared_mutex or ContextSharedMutex) supplied as the + * constructor argument. Think of it as std::lock_guard which locks shared. + * * On construction it acquires shared ownership using `lock_shared` method. - * On desruction shared ownership is released using `unlock_shared` method. + * On destruction shared ownership is released using `unlock_shared` method. */ template class TSA_SCOPED_LOCKABLE SharedLockGuard diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index eed66cca0af..dfcbf27b9fc 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -74,7 +74,7 @@ struct AsyncLoaderTest T randomInt(T from, T to) { std::uniform_int_distribution distribution(from, to); - std::scoped_lock lock(rng_mutex); + std::lock_guard lock(rng_mutex); return distribution(rng); } diff --git a/src/Server/waitServersToFinish.cpp b/src/Server/waitServersToFinish.cpp index 3b07c082067..d6eb5b59b88 100644 --- a/src/Server/waitServersToFinish.cpp +++ b/src/Server/waitServersToFinish.cpp @@ -16,7 +16,7 @@ size_t waitServersToFinish(std::vector & servers, std current_connections = 0; { - std::scoped_lock lock{mutex}; + std::lock_guard lock{mutex}; for (auto & server : servers) { server.stop(); From a7e582fb119f547bc68452e3cc2ef63420e7589c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 25 Oct 2023 09:35:39 +0000 Subject: [PATCH 582/634] Fix style, pt. II --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index ed2575d1110..f9c829209c5 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -2033,7 +2033,7 @@ snowflakeToDateTime64(value[, time_zone]) **Returned value** -- The timestamp component of `value` as a [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) with scale = 3, i.e. millisecond precison. +- The timestamp component of `value` as a [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) with scale = 3, i.e. millisecond precision. **Example** From 8861f8b6ea03f0af872fe97b163c3b576f9b11f8 Mon Sep 17 00:00:00 2001 From: Antonio Bonuccelli Date: Wed, 25 Oct 2023 11:50:41 +0200 Subject: [PATCH 583/634] Fix toDayOfWeek return value description Fix toDayOfWeek return value description --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index f65354921e2..a84f9002b24 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -605,7 +605,7 @@ The first argument can also be specified as [String](../data-types/string.md) in **Returned value** -- The day of the month (1 - 31) of the given date/time +- The day of the week (1-7), depending on the choosen mode, of the given date/time **Example** From ef2740052eee4361fb9c039eb34bc36ddbdea96f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 25 Oct 2023 10:15:35 +0000 Subject: [PATCH 584/634] Attempt to wait until server is responsive after gdb detach --- tests/ci/stress.py | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 446350863aa..bd9f5c7e945 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -131,20 +131,17 @@ def prepare_for_hung_check(drop_databases: bool) -> bool: # We attach gdb to clickhouse-server before running tests # to print stacktraces of all crashes even if clickhouse cannot print it for some reason. - # However, it obstruct checking for hung queries. + # However, it obstructs checking for hung queries. logging.info("Will terminate gdb (if any)") call_with_retry("kill -TERM $(pidof gdb)") call_with_retry("tail --pid=$(pidof gdb) -f /dev/null") # Sometimes there is a message `Child process was stopped by signal 19` in logs after stopping gdb call_with_retry( - "kill -CONT $(cat /var/run/clickhouse-server/clickhouse-server.pid)" + "kill -CONT $(cat /var/run/clickhouse-server/clickhouse-server.pid) && clickhouse client -q 'SELECT 1 FORMAT Null'" ) # ThreadFuzzer significantly slows down server and causes false-positive hung check failures - call_with_retry("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'") - - call_with_retry(make_query_command("SELECT 1 FORMAT Null")) - + call_with_retry(make_query_command("SYSTEM STOP THREAD FUZZER")) # Some tests execute SYSTEM STOP MERGES or similar queries. # It may cause some ALTERs to hang. # Possibly we should fix tests and forbid to use such queries without specifying table. From 50c92dfe11687e29ff424670d53eaafc5906f673 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 25 Oct 2023 11:56:29 +0000 Subject: [PATCH 585/634] Update protobuf to v21.9 --- contrib/google-protobuf | 2 +- contrib/google-protobuf-cmake/CMakeLists.txt | 96 ++++++++++---------- 2 files changed, 48 insertions(+), 50 deletions(-) diff --git a/contrib/google-protobuf b/contrib/google-protobuf index 9650fe3fa9b..2a4fa1a4e95 160000 --- a/contrib/google-protobuf +++ b/contrib/google-protobuf @@ -1 +1 @@ -Subproject commit 9650fe3fa9b326bbe0ecfc52883bf5a0e030f8bb +Subproject commit 2a4fa1a4e95012d754ac55d43c8bc462dd1c78a8 diff --git a/contrib/google-protobuf-cmake/CMakeLists.txt b/contrib/google-protobuf-cmake/CMakeLists.txt index 32657b62c17..268f0fbe0e4 100644 --- a/contrib/google-protobuf-cmake/CMakeLists.txt +++ b/contrib/google-protobuf-cmake/CMakeLists.txt @@ -36,7 +36,6 @@ set(libprotobuf_lite_files ${protobuf_source_dir}/src/google/protobuf/arenastring.cc ${protobuf_source_dir}/src/google/protobuf/extension_set.cc ${protobuf_source_dir}/src/google/protobuf/generated_enum_util.cc - ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven_lite.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_util.cc ${protobuf_source_dir}/src/google/protobuf/implicit_weak_message.cc ${protobuf_source_dir}/src/google/protobuf/io/coded_stream.cc @@ -87,7 +86,6 @@ set(libprotobuf_files ${protobuf_source_dir}/src/google/protobuf/field_mask.pb.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_bases.cc ${protobuf_source_dir}/src/google/protobuf/generated_message_reflection.cc - ${protobuf_source_dir}/src/google/protobuf/generated_message_table_driven.cc ${protobuf_source_dir}/src/google/protobuf/io/gzip_stream.cc ${protobuf_source_dir}/src/google/protobuf/io/printer.cc ${protobuf_source_dir}/src/google/protobuf/io/tokenizer.cc @@ -144,21 +142,21 @@ add_library(protobuf::libprotobuf ALIAS _libprotobuf) set(libprotoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/code_generator.cc ${protobuf_source_dir}/src/google/protobuf/compiler/command_line_interface.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_enum.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_extension.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_file.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_helpers.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_map_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_message.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_padding_optimizer.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_parse_function_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_primitive_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_service.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/cpp_string_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/enum.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/extension.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/file.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/helpers.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/map_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/message.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/padding_optimizer.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/parse_function_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/service.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/cpp/string_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_doc_comment.cc ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_enum.cc ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_enum_field.cc @@ -175,37 +173,35 @@ set(libprotoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_repeated_primitive_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_source_generator_base.cc ${protobuf_source_dir}/src/google/protobuf/compiler/csharp/csharp_wrapper_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_context.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_doc_comment.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_enum_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_extension.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_extension_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_file.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_generator_factory.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_helpers.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_kotlin_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_map_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_map_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_builder.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_builder_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_message_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_name_resolver.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_primitive_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_primitive_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_service.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_shared_code_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_string_field.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/java/java_string_field_lite.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/js/js_generator.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/js/well_known_types_embed.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/context.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/doc_comment.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/enum.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/enum_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/enum_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/enum_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/extension.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/extension_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/file.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/generator_factory.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/helpers.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/kotlin_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/map_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/map_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/message.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/message_builder.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/message_builder_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/message_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/message_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/message_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/name_resolver.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/primitive_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/primitive_field_lite.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/service.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/shared_code_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/string_field.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/java/string_field_lite.cc ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_enum.cc ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_enum_field.cc ${protobuf_source_dir}/src/google/protobuf/compiler/objectivec/objectivec_extension.cc @@ -221,7 +217,9 @@ set(libprotoc_files ${protobuf_source_dir}/src/google/protobuf/compiler/php/php_generator.cc ${protobuf_source_dir}/src/google/protobuf/compiler/plugin.cc ${protobuf_source_dir}/src/google/protobuf/compiler/plugin.pb.cc - ${protobuf_source_dir}/src/google/protobuf/compiler/python/python_generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/python/generator.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/python/helpers.cc + ${protobuf_source_dir}/src/google/protobuf/compiler/python/pyi_generator.cc ${protobuf_source_dir}/src/google/protobuf/compiler/ruby/ruby_generator.cc ${protobuf_source_dir}/src/google/protobuf/compiler/subprocess.cc ${protobuf_source_dir}/src/google/protobuf/compiler/zip_writer.cc From d5feaa72c1917682fb8955663804c9514c3e496f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 25 Oct 2023 20:16:18 +0800 Subject: [PATCH 586/634] improve thrown exception message --- src/IO/readFloatText.h | 10 +++++++--- tests/queries/0_stateless/02900_issue_55858.sql | 8 ++++---- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/IO/readFloatText.h b/src/IO/readFloatText.h index ad14563c859..c0da9129a05 100644 --- a/src/IO/readFloatText.h +++ b/src/IO/readFloatText.h @@ -161,7 +161,10 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf) if (unlikely(res.ec != std::errc())) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value"); + throw ParsingException( + ErrorCodes::CANNOT_PARSE_NUMBER, + "Cannot read floating point value here: {}", + String(initial_position, buf.buffer().end() - initial_position)); else return ReturnType(false); } @@ -248,10 +251,11 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf) res = fast_float::from_chars(tmp_buf, tmp_buf + num_copied_chars, x64); x = static_cast(x64); } - if (unlikely(res.ec != std::errc())) + if (unlikely(res.ec != std::errc() || res.ptr - tmp_buf != num_copied_chars)) { if constexpr (throw_exception) - throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value"); + throw ParsingException( + ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value here: {}", String(tmp_buf, num_copied_chars)); else return ReturnType(false); } diff --git a/tests/queries/0_stateless/02900_issue_55858.sql b/tests/queries/0_stateless/02900_issue_55858.sql index b7b6704cdb5..65fc06d9797 100644 --- a/tests/queries/0_stateless/02900_issue_55858.sql +++ b/tests/queries/0_stateless/02900_issue_55858.sql @@ -1,9 +1,9 @@ set precise_float_parsing = 1; -select cast('2023-01-01' as Float64); -- { serverError 6 } -select cast('2023-01-01' as Float32); -- { serverError 6 } -select toFloat32('2023-01-01'); -- { serverError 6 } -select toFloat64('2023-01-01'); -- { serverError 6 } +select cast('2023-01-01' as Float64); -- { serverError 72 } +select cast('2023-01-01' as Float32); -- { serverError 72 } +select toFloat32('2023-01-01'); -- { serverError 72 } +select toFloat64('2023-01-01'); -- { serverError 72 } select toFloat32OrZero('2023-01-01'); select toFloat64OrZero('2023-01-01'); select toFloat32OrNull('2023-01-01'); From b0bbb4d804eca47352495ed13681d13f79286d99 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 25 Oct 2023 15:53:41 +0200 Subject: [PATCH 587/634] Fix ASan failure --- src/Server/HTTPHandlerRequestFilter.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/HTTPHandlerRequestFilter.h b/src/Server/HTTPHandlerRequestFilter.h index 6e952c2fd9d..94c1ad5b6f2 100644 --- a/src/Server/HTTPHandlerRequestFilter.h +++ b/src/Server/HTTPHandlerRequestFilter.h @@ -98,7 +98,7 @@ static inline auto headersFilter(const Poco::Util::AbstractConfiguration & confi { for (const auto & [header_name, header_expression] : headers_expression) { - const auto & header_value = request.get(header_name, ""); + const auto header_value = request.get(header_name, ""); if (!checkExpression(std::string_view(header_value.data(), header_value.size()), header_expression)) return false; } From 62088772d2948801fb489928a622da48ca33d3fc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 25 Oct 2023 16:12:18 +0200 Subject: [PATCH 588/634] Fuzzer: Use the known pid instead of reading the file The file might not exists yet --- docker/test/fuzzer/run-fuzzer.sh | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 326f2a751e6..af1ce0c4dd4 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -210,7 +210,7 @@ detach quit " > script.gdb - gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" & + gdb -batch -command script.gdb -p $server_pid & sleep 5 # gdb will send SIGSTOP, spend some time loading debug info and then send SIGCONT, wait for it (up to send_timeout, 300s) time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||: @@ -219,13 +219,12 @@ quit # to freeze and the fuzzer will fail. In debug build it can take a lot of time. for _ in {1..180} do - sleep 1 if clickhouse-client --query "select 1" then break fi + sleep 1 done - clickhouse-client --query "select 1" # This checks that the server is responding kill -0 $server_pid # This checks that it is our server that is started and not some other one echo 'Server started and responded' From 3973b243024a8ca74b87835ec91fad5fbfcc97d8 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 25 Oct 2023 16:58:25 +0200 Subject: [PATCH 589/634] Update docs/en/sql-reference/functions/date-time-functions.md --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index a84f9002b24..3fe791a4fb9 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -605,7 +605,7 @@ The first argument can also be specified as [String](../data-types/string.md) in **Returned value** -- The day of the week (1-7), depending on the choosen mode, of the given date/time +- The day of the week (1-7), depending on the chosen mode, of the given date/time **Example** From 3e6ecf1858c9415db44e967a0953f70c04df7f5b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 25 Oct 2023 15:08:27 +0000 Subject: [PATCH 590/634] correct the implementation of function jsonMergePatch --- .../sql-reference/functions/json-functions.md | 10 +-- src/Common/ErrorCodes.cpp | 1 - src/Functions/jsonMergePatch.cpp | 86 +++++++++++++------ ...4_json_merge_patch_function_test.reference | 10 +++ .../02874_json_merge_patch_function_test.sql | 14 ++- 5 files changed, 85 insertions(+), 36 deletions(-) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index 83900532de8..a4c3ba83351 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -4,7 +4,7 @@ sidebar_position: 105 sidebar_label: JSON --- -There are two sets of functions to parse JSON. +There are two sets of functions to parse JSON. - `visitParam*` (`simpleJSON*`) is made to parse a special very limited subset of a JSON, but these functions are extremely fast. - `JSONExtract*` is made to parse normal JSON. @@ -367,7 +367,7 @@ SELECT JSON_EXISTS('{"hello":["world"]}', '$.hello[*]'); SELECT JSON_EXISTS('{"hello":["world"]}', '$.hello[0]'); ``` -:::note +:::note Before version 21.11 the order of arguments was wrong, i.e. JSON_EXISTS(path, json) ::: @@ -394,7 +394,7 @@ Result: [2] String ``` -:::note +:::note Before version 21.11 the order of arguments was wrong, i.e. JSON_QUERY(path, json) ::: @@ -424,7 +424,7 @@ world String ``` -:::note +:::note Before version 21.11 the order of arguments was wrong, i.e. JSON_VALUE(path, json) ::: @@ -513,7 +513,7 @@ SELECT ## jsonMergePatch -Return the merged JSON object string which is formed by merging multiple JSON objects. +Returns the merged JSON object string which is formed by merging multiple JSON objects. **Syntax** diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index d2196b3c2b9..e780ba5ce49 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -586,7 +586,6 @@ M(704, CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS) \ M(705, TABLE_NOT_EMPTY) \ M(706, LIBSSH_ERROR) \ - M(707, ILLEGAL_JSON_OBJECT_FORMAT) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ diff --git a/src/Functions/jsonMergePatch.cpp b/src/Functions/jsonMergePatch.cpp index ba641f70950..382cbfa07b7 100644 --- a/src/Functions/jsonMergePatch.cpp +++ b/src/Functions/jsonMergePatch.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include "config.h" #if USE_RAPIDJSON @@ -20,10 +22,11 @@ namespace DB { + namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_JSON_OBJECT_FORMAT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -42,7 +45,6 @@ namespace static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } - bool isVariadic() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } @@ -54,20 +56,25 @@ namespace if (arguments.empty()) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least one argument.", getName()); + for (const auto & arg : arguments) + if (!isString(arg.type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {} requires string arguments", getName()); + return std::make_shared(); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - rapidjson::Document merged_json; - merged_json.SetObject(); - rapidjson::Document::AllocatorType& allocator = merged_json.GetAllocator(); + chassert(!arguments.empty()); - std::function mergeObjects; - mergeObjects = [&mergeObjects, &allocator](rapidjson::Value& dest, const rapidjson::Value& src) -> void + rapidjson::Document::AllocatorType allocator; + std::function merge_objects; + + merge_objects = [&merge_objects, &allocator](rapidjson::Value & dest, const rapidjson::Value & src) -> void { if (!src.IsObject()) return; + for (auto it = src.MemberBegin(); it != src.MemberEnd(); ++it) { rapidjson::Value key(it->name, allocator); @@ -75,7 +82,7 @@ namespace if (dest.HasMember(key)) { if (dest[key].IsObject() && value.IsObject()) - mergeObjects(dest[key], value); + merge_objects(dest[key], value); else dest[key] = value; } @@ -86,34 +93,57 @@ namespace } }; - for (const auto & arg : arguments) + auto parse_json_document = [](const ColumnString & column, rapidjson::Document & document, size_t i) { - const ColumnPtr column = arg.column; - const ColumnString * col = typeid_cast(column.get()); - if (!col) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "First argument of function {} must be string", getName()); + auto str_ref = column.getDataAt(i); + const char * json = str_ref.data; + + document.Parse(json); + if (document.HasParseError() || !document.IsObject()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong JSON string to merge. Expected JSON object"); + }; + + const auto * first_string = typeid_cast(arguments[0].column.get()); + if (!first_string) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Arguments of function {} must be strings", getName()); + + std::vector merged_jsons; + merged_jsons.reserve(input_rows_count); + + for (size_t i = 0; i < input_rows_count; ++i) + { + auto & merged_json = merged_jsons.emplace_back(rapidjson::Type::kObjectType, &allocator); + parse_json_document(*first_string, merged_json, i); + } + + for (size_t col_idx = 1; col_idx < arguments.size(); ++col_idx) + { + const auto * column_string = typeid_cast(arguments[col_idx].column.get()); + if (!column_string) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Arguments of function {} must be strings", getName()); for (size_t i = 0; i < input_rows_count; ++i) { - auto str_ref = col->getDataAt(i); - const char* json = str_ref.data; - rapidjson::Document document; - document.Parse(json); - if (!document.IsObject()) - throw Exception(ErrorCodes::ILLEGAL_JSON_OBJECT_FORMAT, "Wrong input Json object format"); - mergeObjects(merged_json, document); + rapidjson::Document document(&allocator); + parse_json_document(*column_string, document, i); + merge_objects(merged_jsons[i], document); } } - rapidjson::StringBuffer buffer; - rapidjson::Writer writer(buffer); - merged_json.Accept(writer); - std::string json_string = buffer.GetString(); + auto result = ColumnString::create(); + auto & result_string = assert_cast(*result); + rapidjson::CrtAllocator buffer_allocator; - auto res = ColumnString::create(); - res->insertData(json_string.c_str(), json_string.size()); + for (size_t i = 0; i < input_rows_count; ++i) + { + rapidjson::StringBuffer buffer(&buffer_allocator); + rapidjson::Writer writer(buffer); - return res; + merged_jsons[i].Accept(writer); + result_string.insertData(buffer.GetString(), buffer.GetSize()); + } + + return result; } }; @@ -122,7 +152,7 @@ namespace REGISTER_FUNCTION(jsonMergePatch) { factory.registerFunction(FunctionDocumentation{ - .description="Return the merged JSON object string, which is formed by merging multiple JSON objects."}); + .description="Returns the merged JSON object string, which is formed by merging multiple JSON objects."}); } } diff --git a/tests/queries/0_stateless/02874_json_merge_patch_function_test.reference b/tests/queries/0_stateless/02874_json_merge_patch_function_test.reference index 85caab63611..29283d0f232 100644 --- a/tests/queries/0_stateless/02874_json_merge_patch_function_test.reference +++ b/tests/queries/0_stateless/02874_json_merge_patch_function_test.reference @@ -8,3 +8,13 @@ {"a":{"b":[3,4],"c":2}} {"a":5,"b":2,"c":4,"d":6} {"a":1,"b":null} +{"k0":"0","k1":"1"} +{"k2":"2","k3":"3"} +{"k4":"4","k5":"5"} +{"k6":"6","k7":"7"} +{"k8":"8","k9":"9"} +{"k10":"222","k11":"11"} +{"k12":"222","k13":"13"} +{"k14":"222","k15":"15"} +{"k16":"222","k17":"17"} +{"k18":"222","k19":"19"} diff --git a/tests/queries/0_stateless/02874_json_merge_patch_function_test.sql b/tests/queries/0_stateless/02874_json_merge_patch_function_test.sql index 51e520bd0e8..4f4b2e20f30 100644 --- a/tests/queries/0_stateless/02874_json_merge_patch_function_test.sql +++ b/tests/queries/0_stateless/02874_json_merge_patch_function_test.sql @@ -10,5 +10,15 @@ select jsonMergePatch('{"a": {"b": 1, "c": 2}}', '{"a": {"b": [3, 4]}}'); select jsonMergePatch('{ "a": 1, "b":2 }','{ "a": 3, "c":4 }','{ "a": 5, "d":6 }'); select jsonMergePatch('{"a":1, "b":2}', '{"b":null}'); -select jsonMergePatch('[1]'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } -select jsonMergePatch('{"a": "1","b": 2,"c": [true,"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); -- { serverError ILLEGAL_JSON_OBJECT_FORMAT } +select jsonMergePatch('[1]'); -- { serverError BAD_ARGUMENTS } +select jsonMergePatch('{"a": "1","b": 2,"c": [true,"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); -- { serverError BAD_ARGUMENTS } + +drop table if exists t_json_merge; +create table t_json_merge (s1 String, s2 String) engine = Memory; + +insert into t_json_merge select toJSONString(map('k' || toString(number * 2), number * 2)), toJSONString(map('k' || toString(number * 2 + 1), number * 2 + 1)) from numbers(5); +insert into t_json_merge select toJSONString(map('k' || toString(number * 2), number * 2)), toJSONString(map('k' || toString(number * 2 + 1), number * 2 + 1, 'k' || toString(number * 2), 222)) from numbers(5, 5); + +select jsonMergePatch(s1, s2) from t_json_merge; + +drop table t_json_merge; From 82d4d52b06476c9a6f46cbac54ab86b731313583 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 24 Oct 2023 14:50:33 +0000 Subject: [PATCH 591/634] Add SHOW SETTING query --- src/Interpreters/InterpreterFactory.cpp | 6 +++ .../InterpreterShowSettingQuery.cpp | 37 +++++++++++++++++++ .../InterpreterShowSettingQuery.h | 33 +++++++++++++++++ src/Parsers/ASTShowSettingQuery.cpp | 25 +++++++++++++ src/Parsers/ASTShowSettingQuery.h | 31 ++++++++++++++++ src/Parsers/ParserQueryWithOutput.cpp | 3 ++ src/Parsers/ParserShowSettingQuery.cpp | 30 +++++++++++++++ src/Parsers/ParserShowSettingQuery.h | 20 ++++++++++ 8 files changed, 185 insertions(+) create mode 100644 src/Interpreters/InterpreterShowSettingQuery.cpp create mode 100644 src/Interpreters/InterpreterShowSettingQuery.h create mode 100644 src/Parsers/ASTShowSettingQuery.cpp create mode 100644 src/Parsers/ASTShowSettingQuery.h create mode 100644 src/Parsers/ParserShowSettingQuery.cpp create mode 100644 src/Parsers/ParserShowSettingQuery.h diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index de3a3d68d39..131361acd91 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -86,6 +87,7 @@ #include #include #include +#include #include #include #include @@ -201,6 +203,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); diff --git a/src/Interpreters/InterpreterShowSettingQuery.cpp b/src/Interpreters/InterpreterShowSettingQuery.cpp new file mode 100644 index 00000000000..aa3556a8837 --- /dev/null +++ b/src/Interpreters/InterpreterShowSettingQuery.cpp @@ -0,0 +1,37 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + + +InterpreterShowSettingQuery::InterpreterShowSettingQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_) + , query_ptr(query_ptr_) +{ +} + + +String InterpreterShowSettingQuery::getRewrittenQuery() +{ + const auto & query = query_ptr->as(); + return fmt::format(R"(SELECT value FROM system.settings WHERE name = '{0}')", query.getSettingName()); +} + + +BlockIO InterpreterShowSettingQuery::execute() +{ + return executeQuery(getRewrittenQuery(), getContext(), true).second; +} + + +} + diff --git a/src/Interpreters/InterpreterShowSettingQuery.h b/src/Interpreters/InterpreterShowSettingQuery.h new file mode 100644 index 00000000000..6fa8869a284 --- /dev/null +++ b/src/Interpreters/InterpreterShowSettingQuery.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class Context; + + +/// Returns setting value for specified setting name +class InterpreterShowSettingQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterShowSettingQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_); + + BlockIO execute() override; + + /// Ignore quota and limits here because execute() produces a SELECT query which checks quotas/limits by itself. + bool ignoreQuota() const override { return true; } + bool ignoreLimits() const override { return true; } + +private: + ASTPtr query_ptr; + + String getRewrittenQuery(); +}; + + +} + diff --git a/src/Parsers/ASTShowSettingQuery.cpp b/src/Parsers/ASTShowSettingQuery.cpp new file mode 100644 index 00000000000..267d462475d --- /dev/null +++ b/src/Parsers/ASTShowSettingQuery.cpp @@ -0,0 +1,25 @@ +#include + +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTShowSettingQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + cloneOutputOptions(*res); + res->setting_name = setting_name; + return res; +} + +void ASTShowSettingQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW SETTING " << (settings.hilite ? hilite_none : "") + << backQuoteIfNeed(setting_name); +} + +} diff --git a/src/Parsers/ASTShowSettingQuery.h b/src/Parsers/ASTShowSettingQuery.h new file mode 100644 index 00000000000..2b4395f307d --- /dev/null +++ b/src/Parsers/ASTShowSettingQuery.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Query SHOW SETTING setting_name +class ASTShowSettingQuery : public ASTQueryWithOutput +{ +public: + explicit ASTShowSettingQuery(String setting_name_) + : setting_name(setting_name_) + {} + + const String & getSettingName() const { return setting_name; } + + String getID(char) const override { return "ShowSetting"; } + ASTPtr clone() const override; + QueryKind getQueryKind() const override { return QueryKind::Show; } + +protected: + void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; + +private: + String setting_name; +}; + +} + diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 9a71bc222b5..f03df6cacfe 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -43,6 +44,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserShowEnginesQuery show_engine_p; ParserShowFunctionsQuery show_functions_p; ParserShowIndexesQuery show_indexes_p; + ParserShowSettingQuery show_setting_p; ParserSelectWithUnionQuery select_p; ParserTablePropertiesQuery table_p; ParserDescribeTableQuery describe_table_p; @@ -75,6 +77,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec || show_engine_p.parse(pos, query, expected) || show_functions_p.parse(pos, query, expected) || show_indexes_p.parse(pos, query, expected) + || show_setting_p.parse(pos, query, expected) || table_p.parse(pos, query, expected) || describe_cache_p.parse(pos, query, expected) || describe_table_p.parse(pos, query, expected) diff --git a/src/Parsers/ParserShowSettingQuery.cpp b/src/Parsers/ParserShowSettingQuery.cpp new file mode 100644 index 00000000000..92bed51b688 --- /dev/null +++ b/src/Parsers/ParserShowSettingQuery.cpp @@ -0,0 +1,30 @@ +#include + +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +bool ParserShowSettingQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserKeyword("SHOW SETTING").ignore(pos, expected)) + return false; + + ASTPtr setting_name_identifier; + if (!ParserIdentifier().parse(pos, setting_name_identifier, expected)) + return false; + + node = std::make_shared(getIdentifierName(setting_name_identifier)); + + return true; +} + +} + diff --git a/src/Parsers/ParserShowSettingQuery.h b/src/Parsers/ParserShowSettingQuery.h new file mode 100644 index 00000000000..56c835eb349 --- /dev/null +++ b/src/Parsers/ParserShowSettingQuery.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +namespace DB +{ + +/** Parses queries of the form + * SHOW SETTING setting_name + */ +class ParserShowSettingQuery : public IParserBase +{ +protected: + const char * getName() const override { return "SHOW SETTING query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + From 258a71a25d2e1b91b57716d13d3d429701836ff5 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 24 Oct 2023 14:54:50 +0000 Subject: [PATCH 592/634] Added tests --- tests/queries/0_stateless/02905_show_setting_query.reference | 1 + tests/queries/0_stateless/02905_show_setting_query.sql | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/02905_show_setting_query.reference create mode 100644 tests/queries/0_stateless/02905_show_setting_query.sql diff --git a/tests/queries/0_stateless/02905_show_setting_query.reference b/tests/queries/0_stateless/02905_show_setting_query.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02905_show_setting_query.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02905_show_setting_query.sql b/tests/queries/0_stateless/02905_show_setting_query.sql new file mode 100644 index 00000000000..ee7c9379146 --- /dev/null +++ b/tests/queries/0_stateless/02905_show_setting_query.sql @@ -0,0 +1,2 @@ +SET max_threads = 1; +SHOW SETTING max_threads; From 108df081b7b51acd2280f3df4105667e6b43560e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 24 Oct 2023 22:59:41 +0300 Subject: [PATCH 593/634] Add documentation --- docs/en/sql-reference/statements/show.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 70e76d33549..d19bf05aca8 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -609,6 +609,18 @@ Result: └──────────────────┴────────┴─────────────┘ ``` +## SHOW SETTING + +``` sql +SHOW SETTING +``` + +Outputs setting value for specified setting name. + +**See Also** +- [system.settings](../../operations/system-tables/settings.md) table + + ## SHOW FILESYSTEM CACHES ```sql From 6aa90117eded1e7688feaad3ed6587954b4744b6 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 25 Oct 2023 17:51:57 +0200 Subject: [PATCH 594/634] Poke CI From 6e735662d992e3ea3d8f4ffd47538474d4e70bcb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 25 Oct 2023 18:24:01 +0200 Subject: [PATCH 595/634] fix --- .../0_stateless/02438_sync_replica_lightweight.reference | 5 +++-- tests/queries/0_stateless/02438_sync_replica_lightweight.sql | 3 ++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02438_sync_replica_lightweight.reference b/tests/queries/0_stateless/02438_sync_replica_lightweight.reference index 25abaad13e2..00d7f58bdff 100644 --- a/tests/queries/0_stateless/02438_sync_replica_lightweight.reference +++ b/tests/queries/0_stateless/02438_sync_replica_lightweight.reference @@ -5,8 +5,9 @@ GET_PART all_1_1_0 MERGE_PARTS all_0_1_1 3 1 all_0_1_1 3 2 all_0_1_1 -4 1 all_0_1_1 -4 2 all_0_1_1 +4 1 +4 2 +MERGE_PARTS all_0_1_1 5 1 all_0_2_2 5 2 all_0_2_2 5 3 all_0_2_2 diff --git a/tests/queries/0_stateless/02438_sync_replica_lightweight.sql b/tests/queries/0_stateless/02438_sync_replica_lightweight.sql index 1da48d95d9b..beddc771141 100644 --- a/tests/queries/0_stateless/02438_sync_replica_lightweight.sql +++ b/tests/queries/0_stateless/02438_sync_replica_lightweight.sql @@ -24,7 +24,8 @@ system start replicated sends rmt1; system sync replica rmt2 lightweight; -- waits for fetches, not merges select type, new_part_name from system.replication_queue where database=currentDatabase() and table='rmt2' order by new_part_name; select 3, n, _part from rmt1 order by n; -select 4, n, _part from rmt2 order by n; +select 4, n from rmt2 order by n; +select type, new_part_name from system.replication_queue where database=currentDatabase() and table='rmt2' order by new_part_name; system start merges rmt2; system sync replica rmt2; From 0f4260bf2694125b64e89ffbd45c1a9143533bb3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 25 Oct 2023 19:51:51 +0200 Subject: [PATCH 596/634] fix some bad code by making it worse --- src/Databases/DatabaseReplicated.cpp | 9 ++++++--- src/Interpreters/InterpreterCreateQuery.cpp | 4 ++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 91153f2302f..14b921f5b97 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -585,7 +585,8 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_ if (auto * create = query->as()) { - bool replicated_table = create->storage && create->storage->engine && startsWith(create->storage->engine->name, "Replicated"); + bool replicated_table = create->storage && create->storage->engine && + (startsWith(create->storage->engine->name, "Replicated") || startsWith(create->storage->engine->name, "Shared")); if (!replicated_table || !create->storage->engine->arguments) return; @@ -711,8 +712,9 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex static UUID getTableUUIDIfReplicated(const String & metadata, ContextPtr context) { bool looks_like_replicated = metadata.find("Replicated") != std::string::npos; + bool looks_like_shared = metadata.find("Shared") != std::string::npos; bool looks_like_merge_tree = metadata.find("MergeTree") != std::string::npos; - if (!looks_like_replicated || !looks_like_merge_tree) + if (!(looks_like_replicated || looks_like_shared) || !looks_like_merge_tree) return UUIDHelpers::Nil; ParserCreateQuery parser; @@ -722,7 +724,8 @@ static UUID getTableUUIDIfReplicated(const String & metadata, ContextPtr context const ASTCreateQuery & create = query->as(); if (!create.storage || !create.storage->engine) return UUIDHelpers::Nil; - if (!startsWith(create.storage->engine->name, "Replicated") || !endsWith(create.storage->engine->name, "MergeTree")) + if (!(startsWith(create.storage->engine->name, "Replicated") || startsWith(create.storage->engine->name, "Shared")) + || !endsWith(create.storage->engine->name, "MergeTree")) return UUIDHelpers::Nil; chassert(create.uuid != UUIDHelpers::Nil); return create.uuid; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 935db6dc7ae..252f45677ef 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -902,8 +902,8 @@ namespace { void checkTemporaryTableEngineName(const String& name) { - if (name.starts_with("Replicated") || name == "KeeperMap") - throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables cannot be created with Replicated or KeeperMap table engines"); + if (name.starts_with("Replicated") || name.starts_with("Shared") || name == "KeeperMap") + throw Exception(ErrorCodes::INCORRECT_QUERY, "Temporary tables cannot be created with Replicated, Shared or KeeperMap table engines"); } void setDefaultTableEngine(ASTStorage &storage, DefaultTableEngine engine) From 901294d35206d52c29f876dda7658a4d8252d191 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 25 Oct 2023 21:41:26 +0300 Subject: [PATCH 597/634] Fixed code review issues --- src/Common/escapeString.cpp | 16 ++++++++++++++++ src/Common/escapeString.h | 10 ++++++++++ src/Interpreters/InterpreterShowColumnsQuery.cpp | 9 +++------ src/Interpreters/InterpreterShowIndexesQuery.cpp | 12 +++--------- src/Interpreters/InterpreterShowSettingQuery.cpp | 7 ++----- src/Parsers/ParserShowSettingQuery.cpp | 3 --- src/Parsers/ParserShowSettingQuery.h | 6 +++--- .../02905_show_setting_query.reference | 1 + .../0_stateless/02905_show_setting_query.sql | 5 +++++ 9 files changed, 43 insertions(+), 26 deletions(-) create mode 100644 src/Common/escapeString.cpp create mode 100644 src/Common/escapeString.h diff --git a/src/Common/escapeString.cpp b/src/Common/escapeString.cpp new file mode 100644 index 00000000000..621726d38ac --- /dev/null +++ b/src/Common/escapeString.cpp @@ -0,0 +1,16 @@ +#include + +#include +#include + +namespace DB +{ + +String escapeString(std::string_view value) +{ + WriteBufferFromOwnString buf; + writeEscapedString(value, buf); + return buf.str(); +} + +} diff --git a/src/Common/escapeString.h b/src/Common/escapeString.h new file mode 100644 index 00000000000..0018296889c --- /dev/null +++ b/src/Common/escapeString.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace DB +{ + +String escapeString(std::string_view value); + +} diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index d14a36ef7e1..c8fb64e37f2 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -31,12 +32,8 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() WriteBufferFromOwnString buf_database; String resolved_database = getContext()->resolveDatabase(query.database); - writeEscapedString(resolved_database, buf_database); - String database = buf_database.str(); - - WriteBufferFromOwnString buf_table; - writeEscapedString(query.table, buf_table); - String table = buf_table.str(); + String database = escapeString(resolved_database); + String table = escapeString(query.table); String rewritten_query; if (use_mysql_types) diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index 9b36f1496e7..63cda814683 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -23,16 +24,9 @@ InterpreterShowIndexesQuery::InterpreterShowIndexesQuery(const ASTPtr & query_pt String InterpreterShowIndexesQuery::getRewrittenQuery() { const auto & query = query_ptr->as(); - - WriteBufferFromOwnString buf_table; - writeEscapedString(query.table, buf_table); - String table = buf_table.str(); - - WriteBufferFromOwnString buf_database; + String table = escapeString(query.table); String resolved_database = getContext()->resolveDatabase(query.database); - writeEscapedString(resolved_database, buf_database); - String database = buf_database.str(); - + String database = escapeString(resolved_database); String where_expression = query.where_expression ? fmt::format("WHERE ({})", query.where_expression) : ""; String rewritten_query = fmt::format(R"( diff --git a/src/Interpreters/InterpreterShowSettingQuery.cpp b/src/Interpreters/InterpreterShowSettingQuery.cpp index aa3556a8837..7567e77d28f 100644 --- a/src/Interpreters/InterpreterShowSettingQuery.cpp +++ b/src/Interpreters/InterpreterShowSettingQuery.cpp @@ -1,10 +1,7 @@ #include -#include -#include -#include +#include #include -#include #include #include @@ -23,7 +20,7 @@ InterpreterShowSettingQuery::InterpreterShowSettingQuery(const ASTPtr & query_pt String InterpreterShowSettingQuery::getRewrittenQuery() { const auto & query = query_ptr->as(); - return fmt::format(R"(SELECT value FROM system.settings WHERE name = '{0}')", query.getSettingName()); + return fmt::format(R"(SELECT value FROM system.settings WHERE name = '{0}')", escapeString(query.getSettingName())); } diff --git a/src/Parsers/ParserShowSettingQuery.cpp b/src/Parsers/ParserShowSettingQuery.cpp index 92bed51b688..2586cbdfb43 100644 --- a/src/Parsers/ParserShowSettingQuery.cpp +++ b/src/Parsers/ParserShowSettingQuery.cpp @@ -1,13 +1,10 @@ #include #include -#include #include #include #include -#include -#include namespace DB { diff --git a/src/Parsers/ParserShowSettingQuery.h b/src/Parsers/ParserShowSettingQuery.h index 56c835eb349..ef166133d09 100644 --- a/src/Parsers/ParserShowSettingQuery.h +++ b/src/Parsers/ParserShowSettingQuery.h @@ -5,9 +5,9 @@ namespace DB { -/** Parses queries of the form - * SHOW SETTING setting_name - */ +/** Parses queries of the form: + * SHOW SETTING [setting_name] + */ class ParserShowSettingQuery : public IParserBase { protected: diff --git a/tests/queries/0_stateless/02905_show_setting_query.reference b/tests/queries/0_stateless/02905_show_setting_query.reference index d00491fd7e5..1191247b6d9 100644 --- a/tests/queries/0_stateless/02905_show_setting_query.reference +++ b/tests/queries/0_stateless/02905_show_setting_query.reference @@ -1 +1,2 @@ 1 +2 diff --git a/tests/queries/0_stateless/02905_show_setting_query.sql b/tests/queries/0_stateless/02905_show_setting_query.sql index ee7c9379146..bbbb1a7e237 100644 --- a/tests/queries/0_stateless/02905_show_setting_query.sql +++ b/tests/queries/0_stateless/02905_show_setting_query.sql @@ -1,2 +1,7 @@ SET max_threads = 1; SHOW SETTING max_threads; + +SET max_threads = 2; +SHOW SETTING max_threads; + +SHOW SETTING `max_threads' OR name = 'max_memory_usage`; From aa5fc05a5571675738974a7172118fc59e143f0e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 18 Oct 2023 13:21:43 +0300 Subject: [PATCH 598/634] Revert "Merge pull request #55682 from ClickHouse/revert-35961-decimal-column-improve-get-permutation" This reverts commit f6dee5fe3ce9e475940673c30c9a5014b0b8e1e5, reversing changes made to f96bda1deb8fd2a9300dd4ceacbd1b8c0a8de9b9. --- base/base/sort.h | 26 +++ contrib/pdqsort/pdqsort.h | 200 +++++++++++++++++++++++- src/Columns/ColumnDecimal.cpp | 87 ++++++++++- src/Columns/ColumnVector.cpp | 116 ++++++-------- src/Columns/RadixSortHelper.h | 25 +++ tests/performance/merge_tree_insert.xml | 19 +++ tests/performance/sort_patterns.xml | 28 ++++ 7 files changed, 434 insertions(+), 67 deletions(-) create mode 100644 src/Columns/RadixSortHelper.h create mode 100644 tests/performance/sort_patterns.xml diff --git a/base/base/sort.h b/base/base/sort.h index 912545979dc..1a814587763 100644 --- a/base/base/sort.h +++ b/base/base/sort.h @@ -131,3 +131,29 @@ void sort(RandomIt first, RandomIt last) using comparator = std::less; ::sort(first, last, comparator()); } + +/** Try to fast sort elements for common sorting patterns: + * 1. If elements are already sorted. + * 2. If elements are already almost sorted. + * 3. If elements are already sorted in reverse order. + * + * Returns true if fast sort was performed or elements were already sorted, false otherwise. + */ +template +bool trySort(RandomIt first, RandomIt last, Compare compare) +{ +#ifndef NDEBUG + ::shuffle(first, last); +#endif + + ComparatorWrapper compare_wrapper = compare; + return ::pdqsort_try_sort(first, last, compare_wrapper); +} + +template +bool trySort(RandomIt first, RandomIt last) +{ + using value_type = typename std::iterator_traits::value_type; + using comparator = std::less; + return ::trySort(first, last, comparator()); +} diff --git a/contrib/pdqsort/pdqsort.h b/contrib/pdqsort/pdqsort.h index 01e82b710ee..cbfc82a4f41 100644 --- a/contrib/pdqsort/pdqsort.h +++ b/contrib/pdqsort/pdqsort.h @@ -54,8 +54,10 @@ namespace pdqsort_detail { block_size = 64, // Cacheline size, assumes power of two. - cacheline_size = 64 + cacheline_size = 64, + /// Try sort allowed iterations + try_sort_iterations = 3, }; #if __cplusplus >= 201103L @@ -501,6 +503,167 @@ namespace pdqsort_detail { leftmost = false; } } + + template + inline bool pdqsort_try_sort_loop(Iter begin, + Iter end, + Compare comp, + size_t bad_allowed, + size_t iterations_allowed, + bool force_sort = false, + bool leftmost = true) { + typedef typename std::iterator_traits::difference_type diff_t; + + // Use a while loop for tail recursion elimination. + while (true) { + if (!force_sort && iterations_allowed == 0) { + return false; + } + + diff_t size = end - begin; + + // Insertion sort is faster for small arrays. + if (size < insertion_sort_threshold) { + if (leftmost) insertion_sort(begin, end, comp); + else unguarded_insertion_sort(begin, end, comp); + + return true; + } + + // Choose pivot as median of 3 or pseudomedian of 9. + diff_t s2 = size / 2; + if (size > ninther_threshold) { + sort3(begin, begin + s2, end - 1, comp); + sort3(begin + 1, begin + (s2 - 1), end - 2, comp); + sort3(begin + 2, begin + (s2 + 1), end - 3, comp); + sort3(begin + (s2 - 1), begin + s2, begin + (s2 + 1), comp); + std::iter_swap(begin, begin + s2); + } else sort3(begin + s2, begin, end - 1, comp); + + // If *(begin - 1) is the end of the right partition of a previous partition operation + // there is no element in [begin, end) that is smaller than *(begin - 1). Then if our + // pivot compares equal to *(begin - 1) we change strategy, putting equal elements in + // the left partition, greater elements in the right partition. We do not have to + // recurse on the left partition, since it's sorted (all equal). + if (!leftmost && !comp(*(begin - 1), *begin)) { + begin = partition_left(begin, end, comp) + 1; + continue; + } + + // Partition and get results. + std::pair part_result = + Branchless ? partition_right_branchless(begin, end, comp) + : partition_right(begin, end, comp); + Iter pivot_pos = part_result.first; + bool already_partitioned = part_result.second; + + // Check for a highly unbalanced partition. + diff_t l_size = pivot_pos - begin; + diff_t r_size = end - (pivot_pos + 1); + bool highly_unbalanced = l_size < size / 8 || r_size < size / 8; + + // If we got a highly unbalanced partition we shuffle elements to break many patterns. + if (highly_unbalanced) { + if (!force_sort) { + return false; + } + + // If we had too many bad partitions, switch to heapsort to guarantee O(n log n). + if (--bad_allowed == 0) { + std::make_heap(begin, end, comp); + std::sort_heap(begin, end, comp); + return true; + } + + if (l_size >= insertion_sort_threshold) { + std::iter_swap(begin, begin + l_size / 4); + std::iter_swap(pivot_pos - 1, pivot_pos - l_size / 4); + + if (l_size > ninther_threshold) { + std::iter_swap(begin + 1, begin + (l_size / 4 + 1)); + std::iter_swap(begin + 2, begin + (l_size / 4 + 2)); + std::iter_swap(pivot_pos - 2, pivot_pos - (l_size / 4 + 1)); + std::iter_swap(pivot_pos - 3, pivot_pos - (l_size / 4 + 2)); + } + } + + if (r_size >= insertion_sort_threshold) { + std::iter_swap(pivot_pos + 1, pivot_pos + (1 + r_size / 4)); + std::iter_swap(end - 1, end - r_size / 4); + + if (r_size > ninther_threshold) { + std::iter_swap(pivot_pos + 2, pivot_pos + (2 + r_size / 4)); + std::iter_swap(pivot_pos + 3, pivot_pos + (3 + r_size / 4)); + std::iter_swap(end - 2, end - (1 + r_size / 4)); + std::iter_swap(end - 3, end - (2 + r_size / 4)); + } + } + } else { + // If we were decently balanced and we tried to sort an already partitioned + // sequence try to use insertion sort. + if (already_partitioned && partial_insertion_sort(begin, pivot_pos, comp) + && partial_insertion_sort(pivot_pos + 1, end, comp)) { + return true; + } + } + + // Sort the left partition first using recursion and do tail recursion elimination for + // the right-hand partition. + if (pdqsort_try_sort_loop(begin, + pivot_pos, + comp, + bad_allowed, + iterations_allowed - 1, + force_sort, + leftmost)) { + force_sort = true; + } else { + return false; + } + + --iterations_allowed; + begin = pivot_pos + 1; + leftmost = false; + } + + return false; + } + + template + inline bool pdqsort_try_sort_impl(Iter begin, Iter end, Compare comp, size_t bad_allowed) + { + typedef typename std::iterator_traits::difference_type diff_t; + + static constexpr size_t iterations_allowed = pdqsort_detail::try_sort_iterations; + static constexpr size_t num_to_try = 16; + + diff_t size = end - begin; + + if (size > num_to_try * 10) + { + size_t out_of_order_elements = 0; + + for (size_t i = 1; i < num_to_try; ++i) + { + diff_t offset = size / num_to_try; + + diff_t prev_position = offset * (i - 1); + diff_t curr_position = offset * i; + diff_t next_position = offset * (i + 1) - 1; + + bool prev_less_than_curr = comp(*(begin + prev_position), *(begin + curr_position)); + bool curr_less_than_next = comp(*(begin + curr_position), *(begin + next_position)); + if ((prev_less_than_curr && curr_less_than_next) || (!prev_less_than_curr && !curr_less_than_next)) + continue; + + ++out_of_order_elements; + if (out_of_order_elements > iterations_allowed) + return false; + } + } + + return pdqsort_try_sort_loop(begin, end, comp, bad_allowed, iterations_allowed); + } } @@ -538,6 +701,41 @@ inline void pdqsort_branchless(Iter begin, Iter end) { pdqsort_branchless(begin, end, std::less()); } +template +inline bool pdqsort_try_sort(Iter begin, Iter end, Compare comp) { + if (begin == end) return true; + +#if __cplusplus >= 201103L + return pdqsort_detail::pdqsort_try_sort_impl::type>::value && + std::is_arithmetic::value_type>::value>( + begin, end, comp, pdqsort_detail::log2(end - begin)); +#else + return pdqsort_detail::pdqsort_try_sort_impl( + begin, end, comp, pdqsort_detail::log2(end - begin)); +#endif +} + +template +inline bool pdqsort_try_sort(Iter begin, Iter end) { + typedef typename std::iterator_traits::value_type T; + return pdqsort_try_sort(begin, end, std::less()); +} + +template +inline bool pdqsort_try_sort_branchless(Iter begin, Iter end, Compare comp) { + if (begin == end) return true; + + return pdqsort_detail::pdqsort_try_sort_impl( + begin, end, comp, pdqsort_detail::log2(end - begin)); +} + +template +inline bool pdqsort_try_sort_branchless(Iter begin, Iter end) { + typedef typename std::iterator_traits::value_type T; + return pdqsort_try_sort_branchless(begin, end, std::less()); +} + #undef PDQSORT_PREFER_MOVE diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 0d82818a431..111c0e3cb1c 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -15,6 +16,7 @@ #include #include #include +#include #include @@ -159,6 +161,59 @@ void ColumnDecimal::getPermutation(IColumn::PermutationSortDirection directio return data[lhs] > data[rhs]; }; + size_t data_size = data.size(); + res.resize(data_size); + + if (limit >= data_size) + limit = 0; + + for (size_t i = 0; i < data_size; ++i) + res[i] = i; + + if constexpr (is_arithmetic_v && !is_big_int_v) + { + if (!limit) + { + /// A case for radix sort + /// LSD RadixSort is stable + + bool reverse = direction == IColumn::PermutationSortDirection::Descending; + bool ascending = direction == IColumn::PermutationSortDirection::Ascending; + bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable; + + /// TODO: LSD RadixSort is currently not stable if direction is descending + bool use_radix_sort = (sort_is_stable && ascending) || !sort_is_stable; + + /// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters. + if (data_size >= 256 && data_size <= std::numeric_limits::max() && use_radix_sort) + { + for (size_t i = 0; i < data_size; ++i) + res[i] = i; + + bool try_sort = false; + + if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) + try_sort = trySort(res.begin(), res.end(), comparator_ascending); + else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) + try_sort = trySort(res.begin(), res.end(), comparator_ascending_stable); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) + try_sort = trySort(res.begin(), res.end(), comparator_descending); + else + try_sort = trySort(res.begin(), res.end(), comparator_descending_stable); + + if (try_sort) + return; + + PaddedPODArray> pairs(data_size); + for (UInt32 i = 0; i < static_cast(data_size); ++i) + pairs[i] = {data[i].value, i}; + + RadixSort>::executeLSD(pairs.data(), data_size, reverse, res.data()); + return; + } + } + } + if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) this->getPermutationImpl(limit, res, comparator_ascending, DefaultSort(), DefaultPartialSort()); else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) @@ -191,7 +246,37 @@ void ColumnDecimal::updatePermutation(IColumn::PermutationSortDirection direc return data[lhs] < data[rhs]; }; auto equals_comparator = [this](size_t lhs, size_t rhs) { return data[lhs] == data[rhs]; }; - auto sort = [](auto begin, auto end, auto pred) { ::sort(begin, end, pred); }; + auto sort = [&](auto begin, auto end, auto pred) + { + bool reverse = direction == IColumn::PermutationSortDirection::Descending; + bool ascending = direction == IColumn::PermutationSortDirection::Ascending; + bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable; + + /// TODO: LSD RadixSort is currently not stable if direction is descending + bool use_radix_sort = (sort_is_stable && ascending) || !sort_is_stable; + size_t size = end - begin; + + if (size >= 256 && size <= std::numeric_limits::max() && use_radix_sort) + { + bool try_sort = trySort(begin, end, pred); + if (try_sort) + return; + + PaddedPODArray> pairs(size); + size_t index = 0; + + for (auto * it = begin; it != end; ++it) + { + pairs[index] = {data[*it].value, static_cast(*it)}; + ++index; + } + + RadixSort>::executeLSD(pairs.data(), size, reverse, res.data()); + return; + } + + ::sort(begin, end, pred); + }; auto partial_sort = [](auto begin, auto mid, auto end, auto pred) { ::partial_sort(begin, mid, end, pred); }; if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index e46384e4d03..37e62c76596 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -192,26 +193,6 @@ struct ColumnVector::equals bool operator()(size_t lhs, size_t rhs) const { return CompareHelper::equals(parent.data[lhs], parent.data[rhs], nan_direction_hint); } }; -namespace -{ - template - struct ValueWithIndex - { - T value; - UInt32 index; - }; - - template - struct RadixSortTraits : RadixSortNumTraits - { - using Element = ValueWithIndex; - using Result = size_t; - - static T & extractKey(Element & elem) { return elem.value; } - static size_t extractResult(Element & elem) { return elem.index; } - }; -} - #if USE_EMBEDDED_COMPILER template @@ -254,35 +235,25 @@ template void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const { - size_t s = data.size(); - res.resize(s); + size_t data_size = data.size(); + res.resize(data_size); - if (s == 0) + if (data_size == 0) return; - if (limit >= s) + if (limit >= data_size) limit = 0; - if (limit) - { - for (size_t i = 0; i < s; ++i) - res[i] = i; + for (size_t i = 0; i < data_size; ++i) + res[i] = i; - if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) - ::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint)); - else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) - ::partial_sort(res.begin(), res.begin() + limit, res.end(), less_stable(*this, nan_direction_hint)); - else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) - ::partial_sort(res.begin(), res.begin() + limit, res.end(), greater(*this, nan_direction_hint)); - else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable) - ::partial_sort(res.begin(), res.begin() + limit, res.end(), greater_stable(*this, nan_direction_hint)); - } - else + if constexpr (is_arithmetic_v && !is_big_int_v) { - /// A case for radix sort - /// LSD RadixSort is stable - if constexpr (is_arithmetic_v && !is_big_int_v) + if (!limit) { + /// A case for radix sort + /// LSD RadixSort is stable + bool reverse = direction == IColumn::PermutationSortDirection::Descending; bool ascending = direction == IColumn::PermutationSortDirection::Ascending; bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable; @@ -291,13 +262,27 @@ void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction bool use_radix_sort = (sort_is_stable && ascending && !std::is_floating_point_v) || !sort_is_stable; /// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters. - if (s >= 256 && s <= std::numeric_limits::max() && use_radix_sort) + if (data_size >= 256 && data_size <= std::numeric_limits::max() && use_radix_sort) { - PaddedPODArray> pairs(s); - for (UInt32 i = 0; i < static_cast(s); ++i) + bool try_sort = false; + + if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) + try_sort = trySort(res.begin(), res.end(), less(*this, nan_direction_hint)); + else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) + try_sort = trySort(res.begin(), res.end(), less_stable(*this, nan_direction_hint)); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) + try_sort = trySort(res.begin(), res.end(), greater(*this, nan_direction_hint)); + else + try_sort = trySort(res.begin(), res.end(), greater_stable(*this, nan_direction_hint)); + + if (try_sort) + return; + + PaddedPODArray> pairs(data_size); + for (UInt32 i = 0; i < static_cast(data_size); ++i) pairs[i] = {data[i], i}; - RadixSort>::executeLSD(pairs.data(), s, reverse, res.data()); + RadixSort>::executeLSD(pairs.data(), data_size, reverse, res.data()); /// Radix sort treats all NaNs to be greater than all numbers. /// If the user needs the opposite, we must move them accordingly. @@ -305,9 +290,9 @@ void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction { size_t nans_to_move = 0; - for (size_t i = 0; i < s; ++i) + for (size_t i = 0; i < data_size; ++i) { - if (isNaN(data[res[reverse ? i : s - 1 - i]])) + if (isNaN(data[res[reverse ? i : data_size - 1 - i]])) ++nans_to_move; else break; @@ -315,38 +300,35 @@ void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction if (nans_to_move) { - std::rotate(std::begin(res), std::begin(res) + (reverse ? nans_to_move : s - nans_to_move), std::end(res)); + std::rotate(std::begin(res), std::begin(res) + (reverse ? nans_to_move : data_size - nans_to_move), std::end(res)); } } + return; } } - - /// Default sorting algorithm. - for (size_t i = 0; i < s; ++i) - res[i] = i; - - if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) - ::sort(res.begin(), res.end(), less(*this, nan_direction_hint)); - else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) - ::sort(res.begin(), res.end(), less_stable(*this, nan_direction_hint)); - else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) - ::sort(res.begin(), res.end(), greater(*this, nan_direction_hint)); - else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable) - ::sort(res.begin(), res.end(), greater_stable(*this, nan_direction_hint)); } + + if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) + this->getPermutationImpl(limit, res, less(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) + this->getPermutationImpl(limit, res, less_stable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); + else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) + this->getPermutationImpl(limit, res, greater(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); + else + this->getPermutationImpl(limit, res, greater_stable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort()); } template void ColumnVector::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const { - bool reverse = direction == IColumn::PermutationSortDirection::Descending; - bool ascending = direction == IColumn::PermutationSortDirection::Ascending; - bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable; - auto sort = [&](auto begin, auto end, auto pred) { + bool reverse = direction == IColumn::PermutationSortDirection::Descending; + bool ascending = direction == IColumn::PermutationSortDirection::Ascending; + bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable; + /// A case for radix sort if constexpr (is_arithmetic_v && !is_big_int_v) { @@ -357,6 +339,10 @@ void ColumnVector::updatePermutation(IColumn::PermutationSortDirection direct /// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters. if (size >= 256 && size <= std::numeric_limits::max() && use_radix_sort) { + bool try_sort = trySort(begin, end, pred); + if (try_sort) + return; + PaddedPODArray> pairs(size); size_t index = 0; diff --git a/src/Columns/RadixSortHelper.h b/src/Columns/RadixSortHelper.h new file mode 100644 index 00000000000..e7d8ea6e535 --- /dev/null +++ b/src/Columns/RadixSortHelper.h @@ -0,0 +1,25 @@ +#pragma once + +#include + +namespace DB +{ + +template +struct ValueWithIndex +{ + T value; + UInt32 index; +}; + +template +struct RadixSortTraits : RadixSortNumTraits +{ + using Element = ValueWithIndex; + using Result = size_t; + + static T & extractKey(Element & elem) { return elem.value; } + static size_t extractResult(Element & elem) { return elem.index; } +}; + +} diff --git a/tests/performance/merge_tree_insert.xml b/tests/performance/merge_tree_insert.xml index 1e987d27d50..3e1d2541480 100644 --- a/tests/performance/merge_tree_insert.xml +++ b/tests/performance/merge_tree_insert.xml @@ -18,15 +18,29 @@ merge_tree_insert_6 + + + decimal_primary_key_table_name + + merge_tree_insert_7 + merge_tree_insert_8 + merge_tree_insert_9 + + CREATE TABLE merge_tree_insert_1 (value_1 UInt64, value_2 UInt64, value_3 UInt64) ENGINE = MergeTree ORDER BY (value_1) CREATE TABLE merge_tree_insert_2 (value_1 UInt64, value_2 UInt64, value_3 UInt64) ENGINE = MergeTree ORDER BY (value_1, value_2) CREATE TABLE merge_tree_insert_3 (value_1 UInt64, value_2 UInt64, value_3 UInt64) ENGINE = MergeTree ORDER BY (value_1, value_2, value_3) + CREATE TABLE merge_tree_insert_4 (value_1 String, value_2 String, value_3 String) ENGINE = MergeTree ORDER BY (value_1) CREATE TABLE merge_tree_insert_5 (value_1 String, value_2 String, value_3 String) ENGINE = MergeTree ORDER BY (value_1, value_2) CREATE TABLE merge_tree_insert_6 (value_1 String, value_2 String, value_3 String) ENGINE = MergeTree ORDER BY (value_1, value_2, value_3) + CREATE TABLE merge_tree_insert_7 (value_1 Decimal64(8), value_2 Decimal64(8), value_3 Decimal64(8)) ENGINE = MergeTree ORDER BY (value_1) + CREATE TABLE merge_tree_insert_8 (value_1 Decimal64(8), value_2 Decimal64(8), value_3 Decimal64(8)) ENGINE = MergeTree ORDER BY (value_1, value_2) + CREATE TABLE merge_tree_insert_9 (value_1 Decimal64(8), value_2 Decimal64(8), value_3 Decimal64(8)) ENGINE = MergeTree ORDER BY (value_1, value_2, value_3) + INSERT INTO {integer_primary_key_table_name} SELECT rand64(0), rand64(1), rand64(2) FROM system.numbers LIMIT 500000 INSERT INTO {integer_primary_key_table_name} SELECT rand64(0), rand64(1), rand64(2) FROM system.numbers LIMIT 1000000 INSERT INTO {integer_primary_key_table_name} SELECT rand64(0), rand64(1), rand64(2) FROM system.numbers LIMIT 1500000 @@ -35,7 +49,12 @@ INSERT INTO {string_primary_key_table_name} SELECT toString(rand64(0)), toString(rand64(1)), toString(rand64(2)) FROM system.numbers LIMIT 1000000 INSERT INTO {string_primary_key_table_name} SELECT toString(rand64(0)), toString(rand64(1)), toString(rand64(2)) FROM system.numbers LIMIT 1500000 + INSERT INTO {decimal_primary_key_table_name} SELECT rand64(0) % 1000000, rand64(1) % 1500000, rand64(2) % 2000000 FROM system.numbers LIMIT 500000 + INSERT INTO {decimal_primary_key_table_name} SELECT rand64(0) % 1000000, rand64(1) % 1500000, rand64(2) % 2000000 FROM system.numbers LIMIT 1000000 + INSERT INTO {decimal_primary_key_table_name} SELECT rand64(0) % 1000000, rand64(1) % 1500000, rand64(2) % 2000000 FROM system.numbers LIMIT 1500000 + DROP TABLE IF EXISTS {integer_primary_key_table_name} DROP TABLE IF EXISTS {string_primary_key_table_name} + DROP TABLE IF EXISTS {decimal_primary_key_table_name} diff --git a/tests/performance/sort_patterns.xml b/tests/performance/sort_patterns.xml new file mode 100644 index 00000000000..fc49b20cc8c --- /dev/null +++ b/tests/performance/sort_patterns.xml @@ -0,0 +1,28 @@ + + + + integer_type + + UInt32 + UInt64 + + + + sort_expression + + key + key, value + key DESC + key DESC, value DESC + + + + + CREATE TABLE sequential_{integer_type} (key {integer_type}, value {integer_type}) Engine = Memory + + INSERT INTO sequential_{integer_type} SELECT number, number FROM numbers(500000000) + + SELECT key, value FROM sequential_{integer_type} ORDER BY {sort_expression} FORMAT Null + + DROP TABLE IF EXISTS sequential_{integer_type} + From 78977f7d5cba4084235d4beddebba8917c7bf30b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 18 Oct 2023 13:24:41 +0300 Subject: [PATCH 599/634] ColumnDecimal updatePermutation fix --- src/Columns/ColumnDecimal.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 111c0e3cb1c..012ae03bab2 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -271,7 +271,7 @@ void ColumnDecimal::updatePermutation(IColumn::PermutationSortDirection direc ++index; } - RadixSort>::executeLSD(pairs.data(), size, reverse, res.data()); + RadixSort>::executeLSD(pairs.data(), size, reverse, begin); return; } From 0ca1b006f3ad9f0eab34d6def97d37163c58d2aa Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 25 Oct 2023 18:53:58 +0000 Subject: [PATCH 600/634] fix build --- src/Functions/jsonMergePatch.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Functions/jsonMergePatch.cpp b/src/Functions/jsonMergePatch.cpp index 382cbfa07b7..ff790ba86b4 100644 --- a/src/Functions/jsonMergePatch.cpp +++ b/src/Functions/jsonMergePatch.cpp @@ -6,8 +6,6 @@ #include #include #include -#include -#include #include "config.h" #if USE_RAPIDJSON @@ -28,6 +26,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace From 97c99132e02dd90cc4f6eda558c5749939187df2 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 25 Oct 2023 19:02:48 +0000 Subject: [PATCH 601/634] Simplify the implementation --- docs/en/sql-reference/statements/system.md | 8 +- src/Databases/DatabaseReplicated.cpp | 118 +++++++++--------- src/Databases/DatabaseReplicated.h | 13 +- src/Interpreters/Cluster.cpp | 3 +- src/Interpreters/Cluster.h | 2 - src/Interpreters/InterpreterSystemQuery.cpp | 8 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 12 +- src/Parsers/ASTSystemQuery.cpp | 3 - src/Parsers/ASTSystemQuery.h | 1 - src/Parsers/ParserSystemQuery.cpp | 8 -- .../test.py | 2 +- 11 files changed, 75 insertions(+), 103 deletions(-) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 578ff38574a..1558e64f99b 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -97,12 +97,12 @@ The fourth one is useful to remove metadata of dead replica when all other repli Dead replicas of `Replicated` databases can be dropped using following syntax: ``` sql -SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] [FROM GROUP 'group_name'] FROM DATABASE database; -SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] [FROM GROUP 'group_name']; -SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] [FROM GROUP 'group_name'] FROM ZKPATH '/path/to/table/in/zk'; +SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] FROM DATABASE database; +SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name']; +SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] FROM ZKPATH '/path/to/table/in/zk'; ``` -Similar to `SYSTEM DROP REPLICA`, but removes the `Replicated` database replica path from ZooKeeper when there's no database to run `DROP DATABASE`. Please note that it does not remove `ReplicatedMergeTree` replicas (so you may need `SYSTEM DROP REPLICA` as well). Shard and replica names are the names that were specified in `Replicated` engine arguments when creating the database. Also, these names can be obtained from `database_shard_name` and `database_replica_name` columns in `system.clusters`. Replica group name is the name defined by `replica_group_name` [setting](../../operations/server-configuration-parameters/settings.md#replica_group_name) in the server configuration. If the `FROM SHARD` clause is missing, then `replica_name` must be a full replica name in `shard_name|replica_name` format if replica groups are not used and in `shard_name|replica_name|group_name` otherwise. +Similar to `SYSTEM DROP REPLICA`, but removes the `Replicated` database replica path from ZooKeeper when there's no database to run `DROP DATABASE`. Please note that it does not remove `ReplicatedMergeTree` replicas (so you may need `SYSTEM DROP REPLICA` as well). Shard and replica names are the names that were specified in `Replicated` engine arguments when creating the database. Also, these names can be obtained from `database_shard_name` and `database_replica_name` columns in `system.clusters`. If the `FROM SHARD` clause is missing, then `replica_name` must be a full replica name in `shard_name|replica_name` format. ## DROP UNCOMPRESSED CACHE diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 7234baa818a..f68264d60fb 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -118,50 +118,28 @@ DatabaseReplicated::DatabaseReplicated( fillClusterAuthInfo(db_settings.collection_name.value, context_->getConfigRef()); replica_group_name = context_->getConfigRef().getString("replica_group_name", ""); - - if (replica_group_name.find('/') != std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica group name should not contain '/': {}", replica_group_name); - if (replica_group_name.find('|') != std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica group name should not contain '|': {}", replica_group_name); } -String DatabaseReplicated::getFullReplicaName(const String & shard, const String & replica, const String & replica_group) +String DatabaseReplicated::getFullReplicaName(const String & shard, const String & replica) { - if (replica_group.empty()) - return shard + '|' + replica; - else - return shard + '|' + replica + '|' + replica_group; + return shard + '|' + replica; } String DatabaseReplicated::getFullReplicaName() const { - return getFullReplicaName(shard_name, replica_name, replica_group_name); + return getFullReplicaName(shard_name, replica_name); } -DatabaseReplicated::NameParts DatabaseReplicated::parseFullReplicaName(const String & name) +std::pair DatabaseReplicated::parseFullReplicaName(const String & name) { - NameParts parts; - - auto pos_first = name.find('|'); - if (pos_first == std::string::npos) + String shard; + String replica; + auto pos = name.find('|'); + if (pos == std::string::npos || name.find('|', pos + 1) != std::string::npos) throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect replica identifier: {}", name); - - parts.shard = name.substr(0, pos_first); - - auto pos_second = name.find('|', pos_first + 1); - if (pos_second == std::string::npos) - { - parts.replica = name.substr(pos_first + 1); - return parts; - } - - if (name.find('|', pos_second + 1) != std::string::npos) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect replica identifier: {}", name); - - parts.replica = name.substr(pos_first + 1, pos_second - pos_first - 1); - parts.replica_group = name.substr(pos_second + 1); - - return parts; + shard = name.substr(0, pos); + replica = name.substr(pos + 1); + return {shard, replica}; } ClusterPtr DatabaseReplicated::tryGetCluster() const @@ -217,10 +195,17 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const "It's possible if the first replica is not fully created yet " "or if the last replica was just dropped or due to logical error", zookeeper_path); + hosts.clear(); + std::vector paths; for (const auto & host : unfiltered_hosts) + paths.push_back(zookeeper_path + "/replicas/" + host + "/replica_group"); + + auto replica_groups = zookeeper->tryGet(paths); + + for (size_t i = 0; i < paths.size(); ++i) { - if (replica_group_name == parseFullReplicaName(host).replica_group) - hosts.push_back(host); + if (replica_groups[i].data == replica_group_name) + hosts.push_back(unfiltered_hosts[i]); } Int32 cversion = stat.cversion; @@ -253,7 +238,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const assert(!hosts.empty()); assert(hosts.size() == host_ids.size()); - String current_shard = parseFullReplicaName(hosts.front()).shard; + String current_shard = parseFullReplicaName(hosts.front()).first; std::vector> shards; shards.emplace_back(); for (size_t i = 0; i < hosts.size(); ++i) @@ -261,17 +246,17 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const const auto & id = host_ids[i]; if (id == DROPPED_MARK) continue; - auto parts = parseFullReplicaName(hosts[i]); + auto [shard, replica] = parseFullReplicaName(hosts[i]); auto pos = id.rfind(':'); String host_port = id.substr(0, pos); - if (parts.shard != current_shard) + if (shard != current_shard) { - current_shard = parts.shard; + current_shard = shard; if (!shards.back().empty()) shards.emplace_back(); } String hostname = unescapeForFileName(host_port); - shards.back().push_back(DatabaseReplicaInfo{std::move(hostname), std::move(parts.shard), std::move(parts.replica), std::move(parts.replica_group)}); + shards.back().push_back(DatabaseReplicaInfo{std::move(hostname), std::move(shard), std::move(replica)}); } UInt16 default_port = getContext()->getTCPPort(); @@ -301,7 +286,7 @@ std::vector DatabaseReplicated::tryGetAreReplicasActive(const ClusterPtr { for (const auto & replica : addresses_with_failover[shard_index]) { - String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name, replica.database_replica_group_name); + String full_name = getFullReplicaName(replica.database_shard_name, replica.database_replica_name); paths.emplace_back(fs::path(zookeeper_path) / "replicas" / full_name / "active"); } } @@ -381,6 +366,21 @@ void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(LoadingStrictnessL "Replica {} of shard {} of replicated database at {} already exists. Replica host ID: '{}', current host ID: '{}'", replica_name, shard_name, zookeeper_path, replica_host_id, host_id); } + + /// Check that replica_group_name in ZooKeeper matches the local one and change it if necessary. + String zk_replica_group_name; + if (!current_zookeeper->tryGet(replica_path + "/replica_group", zk_replica_group_name)) + { + /// Replica groups were introduced in 23.10, so the node might not exist + current_zookeeper->create(replica_path + "/replica_group", replica_group_name, zkutil::CreateMode::Persistent); + if (!replica_group_name.empty()) + createEmptyLogEntry(current_zookeeper); + } + else if (zk_replica_group_name != replica_group_name) + { + current_zookeeper->set(replica_path + "/replica_group", replica_group_name, -1); + createEmptyLogEntry(current_zookeeper); + } } else if (is_create_query) { @@ -497,21 +497,8 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt for (int attempts = 10; attempts > 0; --attempts) { - Coordination::Stat stat_max_log_ptr; - Coordination::Stat stat_replicas; - String max_log_ptr_str = current_zookeeper->get(zookeeper_path + "/max_log_ptr", &stat_max_log_ptr); - Strings replicas = current_zookeeper->getChildren(zookeeper_path + "/replicas", &stat_replicas); - for (const auto & replica : replicas) - { - NameParts parts = parseFullReplicaName(replica); - if (parts.shard == shard_name && parts.replica == replica_name) - { - throw Exception( - ErrorCodes::REPLICA_ALREADY_EXISTS, - "Replica {} of shard {} of replicated database already exists in the replica group {} at {}", - replica_name, shard_name, parts.replica_group, zookeeper_path); - } - } + Coordination::Stat stat; + String max_log_ptr_str = current_zookeeper->get(zookeeper_path + "/max_log_ptr", &stat); /// This way we make sure that other replica with the same replica_name and shard_name /// but with a different replica_group_name was not created at the same time. @@ -521,11 +508,12 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/digest", "0", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/replica_group", replica_group_name, zkutil::CreateMode::Persistent)); /// In addition to creating the replica nodes, we record the max_log_ptr at the instant where /// we declared ourself as an existing replica. We'll need this during recoverLostReplica to /// notify other nodes that issued new queries while this node was recovering. - ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/max_log_ptr", stat_max_log_ptr.version)); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/replicas", replica_value, stat_replicas.version)); + ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/max_log_ptr", stat.version)); + Coordination::Responses responses; const auto code = current_zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZOK) @@ -759,10 +747,16 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex Strings hosts_to_wait; Strings unfiltered_hosts = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); + std::vector paths; for (const auto & host : unfiltered_hosts) + paths.push_back(zookeeper_path + "/replicas/" + host + "/replica_group"); + + auto replica_groups = getZooKeeper()->tryGet(paths); + + for (size_t i = 0; i < paths.size(); ++i) { - if (replica_group_name == parseFullReplicaName(host).replica_group) - hosts_to_wait.push_back(host); + if (replica_groups[i].data == replica_group_name) + hosts_to_wait.push_back(unfiltered_hosts[i]); } return getDistributedDDLStatus(node_path, entry, query_context, &hosts_to_wait); @@ -1172,11 +1166,11 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node } void DatabaseReplicated::dropReplica( - DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, const String & replica_group) + DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica) { assert(!database || database_zookeeper_path == database->zookeeper_path); - String full_replica_name = shard.empty() ? replica : getFullReplicaName(shard, replica, replica_group); + String full_replica_name = shard.empty() ? replica : getFullReplicaName(shard, replica); if (full_replica_name.find('/') != std::string::npos) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid replica name, '/' is not allowed: {}", full_replica_name); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 1622578f3d9..005180624ed 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -54,19 +54,12 @@ public: void stopReplication() override; - struct NameParts - { - String shard; - String replica; - String replica_group; - }; - String getShardName() const { return shard_name; } String getReplicaName() const { return replica_name; } String getReplicaGroupName() const { return replica_group_name; } String getFullReplicaName() const; - static String getFullReplicaName(const String & shard, const String & replica, const String & replica_group); - static NameParts parseFullReplicaName(const String & name); + static String getFullReplicaName(const String & shard, const String & replica); + static std::pair parseFullReplicaName(const String & name); const String & getZooKeeperPath() const { return zookeeper_path; } @@ -88,7 +81,7 @@ public: bool shouldReplicateQuery(const ContextPtr & query_context, const ASTPtr & query_ptr) const override; - static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica, const String & replica_group); + static void dropReplica(DatabaseReplicated * database, const String & database_zookeeper_path, const String & shard, const String & replica); std::vector tryGetAreReplicasActive(const ClusterPtr & cluster_) const; diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index fbc760bc486..82c3d48bc05 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -159,7 +159,6 @@ Cluster::Address::Address( host_name = parsed_host_port.first; database_shard_name = info.shard_name; database_replica_name = info.replica_name; - database_replica_group_name = info.replica_group_name; port = parsed_host_port.second; secure = params.secure ? Protocol::Secure::Enable : Protocol::Secure::Disable; priority = params.priority; @@ -517,7 +516,7 @@ Cluster::Cluster( Addresses current; for (const auto & replica : shard) current.emplace_back( - DatabaseReplicaInfo{replica, "", "", ""}, + DatabaseReplicaInfo{replica, "", ""}, params, current_shard_num, current.size() + 1); diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index acda6d9afec..b2bc03dd74d 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -35,7 +35,6 @@ struct DatabaseReplicaInfo String hostname; String shard_name; String replica_name; - String replica_group_name; }; struct ClusterConnectionParameters @@ -112,7 +111,6 @@ public: String host_name; String database_shard_name; String database_replica_name; - String database_replica_group_name; UInt16 port{0}; String user; String password; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index d11c2d9a969..07a1ae7d170 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -927,7 +927,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) if (!query_.replica_zk_path.empty() && fs::path(replicated->getZooKeeperPath()) != fs::path(query_.replica_zk_path)) return; String full_replica_name = query_.shard.empty() ? query_.replica - : DatabaseReplicated::getFullReplicaName(query_.shard, query_.replica, query_.replica_group); + : DatabaseReplicated::getFullReplicaName(query_.shard, query_.replica); if (replicated->getFullReplicaName() != full_replica_name) return; @@ -943,7 +943,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) if (auto * replicated = dynamic_cast(database.get())) { check_not_local_replica(replicated, query); - DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica, query.replica_group); + DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica); } else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database {} is not Replicated, cannot drop replica", query.getDatabase()); @@ -968,7 +968,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) } check_not_local_replica(replicated, query); - DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica, query.replica_group); + DatabaseReplicated::dropReplica(replicated, replicated->getZooKeeperPath(), query.shard, query.replica); LOG_TRACE(log, "Dropped replica {} of Replicated database {}", query.replica, backQuoteIfNeed(database->getDatabaseName())); } } @@ -981,7 +981,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query) if (auto * replicated = dynamic_cast(elem.second.get())) check_not_local_replica(replicated, query); - DatabaseReplicated::dropReplica(nullptr, query.replica_zk_path, query.shard, query.replica, query.replica_group); + DatabaseReplicated::dropReplica(nullptr, query.replica_zk_path, query.shard, query.replica); LOG_INFO(log, "Dropped replica {} of Replicated database with path {}", query.replica, query.replica_zk_path); } else diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 188865cb35c..750affdfe71 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -357,9 +357,9 @@ Chunk DDLQueryStatusSource::generateChunkWithUnfinishedHosts() const size_t num = 0; if (is_replicated_database) { - auto parts = DatabaseReplicated::parseFullReplicaName(host_id); - columns[num++]->insert(parts.shard); - columns[num++]->insert(parts.replica); + auto [shard, replica] = DatabaseReplicated::parseFullReplicaName(host_id); + columns[num++]->insert(shard); + columns[num++]->insert(replica); if (active_hosts_set.contains(host_id)) columns[num++]->insert(IN_PROGRESS); else @@ -511,9 +511,9 @@ Chunk DDLQueryStatusSource::generate() { if (status.code != 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "There was an error on {}: {} (probably it's a bug)", host_id, status.message); - auto parts = DatabaseReplicated::parseFullReplicaName(host_id); - columns[num++]->insert(parts.shard); - columns[num++]->insert(parts.replica); + auto [shard, replica] = DatabaseReplicated::parseFullReplicaName(host_id); + columns[num++]->insert(shard); + columns[num++]->insert(replica); columns[num++]->insert(OK); } else diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index b6817bb8a3a..77235dfb6c2 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -116,9 +116,6 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, if (!shard.empty()) print_keyword(" FROM SHARD ") << quoteString(shard); - if (!replica_group.empty()) - print_keyword(" FROM GROUP ") << quoteString(replica_group); - if (table) { print_keyword(" FROM TABLE "); diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 3315d7dd3b6..cc06e0fdcb5 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -107,7 +107,6 @@ public: String replica; String shard; String replica_zk_path; - String replica_group; bool is_drop_whole_replica{}; String storage_policy; String volume; diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 979debeb75f..a26fdc1396b 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -165,14 +165,6 @@ enum class SystemQueryTargetType if (!ParserStringLiteral{}.parse(pos, ast, expected)) return false; res->shard = ast->as().value.safeGet(); - - if (database && ParserKeyword{"FROM GROUP"}.ignore(pos, expected)) - { - ASTPtr group_ast; - if (!ParserStringLiteral{}.parse(pos, group_ast, expected)) - return false; - res->replica_group = group_ast->as().value.safeGet(); - } } if (ParserKeyword{"FROM"}.ignore(pos, expected)) diff --git a/tests/integration/test_replicated_database_cluster_groups/test.py b/tests/integration/test_replicated_database_cluster_groups/test.py index db43c37bc6e..b14581c1fe6 100644 --- a/tests/integration/test_replicated_database_cluster_groups/test.py +++ b/tests/integration/test_replicated_database_cluster_groups/test.py @@ -118,7 +118,7 @@ def test_cluster_groups(started_cluster): # 4. SYSTEM DROP DATABASE REPLICA backup_node_2.stop_clickhouse() backup_node_1.query( - "SYSTEM DROP DATABASE REPLICA '4' FROM SHARD '1' FROM GROUP 'backups' FROM DATABASE cluster_groups" + "SYSTEM DROP DATABASE REPLICA '1|4' FROM DATABASE cluster_groups" ) assert_eq_with_retry(backup_node_1, cluster_query, "backup_node_1\n") From 95f58777e90a3bc628dca0d6ff3827bbb4cda58a Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 25 Oct 2023 19:04:30 +0000 Subject: [PATCH 602/634] Fix style --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index fa9f8e70692..434ba557859 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1401,7 +1401,7 @@ For more information, see the section [Creating replicated tables](../../engines Replica group name for database Replicated. The cluster created by Replicated database will consist of replicas in the same group. -DDL queries will only wail for the replicas in the same group. +DDL queries will only wait for the replicas in the same group. Empty by default. From d5ee8635941deb231bf7eedf1783f59f2eed9fdf Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 25 Oct 2023 19:06:50 +0000 Subject: [PATCH 603/634] Fix --- src/Databases/DatabaseReplicated.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index f68264d60fb..0cf1dbfd675 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -500,10 +500,6 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt Coordination::Stat stat; String max_log_ptr_str = current_zookeeper->get(zookeeper_path + "/max_log_ptr", &stat); - /// This way we make sure that other replica with the same replica_name and shard_name - /// but with a different replica_group_name was not created at the same time. - String replica_value = "Last added replica: " + getFullReplicaName(); - Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent)); From b2114fe5616451d5fce479c37a5207d059a0218f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 25 Oct 2023 22:08:57 +0200 Subject: [PATCH 604/634] Address review suggestions --- src/Functions/currentProfiles.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Functions/currentProfiles.cpp b/src/Functions/currentProfiles.cpp index b7350eed4e0..71b0eda5f26 100644 --- a/src/Functions/currentProfiles.cpp +++ b/src/Functions/currentProfiles.cpp @@ -32,7 +32,7 @@ namespace } } - class FunctionCurrentProfiles : public IFunction + class FunctionProfiles : public IFunction { public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override @@ -45,7 +45,7 @@ namespace return toString(kind); } - explicit FunctionCurrentProfiles(const ContextPtr & context, Kind kind_) + explicit FunctionProfiles(const ContextPtr & context, Kind kind_) : kind(kind_) { const auto & manager = context->getAccessControl(); @@ -87,11 +87,11 @@ namespace }; } -REGISTER_FUNCTION(CurrentProfiles) +REGISTER_FUNCTION(Profiles) { - factory.registerFunction("currentProfiles", [](ContextPtr context){ return std::make_unique(std::make_shared(context, Kind::currentProfiles)); }); - factory.registerFunction("enabledProfiles", [](ContextPtr context){ return std::make_unique(std::make_shared(context, Kind::enabledProfiles)); }); - factory.registerFunction("defaultProfiles", [](ContextPtr context){ return std::make_unique(std::make_shared(context, Kind::defaultProfiles)); }); + factory.registerFunction("currentProfiles", [](ContextPtr context){ return std::make_unique(std::make_shared(context, Kind::currentProfiles)); }); + factory.registerFunction("enabledProfiles", [](ContextPtr context){ return std::make_unique(std::make_shared(context, Kind::enabledProfiles)); }); + factory.registerFunction("defaultProfiles", [](ContextPtr context){ return std::make_unique(std::make_shared(context, Kind::defaultProfiles)); }); } } From 1c38f387a57a1ff8ef4122c4e2927b4990dabe8d Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 25 Oct 2023 22:19:11 +0200 Subject: [PATCH 605/634] Poke CI From 077074403203849675e80822a314b82747c00a33 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 25 Oct 2023 21:07:03 +0000 Subject: [PATCH 606/634] Try to stabilize test --- .../0_stateless/01942_dateTimeToSnowflake.sql | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql index 3789f130b99..7a3f874b0ec 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql @@ -1,3 +1,5 @@ +SET session_timezone = 'Africa/Juba'; + -- Error cases SELECT dateTimeToSnowflake(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} SELECT dateTime64ToSnowflake(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} @@ -30,11 +32,11 @@ SELECT dateTime64ToSnowflake(dt64_0), dateTime64ToSnowflake(dt64_4); -- DateTime64-to-Snowflake-to-DateTime64 is idempotent *if* the scale is <=3 (millisecond precision) -WITH now64(0) AS dt64_0, - now64(1) AS dt64_1, - now64(2) AS dt64_2, - now64(3) AS dt64_3, - now64(4) AS dt64_4 +WITH now64(0, 'UTC') AS dt64_0, + now64(1, 'UTC') AS dt64_1, + now64(2, 'UTC') AS dt64_2, + now64(3, 'UTC') AS dt64_3, + now64(4, 'UTC') AS dt64_4 SELECT snowflakeToDateTime64(dateTime64ToSnowflake(dt64_0)) == dt64_0, snowflakeToDateTime64(dateTime64ToSnowflake(dt64_1)) == dt64_1, snowflakeToDateTime64(dateTime64ToSnowflake(dt64_2)) == dt64_2, From 2f3695add8dfd763fe41ffdb6beb9ed2911282af Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 19 Oct 2023 11:11:45 +0200 Subject: [PATCH 607/634] Introduce setting for dt overflow exception Added tests and docs --- docs/en/operations/settings/settings.md | 12 + docs/ru/operations/settings/settings.md | 12 + src/Common/ErrorCodes.cpp | 1 + src/Core/Settings.h | 3 +- src/Core/SettingsEnums.cpp | 5 + src/Core/SettingsEnums.h | 9 + src/Functions/DateTimeTransforms.cpp | 1 - src/Functions/DateTimeTransforms.h | 44 +- src/Functions/FunctionsConversion.h | 603 +++++++++++++----- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 24 +- .../02900_date_time_check_overflow.reference | 23 + .../02900_date_time_check_overflow.sql | 50 ++ 12 files changed, 613 insertions(+), 174 deletions(-) create mode 100644 tests/queries/0_stateless/02900_date_time_check_overflow.reference create mode 100644 tests/queries/0_stateless/02900_date_time_check_overflow.sql diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c5f32dfe254..e3c8f729f3f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4771,3 +4771,15 @@ LIFETIME(MIN 0 MAX 3600) LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) SETTINGS(dictionary_use_async_executor=1, max_threads=8); ``` +## date_time_overflow_mode {#date_time_overflow_mode} + +Defines behavior on overflow when converting [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md). + +Possible values: + +- `ignore` — Silently ignore overflow. The result may or may not be saturated to fit the resulting data type values range. The behavior is different for different data types. +- `throw` — Throw an exception in case of conversion overflow. +- `saturate` — Silently saturate the result to fit the result type values range. + +Default value: `ignore`. + diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index c58b3ae8107..087c3601f54 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4209,3 +4209,15 @@ SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_pars │ 1.7091 │ 15008753 │ └─────────────────────┴──────────────────────────┘ ``` + +## date_time_overflow_mode {#date_time_overflow_mode} + +Определяет поведение при переполнении во время преобразования типов [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md). + +Возможные значения: + +- `ignore` — Молча игнорирует переполнение. Полученный результат может быть округлём до ближайшего действительного значения области значений результирующего типа. А может и не быть, тогда результатом будет просто какое-то значение из всех возможных. +- `throw` — Выкинуть исключение при переполнении. +- `saturate` — Молча округлить до ближайшего (то есть наибольшего или наименьшего) значения из области значений результирующего типа. + +Значение по умолчанию: `ignore`. diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index d2196b3c2b9..336512d9be1 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -587,6 +587,7 @@ M(705, TABLE_NOT_EMPTY) \ M(706, LIBSSH_ERROR) \ M(707, ILLEGAL_JSON_OBJECT_FORMAT) \ + M(708, DATE_TIME_OVERFLOW) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index de5b37101a1..96a7acb9d51 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -810,9 +810,10 @@ class IColumn; M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \ M(Bool, print_pretty_type_names, false, "Print pretty type names in DESCRIBE query and toTypeName() function", 0) \ M(Bool, create_table_empty_primary_key_by_default, false, "Allow to create *MergeTree tables with empty primary key when ORDER BY and PRIMARY KEY not specified", 0) \ + M(DateTimeOverflowMode, date_time_overflow_mode, DateTimeOverflowMode::IGNORE, "Sets overflow mode for Date, Date32, DateTime, DateTime64 types.", 0) \ // End of COMMON_SETTINGS - // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. +// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. #define MAKE_OBSOLETE(M, TYPE, NAME, DEFAULT) \ M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", BaseSettingsHelpers::Flags::OBSOLETE) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 82e158877c5..bb889cd953b 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -190,4 +190,9 @@ IMPLEMENT_SETTING_ENUM(ExternalCommandStderrReaction, ErrorCodes::BAD_ARGUMENTS, {"log_last", ExternalCommandStderrReaction::LOG_LAST}, {"throw", ExternalCommandStderrReaction::THROW}}) +IMPLEMENT_SETTING_ENUM(DateTimeOverflowMode, ErrorCodes::BAD_ARGUMENTS, + {{"throw", DateTimeOverflowMode::THROW}, + {"ignore", DateTimeOverflowMode::IGNORE}, + {"saturate", DateTimeOverflowMode::SATURATE}}) + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 7db8c88c53d..ab90aa740f5 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -242,4 +242,13 @@ DECLARE_SETTING_ENUM(S3QueueAction) DECLARE_SETTING_ENUM(ExternalCommandStderrReaction) +enum class DateTimeOverflowMode +{ + THROW, + IGNORE, + SATURATE, +}; + +DECLARE_SETTING_ENUM(DateTimeOverflowMode) + } diff --git a/src/Functions/DateTimeTransforms.cpp b/src/Functions/DateTimeTransforms.cpp index af1d9a4a5bb..7ec13be9d6d 100644 --- a/src/Functions/DateTimeTransforms.cpp +++ b/src/Functions/DateTimeTransforms.cpp @@ -22,5 +22,4 @@ void throwDate32IsNotSupported(const char * name) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date32 of argument for function {}", name); } - } diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 0814d640f2b..aca9311e18d 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -23,12 +23,15 @@ namespace DB static constexpr auto microsecond_multiplier = 1000000; static constexpr auto millisecond_multiplier = 1000; +static constexpr DateTimeOverflowMode default_date_time_overflow_mode = DateTimeOverflowMode::IGNORE; + namespace ErrorCodes { extern const int CANNOT_CONVERT_TYPE; extern const int DECIMAL_OVERFLOW; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int DATE_TIME_OVERFLOW; } /** Transformations. @@ -57,25 +60,51 @@ struct ZeroTransform static UInt16 execute(UInt16, const DateLUTImpl &) { return 0; } }; +template struct ToDateImpl { static constexpr auto name = "toDate"; static UInt16 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { - return static_cast(time_zone.toDayNum(t.whole)); + return execute(t.whole, time_zone); } + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { - return UInt16(time_zone.toDayNum(t)); + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::SATURATE) + { + if (t < 0) + t = 0; + else if (t > 5662310399) // 2149-06-06 23:59:59 UTC + t = 5662310399; + } + else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (unlikely(t < 0 || t > 5662310399)) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type Date", t); + } + return static_cast(time_zone.toDayNum(t)); } static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { - return UInt16(time_zone.toDayNum(t)); + return UInt16(time_zone.toDayNum(t)); /// never causes overflow by design } - static UInt16 execute(Int32, const DateLUTImpl &) + static UInt16 execute(Int32 t, const DateLUTImpl &) { - throwDateIsNotSupported(name); + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::SATURATE) + { + if (t < 0) + t = 0; + else if (t > DATE_LUT_MAX_DAY_NUM) + t = DATE_LUT_MAX_DAY_NUM; + } + else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (unlikely(t < 0 || t > DATE_LUT_MAX_DAY_NUM)) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type Date", t); + } + return static_cast(t); } static UInt16 execute(UInt16 d, const DateLUTImpl &) { @@ -89,6 +118,7 @@ struct ToDateImpl using FactorTransform = ZeroTransform; }; +template struct ToDate32Impl { static constexpr auto name = "toDate32"; @@ -750,7 +780,7 @@ struct ToTimeImpl } static constexpr bool hasPreimage() { return false; } - using FactorTransform = ToDateImpl; + using FactorTransform = ToDateImpl<>; }; struct ToStartOfMinuteImpl @@ -1401,7 +1431,7 @@ struct ToHourImpl } static constexpr bool hasPreimage() { return false; } - using FactorTransform = ToDateImpl; + using FactorTransform = ToDateImpl<>; }; struct TimezoneOffsetImpl diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 69c5aa48155..5311deb45fc 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -90,8 +90,13 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; extern const int CANNOT_PARSE_BOOL; -} + extern const int DATE_TIME_OVERFLOW; +constexpr time_t MAX_DT64_TIMESTAMP = 10413791999LL; // 1900-01-01 00:00:00 UTC +constexpr time_t MIN_DT64_TIMESTAMP = -2208988800LL; // 2299-12-31 23:59:59 UTC +constexpr time_t MAX_DT_TIMESTAMP = 0xFFFFFFFF; +constexpr time_t MAX_DATE_TIMESTAMP = 5662310399; +constexpr time_t MAX_DT_DAY_NUM = 49710; /** Type conversion functions. * toType - conversion in "natural way"; @@ -133,7 +138,8 @@ struct ConvertReturnZeroOnErrorTag {}; /** Conversion of number types to each other, enums to numbers, dates and datetimes to numbers and back: done by straight assignment. * (Date is represented internally as number of days from some day; DateTime - as unix timestamp) */ -template +template struct ConvertImpl { using FromFieldType = typename FromDataType::FieldType; @@ -388,28 +394,53 @@ struct ConvertImpl /** Conversion of DateTime to Date: throw off time component. */ -template struct ConvertImpl - : DateTimeTransformImpl {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; /** Conversion of DateTime to Date32: throw off time component. */ -template struct ConvertImpl - : DateTimeTransformImpl {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; /** Conversion of Date to DateTime: adding 00:00:00 time component. */ +template struct ToDateTimeImpl { static constexpr auto name = "toDateTime"; static UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (d > MAX_DT_DAY_NUM) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Day number {} is out of bounds of type DateTime", d); + } + else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::SATURATE) + { + if (d > MAX_DT_DAY_NUM) + d = MAX_DT_DAY_NUM; + } return static_cast(time_zone.fromDayNum(DayNum(d))); } - static Int64 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt32 execute(Int32 d, const DateLUTImpl & time_zone) { - return time_zone.fromDayNum(ExtendedDayNum(d)); + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::SATURATE) + { + if (d < 0) + return 0; + else if (d > MAX_DT_DAY_NUM) + d = MAX_DT_DAY_NUM; + } + else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (unlikely(d < 0 || d > MAX_DT_DAY_NUM)) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type DateTime", d); + } + return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))); } static UInt32 execute(UInt32 dt, const DateLUTImpl & /*time_zone*/) @@ -417,36 +448,63 @@ struct ToDateTimeImpl return dt; } - // TODO: return UInt32 ??? - static Int64 execute(Int64 dt64, const DateLUTImpl & /*time_zone*/) + static UInt32 execute(Int64 dt64, const DateLUTImpl & /*time_zone*/) { - return dt64; + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::IGNORE) + return static_cast(dt64); + else + { + if (unlikely(dt64 < 0 || dt64 >= MAX_DT_TIMESTAMP)) + { + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::SATURATE) + return dt64 < 0 ? 0 : std::numeric_limits::max(); + else + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type DateTime", dt64); + } + else + return static_cast(dt64); + } } }; -template struct ConvertImpl - : DateTimeTransformImpl {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template struct ConvertImpl - : DateTimeTransformImpl {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; /// Implementation of toDate function. -template +template struct ToDateTransform32Or64 { static constexpr auto name = "toDate"; static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { - // since converting to Date, no need in values outside of default LUT range. - return (from <= DATE_LUT_MAX_DAY_NUM) - ? from - : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (from > MAX_DT_TIMESTAMP) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type Date", from); + } + /// if value is smaller (or equal) than maximum day value for Date, than treat it as day num, + /// otherwise treat it as unix timestamp. This is a bit weird, but we leave this behavior. + if (from <= DATE_LUT_MAX_DAY_NUM) + return from; + else + return time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DT_TIMESTAMP))); } }; -template +/** Conversion of Date32 to Date. + */ +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template struct ToDateTransform32Or64Signed { static constexpr auto name = "toDate"; @@ -454,16 +512,23 @@ struct ToDateTransform32Or64Signed static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { // TODO: decide narrow or extended range based on FromType - /// The function should be monotonic (better for query optimizations), so we saturate instead of overflow. - if (from < 0) - return 0; + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (from < 0 || from > MAX_DATE_TIMESTAMP) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type Date", from); + } + else + { + if (from < 0) + return 0; + } return (from <= DATE_LUT_MAX_DAY_NUM) ? static_cast(from) - : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); + : time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DATE_TIMESTAMP))); } }; -template +template struct ToDateTransform8Or16Signed { static constexpr auto name = "toDate"; @@ -471,30 +536,44 @@ struct ToDateTransform8Or16Signed static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { if (from < 0) - return 0; + { + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type Date", from); + else + return 0; + } return from; } }; -template struct ConvertImpl - : DateTimeTransformImpl> {}; +template +struct ConvertImpl + : DateTimeTransformImpl>, false> {}; /// Implementation of toDate32 function. -template +template struct ToDate32Transform32Or64 { static constexpr auto name = "toDate32"; static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { - return (from < DATE_LUT_MAX_EXTEND_DAY_NUM) - ? static_cast(from) - : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); + if (from < DATE_LUT_MAX_EXTEND_DAY_NUM) + return static_cast(from); + else + { + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (from > MAX_DT64_TIMESTAMP) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type Date32", from); + } + return time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DT64_TIMESTAMP))); + } } }; -template +template struct ToDate32Transform32Or64Signed { static constexpr auto name = "toDate32"; @@ -502,11 +581,19 @@ struct ToDate32Transform32Or64Signed static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { static const Int32 daynum_min_offset = -static_cast(time_zone.getDayNumOffsetEpoch()); + + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (from < daynum_min_offset || from > MAX_DT64_TIMESTAMP) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type Date32", from); + } + if (from < daynum_min_offset) return daynum_min_offset; + return (from < DATE_LUT_MAX_EXTEND_DAY_NUM) ? static_cast(from) - : time_zone.toDayNum(std::min(time_t(Int64(from)), time_t(0xFFFFFFFF))); + : time_zone.toDayNum(std::min(time_t(Int64(from)), time_t(MAX_DT64_TIMESTAMP))); } }; @@ -522,7 +609,7 @@ struct ToDate32Transform8Or16Signed }; /** Special case of converting Int8, Int16, (U)Int32 or (U)Int64 (and also, for convenience, - * Float32, Float64) to Date. If the number is negative, saturate it to unix epoch time. If the + * Float32, Float64) to Date. If the * number is less than 65536, then it is treated as DayNum, and if it's greater or equals to 65536, * then treated as unix timestamp. If the number exceeds UInt32, saturate to MAX_UINT32 then as DayNum. * It's a bit illogical, as we actually have two functions in one. @@ -530,53 +617,89 @@ struct ToDate32Transform8Or16Signed * when user write toDate(UInt32), expecting conversion of unix timestamp to Date. * (otherwise such usage would be frequent mistake). */ -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + + +template struct ToDateTimeTransform64 { static constexpr auto name = "toDateTime"; static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { - return static_cast(std::min(time_t(from), time_t(0xFFFFFFFF))); + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (from > MAX_DT_TIMESTAMP) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type DateTime", from); + } + return static_cast(std::min(time_t(from), time_t(MAX_DT_TIMESTAMP))); } }; -template +template struct ToDateTimeTransformSigned { static constexpr auto name = "toDateTime"; @@ -584,51 +707,68 @@ struct ToDateTimeTransformSigned static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { if (from < 0) - return 0; + { + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type DateTime", from); + else + return 0; + } return from; } }; -template +template struct ToDateTimeTransform64Signed { static constexpr auto name = "toDateTime"; static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (from < 0 || from > MAX_DT_TIMESTAMP) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type DateTime", from); + } + if (from < 0) return 0; - return static_cast(std::min(time_t(from), time_t(0xFFFFFFFF))); + return static_cast(std::min(time_t(from), time_t(MAX_DT_TIMESTAMP))); } }; -/** Special case of converting Int8, Int16, Int32 or (U)Int64 (and also, for convenience, Float32, - * Float64) to DateTime. If the number is negative, saturate it to unix epoch time. If the number - * exceeds UInt32, saturate to MAX_UINT32. - */ -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; +/// Special case of converting Int8, Int16, Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to DateTime. +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -constexpr time_t LUT_MIN_TIME = -2208988800l; // 1900-01-01 UTC +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -constexpr time_t LUT_MAX_TIME = 10413791999l; // 2299-12-31 UTC +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; /** Conversion of numeric to DateTime64 */ -template +template struct ToDateTime64TransformUnsigned { static constexpr auto name = "toDateTime64"; @@ -641,11 +781,18 @@ struct ToDateTime64TransformUnsigned NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const { - from = std::min(from, LUT_MAX_TIME); - return DecimalUtils::decimalFromComponentsWithMultiplier(from, 0, scale_multiplier); + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (from > MAX_DT64_TIMESTAMP) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type DateTime64", from); + else + return DecimalUtils::decimalFromComponentsWithMultiplier(from, 0, scale_multiplier); + } + else + return DecimalUtils::decimalFromComponentsWithMultiplier(std::min(from, MAX_DT64_TIMESTAMP), 0, scale_multiplier); } }; -template +template struct ToDateTime64TransformSigned { static constexpr auto name = "toDateTime64"; @@ -658,12 +805,18 @@ struct ToDateTime64TransformSigned NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const { - from = static_cast(std::max(from, LUT_MIN_TIME)); - from = static_cast(std::min(from, LUT_MAX_TIME)); + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (from < MIN_DT64_TIMESTAMP || from > MAX_DT64_TIMESTAMP) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type DateTime64", from); + } + from = static_cast(std::max(from, MIN_DT64_TIMESTAMP)); + from = static_cast(std::min(from, MAX_DT64_TIMESTAMP)); + return DecimalUtils::decimalFromComponentsWithMultiplier(from, 0, scale_multiplier); } }; -template +template struct ToDateTime64TransformFloat { static constexpr auto name = "toDateTime64"; @@ -676,26 +829,45 @@ struct ToDateTime64TransformFloat NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const { - from = std::max(from, static_cast(LUT_MIN_TIME)); - from = std::min(from, static_cast(LUT_MAX_TIME)); + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + { + if (from < MIN_DT64_TIMESTAMP || from > MAX_DT64_TIMESTAMP) + throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type DateTime64", from); + } + + from = std::max(from, static_cast(MIN_DT64_TIMESTAMP)); + from = std::min(from, static_cast(MAX_DT64_TIMESTAMP)); return convertToDecimal(from, scale); } }; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; /** Conversion of DateTime64 to Date or DateTime: discards fractional part. @@ -720,10 +892,13 @@ struct FromDateTime64Transform /** Conversion of DateTime64 to Date or DateTime: discards fractional part. */ -template struct ConvertImpl - : DateTimeTransformImpl> {}; -template struct ConvertImpl - : DateTimeTransformImpl> {}; +template +struct ConvertImpl + : DateTimeTransformImpl>, false> {}; + +template +struct ConvertImpl + : DateTimeTransformImpl>, false> {}; struct ToDateTime64Transform { @@ -737,13 +912,13 @@ struct ToDateTime64Transform DateTime64::NativeType execute(UInt16 d, const DateLUTImpl & time_zone) const { - const auto dt = ToDateTimeImpl::execute(d, time_zone); + const auto dt = ToDateTimeImpl<>::execute(d, time_zone); return execute(dt, time_zone); } DateTime64::NativeType execute(Int32 d, const DateLUTImpl & time_zone) const { - const auto dt = ToDateTimeImpl::execute(d, time_zone); + Int64 dt = static_cast(time_zone.fromDayNum(ExtendedDayNum(d))); return DecimalUtils::decimalFromComponentsWithMultiplier(dt, 0, scale_multiplier); } @@ -755,11 +930,16 @@ struct ToDateTime64Transform /** Conversion of Date or DateTime to DateTime64: add zero sub-second part. */ -template struct ConvertImpl +template +struct ConvertImpl : DateTimeTransformImpl {}; -template struct ConvertImpl + +template +struct ConvertImpl : DateTimeTransformImpl {}; -template struct ConvertImpl + +template +struct ConvertImpl : DateTimeTransformImpl {}; @@ -1614,29 +1794,29 @@ struct ConvertThroughParsing }; -template +template requires (!std::is_same_v) -struct ConvertImpl +struct ConvertImpl : ConvertThroughParsing {}; -template +template requires (!std::is_same_v) -struct ConvertImpl +struct ConvertImpl : ConvertThroughParsing {}; -template +template requires (!std::is_same_v) -struct ConvertImpl +struct ConvertImpl : ConvertThroughParsing {}; -template +template requires (!std::is_same_v) -struct ConvertImpl +struct ConvertImpl : ConvertThroughParsing {}; -template +template requires (is_any_of && is_any_of) -struct ConvertImpl +struct ConvertImpl : ConvertThroughParsing {}; /// Generic conversion of any type from String. Used for complex types: Array and Tuple or types with custom serialization. @@ -2093,6 +2273,11 @@ private: const DataTypePtr from_type = removeNullable(arguments[0].type); ColumnPtr result_column; + [[maybe_unused]] DateTimeOverflowMode date_time_overflow_mode = default_date_time_overflow_mode; + + if (context) + date_time_overflow_mode = context->getSettingsRef().date_time_overflow_mode.value; + auto call = [&](const auto & types, const auto & tag) -> bool { using Types = std::decay_t; @@ -2116,13 +2301,42 @@ private: const ColumnWithTypeAndName & scale_column = arguments[1]; UInt32 scale = extractToDecimalScale(scale_column); - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + switch (date_time_overflow_mode) + { + case DateTimeOverflowMode::THROW: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + break; + case DateTimeOverflowMode::IGNORE: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + break; + case DateTimeOverflowMode::SATURATE: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + break; + } + } else if constexpr (IsDataTypeDateOrDateTime && std::is_same_v) { const auto * dt64 = assert_cast(arguments[0].type.get()); - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); + switch (date_time_overflow_mode) + { + case DateTimeOverflowMode::THROW: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); + break; + case DateTimeOverflowMode::IGNORE: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); + break; + case DateTimeOverflowMode::SATURATE: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); + break; + } } +#define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE) \ + case DateTimeOverflowMode::OVERFLOW_MODE: \ + result_column = ConvertImpl::execute( \ + arguments, result_type, input_rows_count); \ + break; + else if constexpr (IsDataTypeDecimalOrNumber && IsDataTypeDecimalOrNumber) { using LeftT = typename LeftDataType::FieldType; @@ -2141,14 +2355,27 @@ private: } else { - result_column - = ConvertImpl::execute(arguments, result_type, input_rows_count); + switch (date_time_overflow_mode) + { + GENERATE_OVERFLOW_MODE_CASE(THROW) + GENERATE_OVERFLOW_MODE_CASE(IGNORE) + GENERATE_OVERFLOW_MODE_CASE(SATURATE) + } } } - else + else if constexpr ((IsDataTypeNumber || IsDataTypeDateOrDateTime) + && IsDataTypeDateOrDateTime) { - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count); + switch (date_time_overflow_mode) + { + GENERATE_OVERFLOW_MODE_CASE(THROW) + GENERATE_OVERFLOW_MODE_CASE(IGNORE) + GENERATE_OVERFLOW_MODE_CASE(SATURATE) + } } +#undef GENERATE_OVERFLOW_MODE_CASE + else + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count); return true; }; @@ -2559,16 +2786,19 @@ struct ToDateMonotonicity static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) { auto which = WhichDataType(type); - if (which.isDateOrDate32() || which.isDateTime() || which.isDateTime64() || which.isInt8() || which.isInt16() || which.isUInt8() || which.isUInt16()) + if (which.isDateOrDate32() || which.isDateTime() || which.isDateTime64() || which.isInt8() || which.isInt16() || which.isUInt8() + || which.isUInt16()) { - return { .is_monotonic = true, .is_always_monotonic = true }; + return {.is_monotonic = true, .is_always_monotonic = true}; } else if ( ((left.getType() == Field::Types::UInt64 || left.isNull()) && (right.getType() == Field::Types::UInt64 || right.isNull()) - && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) + && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) || ((left.getType() == Field::Types::Int64 || left.isNull()) && (right.getType() == Field::Types::Int64 || right.isNull()) && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF))) - || (((left.getType() == Field::Types::Float64 || left.isNull()) && (right.getType() == Field::Types::Float64 || right.isNull()) + || (( + (left.getType() == Field::Types::Float64 || left.isNull()) + && (right.getType() == Field::Types::Float64 || right.isNull()) && ((left.isNull() || left.get() < 0xFFFF) && (right.isNull() || right.get() >= 0xFFFF)))) || !isNativeNumber(type)) { @@ -2576,7 +2806,7 @@ struct ToDateMonotonicity } else { - return { .is_monotonic = true, .is_always_monotonic = true }; + return {.is_monotonic = true, .is_always_monotonic = true}; } } }; @@ -2588,7 +2818,7 @@ struct ToDateTimeMonotonicity static IFunction::Monotonicity get(const IDataType & type, const Field &, const Field &) { if (type.isValueRepresentedByNumber()) - return { .is_monotonic = true, .is_always_monotonic = true }; + return {.is_monotonic = true, .is_always_monotonic = true}; else return {}; } @@ -2674,11 +2904,17 @@ using FunctionToInt128 = FunctionConvert>; using FunctionToFloat32 = FunctionConvert>; using FunctionToFloat64 = FunctionConvert>; + using FunctionToDate = FunctionConvert; + using FunctionToDate32 = FunctionConvert; + using FunctionToDateTime = FunctionConvert; + using FunctionToDateTime32 = FunctionConvert; + using FunctionToDateTime64 = FunctionConvert; + using FunctionToUUID = FunctionConvert>; using FunctionToIPv4 = FunctionConvert>; using FunctionToIPv6 = FunctionConvert>; @@ -2689,8 +2925,7 @@ using FunctionToDecimal64 = FunctionConvert, NameToDe using FunctionToDecimal128 = FunctionConvert, NameToDecimal128, UnknownMonotonicity>; using FunctionToDecimal256 = FunctionConvert, NameToDecimal256, UnknownMonotonicity>; - -template struct FunctionTo; +template struct FunctionTo; template <> struct FunctionTo { using Type = FunctionToUInt8; }; template <> struct FunctionTo { using Type = FunctionToUInt16; }; @@ -2706,10 +2941,19 @@ template <> struct FunctionTo { using Type = FunctionToInt128; } template <> struct FunctionTo { using Type = FunctionToInt256; }; template <> struct FunctionTo { using Type = FunctionToFloat32; }; template <> struct FunctionTo { using Type = FunctionToFloat64; }; -template <> struct FunctionTo { using Type = FunctionToDate; }; -template <> struct FunctionTo { using Type = FunctionToDate32; }; -template <> struct FunctionTo { using Type = FunctionToDateTime; }; -template <> struct FunctionTo { using Type = FunctionToDateTime64; }; + +template +struct FunctionTo { using Type = FunctionToDate; }; + +template +struct FunctionTo { using Type = FunctionToDate32; }; + +template +struct FunctionTo { using Type = FunctionToDateTime; }; + +template +struct FunctionTo { using Type = FunctionToDateTime64; }; + template <> struct FunctionTo { using Type = FunctionToUUID; }; template <> struct FunctionTo { using Type = FunctionToIPv4; }; template <> struct FunctionTo { using Type = FunctionToIPv6; }; @@ -3044,6 +3288,10 @@ private: bool can_apply_accurate_cast = (cast_type == CastType::accurate || cast_type == CastType::accurateOrNull) && (which.isInt() || which.isUInt() || which.isFloat()); + DateTimeOverflowMode date_time_overflow_mode = default_date_time_overflow_mode; + if (context) + date_time_overflow_mode = context->getSettingsRef().date_time_overflow_mode; + if (requested_result_is_nullable && checkAndGetDataType(from_type.get())) { /// In case when converting to Nullable type, we apply different parsing rule, @@ -3060,7 +3308,7 @@ private: auto wrapper_cast_type = cast_type; - return [wrapper_cast_type, from_type_index, to_type] + return [wrapper_cast_type, from_type_index, to_type, date_time_overflow_mode] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *column_nullable, size_t input_rows_count) { ColumnPtr result_column; @@ -3073,31 +3321,60 @@ private: { if constexpr (IsDataTypeNumber) { +#define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE, ADDITIONS) \ + case DateTimeOverflowMode::OVERFLOW_MODE: \ + result_column = ConvertImpl::execute( \ + arguments, result_type, input_rows_count, ADDITIONS()); \ + break; if (wrapper_cast_type == CastType::accurate) { - result_column = ConvertImpl::execute( - arguments, result_type, input_rows_count, AccurateConvertStrategyAdditions()); + switch (date_time_overflow_mode) + { + GENERATE_OVERFLOW_MODE_CASE(THROW, AccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(IGNORE, AccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(SATURATE, AccurateConvertStrategyAdditions) + } } else { - result_column = ConvertImpl::execute( - arguments, result_type, input_rows_count, AccurateOrNullConvertStrategyAdditions()); + switch (date_time_overflow_mode) + { + GENERATE_OVERFLOW_MODE_CASE(THROW, AccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(IGNORE, AccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(SATURATE, AccurateOrNullConvertStrategyAdditions) + } } +#undef GENERATE_OVERFLOW_MODE_CASE + return true; } if constexpr (std::is_same_v || std::is_same_v) { +#define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE, ADDITIONS) \ + case DateTimeOverflowMode::OVERFLOW_MODE: \ + result_column = ConvertImpl::template execute( \ +arguments, result_type, input_rows_count); \ + break; if (wrapper_cast_type == CastType::accurate) { - result_column = ConvertImpl::template execute( - arguments, result_type, input_rows_count); + switch (date_time_overflow_mode) + { + GENERATE_OVERFLOW_MODE_CASE(THROW, DateTimeAccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(IGNORE, DateTimeAccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(SATURATE, DateTimeAccurateConvertStrategyAdditions) + } } else { - result_column = ConvertImpl::template execute( - arguments, result_type, input_rows_count); + switch (date_time_overflow_mode) + { + GENERATE_OVERFLOW_MODE_CASE(THROW, DateTimeAccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(IGNORE, DateTimeAccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(SATURATE, DateTimeAccurateOrNullConvertStrategyAdditions) + } } +#undef GENERATE_OVERFLOW_MODE_CASE return true; } } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 2a9892b7219..e50f89113a8 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -252,6 +253,17 @@ static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptrgetSettingsRef().date_time_overflow_mode; + } + if (!type_hint || (type_hint && isDate32(*type_hint))) { internal_type = std::make_shared(); @@ -277,8 +289,16 @@ static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr(chunk.Value(value_i)); if (days_num > DATE_LUT_MAX_EXTEND_DAY_NUM || days_num < -DAYNUM_OFFSET_EPOCH) - throw Exception{ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, - "Input value {} of a column \"{}\" is out of allowed Date32 range, which is [{}, {}]", days_num, column_name, DAYNUM_OFFSET_EPOCH, DATE_LUT_MAX_EXTEND_DAY_NUM}; + { + if (likely(date_time_overflow_mode == DateTimeOverflowMode::SATURATE)) + days_num = (days_num < -DAYNUM_OFFSET_EPOCH) ? -DAYNUM_OFFSET_EPOCH : DATE_LUT_MAX_EXTEND_DAY_NUM; + else + { + throw Exception{ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, + "Input value {} of a column \"{}\" is out of allowed Date32 range, which is [{}, {}]", + days_num,column_name, -DAYNUM_OFFSET_EPOCH, DATE_LUT_MAX_EXTEND_DAY_NUM}; + } + } column_data.emplace_back(days_num); } diff --git a/tests/queries/0_stateless/02900_date_time_check_overflow.reference b/tests/queries/0_stateless/02900_date_time_check_overflow.reference new file mode 100644 index 00000000000..5f87903e367 --- /dev/null +++ b/tests/queries/0_stateless/02900_date_time_check_overflow.reference @@ -0,0 +1,23 @@ +ignore +2036-02-07 06:28:16 +2027-10-18 11:03:27 +2036-02-07 06:28:16 +2027-10-17 11:03:28 +2013-04-29 17:31:44 +2079-06-07 +1970-01-01 +2120-07-26 +2079-06-07 +2120-07-26 +saturate +1970-01-01 00:00:00 +2106-02-07 06:28:15 +1970-01-01 00:00:00 +2106-02-07 00:00:00 +2106-02-07 00:00:00 +1970-01-01 +2149-06-06 +2149-06-06 +1970-01-01 +2149-06-06 +No output on `throw` diff --git a/tests/queries/0_stateless/02900_date_time_check_overflow.sql b/tests/queries/0_stateless/02900_date_time_check_overflow.sql new file mode 100644 index 00000000000..3d9c567208f --- /dev/null +++ b/tests/queries/0_stateless/02900_date_time_check_overflow.sql @@ -0,0 +1,50 @@ +-- I hope we will get rid of "ignore" option one day :) +-- It gives unpredictable result, but it is faster than others. +SELECT 'ignore'; +SET date_time_overflow_mode = 'ignore'; +SELECT toDateTime(toDateTime64('1900-01-01 00:00:00.123', 3, 'UTC'), 'UTC'); +SELECT toDateTime(toDateTime64('2299-12-31 23:59:59.999', 3, 'UTC'), 'UTC'); + +SELECT toDateTime(toDate32('1900-01-01', 'UTC'), 'UTC'); +SELECT toDateTime(toDate32('2299-12-31', 'UTC'), 'UTC'); + +SELECT toDateTime(toDate('2149-06-06', 'UTC'), 'UTC'); + +SELECT toDate(toDateTime64('1900-01-01 00:00:00.123', 3, 'UTC'), 'UTC'); +SELECT toDate(toDateTime64('2149-06-07 00:00:00.123', 3, 'UTC'), 'UTC'); +SELECT toDate(toDateTime64('2299-12-31 23:59:59.999', 3, 'UTC'), 'UTC'); + +SELECT toDate(toDate32('1900-01-01', 'UTC'), 'UTC'); +SELECT toDate(toDate32('2299-12-31', 'UTC'), 'UTC'); + + +SELECT 'saturate'; +SET date_time_overflow_mode = 'saturate'; + +SELECT toDateTime(toDateTime64('1900-01-01 00:00:00.123', 3, 'UTC'), 'UTC'); +SELECT toDateTime(toDateTime64('2299-12-31 23:59:59.999', 3, 'UTC'), 'UTC'); + +SELECT toDateTime(toDate32('1900-01-01', 'UTC'), 'UTC'); +SELECT toDateTime(toDate32('2299-12-31', 'UTC'), 'UTC'); + +SELECT toDateTime(toDate('2149-06-06', 'UTC'), 'UTC'); + +SELECT toDate(toDateTime64('1900-01-01 00:00:00.123', 3, 'UTC'), 'UTC'); +SELECT toDate(toDateTime64('2149-06-07 00:00:00.123', 3, 'UTC'), 'UTC'); +SELECT toDate(toDateTime64('2299-12-31 23:59:59.999', 3, 'UTC'), 'UTC'); + +SELECT toDate(toDate32('1900-01-01', 'UTC'), 'UTC'); +SELECT toDate(toDate32('2299-12-31', 'UTC'), 'UTC'); + + +SELECT 'No output on `throw`'; +SET date_time_overflow_mode = 'throw'; +SELECT toDateTime(toDateTime64('1900-01-01 00:00:00.123', 3, 'UTC'), 'UTC'); -- { serverError 707 } +SELECT toDateTime(toDateTime64('2299-12-31 23:59:59.999', 3, 'UTC'), 'UTC'); -- { serverError 707 } +SELECT toDateTime(toDate32('1900-01-01', 'UTC'), 'UTC'); -- { serverError 707 } +SELECT toDateTime(toDate32('2299-12-31', 'UTC'), 'UTC'); -- { serverError 707 } +SELECT toDateTime(toDate('2149-06-06', 'UTC'), 'UTC'); -- { serverError 707 } +SELECT toDate(toDateTime64('1900-01-01 00:00:00.123', 3, 'UTC'), 'UTC'); -- { serverError 707 } +SELECT toDate(toDateTime64('2299-12-31 23:59:59.999', 3, 'UTC'), 'UTC'); -- { serverError 707 } +SELECT toDate(toDate32('1900-01-01', 'UTC'), 'UTC'); -- { serverError 707 } +SELECT toDate(toDate32('2299-12-31', 'UTC'), 'UTC'); -- { serverError 707 } From 5b86e8c71457c76b81e54ada6685a7332e2a0592 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 23 Oct 2023 15:01:45 +0200 Subject: [PATCH 608/634] updated after review --- docs/en/operations/settings/settings.md | 12 ++ docs/ru/operations/settings/settings.md | 24 +-- src/Common/ErrorCodes.cpp | 1 - src/Core/SettingsEnums.cpp | 6 +- src/Core/SettingsEnums.h | 6 +- src/Functions/DateTimeTransforms.h | 34 +++-- src/Functions/FunctionsConversion.h | 139 +++++++++--------- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 4 +- .../02900_date_time_check_overflow.reference | 2 +- .../02900_date_time_check_overflow.sql | 70 ++++----- 10 files changed, 155 insertions(+), 143 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e3c8f729f3f..b165420103c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4144,6 +4144,18 @@ Possible values: Default value: `0`. +## date_time_overflow_mode {#date_time_overflow_mode} + +Defines the behavior when [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md) or integers are converted into Date, Date32, DateTime or DateTime64 but the value cannot be represented in the result type. + +Possible values: + +- `ignore` — Silently ignore overflows. The result is random. +- `throw` — Throw an exception in case of conversion overflow. +- `saturate` — Silently saturate the result. If the value is smaller than the smallest value that can be represented by the target type, the result is chosen as the smallest representable value. If the value is bigger than the largest value that can be represented by the target type, the result is chosen as the largest representable value. + +Default value: `ignore`. + ## optimize_move_to_prewhere {#optimize_move_to_prewhere} Enables or disables automatic [PREWHERE](../../sql-reference/statements/select/prewhere.md) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 087c3601f54..6693824ca97 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3838,6 +3838,18 @@ SELECT * FROM positional_arguments ORDER BY 2,3; Значение по умолчанию: `0`. +## date_time_overflow_mode {#date_time_overflow_mode} + +Задаёт поведение при преобразовании [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md), а также численных типов данных к Date, Date32, DateTime, DateTime64 в случае, если результат выходит за пределы диапазона значений необходимого типа. + +Возможные значения: + +- `ignore` — Молча игнорирует переполнение. В таком случае, результатом будет случайное значение. +- `throw` — Выкинуть исключение при переполнении. +- `saturate` — Молча округлить до ближайшего (то есть наибольшего или наименьшего) значения из диапазона значений результата. + +Значение по умолчанию: `ignore`. + ## optimize_move_to_prewhere {#optimize_move_to_prewhere} Включает или отключает автоматическую оптимизацию [PREWHERE](../../sql-reference/statements/select/prewhere.md) в запросах [SELECT](../../sql-reference/statements/select/index.md). @@ -4209,15 +4221,3 @@ SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_pars │ 1.7091 │ 15008753 │ └─────────────────────┴──────────────────────────┘ ``` - -## date_time_overflow_mode {#date_time_overflow_mode} - -Определяет поведение при переполнении во время преобразования типов [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md). - -Возможные значения: - -- `ignore` — Молча игнорирует переполнение. Полученный результат может быть округлём до ближайшего действительного значения области значений результирующего типа. А может и не быть, тогда результатом будет просто какое-то значение из всех возможных. -- `throw` — Выкинуть исключение при переполнении. -- `saturate` — Молча округлить до ближайшего (то есть наибольшего или наименьшего) значения из области значений результирующего типа. - -Значение по умолчанию: `ignore`. diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 336512d9be1..d2196b3c2b9 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -587,7 +587,6 @@ M(705, TABLE_NOT_EMPTY) \ M(706, LIBSSH_ERROR) \ M(707, ILLEGAL_JSON_OBJECT_FORMAT) \ - M(708, DATE_TIME_OVERFLOW) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index bb889cd953b..61cb769af15 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -191,8 +191,8 @@ IMPLEMENT_SETTING_ENUM(ExternalCommandStderrReaction, ErrorCodes::BAD_ARGUMENTS, {"throw", ExternalCommandStderrReaction::THROW}}) IMPLEMENT_SETTING_ENUM(DateTimeOverflowMode, ErrorCodes::BAD_ARGUMENTS, - {{"throw", DateTimeOverflowMode::THROW}, - {"ignore", DateTimeOverflowMode::IGNORE}, - {"saturate", DateTimeOverflowMode::SATURATE}}) + {{"throw", DateTimeOverflowMode::Throw}, + {"ignore", DateTimeOverflowMode::Ignore}, + {"saturate", DateTimeOverflowMode::Saturate}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index ab90aa740f5..e1b435f9434 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -244,9 +244,9 @@ DECLARE_SETTING_ENUM(ExternalCommandStderrReaction) enum class DateTimeOverflowMode { - THROW, - IGNORE, - SATURATE, + Throw, + Ignore, + Saturate, }; DECLARE_SETTING_ENUM(DateTimeOverflowMode) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index aca9311e18d..773fc05a5ef 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -23,7 +23,7 @@ namespace DB static constexpr auto microsecond_multiplier = 1000000; static constexpr auto millisecond_multiplier = 1000; -static constexpr DateTimeOverflowMode default_date_time_overflow_mode = DateTimeOverflowMode::IGNORE; +static constexpr DateTimeOverflowMode default_date_time_overflow_mode = DateTimeOverflowMode::Ignore; namespace ErrorCodes { @@ -31,7 +31,7 @@ namespace ErrorCodes extern const int DECIMAL_OVERFLOW; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int DATE_TIME_OVERFLOW; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; } /** Transformations. @@ -47,6 +47,12 @@ namespace ErrorCodes * factor-transformation F is "round to the nearest month" (2015-02-03 -> 2015-02-01). */ +constexpr time_t MAX_DT64_TIMESTAMP = 10413791999LL; // 1900-01-01 00:00:00 UTC +constexpr time_t MIN_DT64_TIMESTAMP = -2208988800LL; // 2299-12-31 23:59:59 UTC +constexpr time_t MAX_DT_TIMESTAMP = 0xFFFFFFFF; +constexpr time_t MAX_DATE_TIMESTAMP = 5662310399; // 2149-06-06 23:59:59 UTC +constexpr time_t MAX_DT_DAY_NUM = 49710; // 2106-02-07 + [[noreturn]] void throwDateIsNotSupported(const char * name); [[noreturn]] void throwDateTimeIsNotSupported(const char * name); [[noreturn]] void throwDate32IsNotSupported(const char * name); @@ -72,17 +78,17 @@ struct ToDateImpl static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::SATURATE) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Saturate) { if (t < 0) t = 0; - else if (t > 5662310399) // 2149-06-06 23:59:59 UTC - t = 5662310399; + else if (t > MAX_DATE_TIMESTAMP) + t = MAX_DATE_TIMESTAMP; } - else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { - if (unlikely(t < 0 || t > 5662310399)) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type Date", t); + if (t < 0 || t > MAX_DATE_TIMESTAMP) + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", t); } return static_cast(time_zone.toDayNum(t)); } @@ -92,17 +98,17 @@ struct ToDateImpl } static UInt16 execute(Int32 t, const DateLUTImpl &) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::SATURATE) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Saturate) { if (t < 0) - t = 0; + return UInt16(0); else if (t > DATE_LUT_MAX_DAY_NUM) - t = DATE_LUT_MAX_DAY_NUM; + return UInt16(DATE_LUT_MAX_DAY_NUM); } - else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { - if (unlikely(t < 0 || t > DATE_LUT_MAX_DAY_NUM)) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type Date", t); + if (t < 0 || t > DATE_LUT_MAX_DAY_NUM) + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", t); } return static_cast(t); } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 5311deb45fc..d72e3d7b0b9 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -90,13 +90,8 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; extern const int CANNOT_PARSE_BOOL; - extern const int DATE_TIME_OVERFLOW; - -constexpr time_t MAX_DT64_TIMESTAMP = 10413791999LL; // 1900-01-01 00:00:00 UTC -constexpr time_t MIN_DT64_TIMESTAMP = -2208988800LL; // 2299-12-31 23:59:59 UTC -constexpr time_t MAX_DT_TIMESTAMP = 0xFFFFFFFF; -constexpr time_t MAX_DATE_TIMESTAMP = 5662310399; -constexpr time_t MAX_DT_DAY_NUM = 49710; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; +} /** Type conversion functions. * toType - conversion in "natural way"; @@ -413,12 +408,12 @@ struct ToDateTimeImpl static UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { if (d > MAX_DT_DAY_NUM) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Day number {} is out of bounds of type DateTime", d); + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Day number {} is out of bounds of type DateTime", d); } - else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::SATURATE) + else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Saturate) { if (d > MAX_DT_DAY_NUM) d = MAX_DT_DAY_NUM; @@ -428,17 +423,17 @@ struct ToDateTimeImpl static UInt32 execute(Int32 d, const DateLUTImpl & time_zone) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::SATURATE) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Saturate) { if (d < 0) return 0; else if (d > MAX_DT_DAY_NUM) d = MAX_DT_DAY_NUM; } - else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { - if (unlikely(d < 0 || d > MAX_DT_DAY_NUM)) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type DateTime", d); + if (d < 0 || d > MAX_DT_DAY_NUM) + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type DateTime", d); } return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))); } @@ -450,16 +445,16 @@ struct ToDateTimeImpl static UInt32 execute(Int64 dt64, const DateLUTImpl & /*time_zone*/) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::IGNORE) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Ignore) return static_cast(dt64); else { - if (unlikely(dt64 < 0 || dt64 >= MAX_DT_TIMESTAMP)) + if (dt64 < 0 || dt64 >= MAX_DT_TIMESTAMP) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::SATURATE) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Saturate) return dt64 < 0 ? 0 : std::numeric_limits::max(); else - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type DateTime", dt64); + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type DateTime", dt64); } else return static_cast(dt64); @@ -484,10 +479,10 @@ struct ToDateTransform32Or64 static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { if (from > MAX_DT_TIMESTAMP) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type Date", from); + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); } /// if value is smaller (or equal) than maximum day value for Date, than treat it as day num, /// otherwise treat it as unix timestamp. This is a bit weird, but we leave this behavior. @@ -512,10 +507,10 @@ struct ToDateTransform32Or64Signed static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { // TODO: decide narrow or extended range based on FromType - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { if (from < 0 || from > MAX_DATE_TIMESTAMP) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type Date", from); + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); } else { @@ -537,8 +532,8 @@ struct ToDateTransform8Or16Signed { if (from < 0) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Value {} is out of bounds of type Date", from); + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); else return 0; } @@ -563,10 +558,10 @@ struct ToDate32Transform32Or64 return static_cast(from); else { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { if (from > MAX_DT64_TIMESTAMP) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type Date32", from); + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type Date32", from); } return time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DT64_TIMESTAMP))); } @@ -582,10 +577,10 @@ struct ToDate32Transform32Or64Signed { static const Int32 daynum_min_offset = -static_cast(time_zone.getDayNumOffsetEpoch()); - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { if (from < daynum_min_offset || from > MAX_DT64_TIMESTAMP) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type Date32", from); + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type Date32", from); } if (from < daynum_min_offset) @@ -690,10 +685,10 @@ struct ToDateTimeTransform64 static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { if (from > MAX_DT_TIMESTAMP) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type DateTime", from); + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); } return static_cast(std::min(time_t(from), time_t(MAX_DT_TIMESTAMP))); } @@ -708,8 +703,8 @@ struct ToDateTimeTransformSigned { if (from < 0) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type DateTime", from); + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); else return 0; } @@ -724,10 +719,10 @@ struct ToDateTimeTransform64Signed static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { if (from < 0 || from > MAX_DT_TIMESTAMP) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type DateTime", from); + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); } if (from < 0) @@ -781,10 +776,10 @@ struct ToDateTime64TransformUnsigned NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { if (from > MAX_DT64_TIMESTAMP) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type DateTime64", from); + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); else return DecimalUtils::decimalFromComponentsWithMultiplier(from, 0, scale_multiplier); } @@ -805,10 +800,10 @@ struct ToDateTime64TransformSigned NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { if (from < MIN_DT64_TIMESTAMP || from > MAX_DT64_TIMESTAMP) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type DateTime64", from); + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); } from = static_cast(std::max(from, MIN_DT64_TIMESTAMP)); from = static_cast(std::min(from, MAX_DT64_TIMESTAMP)); @@ -829,10 +824,10 @@ struct ToDateTime64TransformFloat NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::THROW) + if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) { if (from < MIN_DT64_TIMESTAMP || from > MAX_DT64_TIMESTAMP) - throw Exception(ErrorCodes::DATE_TIME_OVERFLOW, "Timestamp value {} is out of bounds of type DateTime64", from); + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); } from = std::max(from, static_cast(MIN_DT64_TIMESTAMP)); @@ -1295,8 +1290,8 @@ inline void convertFromTime(DataTypeDateTime::FieldType & x, t { if (unlikely(time < 0)) x = 0; - else if (unlikely(time > 0xFFFFFFFF)) - x = 0xFFFFFFFF; + else if (unlikely(time > MAX_DT_TIMESTAMP)) + x = MAX_DT_TIMESTAMP; else x = static_cast(time); } @@ -2303,14 +2298,14 @@ private: switch (date_time_overflow_mode) { - case DateTimeOverflowMode::THROW: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + case DateTimeOverflowMode::Throw: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); break; - case DateTimeOverflowMode::IGNORE: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + case DateTimeOverflowMode::Ignore: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); break; - case DateTimeOverflowMode::SATURATE: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + case DateTimeOverflowMode::Saturate: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); break; } @@ -2320,14 +2315,14 @@ private: const auto * dt64 = assert_cast(arguments[0].type.get()); switch (date_time_overflow_mode) { - case DateTimeOverflowMode::THROW: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); + case DateTimeOverflowMode::Throw: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); break; - case DateTimeOverflowMode::IGNORE: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); + case DateTimeOverflowMode::Ignore: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); break; - case DateTimeOverflowMode::SATURATE: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); + case DateTimeOverflowMode::Saturate: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); break; } } @@ -2357,9 +2352,9 @@ private: { switch (date_time_overflow_mode) { - GENERATE_OVERFLOW_MODE_CASE(THROW) - GENERATE_OVERFLOW_MODE_CASE(IGNORE) - GENERATE_OVERFLOW_MODE_CASE(SATURATE) + GENERATE_OVERFLOW_MODE_CASE(Throw) + GENERATE_OVERFLOW_MODE_CASE(Ignore) + GENERATE_OVERFLOW_MODE_CASE(Saturate) } } } @@ -2368,9 +2363,9 @@ private: { switch (date_time_overflow_mode) { - GENERATE_OVERFLOW_MODE_CASE(THROW) - GENERATE_OVERFLOW_MODE_CASE(IGNORE) - GENERATE_OVERFLOW_MODE_CASE(SATURATE) + GENERATE_OVERFLOW_MODE_CASE(Throw) + GENERATE_OVERFLOW_MODE_CASE(Ignore) + GENERATE_OVERFLOW_MODE_CASE(Saturate) } } #undef GENERATE_OVERFLOW_MODE_CASE @@ -3330,18 +3325,18 @@ private: { switch (date_time_overflow_mode) { - GENERATE_OVERFLOW_MODE_CASE(THROW, AccurateConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(IGNORE, AccurateConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(SATURATE, AccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Throw, AccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Ignore, AccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Saturate, AccurateConvertStrategyAdditions) } } else { switch (date_time_overflow_mode) { - GENERATE_OVERFLOW_MODE_CASE(THROW, AccurateOrNullConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(IGNORE, AccurateOrNullConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(SATURATE, AccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Throw, AccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Ignore, AccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Saturate, AccurateOrNullConvertStrategyAdditions) } } #undef GENERATE_OVERFLOW_MODE_CASE @@ -3360,18 +3355,18 @@ arguments, result_type, input_rows_count); \ { switch (date_time_overflow_mode) { - GENERATE_OVERFLOW_MODE_CASE(THROW, DateTimeAccurateConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(IGNORE, DateTimeAccurateConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(SATURATE, DateTimeAccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Throw, DateTimeAccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Ignore, DateTimeAccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Saturate, DateTimeAccurateConvertStrategyAdditions) } } else { switch (date_time_overflow_mode) { - GENERATE_OVERFLOW_MODE_CASE(THROW, DateTimeAccurateOrNullConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(IGNORE, DateTimeAccurateOrNullConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(SATURATE, DateTimeAccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Throw, DateTimeAccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Ignore, DateTimeAccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Saturate, DateTimeAccurateOrNullConvertStrategyAdditions) } } #undef GENERATE_OVERFLOW_MODE_CASE diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index e50f89113a8..2e960cf2eff 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -254,7 +254,7 @@ static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr(chunk.Value(value_i)); if (days_num > DATE_LUT_MAX_EXTEND_DAY_NUM || days_num < -DAYNUM_OFFSET_EPOCH) { - if (likely(date_time_overflow_mode == DateTimeOverflowMode::SATURATE)) + if (date_time_overflow_mode == DateTimeOverflowMode::Saturate) days_num = (days_num < -DAYNUM_OFFSET_EPOCH) ? -DAYNUM_OFFSET_EPOCH : DATE_LUT_MAX_EXTEND_DAY_NUM; else { diff --git a/tests/queries/0_stateless/02900_date_time_check_overflow.reference b/tests/queries/0_stateless/02900_date_time_check_overflow.reference index 5f87903e367..26bbfc8752f 100644 --- a/tests/queries/0_stateless/02900_date_time_check_overflow.reference +++ b/tests/queries/0_stateless/02900_date_time_check_overflow.reference @@ -9,6 +9,7 @@ ignore 2120-07-26 2079-06-07 2120-07-26 +No output on `throw` saturate 1970-01-01 00:00:00 2106-02-07 06:28:15 @@ -20,4 +21,3 @@ saturate 2149-06-06 1970-01-01 2149-06-06 -No output on `throw` diff --git a/tests/queries/0_stateless/02900_date_time_check_overflow.sql b/tests/queries/0_stateless/02900_date_time_check_overflow.sql index 3d9c567208f..f466cf5a018 100644 --- a/tests/queries/0_stateless/02900_date_time_check_overflow.sql +++ b/tests/queries/0_stateless/02900_date_time_check_overflow.sql @@ -1,50 +1,50 @@ --- I hope we will get rid of "ignore" option one day :) --- It gives unpredictable result, but it is faster than others. +SET session_timezone = 'UTC'; + SELECT 'ignore'; SET date_time_overflow_mode = 'ignore'; -SELECT toDateTime(toDateTime64('1900-01-01 00:00:00.123', 3, 'UTC'), 'UTC'); -SELECT toDateTime(toDateTime64('2299-12-31 23:59:59.999', 3, 'UTC'), 'UTC'); +SELECT toDateTime(toDateTime64('1900-01-01 00:00:00.123', 3)); +SELECT toDateTime(toDateTime64('2299-12-31 23:59:59.999', 3)); -SELECT toDateTime(toDate32('1900-01-01', 'UTC'), 'UTC'); -SELECT toDateTime(toDate32('2299-12-31', 'UTC'), 'UTC'); +SELECT toDateTime(toDate32('1900-01-01')); +SELECT toDateTime(toDate32('2299-12-31')); -SELECT toDateTime(toDate('2149-06-06', 'UTC'), 'UTC'); +SELECT toDateTime(toDate('2149-06-06')); -SELECT toDate(toDateTime64('1900-01-01 00:00:00.123', 3, 'UTC'), 'UTC'); -SELECT toDate(toDateTime64('2149-06-07 00:00:00.123', 3, 'UTC'), 'UTC'); -SELECT toDate(toDateTime64('2299-12-31 23:59:59.999', 3, 'UTC'), 'UTC'); +SELECT toDate(toDateTime64('1900-01-01 00:00:00.123', 3)); +SELECT toDate(toDateTime64('2149-06-07 00:00:00.123', 3)); +SELECT toDate(toDateTime64('2299-12-31 23:59:59.999', 3)); -SELECT toDate(toDate32('1900-01-01', 'UTC'), 'UTC'); -SELECT toDate(toDate32('2299-12-31', 'UTC'), 'UTC'); +SELECT toDate(toDate32('1900-01-01')); +SELECT toDate(toDate32('2299-12-31')); + + +SELECT 'No output on `throw`'; +SET date_time_overflow_mode = 'throw'; +SELECT toDateTime(toDateTime64('1900-01-01 00:00:00.123', 3)); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } +SELECT toDateTime(toDateTime64('2299-12-31 23:59:59.999', 3)); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } +SELECT toDateTime(toDate32('1900-01-01')); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } +SELECT toDateTime(toDate32('2299-12-31')); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } +SELECT toDateTime(toDate('2149-06-06')); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } +SELECT toDate(toDateTime64('1900-01-01 00:00:00.123', 3)); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } +SELECT toDate(toDateTime64('2299-12-31 23:59:59.999', 3)); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } +SELECT toDate(toDate32('1900-01-01')); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } +SELECT toDate(toDate32('2299-12-31')); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } SELECT 'saturate'; SET date_time_overflow_mode = 'saturate'; -SELECT toDateTime(toDateTime64('1900-01-01 00:00:00.123', 3, 'UTC'), 'UTC'); -SELECT toDateTime(toDateTime64('2299-12-31 23:59:59.999', 3, 'UTC'), 'UTC'); +SELECT toDateTime(toDateTime64('1900-01-01 00:00:00.123', 3)); +SELECT toDateTime(toDateTime64('2299-12-31 23:59:59.999', 3)); -SELECT toDateTime(toDate32('1900-01-01', 'UTC'), 'UTC'); -SELECT toDateTime(toDate32('2299-12-31', 'UTC'), 'UTC'); +SELECT toDateTime(toDate32('1900-01-01')); +SELECT toDateTime(toDate32('2299-12-31')); -SELECT toDateTime(toDate('2149-06-06', 'UTC'), 'UTC'); +SELECT toDateTime(toDate('2149-06-06')); -SELECT toDate(toDateTime64('1900-01-01 00:00:00.123', 3, 'UTC'), 'UTC'); -SELECT toDate(toDateTime64('2149-06-07 00:00:00.123', 3, 'UTC'), 'UTC'); -SELECT toDate(toDateTime64('2299-12-31 23:59:59.999', 3, 'UTC'), 'UTC'); +SELECT toDate(toDateTime64('1900-01-01 00:00:00.123', 3)); +SELECT toDate(toDateTime64('2149-06-07 00:00:00.123', 3)); +SELECT toDate(toDateTime64('2299-12-31 23:59:59.999', 3)); -SELECT toDate(toDate32('1900-01-01', 'UTC'), 'UTC'); -SELECT toDate(toDate32('2299-12-31', 'UTC'), 'UTC'); - - -SELECT 'No output on `throw`'; -SET date_time_overflow_mode = 'throw'; -SELECT toDateTime(toDateTime64('1900-01-01 00:00:00.123', 3, 'UTC'), 'UTC'); -- { serverError 707 } -SELECT toDateTime(toDateTime64('2299-12-31 23:59:59.999', 3, 'UTC'), 'UTC'); -- { serverError 707 } -SELECT toDateTime(toDate32('1900-01-01', 'UTC'), 'UTC'); -- { serverError 707 } -SELECT toDateTime(toDate32('2299-12-31', 'UTC'), 'UTC'); -- { serverError 707 } -SELECT toDateTime(toDate('2149-06-06', 'UTC'), 'UTC'); -- { serverError 707 } -SELECT toDate(toDateTime64('1900-01-01 00:00:00.123', 3, 'UTC'), 'UTC'); -- { serverError 707 } -SELECT toDate(toDateTime64('2299-12-31 23:59:59.999', 3, 'UTC'), 'UTC'); -- { serverError 707 } -SELECT toDate(toDate32('1900-01-01', 'UTC'), 'UTC'); -- { serverError 707 } -SELECT toDate(toDate32('2299-12-31', 'UTC'), 'UTC'); -- { serverError 707 } +SELECT toDate(toDate32('1900-01-01')); +SELECT toDate(toDate32('2299-12-31')); From 0766c73aab1adc04c4a0f90fcbcd0954b741f4b3 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 24 Oct 2023 01:49:40 +0200 Subject: [PATCH 609/634] Rename date_time_overflow_mode -> date_time_overflow_behavior, moved it to format settings --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- src/Core/Settings.h | 3 +- src/Core/SettingsEnums.cpp | 8 +- src/Core/SettingsEnums.h | 9 +- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 9 + src/Functions/DateTimeTransforms.h | 23 +- src/Functions/FunctionsConversion.h | 451 +++++++++--------- .../Formats/Impl/ArrowBlockInputFormat.cpp | 1 + .../Formats/Impl/ArrowColumnToCHColumn.cpp | 56 +-- .../Formats/Impl/ArrowColumnToCHColumn.h | 4 +- .../Formats/Impl/ORCBlockInputFormat.cpp | 1 + .../Formats/Impl/ParquetBlockInputFormat.cpp | 1 + .../DataLakes/DeltaLakeMetadataParser.cpp | 1 + .../02900_date_time_check_overflow.sql | 6 +- 16 files changed, 294 insertions(+), 284 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b165420103c..19a3693a43b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4144,7 +4144,7 @@ Possible values: Default value: `0`. -## date_time_overflow_mode {#date_time_overflow_mode} +## date_time_overflow_behavior {#date_time_overflow_behavior} Defines the behavior when [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md) or integers are converted into Date, Date32, DateTime or DateTime64 but the value cannot be represented in the result type. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 6693824ca97..8757cf2f5e7 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3838,7 +3838,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; Значение по умолчанию: `0`. -## date_time_overflow_mode {#date_time_overflow_mode} +## date_time_overflow_behavior {#date_time_overflow_behavior} Задаёт поведение при преобразовании [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md), а также численных типов данных к Date, Date32, DateTime, DateTime64 в случае, если результат выходит за пределы диапазона значений необходимого типа. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 96a7acb9d51..ddb531c40cd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -810,7 +810,6 @@ class IColumn; M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \ M(Bool, print_pretty_type_names, false, "Print pretty type names in DESCRIBE query and toTypeName() function", 0) \ M(Bool, create_table_empty_primary_key_by_default, false, "Allow to create *MergeTree tables with empty primary key when ORDER BY and PRIMARY KEY not specified", 0) \ - M(DateTimeOverflowMode, date_time_overflow_mode, DateTimeOverflowMode::IGNORE, "Sets overflow mode for Date, Date32, DateTime, DateTime64 types.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. @@ -1077,6 +1076,8 @@ class IColumn; \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading dictionary source in several threads. It's supported only by dictionaries with local CLICKHOUSE source.", 0) \ M(Bool, precise_float_parsing, false, "Prefer more precise (but slower) float parsing algorithm", 0) \ + M(DateTimeOverflowBehavior, date_time_overflow_behavior, "ignore", "Overflow mode for Date, Date32, DateTime, DateTime64 types. Possible values: 'ignore', 'throw', 'saturate'.", 0) \ + // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 61cb769af15..9b84d092fd0 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -190,9 +190,9 @@ IMPLEMENT_SETTING_ENUM(ExternalCommandStderrReaction, ErrorCodes::BAD_ARGUMENTS, {"log_last", ExternalCommandStderrReaction::LOG_LAST}, {"throw", ExternalCommandStderrReaction::THROW}}) -IMPLEMENT_SETTING_ENUM(DateTimeOverflowMode, ErrorCodes::BAD_ARGUMENTS, - {{"throw", DateTimeOverflowMode::Throw}, - {"ignore", DateTimeOverflowMode::Ignore}, - {"saturate", DateTimeOverflowMode::Saturate}}) +IMPLEMENT_SETTING_ENUM(DateTimeOverflowBehavior, ErrorCodes::BAD_ARGUMENTS, + {{"throw", FormatSettings::DateTimeOverflowBehavior::Throw}, + {"ignore", FormatSettings::DateTimeOverflowBehavior::Ignore}, + {"saturate", FormatSettings::DateTimeOverflowBehavior::Saturate}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index e1b435f9434..bf32881cca8 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -242,13 +242,6 @@ DECLARE_SETTING_ENUM(S3QueueAction) DECLARE_SETTING_ENUM(ExternalCommandStderrReaction) -enum class DateTimeOverflowMode -{ - Throw, - Ignore, - Saturate, -}; - -DECLARE_SETTING_ENUM(DateTimeOverflowMode) +DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOverflowBehavior, FormatSettings::DateTimeOverflowBehavior) } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 920a3cb1bb5..d51ea9ad2d0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -227,6 +227,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.native.allow_types_conversion = settings.input_format_native_allow_types_conversion; format_settings.max_parser_depth = context->getSettingsRef().max_parser_depth; format_settings.client_protocol_version = context->getClientProtocolVersion(); + format_settings.date_time_overflow_behavior = settings.date_time_overflow_behavior; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (format_settings.schema.is_server) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 77fd8e1fcbd..382f8b7173a 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -88,6 +88,15 @@ struct FormatSettings IntervalOutputFormat output_format = IntervalOutputFormat::Numeric; } interval; + enum class DateTimeOverflowBehavior + { + Ignore, + Throw, + Saturate + }; + + DateTimeOverflowBehavior date_time_overflow_behavior = DateTimeOverflowBehavior::Ignore; + bool input_format_ipv4_default_on_conversion_error = false; bool input_format_ipv6_default_on_conversion_error = false; diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 773fc05a5ef..d371e19e6ee 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -23,7 +23,7 @@ namespace DB static constexpr auto microsecond_multiplier = 1000000; static constexpr auto millisecond_multiplier = 1000; -static constexpr DateTimeOverflowMode default_date_time_overflow_mode = DateTimeOverflowMode::Ignore; +static constexpr FormatSettings::DateTimeOverflowBehavior default_date_time_overflow_behavior = FormatSettings::DateTimeOverflowBehavior::Ignore; namespace ErrorCodes { @@ -47,11 +47,11 @@ namespace ErrorCodes * factor-transformation F is "round to the nearest month" (2015-02-03 -> 2015-02-01). */ -constexpr time_t MAX_DT64_TIMESTAMP = 10413791999LL; // 1900-01-01 00:00:00 UTC -constexpr time_t MIN_DT64_TIMESTAMP = -2208988800LL; // 2299-12-31 23:59:59 UTC -constexpr time_t MAX_DT_TIMESTAMP = 0xFFFFFFFF; +constexpr time_t MAX_DATETIME64_TIMESTAMP = 10413791999LL; // 1900-01-01 00:00:00 UTC +constexpr time_t MIN_DATETIME64_TIMESTAMP = -2208988800LL; // 2299-12-31 23:59:59 UTC +constexpr time_t MAX_DATETIME_TIMESTAMP = 0xFFFFFFFF; constexpr time_t MAX_DATE_TIMESTAMP = 5662310399; // 2149-06-06 23:59:59 UTC -constexpr time_t MAX_DT_DAY_NUM = 49710; // 2106-02-07 +constexpr time_t MAX_DATETIME_DAY_NUM = 49710; // 2106-02-07 [[noreturn]] void throwDateIsNotSupported(const char * name); [[noreturn]] void throwDateTimeIsNotSupported(const char * name); @@ -66,7 +66,7 @@ struct ZeroTransform static UInt16 execute(UInt16, const DateLUTImpl &) { return 0; } }; -template +template struct ToDateImpl { static constexpr auto name = "toDate"; @@ -78,16 +78,16 @@ struct ToDateImpl static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Saturate) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) { if (t < 0) t = 0; else if (t > MAX_DATE_TIMESTAMP) t = MAX_DATE_TIMESTAMP; } - else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + else if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (t < 0 || t > MAX_DATE_TIMESTAMP) + if (t < 0 || t > MAX_DATE_TIMESTAMP) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", t); } return static_cast(time_zone.toDayNum(t)); @@ -98,14 +98,14 @@ struct ToDateImpl } static UInt16 execute(Int32 t, const DateLUTImpl &) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Saturate) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) { if (t < 0) return UInt16(0); else if (t > DATE_LUT_MAX_DAY_NUM) return UInt16(DATE_LUT_MAX_DAY_NUM); } - else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + else if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { if (t < 0 || t > DATE_LUT_MAX_DAY_NUM) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", t); @@ -124,7 +124,6 @@ struct ToDateImpl using FactorTransform = ZeroTransform; }; -template struct ToDate32Impl { static constexpr auto name = "toDate32"; diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index d72e3d7b0b9..66d891230fb 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -134,7 +134,8 @@ struct ConvertReturnZeroOnErrorTag {}; * (Date is represented internally as number of days from some day; DateTime - as unix timestamp) */ template + typename SpecialTag = ConvertDefaultBehaviorTag, + FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior> struct ConvertImpl { using FromFieldType = typename FromDataType::FieldType; @@ -389,50 +390,50 @@ struct ConvertImpl /** Conversion of DateTime to Date: throw off time component. */ -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; /** Conversion of DateTime to Date32: throw off time component. */ -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl {}; /** Conversion of Date to DateTime: adding 00:00:00 time component. */ -template +template struct ToDateTimeImpl { static constexpr auto name = "toDateTime"; static UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (d > MAX_DT_DAY_NUM) + if (d > MAX_DATETIME_DAY_NUM) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Day number {} is out of bounds of type DateTime", d); } - else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Saturate) + else if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) { - if (d > MAX_DT_DAY_NUM) - d = MAX_DT_DAY_NUM; + if (d > MAX_DATETIME_DAY_NUM) + d = MAX_DATETIME_DAY_NUM; } return static_cast(time_zone.fromDayNum(DayNum(d))); } static UInt32 execute(Int32 d, const DateLUTImpl & time_zone) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Saturate) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) { if (d < 0) return 0; - else if (d > MAX_DT_DAY_NUM) - d = MAX_DT_DAY_NUM; + else if (d > MAX_DATETIME_DAY_NUM) + d = MAX_DATETIME_DAY_NUM; } - else if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + else if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (d < 0 || d > MAX_DT_DAY_NUM) + if (d < 0 || d > MAX_DATETIME_DAY_NUM) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type DateTime", d); } return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))); @@ -445,13 +446,13 @@ struct ToDateTimeImpl static UInt32 execute(Int64 dt64, const DateLUTImpl & /*time_zone*/) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Ignore) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Ignore) return static_cast(dt64); else { - if (dt64 < 0 || dt64 >= MAX_DT_TIMESTAMP) + if (dt64 < 0 || dt64 >= MAX_DATETIME_TIMESTAMP) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Saturate) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) return dt64 < 0 ? 0 : std::numeric_limits::max(); else throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type DateTime", dt64); @@ -462,26 +463,26 @@ struct ToDateTimeImpl } }; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; /// Implementation of toDate function. -template +template struct ToDateTransform32Or64 { static constexpr auto name = "toDate"; static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from > MAX_DT_TIMESTAMP) + if (from > MAX_DATETIME_TIMESTAMP) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); } /// if value is smaller (or equal) than maximum day value for Date, than treat it as day num, @@ -489,17 +490,17 @@ struct ToDateTransform32Or64 if (from <= DATE_LUT_MAX_DAY_NUM) return from; else - return time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DT_TIMESTAMP))); + return time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DATETIME_TIMESTAMP))); } }; /** Conversion of Date32 to Date. */ -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template +template struct ToDateTransform32Or64Signed { static constexpr auto name = "toDate"; @@ -507,7 +508,7 @@ struct ToDateTransform32Or64Signed static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { // TODO: decide narrow or extended range based on FromType - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { if (from < 0 || from > MAX_DATE_TIMESTAMP) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); @@ -523,7 +524,7 @@ struct ToDateTransform32Or64Signed } }; -template +template struct ToDateTransform8Or16Signed { static constexpr auto name = "toDate"; @@ -532,7 +533,7 @@ struct ToDateTransform8Or16Signed { if (from < 0) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); else return 0; @@ -541,13 +542,13 @@ struct ToDateTransform8Or16Signed } }; -template -struct ConvertImpl - : DateTimeTransformImpl>, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; /// Implementation of toDate32 function. -template +template struct ToDate32Transform32Or64 { static constexpr auto name = "toDate32"; @@ -558,17 +559,17 @@ struct ToDate32Transform32Or64 return static_cast(from); else { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from > MAX_DT64_TIMESTAMP) + if (from > MAX_DATETIME64_TIMESTAMP) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type Date32", from); } - return time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DT64_TIMESTAMP))); + return time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DATETIME64_TIMESTAMP))); } } }; -template +template struct ToDate32Transform32Or64Signed { static constexpr auto name = "toDate32"; @@ -577,9 +578,9 @@ struct ToDate32Transform32Or64Signed { static const Int32 daynum_min_offset = -static_cast(time_zone.getDayNumOffsetEpoch()); - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from < daynum_min_offset || from > MAX_DT64_TIMESTAMP) + if (from < daynum_min_offset || from > MAX_DATETIME64_TIMESTAMP) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type Date32", from); } @@ -588,7 +589,7 @@ struct ToDate32Transform32Or64Signed return (from < DATE_LUT_MAX_EXTEND_DAY_NUM) ? static_cast(from) - : time_zone.toDayNum(std::min(time_t(Int64(from)), time_t(MAX_DT64_TIMESTAMP))); + : time_zone.toDayNum(std::min(time_t(Int64(from)), time_t(MAX_DATETIME64_TIMESTAMP))); } }; @@ -612,89 +613,89 @@ struct ToDate32Transform8Or16Signed * when user write toDate(UInt32), expecting conversion of unix timestamp to Date. * (otherwise such usage would be frequent mistake). */ -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl +template +struct ConvertImpl : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl +template +struct ConvertImpl : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template +template struct ToDateTimeTransform64 { static constexpr auto name = "toDateTime"; static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from > MAX_DT_TIMESTAMP) + if (from > MAX_DATETIME_TIMESTAMP) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); } - return static_cast(std::min(time_t(from), time_t(MAX_DT_TIMESTAMP))); + return static_cast(std::min(time_t(from), time_t(MAX_DATETIME_TIMESTAMP))); } }; -template +template struct ToDateTimeTransformSigned { static constexpr auto name = "toDateTime"; @@ -703,7 +704,7 @@ struct ToDateTimeTransformSigned { if (from < 0) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); else return 0; @@ -712,58 +713,58 @@ struct ToDateTimeTransformSigned } }; -template +template struct ToDateTimeTransform64Signed { static constexpr auto name = "toDateTime"; static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from < 0 || from > MAX_DT_TIMESTAMP) + if (from < 0 || from > MAX_DATETIME_TIMESTAMP) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); } if (from < 0) return 0; - return static_cast(std::min(time_t(from), time_t(MAX_DT_TIMESTAMP))); + return static_cast(std::min(time_t(from), time_t(MAX_DATETIME_TIMESTAMP))); } }; /// Special case of converting Int8, Int16, Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to DateTime. -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; /** Conversion of numeric to DateTime64 */ -template +template struct ToDateTime64TransformUnsigned { static constexpr auto name = "toDateTime64"; @@ -776,18 +777,18 @@ struct ToDateTime64TransformUnsigned NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from > MAX_DT64_TIMESTAMP) + if (from > MAX_DATETIME64_TIMESTAMP) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); else return DecimalUtils::decimalFromComponentsWithMultiplier(from, 0, scale_multiplier); } else - return DecimalUtils::decimalFromComponentsWithMultiplier(std::min(from, MAX_DT64_TIMESTAMP), 0, scale_multiplier); + return DecimalUtils::decimalFromComponentsWithMultiplier(std::min(from, MAX_DATETIME64_TIMESTAMP), 0, scale_multiplier); } }; -template +template struct ToDateTime64TransformSigned { static constexpr auto name = "toDateTime64"; @@ -800,18 +801,18 @@ struct ToDateTime64TransformSigned NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from < MIN_DT64_TIMESTAMP || from > MAX_DT64_TIMESTAMP) + if (from < MIN_DATETIME64_TIMESTAMP || from > MAX_DATETIME64_TIMESTAMP) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); } - from = static_cast(std::max(from, MIN_DT64_TIMESTAMP)); - from = static_cast(std::min(from, MAX_DT64_TIMESTAMP)); + from = static_cast(std::max(from, MIN_DATETIME64_TIMESTAMP)); + from = static_cast(std::min(from, MAX_DATETIME64_TIMESTAMP)); return DecimalUtils::decimalFromComponentsWithMultiplier(from, 0, scale_multiplier); } }; -template +template struct ToDateTime64TransformFloat { static constexpr auto name = "toDateTime64"; @@ -824,45 +825,45 @@ struct ToDateTime64TransformFloat NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const { - if constexpr (date_time_overflow_mode == DateTimeOverflowMode::Throw) + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from < MIN_DT64_TIMESTAMP || from > MAX_DT64_TIMESTAMP) + if (from < MIN_DATETIME64_TIMESTAMP || from > MAX_DATETIME64_TIMESTAMP) throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); } - from = std::max(from, static_cast(MIN_DT64_TIMESTAMP)); - from = std::min(from, static_cast(MAX_DT64_TIMESTAMP)); + from = std::max(from, static_cast(MIN_DATETIME64_TIMESTAMP)); + from = std::min(from, static_cast(MAX_DATETIME64_TIMESTAMP)); return convertToDecimal(from, scale); } }; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl, false> {}; /** Conversion of DateTime64 to Date or DateTime: discards fractional part. @@ -887,13 +888,13 @@ struct FromDateTime64Transform /** Conversion of DateTime64 to Date or DateTime: discards fractional part. */ -template -struct ConvertImpl - : DateTimeTransformImpl>, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl>, false> {}; -template -struct ConvertImpl - : DateTimeTransformImpl>, false> {}; +template +struct ConvertImpl + : DateTimeTransformImpl>, false> {}; struct ToDateTime64Transform { @@ -925,16 +926,16 @@ struct ToDateTime64Transform /** Conversion of Date or DateTime to DateTime64: add zero sub-second part. */ -template -struct ConvertImpl +template +struct ConvertImpl : DateTimeTransformImpl {}; -template -struct ConvertImpl +template +struct ConvertImpl : DateTimeTransformImpl {}; -template -struct ConvertImpl +template +struct ConvertImpl : DateTimeTransformImpl {}; @@ -1290,8 +1291,8 @@ inline void convertFromTime(DataTypeDateTime::FieldType & x, t { if (unlikely(time < 0)) x = 0; - else if (unlikely(time > MAX_DT_TIMESTAMP)) - x = MAX_DT_TIMESTAMP; + else if (unlikely(time > MAX_DATETIME_TIMESTAMP)) + x = MAX_DATETIME_TIMESTAMP; else x = static_cast(time); } @@ -1789,29 +1790,29 @@ struct ConvertThroughParsing }; -template +template requires (!std::is_same_v) -struct ConvertImpl +struct ConvertImpl : ConvertThroughParsing {}; -template +template requires (!std::is_same_v) -struct ConvertImpl +struct ConvertImpl : ConvertThroughParsing {}; -template +template requires (!std::is_same_v) -struct ConvertImpl +struct ConvertImpl : ConvertThroughParsing {}; -template +template requires (!std::is_same_v) -struct ConvertImpl +struct ConvertImpl : ConvertThroughParsing {}; -template +template requires (is_any_of && is_any_of) -struct ConvertImpl +struct ConvertImpl : ConvertThroughParsing {}; /// Generic conversion of any type from String. Used for complex types: Array and Tuple or types with custom serialization. @@ -2268,10 +2269,10 @@ private: const DataTypePtr from_type = removeNullable(arguments[0].type); ColumnPtr result_column; - [[maybe_unused]] DateTimeOverflowMode date_time_overflow_mode = default_date_time_overflow_mode; + [[maybe_unused]] FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior; if (context) - date_time_overflow_mode = context->getSettingsRef().date_time_overflow_mode.value; + date_time_overflow_behavior = context->getSettingsRef().date_time_overflow_behavior.value; auto call = [&](const auto & types, const auto & tag) -> bool { @@ -2296,16 +2297,16 @@ private: const ColumnWithTypeAndName & scale_column = arguments[1]; UInt32 scale = extractToDecimalScale(scale_column); - switch (date_time_overflow_mode) + switch (date_time_overflow_behavior) { - case DateTimeOverflowMode::Throw: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + case FormatSettings::DateTimeOverflowBehavior::Throw: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); break; - case DateTimeOverflowMode::Ignore: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + case FormatSettings::DateTimeOverflowBehavior::Ignore: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); break; - case DateTimeOverflowMode::Saturate: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + case FormatSettings::DateTimeOverflowBehavior::Saturate: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); break; } @@ -2313,22 +2314,22 @@ private: else if constexpr (IsDataTypeDateOrDateTime && std::is_same_v) { const auto * dt64 = assert_cast(arguments[0].type.get()); - switch (date_time_overflow_mode) + switch (date_time_overflow_behavior) { - case DateTimeOverflowMode::Throw: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); + case FormatSettings::DateTimeOverflowBehavior::Throw: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); break; - case DateTimeOverflowMode::Ignore: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); + case FormatSettings::DateTimeOverflowBehavior::Ignore: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); break; - case DateTimeOverflowMode::Saturate: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); + case FormatSettings::DateTimeOverflowBehavior::Saturate: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); break; } } #define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE) \ - case DateTimeOverflowMode::OVERFLOW_MODE: \ - result_column = ConvertImpl::execute( \ + case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \ + result_column = ConvertImpl::execute( \ arguments, result_type, input_rows_count); \ break; @@ -2350,7 +2351,7 @@ private: } else { - switch (date_time_overflow_mode) + switch (date_time_overflow_behavior) { GENERATE_OVERFLOW_MODE_CASE(Throw) GENERATE_OVERFLOW_MODE_CASE(Ignore) @@ -2361,7 +2362,7 @@ private: else if constexpr ((IsDataTypeNumber || IsDataTypeDateOrDateTime) && IsDataTypeDateOrDateTime) { - switch (date_time_overflow_mode) + switch (date_time_overflow_behavior) { GENERATE_OVERFLOW_MODE_CASE(Throw) GENERATE_OVERFLOW_MODE_CASE(Ignore) @@ -2920,7 +2921,7 @@ using FunctionToDecimal64 = FunctionConvert, NameToDe using FunctionToDecimal128 = FunctionConvert, NameToDecimal128, UnknownMonotonicity>; using FunctionToDecimal256 = FunctionConvert, NameToDecimal256, UnknownMonotonicity>; -template struct FunctionTo; +template struct FunctionTo; template <> struct FunctionTo { using Type = FunctionToUInt8; }; template <> struct FunctionTo { using Type = FunctionToUInt16; }; @@ -2937,17 +2938,17 @@ template <> struct FunctionTo { using Type = FunctionToInt256; } template <> struct FunctionTo { using Type = FunctionToFloat32; }; template <> struct FunctionTo { using Type = FunctionToFloat64; }; -template -struct FunctionTo { using Type = FunctionToDate; }; +template +struct FunctionTo { using Type = FunctionToDate; }; -template -struct FunctionTo { using Type = FunctionToDate32; }; +template +struct FunctionTo { using Type = FunctionToDate32; }; -template -struct FunctionTo { using Type = FunctionToDateTime; }; +template +struct FunctionTo { using Type = FunctionToDateTime; }; -template -struct FunctionTo { using Type = FunctionToDateTime64; }; +template +struct FunctionTo { using Type = FunctionToDateTime64; }; template <> struct FunctionTo { using Type = FunctionToUUID; }; template <> struct FunctionTo { using Type = FunctionToIPv4; }; @@ -3283,9 +3284,9 @@ private: bool can_apply_accurate_cast = (cast_type == CastType::accurate || cast_type == CastType::accurateOrNull) && (which.isInt() || which.isUInt() || which.isFloat()); - DateTimeOverflowMode date_time_overflow_mode = default_date_time_overflow_mode; + FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior; if (context) - date_time_overflow_mode = context->getSettingsRef().date_time_overflow_mode; + date_time_overflow_behavior = context->getSettingsRef().date_time_overflow_behavior; if (requested_result_is_nullable && checkAndGetDataType(from_type.get())) { @@ -3303,7 +3304,7 @@ private: auto wrapper_cast_type = cast_type; - return [wrapper_cast_type, from_type_index, to_type, date_time_overflow_mode] + return [wrapper_cast_type, from_type_index, to_type, date_time_overflow_behavior] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *column_nullable, size_t input_rows_count) { ColumnPtr result_column; @@ -3317,13 +3318,13 @@ private: if constexpr (IsDataTypeNumber) { #define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE, ADDITIONS) \ - case DateTimeOverflowMode::OVERFLOW_MODE: \ - result_column = ConvertImpl::execute( \ + case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \ + result_column = ConvertImpl::execute( \ arguments, result_type, input_rows_count, ADDITIONS()); \ break; if (wrapper_cast_type == CastType::accurate) { - switch (date_time_overflow_mode) + switch (date_time_overflow_behavior) { GENERATE_OVERFLOW_MODE_CASE(Throw, AccurateConvertStrategyAdditions) GENERATE_OVERFLOW_MODE_CASE(Ignore, AccurateConvertStrategyAdditions) @@ -3332,7 +3333,7 @@ private: } else { - switch (date_time_overflow_mode) + switch (date_time_overflow_behavior) { GENERATE_OVERFLOW_MODE_CASE(Throw, AccurateOrNullConvertStrategyAdditions) GENERATE_OVERFLOW_MODE_CASE(Ignore, AccurateOrNullConvertStrategyAdditions) @@ -3347,13 +3348,13 @@ private: if constexpr (std::is_same_v || std::is_same_v) { #define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE, ADDITIONS) \ - case DateTimeOverflowMode::OVERFLOW_MODE: \ - result_column = ConvertImpl::template execute( \ + case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \ + result_column = ConvertImpl::template execute( \ arguments, result_type, input_rows_count); \ break; if (wrapper_cast_type == CastType::accurate) { - switch (date_time_overflow_mode) + switch (date_time_overflow_behavior) { GENERATE_OVERFLOW_MODE_CASE(Throw, DateTimeAccurateConvertStrategyAdditions) GENERATE_OVERFLOW_MODE_CASE(Ignore, DateTimeAccurateConvertStrategyAdditions) @@ -3362,7 +3363,7 @@ arguments, result_type, input_rows_count); \ } else { - switch (date_time_overflow_mode) + switch (date_time_overflow_behavior) { GENERATE_OVERFLOW_MODE_CASE(Throw, DateTimeAccurateOrNullConvertStrategyAdditions) GENERATE_OVERFLOW_MODE_CASE(Ignore, DateTimeAccurateOrNullConvertStrategyAdditions) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index f92406488d8..1e81274f5be 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -157,6 +157,7 @@ void ArrowBlockInputFormat::prepareReader() "Arrow", format_settings.arrow.allow_missing_columns, format_settings.null_as_default, + format_settings.date_time_overflow_behavior, format_settings.arrow.case_insensitive_column_matching); if (stream) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 2e960cf2eff..6f9d49498f2 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -32,7 +32,6 @@ #include #include #include -#include #include #include #include @@ -248,22 +247,13 @@ static ColumnWithTypeAndName readColumnWithBooleanData(std::shared_ptr & arrow_column, const String & column_name, const DataTypePtr & type_hint) +static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr & arrow_column, const String & column_name, + const DataTypePtr & type_hint, FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior) { DataTypePtr internal_type; bool check_date_range = false; /// Make result type Date32 when requested type is actually Date32 or when we use schema inference - DateTimeOverflowMode date_time_overflow_mode = DateTimeOverflowMode::Throw; - - if (DB::CurrentThread::isInitialized()) - { - const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext(); - - if (query_context) - date_time_overflow_mode = query_context->getSettingsRef().date_time_overflow_mode; - } - if (!type_hint || (type_hint && isDate32(*type_hint))) { internal_type = std::make_shared(); @@ -290,13 +280,18 @@ static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr(chunk.Value(value_i)); if (days_num > DATE_LUT_MAX_EXTEND_DAY_NUM || days_num < -DAYNUM_OFFSET_EPOCH) { - if (date_time_overflow_mode == DateTimeOverflowMode::Saturate) - days_num = (days_num < -DAYNUM_OFFSET_EPOCH) ? -DAYNUM_OFFSET_EPOCH : DATE_LUT_MAX_EXTEND_DAY_NUM; - else + switch (date_time_overflow_behavior) { - throw Exception{ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, - "Input value {} of a column \"{}\" is out of allowed Date32 range, which is [{}, {}]", - days_num,column_name, -DAYNUM_OFFSET_EPOCH, DATE_LUT_MAX_EXTEND_DAY_NUM}; + case FormatSettings::DateTimeOverflowBehavior::Saturate: + days_num = (days_num < -DAYNUM_OFFSET_EPOCH) ? -DAYNUM_OFFSET_EPOCH : DATE_LUT_MAX_EXTEND_DAY_NUM; + break; + default: + /// Prior to introducing `date_time_overflow_behavior`, this function threw an error in case value was out of range. + /// In order to leave this behavior as default, we also throw when `date_time_overflow_mode == ignore`, as it is the setting's default value + /// (As we want to make this backwards compatible, not break any workflows.) + throw Exception{ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, + "Input value {} of a column \"{}\" is out of allowed Date32 range, which is [{}, {}]", + days_num,column_name, -DAYNUM_OFFSET_EPOCH, DATE_LUT_MAX_EXTEND_DAY_NUM}; } } @@ -701,6 +696,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( bool allow_null_type, bool skip_columns_with_unsupported_types, bool & skipped, + FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = FormatSettings::DateTimeOverflowBehavior::Ignore, DataTypePtr type_hint = nullptr, bool is_map_nested = false) { @@ -711,7 +707,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( DataTypePtr nested_type_hint; if (type_hint) nested_type_hint = removeNullable(type_hint); - auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint); + auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, date_time_overflow_behavior, nested_type_hint); if (skipped) return {}; auto nullmap_column = readByteMapFromArrowColumn(arrow_column); @@ -776,7 +772,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( case arrow::Type::BOOL: return readColumnWithBooleanData(arrow_column, column_name); case arrow::Type::DATE32: - return readColumnWithDate32Data(arrow_column, column_name, type_hint); + return readColumnWithDate32Data(arrow_column, column_name, type_hint, date_time_overflow_behavior); case arrow::Type::DATE64: return readColumnWithDate64Data(arrow_column, column_name); // ClickHouse writes Date as arrow UINT16 and DateTime as arrow UINT32, @@ -824,7 +820,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( } } auto arrow_nested_column = getNestedArrowColumn(arrow_column); - auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint, true); + auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, date_time_overflow_behavior, nested_type_hint, true); if (skipped) return {}; @@ -859,7 +855,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( nested_type_hint = array_type_hint->getNestedType(); } auto arrow_nested_column = getNestedArrowColumn(arrow_column); - auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint); + auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, date_time_overflow_behavior, nested_type_hint); if (skipped) return {}; auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); @@ -900,7 +896,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( nested_type_hint = tuple_type_hint->getElement(i); } auto nested_arrow_column = std::make_shared(nested_arrow_columns[i]); - auto element = readColumnFromArrowColumn(nested_arrow_column, field_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint); + auto element = readColumnFromArrowColumn(nested_arrow_column, field_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, date_time_overflow_behavior, nested_type_hint); if (skipped) return {}; tuple_elements.emplace_back(std::move(element.column)); @@ -927,7 +923,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( dict_array.emplace_back(dict_chunk.dictionary()); } auto arrow_dict_column = std::make_shared(dict_array); - auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped); + auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, date_time_overflow_behavior); for (size_t i = 0; i != dict_column.column->size(); ++i) { if (dict_column.column->isDefaultAt(i)) @@ -1017,7 +1013,8 @@ static void checkStatus(const arrow::Status & status, const String & column_name Block ArrowColumnToCHColumn::arrowSchemaToCHHeader( - const arrow::Schema & schema, const std::string & format_name, bool skip_columns_with_unsupported_types, const Block * hint_header, bool ignore_case) + const arrow::Schema & schema, const std::string & format_name, + bool skip_columns_with_unsupported_types, const Block * hint_header, bool ignore_case) { ColumnsWithTypeAndName sample_columns; std::unordered_set nested_table_names; @@ -1060,12 +1057,14 @@ ArrowColumnToCHColumn::ArrowColumnToCHColumn( const std::string & format_name_, bool allow_missing_columns_, bool null_as_default_, + FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior_, bool case_insensitive_matching_) : header(header_) , format_name(format_name_) , allow_missing_columns(allow_missing_columns_) , null_as_default(null_as_default_) , case_insensitive_matching(case_insensitive_matching_) + , date_time_overflow_behavior(date_time_overflow_behavior_) { } @@ -1122,8 +1121,9 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & auto nested_table_type = Nested::collect(nested_columns).front().type; std::shared_ptr arrow_column = name_to_column_ptr[search_nested_table_name]; - ColumnsWithTypeAndName cols = {readColumnFromArrowColumn( - arrow_column, nested_table_name, format_name, false, dictionary_infos, true, false, skipped, nested_table_type)}; + ColumnsWithTypeAndName cols = { + readColumnFromArrowColumn(arrow_column, nested_table_name, format_name, false, dictionary_infos, true, false, + skipped, date_time_overflow_behavior, nested_table_type)}; BlockPtr block_ptr = std::make_shared(cols); auto column_extractor = std::make_shared(*block_ptr, case_insensitive_matching); nested_tables[search_nested_table_name] = {block_ptr, column_extractor}; @@ -1158,7 +1158,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & { auto arrow_column = name_to_column_ptr[search_column_name]; column = readColumnFromArrowColumn( - arrow_column, header_column.name, format_name, false, dictionary_infos, true, false, skipped, header_column.type); + arrow_column, header_column.name, format_name, false, dictionary_infos, true, false, skipped, date_time_overflow_behavior, header_column.type); } if (null_as_default) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 57f33069e0e..ff80797dc27 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -8,7 +8,7 @@ #include #include #include - +#include namespace DB { @@ -26,6 +26,7 @@ public: const std::string & format_name_, bool allow_missing_columns_, bool null_as_default_, + FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior_, bool case_insensitive_matching_ = false); void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, size_t num_rows, BlockMissingValues * block_missing_values = nullptr); @@ -56,6 +57,7 @@ private: bool allow_missing_columns; bool null_as_default; bool case_insensitive_matching; + FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior; /// Map {column name : dictionary column}. /// To avoid converting dictionary from Arrow Dictionary diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index c0ac0e2c232..5cde51a4927 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -131,6 +131,7 @@ void ORCBlockInputFormat::prepareReader() "ORC", format_settings.orc.allow_missing_columns, format_settings.null_as_default, + format_settings.date_time_overflow_behavior, format_settings.orc.case_insensitive_column_matching); const bool ignore_case = format_settings.orc.case_insensitive_column_matching; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 3cfeb80afd5..c9ac2438fc0 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -493,6 +493,7 @@ void ParquetBlockInputFormat::initializeRowGroupBatchReader(size_t row_group_bat "Parquet", format_settings.parquet.allow_missing_columns, format_settings.null_as_default, + format_settings.date_time_overflow_behavior, format_settings.parquet.case_insensitive_column_matching); } diff --git a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp b/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp index 1172a40627d..eae17078577 100644 --- a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp +++ b/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp @@ -283,6 +283,7 @@ struct DeltaLakeMetadataParser::Impl header, "Parquet", format_settings.parquet.allow_missing_columns, /* null_as_default */true, + format_settings.date_time_overflow_behavior, /* case_insensitive_column_matching */false); Chunk res; diff --git a/tests/queries/0_stateless/02900_date_time_check_overflow.sql b/tests/queries/0_stateless/02900_date_time_check_overflow.sql index f466cf5a018..ba6125cb4b4 100644 --- a/tests/queries/0_stateless/02900_date_time_check_overflow.sql +++ b/tests/queries/0_stateless/02900_date_time_check_overflow.sql @@ -1,7 +1,7 @@ SET session_timezone = 'UTC'; SELECT 'ignore'; -SET date_time_overflow_mode = 'ignore'; +SET date_time_overflow_behavior = 'ignore'; SELECT toDateTime(toDateTime64('1900-01-01 00:00:00.123', 3)); SELECT toDateTime(toDateTime64('2299-12-31 23:59:59.999', 3)); @@ -19,7 +19,7 @@ SELECT toDate(toDate32('2299-12-31')); SELECT 'No output on `throw`'; -SET date_time_overflow_mode = 'throw'; +SET date_time_overflow_behavior = 'throw'; SELECT toDateTime(toDateTime64('1900-01-01 00:00:00.123', 3)); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } SELECT toDateTime(toDateTime64('2299-12-31 23:59:59.999', 3)); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } SELECT toDateTime(toDate32('1900-01-01')); -- { serverError VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE } @@ -32,7 +32,7 @@ SELECT toDate(toDate32('2299-12-31')); -- { serverError VALUE_IS_OUT_OF_RANGE_OF SELECT 'saturate'; -SET date_time_overflow_mode = 'saturate'; +SET date_time_overflow_behavior = 'saturate'; SELECT toDateTime(toDateTime64('1900-01-01 00:00:00.123', 3)); SELECT toDateTime(toDateTime64('2299-12-31 23:59:59.999', 3)); From 95df08dee2eaeb27723b26825bac2b91ed721636 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 25 Oct 2023 23:55:39 +0000 Subject: [PATCH 610/634] fix test --- ...4_json_merge_patch_function_test.reference | 20 +++++++++---------- .../02874_json_merge_patch_function_test.sql | 8 ++++---- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/02874_json_merge_patch_function_test.reference b/tests/queries/0_stateless/02874_json_merge_patch_function_test.reference index 29283d0f232..d7de5819d77 100644 --- a/tests/queries/0_stateless/02874_json_merge_patch_function_test.reference +++ b/tests/queries/0_stateless/02874_json_merge_patch_function_test.reference @@ -8,13 +8,13 @@ {"a":{"b":[3,4],"c":2}} {"a":5,"b":2,"c":4,"d":6} {"a":1,"b":null} -{"k0":"0","k1":"1"} -{"k2":"2","k3":"3"} -{"k4":"4","k5":"5"} -{"k6":"6","k7":"7"} -{"k8":"8","k9":"9"} -{"k10":"222","k11":"11"} -{"k12":"222","k13":"13"} -{"k14":"222","k15":"15"} -{"k16":"222","k17":"17"} -{"k18":"222","k19":"19"} +{"k0":0,"k1":1} +{"k2":2,"k3":3} +{"k4":4,"k5":5} +{"k6":6,"k7":7} +{"k8":8,"k9":9} +{"k10":222,"k11":11} +{"k12":222,"k13":13} +{"k14":222,"k15":15} +{"k16":222,"k17":17} +{"k18":222,"k19":19} diff --git a/tests/queries/0_stateless/02874_json_merge_patch_function_test.sql b/tests/queries/0_stateless/02874_json_merge_patch_function_test.sql index 4f4b2e20f30..9f0c3fd48cd 100644 --- a/tests/queries/0_stateless/02874_json_merge_patch_function_test.sql +++ b/tests/queries/0_stateless/02874_json_merge_patch_function_test.sql @@ -14,11 +14,11 @@ select jsonMergePatch('[1]'); -- { serverError BAD_ARGUMENTS } select jsonMergePatch('{"a": "1","b": 2,"c": [true,"qrdzkzjvnos": true,"yxqhipj": false,"oesax": "33o8_6AyUy"}]}', '{"c": "1"}'); -- { serverError BAD_ARGUMENTS } drop table if exists t_json_merge; -create table t_json_merge (s1 String, s2 String) engine = Memory; +create table t_json_merge (id UInt64, s1 String, s2 String) engine = Memory; -insert into t_json_merge select toJSONString(map('k' || toString(number * 2), number * 2)), toJSONString(map('k' || toString(number * 2 + 1), number * 2 + 1)) from numbers(5); -insert into t_json_merge select toJSONString(map('k' || toString(number * 2), number * 2)), toJSONString(map('k' || toString(number * 2 + 1), number * 2 + 1, 'k' || toString(number * 2), 222)) from numbers(5, 5); +insert into t_json_merge select number, format('{{ "k{0}": {0} }}', toString(number * 2)), format('{{ "k{0}": {0} }}', toString(number * 2 + 1)) from numbers(5); +insert into t_json_merge select number, format('{{ "k{0}": {0} }}', toString(number * 2)), format('{{ "k{0}": {0}, "k{1}": 222 }}', toString(number * 2 + 1), toString(number * 2)) from numbers(5, 5); -select jsonMergePatch(s1, s2) from t_json_merge; +select jsonMergePatch(s1, s2) from t_json_merge ORDER BY id; drop table t_json_merge; From 3009e535063a66d854ed57aa4d8490d8bbdc8f29 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 26 Oct 2023 15:05:21 +0800 Subject: [PATCH 611/634] fix failed ut --- tests/queries/0_stateless/02900_issue_55858.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02900_issue_55858.sql b/tests/queries/0_stateless/02900_issue_55858.sql index 65fc06d9797..b7b6704cdb5 100644 --- a/tests/queries/0_stateless/02900_issue_55858.sql +++ b/tests/queries/0_stateless/02900_issue_55858.sql @@ -1,9 +1,9 @@ set precise_float_parsing = 1; -select cast('2023-01-01' as Float64); -- { serverError 72 } -select cast('2023-01-01' as Float32); -- { serverError 72 } -select toFloat32('2023-01-01'); -- { serverError 72 } -select toFloat64('2023-01-01'); -- { serverError 72 } +select cast('2023-01-01' as Float64); -- { serverError 6 } +select cast('2023-01-01' as Float32); -- { serverError 6 } +select toFloat32('2023-01-01'); -- { serverError 6 } +select toFloat64('2023-01-01'); -- { serverError 6 } select toFloat32OrZero('2023-01-01'); select toFloat64OrZero('2023-01-01'); select toFloat32OrNull('2023-01-01'); From ba418271c0bf184175210cc928712339234b2c5a Mon Sep 17 00:00:00 2001 From: AN Date: Thu, 26 Oct 2023 12:51:34 +0300 Subject: [PATCH 612/634] Update index.md Minor Russian language fixes (hyphens in compounds, term unification) --- docs/ru/index.md | 34 +++++++++++++++++----------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/docs/ru/index.md b/docs/ru/index.md index 27748fe7051..78bb382753b 100644 --- a/docs/ru/index.md +++ b/docs/ru/index.md @@ -6,7 +6,7 @@ sidebar_label: "Что такое ClickHouse" # Что такое ClickHouse {#what-is-clickhouse} -ClickHouse - столбцовая система управления базами данных (СУБД) для онлайн обработки аналитических запросов (OLAP). +ClickHouse — столбцовая система управления базами данных (СУБД) для онлайн-обработки аналитических запросов (OLAP). В обычной, «строковой» СУБД, данные хранятся в таком порядке: @@ -19,10 +19,10 @@ ClickHouse - столбцовая система управления базам То есть, значения, относящиеся к одной строке, физически хранятся рядом. -Примеры строковых СУБД: MySQL, Postgres, MS SQL Server. +Примеры строковых СУБД: MySQL, PostgreSQL, MS SQL Server. {: .grey } -В столбцовых СУБД, данные хранятся в таком порядке: +В столбцовых СУБД данные хранятся в таком порядке: | Строка: | #0 | #1 | #2 | #N | |-------------|---------------------|---------------------|---------------------|-----| @@ -33,37 +33,37 @@ ClickHouse - столбцовая система управления базам | EventTime: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | … | В примерах изображён только порядок расположения данных. -То есть, значения из разных столбцов хранятся отдельно, а данные одного столбца - вместе. +То есть значения из разных столбцов хранятся отдельно, а данные одного столбца — вместе. Примеры столбцовых СУБД: Vertica, Paraccel (Actian Matrix, Amazon Redshift), Sybase IQ, Exasol, Infobright, InfiniDB, MonetDB (VectorWise, Actian Vector), LucidDB, SAP HANA, Google Dremel, Google PowerDrill, Druid, kdb+. {: .grey } Разный порядок хранения данных лучше подходит для разных сценариев работы. -Сценарий работы с данными - это то, какие производятся запросы, как часто и в каком соотношении; сколько читается данных на запросы каждого вида - строк, столбцов, байт; как соотносятся чтения и обновления данных; какой рабочий размер данных и насколько локально он используется; используются ли транзакции и с какой изолированностью; какие требования к дублированию данных и логической целостности; требования к задержкам на выполнение и пропускной способности запросов каждого вида и т. п. +Сценарий работы с данными — это то, какие производятся запросы, как часто и в каком соотношении; сколько читается данных на запросы каждого вида — строк, столбцов, байтов; как соотносятся чтения и обновления данных; какой рабочий размер данных и насколько локально он используется; используются ли транзакции и с какой изолированностью; какие требования к дублированию данных и логической целостности; требования к задержкам на выполнение и пропускной способности запросов каждого вида и т. п. Чем больше нагрузка на систему, тем более важной становится специализация под сценарий работы, и тем более конкретной становится эта специализация. Не существует системы, одинаково хорошо подходящей под существенно различные сценарии работы. Если система подходит под широкое множество сценариев работы, то при достаточно большой нагрузке, система будет справляться со всеми сценариями работы плохо, или справляться хорошо только с одним из сценариев работы. -## Ключевые особенности OLAP сценария работы {#kliuchevye-osobennosti-olap-stsenariia-raboty} +## Ключевые особенности OLAP-сценария работы {#kliuchevye-osobennosti-olap-stsenariia-raboty} -- подавляющее большинство запросов - на чтение; +- подавляющее большинство запросов — на чтение; - данные обновляются достаточно большими пачками (\> 1000 строк), а не по одной строке, или не обновляются вообще; - данные добавляются в БД, но не изменяются; -- при чтении, вынимается достаточно большое количество строк из БД, но только небольшое подмножество столбцов; -- таблицы являются «широкими», то есть, содержат большое количество столбцов; +- при чтении «вынимается» достаточно большое количество строк из БД, но только небольшое подмножество столбцов; +- таблицы являются «широкими», то есть содержат большое количество столбцов; - запросы идут сравнительно редко (обычно не более сотни в секунду на сервер); - при выполнении простых запросов, допустимы задержки в районе 50 мс; -- значения в столбцах достаточно мелкие - числа и небольшие строки (пример - 60 байт на URL); -- требуется высокая пропускная способность при обработке одного запроса (до миллиардов строк в секунду на один сервер); +- значения в столбцах достаточно мелкие — числа и небольшие строки (например, 60 байт на URL); +- требуется высокая пропускная способность при обработке одного запроса (до миллиардов строк в секунду на один узел); - транзакции отсутствуют; - низкие требования к консистентности данных; - в запросе одна большая таблица, все таблицы кроме одной маленькие; -- результат выполнения запроса существенно меньше исходных данных - то есть, данные фильтруются или агрегируются; результат выполнения помещается в оперативку на одном сервере. +- результат выполнения запроса существенно меньше исходных данных — то есть данные фильтруются или агрегируются; результат выполнения помещается в оперативную память одного узла. -Легко видеть, что OLAP сценарий работы существенно отличается от других распространённых сценариев работы (например, OLTP или Key-Value сценариев работы). Таким образом, не имеет никакого смысла пытаться использовать OLTP или Key-Value БД для обработки аналитических запросов, если вы хотите получить приличную производительность («выше плинтуса»). Например, если вы попытаетесь использовать для аналитики MongoDB или Redis - вы получите анекдотически низкую производительность по сравнению с OLAP-СУБД. +Легко видеть, что OLAP-сценарий работы существенно отличается от других распространённых сценариев работы (например, OLTP или Key-Value сценариев работы). Таким образом, не имеет никакого смысла пытаться использовать OLTP-системы или системы класса «ключ — значение» для обработки аналитических запросов, если вы хотите получить приличную производительность («выше плинтуса»). Например, если вы попытаетесь использовать для аналитики MongoDB или Redis — вы получите анекдотически низкую производительность по сравнению с OLAP-СУБД. -## Причины, по которым столбцовые СУБД лучше подходят для OLAP сценария {#prichiny-po-kotorym-stolbtsovye-subd-luchshe-podkhodiat-dlia-olap-stsenariia} +## Причины, по которым столбцовые СУБД лучше подходят для OLAP-сценария {#prichiny-po-kotorym-stolbtsovye-subd-luchshe-podkhodiat-dlia-olap-stsenariia} -Столбцовые СУБД лучше (от 100 раз по скорости обработки большинства запросов) подходят для OLAP сценария работы. Причины в деталях будут разъяснены ниже, а сам факт проще продемонстрировать визуально: +Столбцовые СУБД лучше (от 100 раз по скорости обработки большинства запросов) подходят для OLAP-сценария работы. Причины в деталях будут разъяснены ниже, а сам факт проще продемонстрировать визуально: **Строковые СУБД** @@ -94,6 +94,6 @@ ClickHouse - столбцовая система управления базам 2. Кодогенерация. Для запроса генерируется код, в котором подставлены все косвенные вызовы. -В «обычных» БД этого не делается, так как не имеет смысла при выполнении простых запросов. Хотя есть исключения. Например, в MemSQL кодогенерация используется для уменьшения latency при выполнении SQL запросов. Для сравнения, в аналитических СУБД требуется оптимизация throughput, а не latency. +В «обычных» СУБД этого не делается, так как не имеет смысла при выполнении простых запросов. Хотя есть исключения. Например, в MemSQL кодогенерация используется для уменьшения времени отклика при выполнении SQL-запросов. Для сравнения: в аналитических СУБД требуется оптимизация по пропускной способности (throughput, ГБ/с), а не времени отклика (latency, с). -Стоит заметить, что для эффективности по CPU требуется, чтобы язык запросов был декларативным (SQL, MDX) или хотя бы векторным (J, K). То есть, чтобы запрос содержал циклы только в неявном виде, открывая возможности для оптимизации. +Стоит заметить, что для эффективности по CPU требуется, чтобы язык запросов был декларативным (SQL, MDX) или хотя бы векторным (J, K). То есть необходимо, чтобы запрос содержал циклы только в неявном виде, открывая возможности для оптимизации. From 6487622d300d8b9436d37615d52db9535040a374 Mon Sep 17 00:00:00 2001 From: AN Date: Thu, 26 Oct 2023 13:00:45 +0300 Subject: [PATCH 613/634] Update distinctive-features.md Minor Russian language-related fixes (term unifications, hyphens, punctuation etc.) --- docs/ru/introduction/distinctive-features.md | 22 ++++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/ru/introduction/distinctive-features.md b/docs/ru/introduction/distinctive-features.md index dc44ca8e03f..dafaf055980 100644 --- a/docs/ru/introduction/distinctive-features.md +++ b/docs/ru/introduction/distinctive-features.md @@ -8,11 +8,11 @@ sidebar_label: "Отличительные возможности ClickHouse" ## По-настоящему столбцовая СУБД {#po-nastoiashchemu-stolbtsovaia-subd} -В по-настоящему столбцовой СУБД рядом со значениями не хранится никаких лишних данных. Например, должны поддерживаться значения постоянной длины, чтобы не хранить рядом со значениями типа «число» их длины. Для примера, миллиард значений типа UInt8 должен действительно занимать в несжатом виде около 1GB, иначе это сильно ударит по эффективности использования CPU. Очень важно хранить данные компактно (без «мусора») в том числе в несжатом виде, так как скорость разжатия (использование CPU) зависит, в основном, от объёма несжатых данных. +В по-настоящему столбцовой СУБД рядом со значениями не хранится никаких лишних данных. Например, должны поддерживаться значения постоянной длины, чтобы не хранить рядом со значениями типа «число» их длины. Для примера, миллиард значений типа UInt8 должен действительно занимать в несжатом виде около 1 ГБ, иначе это сильно ударит по эффективности использования CPU. Очень важно хранить данные компактно (без «мусора») в том числе в несжатом виде, так как скорость разжатия (использование CPU) зависит, в основном, от объёма несжатых данных. Этот пункт пришлось выделить, так как существуют системы, которые могут хранить значения отдельных столбцов по отдельности, но не могут эффективно выполнять аналитические запросы в силу оптимизации под другой сценарий работы. Примеры: HBase, BigTable, Cassandra, HyperTable. В этих системах вы получите пропускную способность в районе сотен тысяч строк в секунду, но не сотен миллионов строк в секунду. -Также стоит заметить, что ClickHouse является системой управления базами данных, а не одной базой данных. То есть, ClickHouse позволяет создавать таблицы и базы данных в runtime, загружать данные и выполнять запросы без переконфигурирования и перезапуска сервера. +Также стоит заметить, что ClickHouse является системой управления базами данных, а не системой для одной базой данных. То есть, ClickHouse позволяет создавать таблицы и базы данных во время выполнения (runtime), загружать данные и выполнять запросы без переконфигурирования и перезапуска сервера. ## Сжатие данных {#szhatie-dannykh} @@ -20,7 +20,7 @@ sidebar_label: "Отличительные возможности ClickHouse" ## Хранение данных на диске {#khranenie-dannykh-na-diske} -Многие столбцовые СУБД (SAP HANA, Google PowerDrill) могут работать только в оперативной памяти. Такой подход стимулирует выделять больший бюджет на оборудование, чем фактически требуется для анализа в реальном времени. ClickHouse спроектирован для работы на обычных жестких дисках, что обеспечивает низкую стоимость хранения на гигабайт данных, но SSD и дополнительная оперативная память тоже полноценно используются, если доступны. +Многие столбцовые СУБД (SAP HANA, Google PowerDrill) могут работать только в оперативной памяти. Такой подход стимулирует выделять больший бюджет на оборудование, чем фактически требуется для анализа в реальном времени. ClickHouse спроектирован для работы на обычных жестких дисках, что обеспечивает низкую стоимость хранения на гигабайт данных. При этом твердотельные накопители (SSD) и дополнительная оперативная память тоже полноценно используются, если доступны. ## Параллельная обработка запроса на многих процессорных ядрах {#parallelnaia-obrabotka-zaprosa-na-mnogikh-protsessornykh-iadrakh} @@ -29,11 +29,11 @@ sidebar_label: "Отличительные возможности ClickHouse" ## Распределённая обработка запроса на многих серверах {#raspredelionnaia-obrabotka-zaprosa-na-mnogikh-serverakh} Почти все перечисленные ранее столбцовые СУБД не поддерживают распределённую обработку запроса. -В ClickHouse данные могут быть расположены на разных шардах. Каждый шард может представлять собой группу реплик, которые используются для отказоустойчивости. Запрос будет выполнен на всех шардах параллельно. Это делается прозрачно для пользователя. +В ClickHouse данные могут быть расположены на разных сегментах (shards). Каждый сегмент может представлять собой группу реплик, которые используются для отказоустойчивости. Запрос будет выполнен на всех сегментах параллельно. Это делается прозрачно для пользователя. ## Поддержка SQL {#sql-support} -ClickHouse поддерживает [декларативный язык запросов на основе SQL](../sql-reference/index.md) и во [многих случаях](../sql-reference/ansi.mdx) совпадающий с SQL стандартом. +ClickHouse поддерживает [декларативный язык запросов на основе SQL](../sql-reference/index.md) и во [многих случаях](../sql-reference/ansi.mdx) совпадающий с SQL-стандартом. Поддерживаются [GROUP BY](../sql-reference/statements/select/group-by.md), [ORDER BY](../sql-reference/statements/select/order-by.md), подзапросы в секциях [FROM](../sql-reference/statements/select/from.md), [IN](../sql-reference/operators/in.md), [JOIN](../sql-reference/statements/select/join.md), [функции window](../sql-reference/window-functions/index.mdx), а также скалярные подзапросы. @@ -41,17 +41,17 @@ ClickHouse поддерживает [декларативный язык зап ## Векторный движок {#vektornyi-dvizhok} -Данные не только хранятся по столбцам, но и обрабатываются по векторам - кусочкам столбцов. За счёт этого достигается высокая эффективность по CPU. +Данные не только хранятся по столбцам, но и обрабатываются по векторам — фрагментам столбцов. За счёт этого достигается высокая эффективность по CPU. ## Обновление данных в реальном времени {#obnovlenie-dannykh-v-realnom-vremeni} -ClickHouse поддерживает таблицы с первичным ключом. Для того, чтобы можно было быстро выполнять запросы по диапазону первичного ключа, данные инкрементально сортируются с помощью merge дерева. За счёт этого, поддерживается постоянное добавление данных в таблицу. Блокировки при добавлении данных отсутствуют. +ClickHouse поддерживает таблицы с первичным ключом. Для того, чтобы можно было быстро выполнять запросы по диапазону первичного ключа, данные инкрементально сортируются с помощью дерева со слиянием (merge tree). За счёт этого поддерживается постоянное добавление данных в таблицу. Блокировки при добавлении данных отсутствуют. ## Наличие индекса {#nalichie-indeksa} -Физическая сортировка данных по первичному ключу позволяет получать данные для конкретных его значений или их диапазонов с низкими задержками - менее десятков миллисекунд. +Физическая сортировка данных по первичному ключу позволяет получать данные для конкретных его значений или их диапазонов с низкими задержками — менее десятков миллисекунд. -## Подходит для онлайн запросов {#podkhodit-dlia-onlain-zaprosov} +## Подходит для онлайн-запросов {#podkhodit-dlia-onlain-zaprosov} Низкие задержки позволяют не откладывать выполнение запроса и не подготавливать ответ заранее, а выполнять его именно в момент загрузки страницы пользовательского интерфейса. То есть, в режиме онлайн. @@ -60,12 +60,12 @@ ClickHouse поддерживает таблицы с первичным клю ClickHouse предоставляет различные способы разменять точность вычислений на производительность: 1. Система содержит агрегатные функции для приближённого вычисления количества различных значений, медианы и квантилей. -2. Поддерживается возможность выполнить запрос на основе части (выборки) данных и получить приближённый результат. При этом, с диска будет считано пропорционально меньше данных. +2. Поддерживается возможность выполнить запрос на основе части (выборки) данных и получить приближённый результат. При этом с диска будет считано пропорционально меньше данных. 3. Поддерживается возможность выполнить агрегацию не для всех ключей, а для ограниченного количества первых попавшихся ключей. При выполнении некоторых условий на распределение ключей в данных, это позволяет получить достаточно точный результат с использованием меньшего количества ресурсов. ## Репликация данных и поддержка целостности {#replikatsiia-dannykh-i-podderzhka-tselostnosti} -Используется асинхронная multimaster репликация. После записи на любую доступную реплику, данные распространяются на все остальные реплики в фоне. Система поддерживает полную идентичность данных на разных репликах. Восстановление после большинства сбоев осуществляется автоматически, а в сложных случаях — полуавтоматически. При необходимости, можно [включить кворумную запись](../operations/settings/settings.md) данных. +Используется асинхронная multimaster-репликация. После записи на любую доступную реплику, данные распространяются на все остальные реплики в фоне. Система поддерживает полную идентичность данных на разных репликах. Восстановление после большинства сбоев осуществляется автоматически, а в сложных случаях — полуавтоматически. При необходимости, можно [включить кворумную запись](../operations/settings/settings.md) данных. Подробнее смотрите раздел [Репликация данных](../engines/table-engines/mergetree-family/replication.md). From 21e0c510c8e715dd308b6c1985e2924e3f91a5a5 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 26 Oct 2023 13:08:11 +0200 Subject: [PATCH 614/634] formatQuery: improve doc --- .../functions/other-functions.md | 52 +------------------ src/Functions/formatQuery.cpp | 8 +-- 2 files changed, 4 insertions(+), 56 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 960f83e229a..2e52d067b45 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2797,7 +2797,7 @@ message Root ## formatQuery -Returns a formatted version of the given SQL query. +Returns a formatted, possibly multi-line, version of the given SQL query. **Syntax** @@ -2815,30 +2815,6 @@ formatQuery(query) **Example** -```sql -SELECT formatQuery('select 1;'); -``` - -Result: - -```result -┌─formatQuery('select 1;')─┐ -│ SELECT 1 │ -└──────────────────────────┘ -``` - -```sql -SELECT formatQuery('SeLecT 1'); -``` - -Result: - -```result -┌─formatQuery('SeLecT 1')──┐ -│ SELECT 1 │ -└──────────────────────────┘ -``` - ```sql SELECT formatQuery('select a, b FRom tab WHERE a > 3 and b < 3'); ``` @@ -2857,7 +2833,7 @@ WHERE (a > 3) AND (b < 3) │ ## formatQueryOneLine -Returns a formatted version of the given SQL query on a single line. +Like formatQuery() but the returned formatted string contains no line breaks. **Syntax** @@ -2881,30 +2857,6 @@ SELECT formatQueryOneLine('select 1;'); Result: -```result -┌─formatQueryOneLine('select 1;')─┐ -│ SELECT 1 │ -└─────────────────────────────────┘ -``` - -```sql -SELECT formatQueryOneLine('SeLecT 1'); -``` - -Result: - -```result -┌─formatQueryOneLine('SeLecT 1')──┐ -│ SELECT 1 │ -└─────────────────────────────────┘ -``` - -```sql -SELECT formatQueryOneLine('select a, b FRom tab WHERE a > 3 and b < 3'); -``` - -Result: - ```result ┌─formatQueryOneLine('select a, b FRom tab WHERE a > 3 and b < 3')─┐ │ SELECT a, b FROM tab WHERE (a > 3) AND (b < 3) │ diff --git a/src/Functions/formatQuery.cpp b/src/Functions/formatQuery.cpp index bd230292607..ca853a155b5 100644 --- a/src/Functions/formatQuery.cpp +++ b/src/Functions/formatQuery.cpp @@ -106,13 +106,11 @@ struct NameFormatQueryOneLine REGISTER_FUNCTION(formatQuery) { factory.registerFunction>(FunctionDocumentation{ - .description = "Returns a formatted version of the given SQL query.\n[example:simple]\n[example:camelcase]", + .description = "Returns a formatted, possibly multi-line, version of the given SQL query.\n[example:multiline]", .syntax = "formatQuery(query)", .arguments = {{"query", "The SQL query to be formatted. [String](../../sql-reference/data-types/string.md)"}}, .returned_value = "The formatted query. [String](../../sql-reference/data-types/string.md).", .examples{ - {"simple", "SELECT formatQuery('select 1;')", "SELECT 1"}, - {"camelcase", "SELECT formatQuery('SeLecT 1')", "SELECT 1"}, {"multiline", "SELECT formatQuery('select a, b FRom tab WHERE a > 3 and b < 3');", "SELECT\n" @@ -126,13 +124,11 @@ REGISTER_FUNCTION(formatQuery) REGISTER_FUNCTION(formatQueryOneLine) { factory.registerFunction>(FunctionDocumentation{ - .description = "Returns a formatted version of the given SQL query on a single line.\n[example:simple]\n[example:camelcase]", + .description = "Like formatQuery() but the returned formatted string contains no line breaks.\n[example:multiline]", .syntax = "formatQueryOneLine(query)", .arguments = {{"query", "The SQL query to be formatted. [String](../../sql-reference/data-types/string.md)"}}, .returned_value = "The formatted query. [String](../../sql-reference/data-types/string.md).", .examples{ - {"simple", "SELECT formatQueryOneLine('select 1;')", "SELECT 1"}, - {"camelcase", "SELECT formatQueryOneLine('SeLecT 1')", "SELECT 1"}, {"multiline", "SELECT formatQuery('select a, b FRom tab WHERE a > 3 and b < 3');", "SELECT a, b FROM tab WHERE (a > 3) AND (b < 3)"}}, From 56bd61d20712e39a913d2639be42b2114f0af2dd Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 26 Oct 2023 13:09:20 +0200 Subject: [PATCH 615/634] Rename formatQueryOneLine -> formatQuerySingleLine --- .../functions/other-functions.md | 12 +++++----- src/Functions/formatQuery.cpp | 12 +++++----- .../0_stateless/02882_formatQuery.reference | 2 +- .../queries/0_stateless/02882_formatQuery.sql | 22 +++++++++---------- .../aspell-ignore/en/aspell-dict.txt | 2 +- 5 files changed, 25 insertions(+), 25 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 2e52d067b45..d7809be86bf 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2831,14 +2831,14 @@ WHERE (a > 3) AND (b < 3) │ └───────────────────────────────────────────────────────────────┘ ``` -## formatQueryOneLine +## formatQuerySingleLine Like formatQuery() but the returned formatted string contains no line breaks. **Syntax** ```sql -formatQueryOneLine(query) +formatQuerySingleLine(query) ``` **Arguments** @@ -2852,13 +2852,13 @@ formatQueryOneLine(query) **Example** ```sql -SELECT formatQueryOneLine('select 1;'); +SELECT formatQuerySingleLine('select a, b FRom tab WHERE a > 3 and b < 3'); ``` Result: ```result -┌─formatQueryOneLine('select a, b FRom tab WHERE a > 3 and b < 3')─┐ -│ SELECT a, b FROM tab WHERE (a > 3) AND (b < 3) │ -└──────────────────────────────────────────────────────────────────────┘ +┌─formatQuerySingleLine('select a, b FRom tab WHERE a > 3 and b < 3')─┐ +│ SELECT a, b FROM tab WHERE (a > 3) AND (b < 3) │ +└─────────────────────────────────────────────────────────────────────────┘ ``` diff --git a/src/Functions/formatQuery.cpp b/src/Functions/formatQuery.cpp index ca853a155b5..f4cb937eed4 100644 --- a/src/Functions/formatQuery.cpp +++ b/src/Functions/formatQuery.cpp @@ -98,9 +98,9 @@ struct NameFormatQuery static constexpr auto name = "formatQuery"; }; -struct NameFormatQueryOneLine +struct NameFormatQuerySingleLine { - static constexpr auto name = "formatQueryOneLine"; + static constexpr auto name = "formatQuerySingleLine"; }; REGISTER_FUNCTION(formatQuery) @@ -121,16 +121,16 @@ REGISTER_FUNCTION(formatQuery) .categories{"Other"}}); } -REGISTER_FUNCTION(formatQueryOneLine) +REGISTER_FUNCTION(formatQuerySingleLine) { - factory.registerFunction>(FunctionDocumentation{ + factory.registerFunction>(FunctionDocumentation{ .description = "Like formatQuery() but the returned formatted string contains no line breaks.\n[example:multiline]", - .syntax = "formatQueryOneLine(query)", + .syntax = "formatQuerySingleLine(query)", .arguments = {{"query", "The SQL query to be formatted. [String](../../sql-reference/data-types/string.md)"}}, .returned_value = "The formatted query. [String](../../sql-reference/data-types/string.md).", .examples{ {"multiline", - "SELECT formatQuery('select a, b FRom tab WHERE a > 3 and b < 3');", + "SELECT formatQuerySingleLine('select a, b FRom tab WHERE a > 3 and b < 3');", "SELECT a, b FROM tab WHERE (a > 3) AND (b < 3)"}}, .categories{"Other"}}); } diff --git a/tests/queries/0_stateless/02882_formatQuery.reference b/tests/queries/0_stateless/02882_formatQuery.reference index a7bd2dbe822..fd84a9505b1 100644 --- a/tests/queries/0_stateless/02882_formatQuery.reference +++ b/tests/queries/0_stateless/02882_formatQuery.reference @@ -7,7 +7,7 @@ INSERT INTO tab FORMAT Values CREATE TABLE default.no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 EXPLAIN SYNTAX\nSELECT\n CAST(1, \'INT\'),\n CEIL(1),\n CEILING(1),\n CHAR(49),\n CHAR_LENGTH(\'1\'),\n CHARACTER_LENGTH(\'1\'),\n COALESCE(1),\n CONCAT(\'1\', \'1\'),\n CORR(1, 1),\n COS(1),\n COUNT(1),\n COVAR_POP(1, 1),\n COVAR_SAMP(1, 1),\n DATABASE(),\n SCHEMA(),\n dateDiff(\'DAY\', toDate(\'2020-10-24\'), toDate(\'2019-10-24\')),\n EXP(1),\n FLATTEN([[1]]),\n FLOOR(1),\n FQDN(),\n GREATEST(1),\n IF(1, 1, 1),\n IFNULL(1, 1),\n LCASE(\'A\'),\n LEAST(1),\n LENGTH(\'1\'),\n LN(1),\n LOCATE(\'1\', \'1\'),\n LOG(1),\n LOG10(1),\n LOG2(1),\n LOWER(\'A\'),\n MAX(1),\n MID(\'123\', 1, 1),\n MIN(1),\n MOD(1, 1),\n NOT 1,\n NOW(),\n NOW64(),\n NULLIF(1, 1),\n PI(),\n position(\'123\', \'2\'),\n POW(1, 1),\n POWER(1, 1),\n RAND(),\n REPLACE(\'1\', \'1\', \'2\'),\n REVERSE(\'123\'),\n ROUND(1),\n SIN(1),\n SQRT(1),\n STDDEV_POP(1),\n STDDEV_SAMP(1),\n SUBSTR(\'123\', 2),\n substring(\'123\', 2),\n SUM(1),\n TAN(1),\n TANH(1),\n TRUNC(1),\n TRUNCATE(1),\n UCASE(\'A\'),\n UPPER(\'A\'),\n USER(),\n VAR_POP(1),\n VAR_SAMP(1),\n WEEK(toDate(\'2020-10-24\')),\n YEARWEEK(toDate(\'2020-10-24\'))\nFORMAT TSVRaw 1 -formatQueryOneLine +formatQuerySingleLine SELECT 1 SELECT 1 SELECT 1 diff --git a/tests/queries/0_stateless/02882_formatQuery.sql b/tests/queries/0_stateless/02882_formatQuery.sql index 9edb07cb0e3..767283552d5 100644 --- a/tests/queries/0_stateless/02882_formatQuery.sql +++ b/tests/queries/0_stateless/02882_formatQuery.sql @@ -8,14 +8,14 @@ SELECT formatQuery('CREATE TABLE default.no_prop_table(`some_column` UInt64) ENG SELECT formatQuery('EXPLAIN SYNTAX SELECT CAST(1 AS INT), CEIL(1), CEILING(1), CHAR(49), CHAR_LENGTH(\'1\'), CHARACTER_LENGTH(\'1\'), COALESCE(1), CONCAT(\'1\', \'1\'), CORR(1, 1), COS(1), COUNT(1), COVAR_POP(1, 1), COVAR_SAMP(1, 1), DATABASE(), SCHEMA(), DATEDIFF(\'DAY\', toDate(\'2020-10-24\'), toDate(\'2019-10-24\')), EXP(1), FLATTEN([[1]]), FLOOR(1), FQDN(), GREATEST(1), IF(1, 1, 1), IFNULL(1, 1), LCASE(\'A\'), LEAST(1), LENGTH(\'1\'), LN(1), LOCATE(\'1\', \'1\'), LOG(1), LOG10(1), LOG2(1), LOWER(\'A\'), MAX(1), MID(\'123\', 1, 1), MIN(1), MOD(1, 1), NOT(1), NOW(), NOW64(), NULLIF(1, 1), PI(), POSITION(\'123\', \'2\'), POW(1, 1), POWER(1, 1), RAND(), REPLACE(\'1\', \'1\', \'2\'), REVERSE(\'123\'), ROUND(1), SIN(1), SQRT(1), STDDEV_POP(1), STDDEV_SAMP(1), SUBSTR(\'123\', 2), SUBSTRING(\'123\', 2), SUM(1), TAN(1), TANH(1), TRUNC(1), TRUNCATE(1), UCASE(\'A\'), UPPER(\'A\'), USER(), VAR_POP(1), VAR_SAMP(1), WEEK(toDate(\'2020-10-24\')), YEARWEEK(toDate(\'2020-10-24\')) format TSVRaw;'); SELECT normalizedQueryHash(formatQuery('select 1')) = normalizedQueryHash(formatQuery('SELECT 1')); SELECT formatQuery('SEECTwrong'); -- { serverError SYNTAX_ERROR } -SELECT 'formatQueryOneLine'; -SELECT formatQueryOneLine('select 1;'); -SELECT formatQueryOneLine('select 1'); -SELECT formatQueryOneLine('SELECT 1;'); -SELECT formatQueryOneLine('SELECT 1'); -SELECT formatQueryOneLine('select 1;') == formatQueryOneLine('SeLecT 1'); -SELECT formatQueryOneLine('INSERT INTO tab VALUES (\'\') (\'test\')'); -SELECT formatQueryOneLine('CREATE TABLE default.no_prop_table(`some_column` UInt64) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192'); -SELECT formatQueryOneLine('EXPLAIN SYNTAX SELECT CAST(1 AS INT), CEIL(1), CEILING(1), CHAR(49), CHAR_LENGTH(\'1\'), CHARACTER_LENGTH(\'1\'), COALESCE(1), CONCAT(\'1\', \'1\'), CORR(1, 1), COS(1), COUNT(1), COVAR_POP(1, 1), COVAR_SAMP(1, 1), DATABASE(), SCHEMA(), DATEDIFF(\'DAY\', toDate(\'2020-10-24\'), toDate(\'2019-10-24\')), EXP(1), FLATTEN([[1]]), FLOOR(1), FQDN(), GREATEST(1), IF(1, 1, 1), IFNULL(1, 1), LCASE(\'A\'), LEAST(1), LENGTH(\'1\'), LN(1), LOCATE(\'1\', \'1\'), LOG(1), LOG10(1), LOG2(1), LOWER(\'A\'), MAX(1), MID(\'123\', 1, 1), MIN(1), MOD(1, 1), NOT(1), NOW(), NOW64(), NULLIF(1, 1), PI(), POSITION(\'123\', \'2\'), POW(1, 1), POWER(1, 1), RAND(), REPLACE(\'1\', \'1\', \'2\'), REVERSE(\'123\'), ROUND(1), SIN(1), SQRT(1), STDDEV_POP(1), STDDEV_SAMP(1), SUBSTR(\'123\', 2), SUBSTRING(\'123\', 2), SUM(1), TAN(1), TANH(1), TRUNC(1), TRUNCATE(1), UCASE(\'A\'), UPPER(\'A\'), USER(), VAR_POP(1), VAR_SAMP(1), WEEK(toDate(\'2020-10-24\')), YEARWEEK(toDate(\'2020-10-24\')) format TSVRaw;'); -SELECT normalizedQueryHash(formatQueryOneLine('select 1')) = normalizedQueryHash(formatQueryOneLine('SELECT 1')); -SELECT formatQueryOneLine('SEECTwrong'); -- { serverError SYNTAX_ERROR } +SELECT 'formatQuerySingleLine'; +SELECT formatQuerySingleLine('select 1;'); +SELECT formatQuerySingleLine('select 1'); +SELECT formatQuerySingleLine('SELECT 1;'); +SELECT formatQuerySingleLine('SELECT 1'); +SELECT formatQuerySingleLine('select 1;') == formatQuerySingleLine('SeLecT 1'); +SELECT formatQuerySingleLine('INSERT INTO tab VALUES (\'\') (\'test\')'); +SELECT formatQuerySingleLine('CREATE TABLE default.no_prop_table(`some_column` UInt64) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192'); +SELECT formatQuerySingleLine('EXPLAIN SYNTAX SELECT CAST(1 AS INT), CEIL(1), CEILING(1), CHAR(49), CHAR_LENGTH(\'1\'), CHARACTER_LENGTH(\'1\'), COALESCE(1), CONCAT(\'1\', \'1\'), CORR(1, 1), COS(1), COUNT(1), COVAR_POP(1, 1), COVAR_SAMP(1, 1), DATABASE(), SCHEMA(), DATEDIFF(\'DAY\', toDate(\'2020-10-24\'), toDate(\'2019-10-24\')), EXP(1), FLATTEN([[1]]), FLOOR(1), FQDN(), GREATEST(1), IF(1, 1, 1), IFNULL(1, 1), LCASE(\'A\'), LEAST(1), LENGTH(\'1\'), LN(1), LOCATE(\'1\', \'1\'), LOG(1), LOG10(1), LOG2(1), LOWER(\'A\'), MAX(1), MID(\'123\', 1, 1), MIN(1), MOD(1, 1), NOT(1), NOW(), NOW64(), NULLIF(1, 1), PI(), POSITION(\'123\', \'2\'), POW(1, 1), POWER(1, 1), RAND(), REPLACE(\'1\', \'1\', \'2\'), REVERSE(\'123\'), ROUND(1), SIN(1), SQRT(1), STDDEV_POP(1), STDDEV_SAMP(1), SUBSTR(\'123\', 2), SUBSTRING(\'123\', 2), SUM(1), TAN(1), TANH(1), TRUNC(1), TRUNCATE(1), UCASE(\'A\'), UPPER(\'A\'), USER(), VAR_POP(1), VAR_SAMP(1), WEEK(toDate(\'2020-10-24\')), YEARWEEK(toDate(\'2020-10-24\')) format TSVRaw;'); +SELECT normalizedQueryHash(formatQuerySingleLine('select 1')) = normalizedQueryHash(formatQuerySingleLine('SELECT 1')); +SELECT formatQuerySingleLine('SEECTwrong'); -- { serverError SYNTAX_ERROR } diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index edafd4e7a55..b506794b5cf 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1477,7 +1477,7 @@ formatDateTime formatDateTimeInJoda formatDateTimeInJodaSyntax formatQuery -formatQueryOneLine +formatQuerySingleLine formatReadableDecimalSize formatReadableQuantity formatReadableSize From 4015176d2739a10719d540c1f5abb69494006883 Mon Sep 17 00:00:00 2001 From: helifu Date: Thu, 26 Oct 2023 19:19:58 +0800 Subject: [PATCH 616/634] A small fix (#55938) --- tests/integration/test_storage_s3_queue/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 079311422cc..29757870f9c 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -159,7 +159,7 @@ def generate_random_files( values_csv = ( "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" ).encode() - print(f"File {filename}, content: {total_values}") + print(f"File {filename}, content: {rand_values}") put_s3_file_content(started_cluster, filename, values_csv) return total_values From 4db31d55fbd08f2a0005bebe84aa726000c234c4 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 26 Oct 2023 16:22:36 +0200 Subject: [PATCH 617/634] Restart CI From b6435b32f05e3f1094da0108f24c3819403a0a48 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 26 Oct 2023 17:30:56 +0200 Subject: [PATCH 618/634] Fix bash completion for mawk $ strings < /usr/bin/awk | grep awk mawk 1.3%s%s %s, Copyright (C) Michael D. Brennan $ awk '{print substr($0, 0, 1)}' << --- programs/bash-completion/completions/clickhouse-bootstrap | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/bash-completion/completions/clickhouse-bootstrap b/programs/bash-completion/completions/clickhouse-bootstrap index 45a7502919d..409c461a816 100644 --- a/programs/bash-completion/completions/clickhouse-bootstrap +++ b/programs/bash-completion/completions/clickhouse-bootstrap @@ -146,7 +146,7 @@ function _clickhouse_quote() # Extract every option (everything that starts with "-") from the --help dialog. function _clickhouse_get_options() { - "$@" --help 2>&1 | awk -F '[ ,=<>]' '{ for (i=1; i <= NF; ++i) { if (substr($i, 0, 1) == "-" && length($i) > 1) print $i; } }' | sort -u + "$@" --help 2>&1 | awk -F '[ ,=<>]' '{ for (i=1; i <= NF; ++i) { if (substr($i, 1, 1) == "-" && length($i) > 1) print $i; } }' | sort -u } function _complete_for_clickhouse_generic_bin_impl() From 4cb8b186e73f454d924eeb75931291d09e06d89c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 26 Oct 2023 17:33:31 +0200 Subject: [PATCH 619/634] Add more delimiters for completion Signed-off-by: Azat Khuzhin --- programs/bash-completion/completions/clickhouse-bootstrap | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/bash-completion/completions/clickhouse-bootstrap b/programs/bash-completion/completions/clickhouse-bootstrap index 409c461a816..218daaf9625 100644 --- a/programs/bash-completion/completions/clickhouse-bootstrap +++ b/programs/bash-completion/completions/clickhouse-bootstrap @@ -146,7 +146,7 @@ function _clickhouse_quote() # Extract every option (everything that starts with "-") from the --help dialog. function _clickhouse_get_options() { - "$@" --help 2>&1 | awk -F '[ ,=<>]' '{ for (i=1; i <= NF; ++i) { if (substr($i, 1, 1) == "-" && length($i) > 1) print $i; } }' | sort -u + "$@" --help 2>&1 | awk -F '[ ,=<>.]' '{ for (i=1; i <= NF; ++i) { if (substr($i, 1, 1) == "-" && length($i) > 1) print $i; } }' | sort -u } function _complete_for_clickhouse_generic_bin_impl() From caec685a49b30c196fb35747cc18c79bd8334318 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 26 Oct 2023 17:34:47 +0200 Subject: [PATCH 620/634] Update formats list Signed-off-by: Azat Khuzhin --- .../bash-completion/completions/clickhouse-bootstrap | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/programs/bash-completion/completions/clickhouse-bootstrap b/programs/bash-completion/completions/clickhouse-bootstrap index 218daaf9625..2862140b528 100644 --- a/programs/bash-completion/completions/clickhouse-bootstrap +++ b/programs/bash-completion/completions/clickhouse-bootstrap @@ -46,6 +46,7 @@ CLICKHOUSE_Format=( ArrowStream Avro AvroConfluent + BSONEachRow CSV CSVWithNames CSVWithNamesAndTypes @@ -56,6 +57,7 @@ CLICKHOUSE_Format=( CustomSeparatedIgnoreSpacesWithNamesAndTypes CustomSeparatedWithNames CustomSeparatedWithNamesAndTypes + DWARF HiveText JSON JSONAsObject @@ -74,7 +76,7 @@ CLICKHOUSE_Format=( JSONEachRow JSONEachRowWithProgress JSONLines - JSONStringEachRow + JSONObjectEachRow JSONStrings JSONStringsEachRow JSONStringsEachRowWithProgress @@ -90,14 +92,19 @@ CLICKHOUSE_Format=( Null ODBCDriver2 ORC + One Parquet + ParquetMetadata PostgreSQLWire Pretty PrettyCompact PrettyCompactMonoBlock PrettyCompactNoEscapes PrettyCompactNoEscapesMonoBlock + PrettyJSONEachRow + PrettyJSONLines PrettyMonoBlock + PrettyNDJSON PrettyNoEscapes PrettyNoEscapesMonoBlock PrettySpace @@ -111,6 +118,7 @@ CLICKHOUSE_Format=( RawBLOB Regexp RowBinary + RowBinaryWithDefaults RowBinaryWithNames RowBinaryWithNamesAndTypes SQLInsert From 7cc4b8b1ae2caf1af82eab3206e5ce97e8ac71c7 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 26 Oct 2023 17:07:26 +0000 Subject: [PATCH 621/634] Analyzer: Fix crash on window resolve --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 4 ++++ .../0_stateless/02901_analyzer_recursive_window.reference | 3 +++ tests/queries/0_stateless/02901_analyzer_recursive_window.sql | 4 ++++ 3 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/02901_analyzer_recursive_window.reference create mode 100644 tests/queries/0_stateless/02901_analyzer_recursive_window.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 5cdd18b4b2f..aeb8c3f84e3 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -970,6 +970,10 @@ private: if (!node->hasAlias()) return; + // We don't should not resolve expressions to WindowNode + if (node->getNodeType() == QueryTreeNodeType::WINDOW) + return; + const auto & alias = node->getAlias(); if (is_lambda_node) diff --git a/tests/queries/0_stateless/02901_analyzer_recursive_window.reference b/tests/queries/0_stateless/02901_analyzer_recursive_window.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/02901_analyzer_recursive_window.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/02901_analyzer_recursive_window.sql b/tests/queries/0_stateless/02901_analyzer_recursive_window.sql new file mode 100644 index 00000000000..49feb897b03 --- /dev/null +++ b/tests/queries/0_stateless/02901_analyzer_recursive_window.sql @@ -0,0 +1,4 @@ +SELECT 1 WINDOW x AS (PARTITION BY x); -- { serverError UNKNOWN_IDENTIFIER } +SELECT 1 WINDOW x AS (PARTITION BY dummy); +SELECT 1 WINDOW dummy AS (PARTITION BY dummy); +SELECT count() OVER dummy WINDOW dummy AS (PARTITION BY dummy); From dbaf66b8f23309b49b78a2c45e549d3d1ba7ed8f Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 26 Oct 2023 19:26:18 +0200 Subject: [PATCH 622/634] Fix function_json_value_return_type_allow_nullable setting name in doc --- docs/en/sql-reference/functions/json-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index a4c3ba83351..2c837ff4a42 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -402,7 +402,7 @@ Before version 21.11 the order of arguments was wrong, i.e. JSON_QUERY(path, jso Parses a JSON and extract a value as JSON scalar. -If the value does not exist, an empty string will be returned by default, and by SET `function_return_type_allow_nullable` = `true`, `NULL` will be returned. If the value is complex type (such as: struct, array, map), an empty string will be returned by default, and by SET `function_json_value_return_type_allow_complex` = `true`, the complex value will be returned. +If the value does not exist, an empty string will be returned by default, and by SET `function_json_value_return_type_allow_nullable` = `true`, `NULL` will be returned. If the value is complex type (such as: struct, array, map), an empty string will be returned by default, and by SET `function_json_value_return_type_allow_complex` = `true`, the complex value will be returned. Example: @@ -411,7 +411,7 @@ SELECT JSON_VALUE('{"hello":"world"}', '$.hello'); SELECT JSON_VALUE('{"array":[[0, 1, 2, 3, 4, 5], [0, -1, -2, -3, -4, -5]]}', '$.array[*][0 to 2, 4]'); SELECT JSON_VALUE('{"hello":2}', '$.hello'); SELECT toTypeName(JSON_VALUE('{"hello":2}', '$.hello')); -select JSON_VALUE('{"hello":"world"}', '$.b') settings function_return_type_allow_nullable=true; +select JSON_VALUE('{"hello":"world"}', '$.b') settings function_json_value_return_type_allow_nullable=true; select JSON_VALUE('{"hello":{"world":"!"}}', '$.hello') settings function_json_value_return_type_allow_complex=true; ``` From b1fd28d0f4a2cec6d10b457628c657f544c8be3c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 26 Oct 2023 19:41:29 +0000 Subject: [PATCH 623/634] Try to stabilize test, pt. II --- .../01942_dateTimeToSnowflake.reference | 3 ++- .../0_stateless/01942_dateTimeToSnowflake.sql | 15 ++++++++------- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference b/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference index 4f51ec55e89..abc27beb5f7 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference @@ -3,4 +3,5 @@ 2021-08-15 18:57:56.492 1426860704886947840 1426860704886947840 -- different DateTime64 scales 1426981498778550272 1426981500456271872 1426981500833759232 1426981500842147840 1426981500842147840 -1 1 1 1 0 +1 1 1 1 +2023-11-11 11:11:11.1231 2023-11-11 11:11:11.123 diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql index 7a3f874b0ec..1090179bb67 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql @@ -35,10 +35,11 @@ SELECT dateTime64ToSnowflake(dt64_0), WITH now64(0, 'UTC') AS dt64_0, now64(1, 'UTC') AS dt64_1, now64(2, 'UTC') AS dt64_2, - now64(3, 'UTC') AS dt64_3, - now64(4, 'UTC') AS dt64_4 -SELECT snowflakeToDateTime64(dateTime64ToSnowflake(dt64_0)) == dt64_0, - snowflakeToDateTime64(dateTime64ToSnowflake(dt64_1)) == dt64_1, - snowflakeToDateTime64(dateTime64ToSnowflake(dt64_2)) == dt64_2, - snowflakeToDateTime64(dateTime64ToSnowflake(dt64_3)) == dt64_3, - snowflakeToDateTime64(dateTime64ToSnowflake(dt64_4)) == dt64_4; + now64(3, 'UTC') AS dt64_3 +SELECT snowflakeToDateTime64(dateTime64ToSnowflake(dt64_0), 'UTC') == dt64_0, + snowflakeToDateTime64(dateTime64ToSnowflake(dt64_1), 'UTC') == dt64_1, + snowflakeToDateTime64(dateTime64ToSnowflake(dt64_2), 'UTC') == dt64_2, + snowflakeToDateTime64(dateTime64ToSnowflake(dt64_3), 'UTC') == dt64_3; + +WITH toDateTime64('2023-11-11 11:11:11.1231', 4, 'UTC') AS dt64_4 +SELECT dt64_4, snowflakeToDateTime64(dateTime64ToSnowflake(dt64_4), 'UTC'); -- not idempotent From 35baf7792dd94c32fd29bf7ac46c7ad6bce5e144 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 26 Oct 2023 23:56:45 +0200 Subject: [PATCH 624/634] cleanup after rebase, add [[unlikely]] --- docs/en/operations/settings/settings.md | 12 ------------ src/Functions/DateTimeTransforms.h | 2 +- src/Functions/FunctionsConversion.h | 24 ++++++++++++------------ 3 files changed, 13 insertions(+), 25 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c9b9074124f..ccf290c8e20 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4794,15 +4794,3 @@ LIFETIME(MIN 0 MAX 3600) LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) SETTINGS(dictionary_use_async_executor=1, max_threads=8); ``` -## date_time_overflow_mode {#date_time_overflow_mode} - -Defines behavior on overflow when converting [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md). - -Possible values: - -- `ignore` — Silently ignore overflow. The result may or may not be saturated to fit the resulting data type values range. The behavior is different for different data types. -- `throw` — Throw an exception in case of conversion overflow. -- `saturate` — Silently saturate the result to fit the result type values range. - -Default value: `ignore`. - diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index d371e19e6ee..9737e69bf34 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -107,7 +107,7 @@ struct ToDateImpl } else if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (t < 0 || t > DATE_LUT_MAX_DAY_NUM) + if (t < 0 || t > DATE_LUT_MAX_DAY_NUM) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", t); } return static_cast(t); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 66d891230fb..27ce8cfb673 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -411,7 +411,7 @@ struct ToDateTimeImpl { if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (d > MAX_DATETIME_DAY_NUM) + if (d > MAX_DATETIME_DAY_NUM) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Day number {} is out of bounds of type DateTime", d); } else if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) @@ -433,7 +433,7 @@ struct ToDateTimeImpl } else if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (d < 0 || d > MAX_DATETIME_DAY_NUM) + if (d < 0 || d > MAX_DATETIME_DAY_NUM) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type DateTime", d); } return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))); @@ -454,7 +454,7 @@ struct ToDateTimeImpl { if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) return dt64 < 0 ? 0 : std::numeric_limits::max(); - else + else [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type DateTime", dt64); } else @@ -482,7 +482,7 @@ struct ToDateTransform32Or64 { if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from > MAX_DATETIME_TIMESTAMP) + if (from > MAX_DATETIME_TIMESTAMP) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); } /// if value is smaller (or equal) than maximum day value for Date, than treat it as day num, @@ -510,7 +510,7 @@ struct ToDateTransform32Or64Signed // TODO: decide narrow or extended range based on FromType if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from < 0 || from > MAX_DATE_TIMESTAMP) + if (from < 0 || from > MAX_DATE_TIMESTAMP) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); } else @@ -561,7 +561,7 @@ struct ToDate32Transform32Or64 { if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from > MAX_DATETIME64_TIMESTAMP) + if (from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type Date32", from); } return time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DATETIME64_TIMESTAMP))); @@ -580,7 +580,7 @@ struct ToDate32Transform32Or64Signed if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from < daynum_min_offset || from > MAX_DATETIME64_TIMESTAMP) + if (from < daynum_min_offset || from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type Date32", from); } @@ -688,7 +688,7 @@ struct ToDateTimeTransform64 { if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from > MAX_DATETIME_TIMESTAMP) + if (from > MAX_DATETIME_TIMESTAMP) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); } return static_cast(std::min(time_t(from), time_t(MAX_DATETIME_TIMESTAMP))); @@ -722,7 +722,7 @@ struct ToDateTimeTransform64Signed { if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from < 0 || from > MAX_DATETIME_TIMESTAMP) + if (from < 0 || from > MAX_DATETIME_TIMESTAMP) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); } @@ -779,7 +779,7 @@ struct ToDateTime64TransformUnsigned { if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from > MAX_DATETIME64_TIMESTAMP) + if (from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); else return DecimalUtils::decimalFromComponentsWithMultiplier(from, 0, scale_multiplier); @@ -803,7 +803,7 @@ struct ToDateTime64TransformSigned { if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from < MIN_DATETIME64_TIMESTAMP || from > MAX_DATETIME64_TIMESTAMP) + if (from < MIN_DATETIME64_TIMESTAMP || from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); } from = static_cast(std::max(from, MIN_DATETIME64_TIMESTAMP)); @@ -827,7 +827,7 @@ struct ToDateTime64TransformFloat { if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { - if (from < MIN_DATETIME64_TIMESTAMP || from > MAX_DATETIME64_TIMESTAMP) + if (from < MIN_DATETIME64_TIMESTAMP || from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); } From 2da12ec291df3a13c5ce5727c922411fa13c7974 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Fri, 27 Oct 2023 01:13:52 +0200 Subject: [PATCH 625/634] Update FunctionsConversion.h --- src/Functions/FunctionsConversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 27ce8cfb673..bda5fce1ac8 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -454,7 +454,7 @@ struct ToDateTimeImpl { if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) return dt64 < 0 ? 0 : std::numeric_limits::max(); - else [[unlikely]] + else throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type DateTime", dt64); } else From 308b2942d7a3168139a38f18e2283fa4596addc0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Fri, 27 Oct 2023 11:42:33 +0800 Subject: [PATCH 626/634] Update readFloatText.h --- src/IO/readFloatText.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/readFloatText.h b/src/IO/readFloatText.h index c0da9129a05..b0682576183 100644 --- a/src/IO/readFloatText.h +++ b/src/IO/readFloatText.h @@ -151,7 +151,6 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf) static constexpr int MAX_LENGTH = 316; ReadBufferFromMemory * buf_from_memory = dynamic_cast(&buf); - /// Fast path (avoid copying) if the buffer have at least MAX_LENGTH bytes or buf is ReadBufferFromMemory if (likely(!buf.eof() && (buf_from_memory || buf.position() + MAX_LENGTH <= buf.buffer().end()))) { From 489e6d9bdcbc28c7dea94fa4942a1298eb57706e Mon Sep 17 00:00:00 2001 From: lgbo Date: Fri, 27 Oct 2023 15:54:25 +0800 Subject: [PATCH 627/634] Optimization for getting value from map, `arrayElement`(1/2) (#55929) --- src/Functions/array/arrayElement.cpp | 47 +++++++++++++++++++++++++--- tests/performance/get_map_value.xml | 22 +++++++++++++ 2 files changed, 64 insertions(+), 5 deletions(-) create mode 100644 tests/performance/get_map_value.xml diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index e972a6846da..d7c29070c91 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -578,7 +578,6 @@ ColumnPtr FunctionArrayElement::executeString( return nullptr; auto col_res = ColumnString::create(); - ArrayElementStringImpl::vector( col_nested->getChars(), col_array->getOffsets(), @@ -819,11 +818,49 @@ void FunctionArrayElement::executeMatchKeyToIndex( const Offsets & offsets, PaddedPODArray & matched_idxs, const Matcher & matcher) { size_t rows = offsets.size(); - for (size_t i = 0; i < rows; ++i) + size_t expected_match_pos = 0; + bool matched = false; + if (!rows) + return; + + /// In practice, map keys are usually in the same order, it is worth a try to + /// predict the next key position. So it can avoid a lot of unnecessary comparisons. + for (size_t j = offsets[-1], end = offsets[0]; j < end; ++j) { - bool matched = false; - size_t begin = offsets[i - 1]; - size_t end = offsets[i]; + if (matcher.match(j, 0)) + { + matched_idxs.push_back(j - offsets[-1] + 1); + matched = true; + expected_match_pos = end + j - offsets[-1]; + break; + } + } + if (!matched) + { + expected_match_pos = offsets[0]; + matched_idxs.push_back(0); + } + size_t i = 1; + for (; i < rows; ++i) + { + const auto & begin = offsets[i - 1]; + const auto & end = offsets[i]; + if (expected_match_pos < end && matcher.match(expected_match_pos, i)) + { + auto map_key_index = expected_match_pos - begin; + matched_idxs.push_back(map_key_index + 1); + expected_match_pos = end + map_key_index; + } + else + break; + } + + // fallback to linear search + for (; i < rows; ++i) + { + matched = false; + const auto & begin = offsets[i - 1]; + const auto & end = offsets[i]; for (size_t j = begin; j < end; ++j) { if (matcher.match(j, i)) diff --git a/tests/performance/get_map_value.xml b/tests/performance/get_map_value.xml new file mode 100644 index 00000000000..c3e0d94f512 --- /dev/null +++ b/tests/performance/get_map_value.xml @@ -0,0 +1,22 @@ + + + CREATE TABLE test_table_map_1 (a Map(String, String)) ENGINE=Memory + + + insert into test_table_map_1 select CAST((['xyz', 'abc', '123'], [cast(number % 10 as String), 'dsdfsd', '123']), 'Map(String, String)') from numbers(10000000) + + select a['xyz'] from test_table_map_1 Format Null settings max_threads=1 + select a['123'] from test_table_map_1 Format Null settings max_threads=1 + + drop table test_table_map_1 + + + CREATE TABLE test_table_map_2 (a Map(String, String)) ENGINE=Memory + + + insert into test_table_map_2 select if (number % 2 = 0, CAST((['xyz', 'abc', '123'], [cast(number % 10 as String), 'dsdfsd', '123']), 'Map(String, String)'), CAST((['123', 'abc', 'xyz'], ['123', 'dsdfsd', cast(number % 10 as String)]), 'Map(String, String)')) from numbers(10000000) + + select a['xyz'] from test_table_map_2 Format Null settings max_threads=1 + + drop table test_table_map_2 + From f5890a5b4c13d19d300e1da6cc09760a16390b5c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 27 Oct 2023 10:53:59 +0200 Subject: [PATCH 628/634] Added healthcheck for LDAP (#55571) --- .../test/integration/runner/compose/docker_compose_ldap.yml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docker/test/integration/runner/compose/docker_compose_ldap.yml b/docker/test/integration/runner/compose/docker_compose_ldap.yml index 857f8fdce62..f199516f315 100644 --- a/docker/test/integration/runner/compose/docker_compose_ldap.yml +++ b/docker/test/integration/runner/compose/docker_compose_ldap.yml @@ -14,3 +14,8 @@ services: LDAP_PORT_NUMBER: ${LDAP_INTERNAL_PORT:-1389} ports: - ${LDAP_EXTERNAL_PORT:-1389}:${LDAP_INTERNAL_PORT:-1389} + healthcheck: + test: "ldapsearch -x -b dc=example,dc=org cn > /dev/null" + interval: 10s + retries: 10 + timeout: 2s From b817ab45bdb60cea8829d7a2aef4acd632d4edbe Mon Sep 17 00:00:00 2001 From: Rory Crispin Date: Fri, 27 Oct 2023 12:03:15 +0200 Subject: [PATCH 629/634] Fix dead link to Virtual Columns --- docs/en/engines/table-engines/special/merge.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/special/merge.md b/docs/en/engines/table-engines/special/merge.md index bd6149406a9..d8babff5d10 100644 --- a/docs/en/engines/table-engines/special/merge.md +++ b/docs/en/engines/table-engines/special/merge.md @@ -84,5 +84,5 @@ SELECT * FROM WatchLog; **See Also** -- [Virtual columns](../../../engines/table-engines/special/index.md#table_engines-virtual_columns) +- [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) - [merge](../../../sql-reference/table-functions/merge.md) table function From 18c50c11b3561671dc27b8f2c4abfccddd003135 Mon Sep 17 00:00:00 2001 From: frinkr Date: Fri, 27 Oct 2023 18:36:28 +0800 Subject: [PATCH 630/634] Multithreading after window functions (#50771) * feat: Preserve number of streams after evaluation the window functions to allow parallel stream processing * fix style * fix style * fix style * setting query_plan_preserve_num_streams_after_window_functions default true * fix tests by SETTINGS query_plan_preserve_num_streams_after_window_functions=0 * fix test references * Resize the streams after the last window function, to keep the order between WindowTransforms (and WindowTransform works on single stream anyway). * feat: Preserve number of streams after evaluation the window functions to allow parallel stream processing * fix style * fix style * fix style * setting query_plan_preserve_num_streams_after_window_functions default true * fix tests by SETTINGS query_plan_preserve_num_streams_after_window_functions=0 * fix test references * Resize the streams after the last window function, to keep the order between WindowTransforms (and WindowTransform works on single stream anyway). * add perf test * perf: change the dataset from 50M to 5M * rename query_plan_preserve_num_streams_after_window_functions -> query_plan_enable_multithreading_after_window_functions * update test reference * fix clang-tidy --------- Co-authored-by: Nikita Taranov --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 6 +- src/Planner/Planner.cpp | 6 +- src/Processors/QueryPlan/WindowStep.cpp | 17 +++-- src/Processors/QueryPlan/WindowStep.h | 4 +- ...ow_functions_downstream_multithreading.xml | 69 +++++++++++++++++++ ...568_window_functions_distributed.reference | 18 ++--- .../01568_window_functions_distributed.sql | 18 ++--- .../01571_window_functions.reference | 3 +- .../0_stateless/01571_window_functions.sql | 4 +- 10 files changed, 119 insertions(+), 27 deletions(-) create mode 100644 tests/performance/window_functions_downstream_multithreading.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5111f48ef6a..97006acd9bb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -681,6 +681,7 @@ class IColumn; M(Bool, query_plan_aggregation_in_order, true, "Use query plan for aggregation-in-order optimisation", 0) \ M(Bool, query_plan_remove_redundant_sorting, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries", 0) \ M(Bool, query_plan_remove_redundant_distinct, true, "Remove redundant Distinct step in query plan", 0) \ + M(Bool, query_plan_enable_multithreading_after_window_functions, true, "Enable multithreading after evaluating window functions to allow parallel stream processing", 0) \ M(UInt64, regexp_max_matches_per_row, 1000, "Max matches of any single regexp per row, used to safeguard 'extractAllGroupsHorizontal' against consuming too much memory with greedy RE.", 0) \ \ M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 3194e753d0d..3b0ba8b375f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2936,7 +2936,11 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan) query_plan.addStep(std::move(sorting_step)); } - auto window_step = std::make_unique(query_plan.getCurrentDataStream(), window, window.window_functions); + // Fan out streams only for the last window to preserve the ordering between windows, + // and WindowTransform works on single stream anyway. + const bool streams_fan_out = settings.query_plan_enable_multithreading_after_window_functions && ((i + 1) == windows_sorted.size()); + + auto window_step = std::make_unique(query_plan.getCurrentDataStream(), window, window.window_functions, streams_fan_out); window_step->setStepDescription("Window step for window '" + window.window_name + "'"); query_plan.addStep(std::move(window_step)); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 7341ee4f1ba..a97edbb6dc2 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -905,8 +905,12 @@ void addWindowSteps(QueryPlan & query_plan, query_plan.addStep(std::move(sorting_step)); } + // Fan out streams only for the last window to preserve the ordering between windows, + // and WindowTransform works on single stream anyway. + const bool streams_fan_out = settings.query_plan_enable_multithreading_after_window_functions && ((i + 1) == window_descriptions_size); + auto window_step - = std::make_unique(query_plan.getCurrentDataStream(), window_description, window_description.window_functions); + = std::make_unique(query_plan.getCurrentDataStream(), window_description, window_description.window_functions, streams_fan_out); window_step->setStepDescription("Window step for window '" + window_description.window_name + "'"); query_plan.addStep(std::move(window_step)); } diff --git a/src/Processors/QueryPlan/WindowStep.cpp b/src/Processors/QueryPlan/WindowStep.cpp index d313b210854..9c68a4b73d1 100644 --- a/src/Processors/QueryPlan/WindowStep.cpp +++ b/src/Processors/QueryPlan/WindowStep.cpp @@ -10,14 +10,14 @@ namespace DB { -static ITransformingStep::Traits getTraits() +static ITransformingStep::Traits getTraits(bool preserves_sorting) { return ITransformingStep::Traits { { .returns_single_stream = false, .preserves_number_of_streams = true, - .preserves_sorting = true, + .preserves_sorting = preserves_sorting, }, { .preserves_number_of_rows = true @@ -46,10 +46,12 @@ static Block addWindowFunctionResultColumns(const Block & block, WindowStep::WindowStep( const DataStream & input_stream_, const WindowDescription & window_description_, - const std::vector & window_functions_) - : ITransformingStep(input_stream_, addWindowFunctionResultColumns(input_stream_.header, window_functions_), getTraits()) + const std::vector & window_functions_, + bool streams_fan_out_) + : ITransformingStep(input_stream_, addWindowFunctionResultColumns(input_stream_.header, window_functions_), getTraits(!streams_fan_out_)) , window_description(window_description_) , window_functions(window_functions_) + , streams_fan_out(streams_fan_out_) { // We don't remove any columns, only add, so probably we don't have to update // the output DataStream::distinct_columns. @@ -60,6 +62,8 @@ WindowStep::WindowStep( void WindowStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { + auto num_threads = pipeline.getNumThreads(); + // This resize is needed for cases such as `over ()` when we don't have a // sort node, and the input might have multiple streams. The sort node would // have resized it. @@ -72,6 +76,11 @@ void WindowStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ input_streams.front().header, output_stream->header, window_description, window_functions); }); + if (streams_fan_out) + { + pipeline.resize(num_threads); + } + assertBlocksHaveEqualStructure(pipeline.getHeader(), output_stream->header, "WindowStep transform for '" + window_description.window_name + "'"); } diff --git a/src/Processors/QueryPlan/WindowStep.h b/src/Processors/QueryPlan/WindowStep.h index 0bafdcbe414..74a0e5930c7 100644 --- a/src/Processors/QueryPlan/WindowStep.h +++ b/src/Processors/QueryPlan/WindowStep.h @@ -16,7 +16,8 @@ class WindowStep : public ITransformingStep public: explicit WindowStep(const DataStream & input_stream_, const WindowDescription & window_description_, - const std::vector & window_functions_); + const std::vector & window_functions_, + bool streams_fan_out_); String getName() const override { return "Window"; } @@ -32,6 +33,7 @@ private: WindowDescription window_description; std::vector window_functions; + bool streams_fan_out; }; } diff --git a/tests/performance/window_functions_downstream_multithreading.xml b/tests/performance/window_functions_downstream_multithreading.xml new file mode 100644 index 00000000000..182936203b9 --- /dev/null +++ b/tests/performance/window_functions_downstream_multithreading.xml @@ -0,0 +1,69 @@ + + + CREATE TABLE + window_test(id Int64, value Int64, partition Int64, msg String) + Engine=MergeTree + ORDER BY id + + + + INSERT INTO window_test + SELECT number, rand(1) % 500, number % 3000, randomPrintableASCII(2) FROM numbers(5000000) + + + + SELECT id, + AVG(value) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS frame1, + MAX(value) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS frame2, + sipHash64(frame1), + sipHash64(frame2) + FROM window_test + + + + SELECT id AS key, + sipHash64(sum(frame)) AS value + FROM ( + SELECT id, + AVG(value) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS frame + FROM window_test) + GROUP BY key + ORDER BY key, value + + + + SELECT id % 100000 AS key, + sipHash64(sum(frame)) AS value + FROM ( + SELECT id, + AVG(value) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS frame + FROM window_test) + GROUP BY key + ORDER BY key, value + + + + WITH 'xxxxyyyyxxxxyyyyxxxxyyyyxxxxyyyy' AS cipherKey + SELECT id, + AVG(value) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS frame, + toString(frame) AS str, + encrypt('aes-256-ofb', str, cipherKey) AS enc, + decrypt('aes-256-ofb', str, cipherKey) AS dec + FROM window_test + + + + SELECT id, + AVG(value) OVER (PARTITION by partition ORDER BY id ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS frame + FROM window_test + ORDER BY id + + + + SELECT DISTINCT AVG(value) OVER (PARTITION by partition ORDER BY id ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS frame + FROM window_test + ORDER BY frame + + + DROP TABLE IF EXISTS window_test + diff --git a/tests/queries/0_stateless/01568_window_functions_distributed.reference b/tests/queries/0_stateless/01568_window_functions_distributed.reference index 0b439ef759a..13ac0769a24 100644 --- a/tests/queries/0_stateless/01568_window_functions_distributed.reference +++ b/tests/queries/0_stateless/01568_window_functions_distributed.reference @@ -1,18 +1,18 @@ -- { echo } -select row_number() over (order by dummy) from (select * from remote('127.0.0.{1,2}', system, one)); +select row_number() over (order by dummy) as x from (select * from remote('127.0.0.{1,2}', system, one)) order by x; 1 2 -select row_number() over (order by dummy) from remote('127.0.0.{1,2}', system, one); +select row_number() over (order by dummy) as x from remote('127.0.0.{1,2}', system, one) order by x; 1 2 -select max(identity(dummy + 1)) over () from remote('127.0.0.{1,2}', system, one); +select max(identity(dummy + 1)) over () as x from remote('127.0.0.{1,2}', system, one) order by x; 1 1 drop table if exists t_01568; create table t_01568 engine Memory as select intDiv(number, 3) p, modulo(number, 3) o, number from numbers(9); -select sum(number) over w, max(number) over w from t_01568 window w as (partition by p); +select sum(number) over w as x, max(number) over w as y from t_01568 window w as (partition by p) order by x, y; 3 2 3 2 3 2 @@ -22,7 +22,7 @@ select sum(number) over w, max(number) over w from t_01568 window w as (partitio 21 8 21 8 21 8 -select sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p); +select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y; 6 2 6 2 6 2 @@ -41,23 +41,23 @@ select sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t 42 8 42 8 42 8 -select distinct sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p); +select distinct sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y; 6 2 24 5 42 8 -- window functions + aggregation w/shards -select groupArray(groupArray(number)) over (rows unbounded preceding) from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3); +select groupArray(groupArray(number)) over (rows unbounded preceding) as x from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) order by x; [[0,3,6,0,3,6]] [[0,3,6,0,3,6],[1,4,7,1,4,7]] [[0,3,6,0,3,6],[1,4,7,1,4,7],[2,5,8,2,5,8]] -select groupArray(groupArray(number)) over (rows unbounded preceding) from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) settings distributed_group_by_no_merge=1; +select groupArray(groupArray(number)) over (rows unbounded preceding) as x from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) order by x settings distributed_group_by_no_merge=1; [[0,3,6]] [[0,3,6],[1,4,7]] [[0,3,6],[1,4,7],[2,5,8]] [[0,3,6]] [[0,3,6],[1,4,7]] [[0,3,6],[1,4,7],[2,5,8]] -select groupArray(groupArray(number)) over (rows unbounded preceding) from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) settings distributed_group_by_no_merge=2; -- { serverError 48 } +select groupArray(groupArray(number)) over (rows unbounded preceding) as x from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) order by x settings distributed_group_by_no_merge=2; -- { serverError 48 } -- proper ORDER BY w/window functions select p, o, count() over (partition by p) from remote('127.0.0.{1,2}', '', t_01568) diff --git a/tests/queries/0_stateless/01568_window_functions_distributed.sql b/tests/queries/0_stateless/01568_window_functions_distributed.sql index 140b02bfa6d..95072d6460f 100644 --- a/tests/queries/0_stateless/01568_window_functions_distributed.sql +++ b/tests/queries/0_stateless/01568_window_functions_distributed.sql @@ -1,11 +1,11 @@ -- Tags: distributed -- { echo } -select row_number() over (order by dummy) from (select * from remote('127.0.0.{1,2}', system, one)); +select row_number() over (order by dummy) as x from (select * from remote('127.0.0.{1,2}', system, one)) order by x; -select row_number() over (order by dummy) from remote('127.0.0.{1,2}', system, one); +select row_number() over (order by dummy) as x from remote('127.0.0.{1,2}', system, one) order by x; -select max(identity(dummy + 1)) over () from remote('127.0.0.{1,2}', system, one); +select max(identity(dummy + 1)) over () as x from remote('127.0.0.{1,2}', system, one) order by x; drop table if exists t_01568; @@ -13,16 +13,16 @@ create table t_01568 engine Memory as select intDiv(number, 3) p, modulo(number, 3) o, number from numbers(9); -select sum(number) over w, max(number) over w from t_01568 window w as (partition by p); +select sum(number) over w as x, max(number) over w as y from t_01568 window w as (partition by p) order by x, y; -select sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p); +select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y; -select distinct sum(number) over w, max(number) over w from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p); +select distinct sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y; -- window functions + aggregation w/shards -select groupArray(groupArray(number)) over (rows unbounded preceding) from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3); -select groupArray(groupArray(number)) over (rows unbounded preceding) from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) settings distributed_group_by_no_merge=1; -select groupArray(groupArray(number)) over (rows unbounded preceding) from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) settings distributed_group_by_no_merge=2; -- { serverError 48 } +select groupArray(groupArray(number)) over (rows unbounded preceding) as x from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) order by x; +select groupArray(groupArray(number)) over (rows unbounded preceding) as x from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) order by x settings distributed_group_by_no_merge=1; +select groupArray(groupArray(number)) over (rows unbounded preceding) as x from remote('127.0.0.{1,2}', '', t_01568) group by mod(number, 3) order by x settings distributed_group_by_no_merge=2; -- { serverError 48 } -- proper ORDER BY w/window functions select p, o, count() over (partition by p) diff --git a/tests/queries/0_stateless/01571_window_functions.reference b/tests/queries/0_stateless/01571_window_functions.reference index 420f7575a52..62741848958 100644 --- a/tests/queries/0_stateless/01571_window_functions.reference +++ b/tests/queries/0_stateless/01571_window_functions.reference @@ -19,7 +19,8 @@ system stop merges order_by_const; INSERT INTO order_by_const(a, b, c, d) VALUES (1, 1, 101, 1), (1, 2, 102, 1), (1, 3, 103, 1), (1, 4, 104, 1); INSERT INTO order_by_const(a, b, c, d) VALUES (1, 5, 104, 1), (1, 6, 105, 1), (2, 1, 106, 2), (2, 1, 107, 2); INSERT INTO order_by_const(a, b, c, d) VALUES (2, 2, 107, 2), (2, 3, 108, 2), (2, 4, 109, 2); -SELECT row_number() OVER (order by 1, a) FROM order_by_const; +-- output 1 sorted stream +SELECT row_number() OVER (order by 1, a) FROM order_by_const SETTINGS query_plan_enable_multithreading_after_window_functions=0; 1 2 3 diff --git a/tests/queries/0_stateless/01571_window_functions.sql b/tests/queries/0_stateless/01571_window_functions.sql index 4aaba19100a..4cad5c5c40b 100644 --- a/tests/queries/0_stateless/01571_window_functions.sql +++ b/tests/queries/0_stateless/01571_window_functions.sql @@ -20,7 +20,9 @@ system stop merges order_by_const; INSERT INTO order_by_const(a, b, c, d) VALUES (1, 1, 101, 1), (1, 2, 102, 1), (1, 3, 103, 1), (1, 4, 104, 1); INSERT INTO order_by_const(a, b, c, d) VALUES (1, 5, 104, 1), (1, 6, 105, 1), (2, 1, 106, 2), (2, 1, 107, 2); INSERT INTO order_by_const(a, b, c, d) VALUES (2, 2, 107, 2), (2, 3, 108, 2), (2, 4, 109, 2); -SELECT row_number() OVER (order by 1, a) FROM order_by_const; + +-- output 1 sorted stream +SELECT row_number() OVER (order by 1, a) FROM order_by_const SETTINGS query_plan_enable_multithreading_after_window_functions=0; drop table order_by_const; From 9a95b82caef6dc6c05f1767aac3adaae1664ec94 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 26 Oct 2023 18:44:21 +0200 Subject: [PATCH 631/634] Avoid traversing potentially big list of watches --- src/Coordination/KeeperStorage.cpp | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 1779415dd30..5534f4c3699 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -2496,25 +2496,15 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) co uint64_t KeeperStorage::getTotalWatchesCount() const { uint64_t ret = 0; - for (const auto & [path, subscribed_sessions] : watches) - ret += subscribed_sessions.size(); - - for (const auto & [path, subscribed_sessions] : list_watches) - ret += subscribed_sessions.size(); + for (const auto & [session, paths] : sessions_and_watchers) + ret += paths.size(); return ret; } uint64_t KeeperStorage::getSessionsWithWatchesCount() const { - std::unordered_set counter; - for (const auto & [path, subscribed_sessions] : watches) - counter.insert(subscribed_sessions.begin(), subscribed_sessions.end()); - - for (const auto & [path, subscribed_sessions] : list_watches) - counter.insert(subscribed_sessions.begin(), subscribed_sessions.end()); - - return counter.size(); + return sessions_and_watchers.size(); } uint64_t KeeperStorage::getTotalEphemeralNodesCount() const From f560e7ee52cd43717759ead2abf7e40d4b318dea Mon Sep 17 00:00:00 2001 From: AN Date: Fri, 27 Oct 2023 16:36:05 +0300 Subject: [PATCH 632/634] Update architecture.md (#56049) Minor fixes (typo found in comment pull/55985 by den-crane, punctuation style unification) --- docs/ru/development/architecture.md | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/ru/development/architecture.md b/docs/ru/development/architecture.md index 9c4a503a276..35741570702 100644 --- a/docs/ru/development/architecture.md +++ b/docs/ru/development/architecture.md @@ -43,7 +43,7 @@ ClickHouse — полноценная столбцовая СУБД. Данны ## Блоки (Block) {#block} -`Block` — это контейнер, который представляет фрагмент (chunk) таблицы в памяти. Это набор троек — `(IColumn, IDataType, имя столбца)`. В процессе выполнения запроса, данные обрабатываются блоками (`Block`). Если есть `Block`, значит у нас есть данные (в объекте `IColumn`), информация о типе (в `IDataType`), которая говорит, как работать со столбцов, и имя столбца (оригинальное имя столбца таблицы или служебное имя, присвоенное для получения промежуточных результатов вычислений). +`Block` — это контейнер, который представляет фрагмент (chunk) таблицы в памяти. Это набор троек — `(IColumn, IDataType, имя столбца)`. В процессе выполнения запроса, данные обрабатываются блоками (`Block`). Если есть `Block`, значит у нас есть данные (в объекте `IColumn`), информация о типе (в `IDataType`), которая говорит, как работать со столбцом, и имя столбца (оригинальное имя столбца таблицы или служебное имя, присвоенное для получения промежуточных результатов вычислений). При вычислении некоторой функции на столбцах в блоке добавляется ещё один столбец с результатами в блок, не трогая колонки аргументов функции, потому что операции иммутабельные. Позже ненужные столбцы могут быть удалены из блока, но не модифицированы. Это удобно для устранения общих подвыражений. @@ -67,15 +67,15 @@ ClickHouse — полноценная столбцовая СУБД. Данны ## Форматы {#formats} -Форматы данных реализуются с помощью потоков блоков. Есть форматы представления (presentational), пригодные только для вывода данных клиенту, такие как `Pretty` формат, который предоставляет только `IBlockOutputStream`. И есть форматы ввода/вывода, такие как `TabSeparated` или `JSONEachRow`. +Форматы данных реализуются с помощью потоков блоков. Есть форматы представления (presentational), пригодные только для вывода данных клиенту, такие как `Pretty`-формат, который предоставляет только `IBlockOutputStream`. И есть форматы ввода-вывода, такие как `TabSeparated` или `JSONEachRow`. -Существуют также потоки строк: `IRowInputStream` и `IRowOutputStream`. Они позволяют вытягивать/выталкивать данные отдельными строками, а не блоками. Они нужны только для упрощения реализации ориентированных на строки форматов. Обертка `BlockInputStreamFromRowInputStream` и `BlockOutputStreamFromRowOutputStream` позволяет конвертировать потоки, ориентированные на строки, в обычные потоки, ориентированные на блоки. +Существуют также потоки строк: `IRowInputStream` и `IRowOutputStream`. Они позволяют вытягивать и выталкивать данные отдельными строками, а не блоками. Они нужны только для упрощения реализации ориентированных на строки форматов. Обертка `BlockInputStreamFromRowInputStream` и `BlockOutputStreamFromRowOutputStream` позволяет конвертировать потоки, ориентированные на строки, в обычные потоки, ориентированные на блоки. ## I/O {#io} Для байт-ориентированного ввода-вывода существуют абстрактные классы `ReadBuffer` и `WriteBuffer`. Они используются вместо `iostream`. Не волнуйтесь: каждый зрелый проект C++ использует что-то другое вместо `iostream` по уважительным причинам. -`ReadBuffer` и `WriteBuffer` это просто непрерывный буфер и курсор, указывающий на позицию в этом буфере. Реализации могут как владеть так и не владеть памятью буфера. Существует виртуальный метод заполнения буфера следующими данными (для `ReadBuffer`) или сброса буфера куда-нибудь (например `WriteBuffer`). Виртуальные методы редко вызываются. +`ReadBuffer` и `WriteBuffer` — это просто непрерывный буфер и курсор, указывающий на позицию в этом буфере. Реализации могут как владеть так и не владеть памятью буфера. Существует виртуальный метод заполнения буфера следующими данными (для `ReadBuffer`) или сброса буфера куда-нибудь (например `WriteBuffer`). Виртуальные методы редко вызываются. Реализации `ReadBuffer`/`WriteBuffer` используются для работы с файлами и файловыми дескрипторами, а также сетевыми сокетами, для реализации сжатия (`CompressedWriteBuffer` инициализируется вместе с другим `WriteBuffer` и осуществляет сжатие данных перед записью в него), и для других целей – названия `ConcatReadBuffer`, `LimitReadBuffer`, и `HashingWriteBuffer` говорят сами за себя. @@ -87,7 +87,7 @@ ClickHouse — полноценная столбцовая СУБД. Данны Интерфейс `IStorage` служит для отображения таблицы. Различные движки таблиц являются реализациями этого интерфейса. Примеры `StorageMergeTree`, `StorageMemory` и так далее. Экземпляры этих классов являются просто таблицами. -Ключевые методы `IStorage` это `read` и `write`. Есть и другие варианты - `alter`, `rename`, `drop` и так далее. Метод `read` принимает следующие аргументы: набор столбцов для чтения из таблицы, `AST` запрос и желаемое количество потоков для вывода. Он возвращает один или несколько объектов `IBlockInputStream` и информацию о стадии обработки данных, которая была завершена внутри табличного движка во время выполнения запроса. +Ключевые методы `IStorage` это `read` и `write`. Есть и другие варианты — `alter`, `rename`, `drop` и так далее. Метод `read` принимает следующие аргументы: набор столбцов для чтения из таблицы, `AST` запрос и желаемое количество потоков для вывода. Он возвращает один или несколько объектов `IBlockInputStream` и информацию о стадии обработки данных, которая была завершена внутри табличного движка во время выполнения запроса. В большинстве случаев метод read отвечает только за чтение указанных столбцов из таблицы, а не за дальнейшую обработку данных. Вся дальнейшая обработка данных осуществляется интерпретатором запросов и не входит в сферу ответственности `IStorage`. @@ -112,7 +112,7 @@ ClickHouse — полноценная столбцовая СУБД. Данны ## Интерпретаторы {#interpreters} -Интерпретаторы отвечают за создание конвейера выполнения запроса из `AST`. Есть простые интерпретаторы, такие как `InterpreterExistsQuery` и `InterpreterDropQuery` или более сложный `InterpreterSelectQuery`. Конвейер выполнения запроса представляет собой комбинацию входных и выходных потоков блоков. Например, результатом интерпретации `SELECT` запроса является `IBlockInputStream` для чтения результирующего набора данных; результат интерпретации `INSERT` запроса - это `IBlockOutputStream`, для записи данных, предназначенных для вставки; результат интерпретации `INSERT SELECT` запроса - это `IBlockInputStream`, который возвращает пустой результирующий набор при первом чтении, но копирует данные из `SELECT` к `INSERT`. +Интерпретаторы отвечают за создание конвейера выполнения запроса из `AST`. Есть простые интерпретаторы, такие как `InterpreterExistsQuery` и `InterpreterDropQuery` или более сложный `InterpreterSelectQuery`. Конвейер выполнения запроса представляет собой комбинацию входных и выходных потоков блоков. Например, результатом интерпретации `SELECT` запроса является `IBlockInputStream` для чтения результирующего набора данных; результат интерпретации `INSERT` запроса — это `IBlockOutputStream`, для записи данных, предназначенных для вставки; результат интерпретации `INSERT SELECT` запроса — это `IBlockInputStream`, который возвращает пустой результирующий набор при первом чтении, но копирует данные из `SELECT` к `INSERT`. `InterpreterSelectQuery` использует `ExpressionAnalyzer` и `ExpressionActions` механизмы для анализа запросов и преобразований. Именно здесь выполняется большинство оптимизаций запросов на основе правил. `ExpressionAnalyzer` написан довольно грязно и должен быть переписан: различные преобразования запросов и оптимизации должны быть извлечены в отдельные классы, чтобы позволить модульные преобразования или запросы. @@ -120,9 +120,9 @@ ClickHouse — полноценная столбцовая СУБД. Данны Существуют обычные функции и агрегатные функции. Агрегатные функции смотрите в следующем разделе. -Обычные функции не изменяют число строк и работают так, как если бы обрабатывали каждую строку независимо. В действительности же, функции вызываются не к отдельным строкам, а блокам данных для реализации векторизованного выполнения запросов. +Обычные функции не изменяют число строк и работают так, как если бы обрабатывали каждую строку независимо. В действительности же функции вызываются не к отдельным строкам, а блокам данных для реализации векторизованного выполнения запросов. -Некоторые функции, такие как [blockSize](../sql-reference/functions/other-functions.md#function-blocksize), [rowNumberInBlock](../sql-reference/functions/other-functions.md#function-rownumberinblock), и [runningAccumulate](../sql-reference/functions/other-functions.md#runningaccumulate), эксплуатируют блочную обработку и нарушают независимость строк. +Некоторые функции, такие как [blockSize](../sql-reference/functions/other-functions.md#function-blocksize), [rowNumberInBlock](../sql-reference/functions/other-functions.md#function-rownumberinblock), и [runningAccumulate](../sql-reference/functions/other-functions.md#runningaccumulate), используют блочную обработку и нарушают независимость строк. ClickHouse имеет сильную типизацию, поэтому нет никакого неявного преобразования типов. Если функция не поддерживает определенную комбинацию типов, она создает исключение. Но функции могут работать (перегружаться) для многих различных комбинаций типов. Например, функция `plus` (для реализации `+` оператор) работает для любой комбинации числовых типов: `UInt8` + `Float32`, `UInt16` + `Int8` и так далее. Кроме того, некоторые вариадические функции, такие как `concat`, могут принимать любое количество аргументов. @@ -161,23 +161,23 @@ ClickHouse имеет сильную типизацию, поэтому нет ::: ## Выполнение распределенных запросов (Distributed Query Execution) {#distributed-query-execution} -Сервера в кластере в основном независимы. Вы можете создать `Распределенную` (`Distributed`) таблицу на одном или всех серверах в кластере. Такая таблица сама по себе не хранит данные - она только предоставляет возможность "просмотра" всех локальных таблиц на нескольких узлах кластера. При выполнении `SELECT` распределенная таблица переписывает запрос, выбирает удаленные узлы в соответствии с настройками балансировки нагрузки и отправляет им запрос. Распределенная таблица просит удаленные сервера обработать запрос до той стадии, когда промежуточные результаты с разных серверов могут быть объединены. Затем он получает промежуточные результаты и объединяет их. Распределенная таблица пытается возложить как можно больше работы на удаленные серверы и сократить объем промежуточных данных, передаваемых по сети. +Сервера в кластере в основном независимы. Вы можете создать `распределённую` (`Distributed`) таблицу на одном или всех серверах в кластере. Такая таблица сама по себе не хранит данные — она только предоставляет возможность “просмотра” всех локальных таблиц на нескольких узлах кластера. При выполнении `SELECT` распределенная таблица переписывает запрос, выбирает удаленные узлы в соответствии с настройками балансировки нагрузки и отправляет им запрос. Распределенная таблица просит удаленные сервера обработать запрос до той стадии, когда промежуточные результаты с разных серверов могут быть объединены. Затем он получает промежуточные результаты и объединяет их. Распределенная таблица пытается возложить как можно больше работы на удаленные серверы и сократить объем промежуточных данных, передаваемых по сети. Ситуация усложняется при использовании подзапросов в случае `IN` или `JOIN`, когда каждый из них использует таблицу `Distributed`. Есть разные стратегии для выполнения таких запросов. -Глобального плана выполнения распределенных запросов не существует. Каждый узел имеет собственный локальный план для своей части работы. У нас есть простое однонаправленное выполнение распределенных запросов: мы отправляем запросы на удаленные узлы и затем объединяем результаты. Но это невозможно для сложных запросов `GROUP BY` высокой кардинальности или запросов с большим числом временных данных в `JOIN`: в таких случаях нам необходимо перераспределить («reshuffle») данные между серверами, что требует дополнительной координации. ClickHouse не поддерживает выполнение запросов такого рода, и нам нужно работать над этим. +Глобального плана выполнения распределённых запросов не существует. Каждый узел имеет собственный локальный план для своей части работы. У нас есть простое однонаправленное выполнение распределенных запросов: мы отправляем запросы на удаленные узлы и затем объединяем результаты. Но это невозможно для сложных запросов `GROUP BY` высокой кардинальности или запросов с большим числом временных данных в `JOIN`: в таких случаях нам необходимо перераспределить (“reshuffle”) данные между узлами, что требует дополнительной координации. ClickHouse не поддерживает выполнение запросов такого рода, и нам нужно работать над этим. ## Merge Tree {#merge-tree} -`MergeTree` — это семейство движков хранения, поддерживающих индексацию по первичному ключу. Первичный ключ может быть произвольным набором (кортежем) столбцов или выражений. Данные в таблице `MergeTree` хранятся "частями" (“parts”). Каждая часть хранит данные отсортированные по первичному ключу (данные упорядочены лексикографически). Все столбцы таблицы хранятся в отдельных файлах `column.bin` в этих частях. Файлы состоят из сжатых блоков. Каждый блок обычно содержит от 64 КБ до 1 МБ несжатых данных, в зависимости от среднего значения размера данных. Блоки состоят из значений столбцов, расположенных последовательно один за другим. Значения столбцов находятся в одинаковом порядке для каждого столбца (порядок определяется первичным ключом), поэтому, когда вы выполняете итерацию по многим столбцам, вы получаете значения для соответствующих строк. +`MergeTree` — это семейство движков хранения, поддерживающих индексацию по первичному ключу. Первичный ключ может быть произвольным набором (кортежем) столбцов или выражений. Данные в таблице `MergeTree` хранятся “частями” (“parts”). Каждая часть хранит данные отсортированные по первичному ключу (данные упорядочены лексикографически). Все столбцы таблицы хранятся в отдельных файлах `column.bin` в этих частях. Файлы состоят из сжатых блоков. Каждый блок обычно содержит от 64 КБ до 1 МБ несжатых данных, в зависимости от среднего значения размера данных. Блоки состоят из значений столбцов, расположенных последовательно один за другим. Значения столбцов находятся в одинаковом порядке для каждого столбца (порядок определяется первичным ключом), поэтому, когда вы выполняете итерацию по многим столбцам, вы получаете значения для соответствующих строк. -Сам первичный ключ является “разреженным” (sparse). Он не относится к каждой отдельной строке, а только к некоторым диапазонам данных. Отдельный файл «primary.idx» имеет значение первичного ключа для каждой N-й строки, где N называется гранулярностью индекса (index_granularity, обычно N = 8192). Также для каждого столбца у нас есть файлы `column.mrk` с "метками" ("marks"), которые обозначают смещение для каждой N-й строки в файле данных. Каждая метка представляет собой пару: смещение начала сжатого блока от начала файла и смещение к началу данных в распакованном блоке. Обычно сжатые блоки выравниваются по меткам, а смещение в распакованном блоке равно нулю. Данные для `primary.idx` всегда находятся в памяти, а данные для файлов `column.mrk` кэшируются. +Сам первичный ключ является “разреженным” (sparse). Он не относится к каждой отдельной строке, а только к некоторым диапазонам данных. Отдельный файл «primary.idx» имеет значение первичного ключа для каждой N-й строки, где N называется гранулярностью индекса (index_granularity, обычно N = 8192). Также для каждого столбца у нас есть файлы `column.mrk` с “метками” (“marks”), которые обозначают смещение для каждой N-й строки в файле данных. Каждая метка представляет собой пару: смещение начала сжатого блока от начала файла и смещение к началу данных в распакованном блоке. Обычно сжатые блоки выравниваются по меткам, а смещение в распакованном блоке равно нулю. Данные для `primary.idx` всегда находятся в памяти, а данные для файлов `column.mrk` кэшируются. Когда мы собираемся читать что-то из части данных `MergeTree`, мы смотрим содержимое `primary.idx` и определяем диапазоны, которые могут содержать запрошенные данные, затем просматриваем содержимое `column.mrk` и вычисляем смещение, чтобы начать чтение этих диапазонов. Из-за разреженности могут быть прочитаны лишние данные. ClickHouse не подходит для простых точечных запросов высокой интенсивности, потому что весь диапазон строк размером `index_granularity` должен быть прочитан для каждого ключа, а сжатый блок должен быть полностью распакован для каждого столбца. Мы сделали индекс разреженным, потому что мы должны иметь возможность поддерживать триллионы строк на один сервер без существенных расходов памяти на индексацию. Кроме того, поскольку первичный ключ является разреженным, он не уникален: он не может проверить наличие ключа в таблице во время INSERT. Вы можете иметь множество строк с одним и тем же ключом в таблице. При выполнении `INSERT` для группы данных в `MergeTree`, элементы группы сортируются по первичному ключу и образует новую “часть”. Фоновые потоки периодически выбирают некоторые части и объединяют их в одну отсортированную часть, чтобы сохранить относительно небольшое количество частей. Вот почему он называется `MergeTree`. Конечно, объединение приводит к повышению интенсивности записи. Все части иммутабельные: они только создаются и удаляются, но не изменяются. Когда выполняется `SELECT`, он содержит снимок таблицы (набор частей). После объединения старые части также сохраняются в течение некоторого времени, чтобы упростить восстановление после сбоя, поэтому, если мы видим, что какая-то объединенная часть, вероятно, повреждена, мы можем заменить ее исходными частями. -`MergeTree` не является LSM (Log-structured merge-tree — журнально-структурированным деревом со слиянием), потому что оно не содержит «memtable» и «log»: вставленные данные записываются непосредственно в файловую систему. Это делает его пригодным только для вставки данных в пакетах, а не по отдельным строкам и не очень часто — примерно раз в секунду это нормально, а тысячу раз в секунду - нет. Мы сделали это для простоты и потому, что мы уже вставляем данные в пакеты в наших приложениях. +`MergeTree` не является LSM (Log-structured merge-tree — журнально-структурированным деревом со слиянием), потому что оно не содержит «memtable» и «log»: вставленные данные записываются непосредственно в файловую систему. Это делает его пригодным только для вставки данных в пакетах, а не по отдельным строкам и не очень часто — примерно раз в секунду это нормально, а тысячу раз в секунду — нет. Мы сделали это для простоты и потому, что мы уже вставляем данные в пакеты в наших приложениях. > Таблицы `MergeTree` могут иметь только один (первичный) индекс: вторичных индексов нет. Было бы неплохо разрешить несколько физических представлениям в одной логической таблице, например, хранить данные в более чем одном физическом порядке или даже разрешить представления с предварительно агрегированными данными вместе с исходными данными. @@ -191,7 +191,7 @@ ClickHouse имеет сильную типизацию, поэтому нет Репликация использует асинхронную multi-master-схему. Вы можете вставить данные в любую реплику, которая имеет открытую сессию в `ZooKeeper`, и данные реплицируются на все другие реплики асинхронно. Поскольку ClickHouse не поддерживает UPDATE, репликация исключает конфликты (conflict-free replication). Поскольку подтверждение вставок кворумом не реализовано, только что вставленные данные могут быть потеряны в случае сбоя одного узла. -Метаданные для репликации хранятся в `ZooKeeper`. Существует журнал репликации, в котором перечислены действия, которые необходимо выполнить. Среди этих действий: получить часть (get the part); объединить части (merge parts); удалить партицию (drop a partition) и так далее. Каждая реплика копирует журнал репликации в свою очередь, а затем выполняет действия из очереди. Например, при вставке в журнале создается действие «получить часть» (get the part), и каждая реплика загружает эту часть. Слияния координируются между репликами, чтобы получить идентичные до байта результаты. Все части объединяются одинаково на всех репликах. Одна из реплик-лидеров инициирует новое слияние кусков первой и записывает действия «слияния частей» в журнал. Несколько реплик (или все) могут быть лидерами одновременно. Реплике можно запретить быть лидером с помощью `merge_tree` настройки `replicated_can_become_leader`. +Метаданные для репликации хранятся в `ZooKeeper`. Существует журнал репликации, в котором перечислены действия, которые необходимо выполнить. Среди этих действий: получить часть (get the part); объединить части (merge parts); удалить партицию (drop a partition) и так далее. Каждая реплика копирует журнал репликации в свою очередь, а затем выполняет действия из очереди. Например, при вставке в журнале создается действие “получить часть” (get the part), и каждая реплика загружает эту часть. Слияния координируются между репликами, чтобы получить идентичные до байта результаты. Все части объединяются одинаково на всех репликах. Одна из реплик-лидеров инициирует новое слияние кусков первой и записывает действия “слияния частей” в журнал. Несколько реплик (или все) могут быть лидерами одновременно. Реплике можно запретить быть лидером с помощью `merge_tree` настройки `replicated_can_become_leader`. Репликация является физической: между узлами передаются только сжатые части, а не запросы. Слияния обрабатываются на каждой реплике независимо, в большинстве случаев, чтобы снизить затраты на сеть, во избежание усиления роли сети. Крупные объединенные части отправляются по сети только в случае значительной задержки репликации. From c634c43662d0d8dd16195966d1b094ce0c0e797d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 27 Oct 2023 16:47:06 +0200 Subject: [PATCH 633/634] Update src/Analyzer/Passes/QueryAnalysisPass.cpp Co-authored-by: Nikita Mikhaylov --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index aeb8c3f84e3..c6fbd728b8f 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -970,7 +970,7 @@ private: if (!node->hasAlias()) return; - // We don't should not resolve expressions to WindowNode + // We should not resolve expressions to WindowNode if (node->getNodeType() == QueryTreeNodeType::WINDOW) return; From b65c498016403f78e5ad83cf066e21b838e55495 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 27 Oct 2023 10:06:43 -0700 Subject: [PATCH 634/634] Fix 'Block structure mismatch' on concurrent ALTER and INSERTs in Buffer table (#55995) --- src/Storages/StorageBuffer.cpp | 19 ++++++----- src/Storages/StorageBuffer.h | 12 +++++++ .../02900_buffer_table_alter_race.reference | 10 ++++++ .../02900_buffer_table_alter_race.sh | 34 +++++++++++++++++++ 4 files changed, 67 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02900_buffer_table_alter_race.reference create mode 100755 tests/queries/0_stateless/02900_buffer_table_alter_race.sh diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index e011565edc1..f3030dadc59 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -165,7 +165,8 @@ public: : ISource(storage_snapshot->getSampleBlockForColumns(column_names_)) , column_names_and_types(storage_snapshot->getColumnsByNames( GetColumnsOptions(GetColumnsOptions::All).withSubcolumns(), column_names_)) - , buffer(buffer_) {} + , buffer(buffer_) + , metadata_version(storage_snapshot->metadata->metadata_version) {} String getName() const override { return "Buffer"; } @@ -180,7 +181,7 @@ protected: std::unique_lock lock(buffer.lockForReading()); - if (!buffer.data.rows()) + if (!buffer.data.rows() || buffer.metadata_version != metadata_version) return res; Columns columns; @@ -198,6 +199,7 @@ protected: private: NamesAndTypesList column_names_and_types; StorageBuffer::Buffer & buffer; + int32_t metadata_version; bool has_been_read = false; }; @@ -615,7 +617,7 @@ public: least_busy_buffer = &storage.buffers[start_shard_num]; least_busy_lock = least_busy_buffer->lockForWriting(); } - insertIntoBuffer(block, *least_busy_buffer); + insertIntoBuffer(block, *least_busy_buffer, metadata_snapshot->metadata_version); least_busy_lock.unlock(); storage.reschedule(); @@ -624,14 +626,15 @@ private: StorageBuffer & storage; StorageMetadataPtr metadata_snapshot; - void insertIntoBuffer(const Block & block, StorageBuffer::Buffer & buffer) + void insertIntoBuffer(const Block & block, StorageBuffer::Buffer & buffer, int32_t metadata_version) { time_t current_time = time(nullptr); /// Sort the columns in the block. This is necessary to make it easier to concatenate the blocks later. Block sorted_block = block.sortColumns(); - if (storage.checkThresholds(buffer, /* direct= */true, current_time, sorted_block.rows(), sorted_block.bytes())) + if (storage.checkThresholds(buffer, /* direct= */true, current_time, sorted_block.rows(), sorted_block.bytes()) || + buffer.metadata_version != metadata_version) { /** If, after inserting the buffer, the constraints are exceeded, then we will reset the buffer. * This also protects against unlimited consumption of RAM, since if it is impossible to write to the table, @@ -639,6 +642,7 @@ private: */ storage.flushBuffer(buffer, false /* check_thresholds */, true /* locked */); + buffer.metadata_version = metadata_version; } if (!buffer.first_write_time) @@ -1062,13 +1066,12 @@ void StorageBuffer::alter(const AlterCommands & params, ContextPtr local_context checkAlterIsPossible(params, local_context); auto metadata_snapshot = getInMemoryMetadataPtr(); - /// Flush all buffers to storages, so that no non-empty blocks of the old - /// structure remain. Structure of empty blocks will be updated during first - /// insert. + /// Flush buffers to the storage because BufferSource skips buffers with old metadata_version. optimize({} /*query*/, metadata_snapshot, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, {}, false /*cleanup*/, local_context); StorageInMemoryMetadata new_metadata = *metadata_snapshot; params.apply(new_metadata, local_context); + new_metadata.metadata_version += 1; DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); setInMemoryMetadata(new_metadata); } diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index db3cde93be5..94873ea04ce 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -128,6 +128,18 @@ private: time_t first_write_time = 0; Block data; + /// Schema version, checked to avoid mixing blocks with different sets of columns, from + /// before and after an ALTER. There are some remaining mild problems if an ALTER happens + /// in the middle of a long-running INSERT: + /// * The data produced by the INSERT after the ALTER is not visible to SELECTs until flushed. + /// That's because BufferSource skips buffers with old metadata_version instead of converting + /// them to the latest schema, for simplicity. + /// * If there are concurrent INSERTs, some of which started before the ALTER and some started + /// after, then the buffer's metadata_version will oscillate back and forth between the two + /// schemas, flushing the buffer each time. This is probably fine because long-running INSERTs + /// usually don't produce lots of small blocks. + int32_t metadata_version = 0; + std::unique_lock lockForReading() const; std::unique_lock lockForWriting() const; std::unique_lock tryLock() const; diff --git a/tests/queries/0_stateless/02900_buffer_table_alter_race.reference b/tests/queries/0_stateless/02900_buffer_table_alter_race.reference new file mode 100644 index 00000000000..7b4a251dfed --- /dev/null +++ b/tests/queries/0_stateless/02900_buffer_table_alter_race.reference @@ -0,0 +1,10 @@ +0 +1 +2 bobr +0 +1 +1 +1 +1 +1 +2 bobr diff --git a/tests/queries/0_stateless/02900_buffer_table_alter_race.sh b/tests/queries/0_stateless/02900_buffer_table_alter_race.sh new file mode 100755 index 00000000000..62865e4a4cd --- /dev/null +++ b/tests/queries/0_stateless/02900_buffer_table_alter_race.sh @@ -0,0 +1,34 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT -q "drop table if exists 02900_buffer" +$CLICKHOUSE_CLIENT -q "drop table if exists 02900_destination" + +$CLICKHOUSE_CLIENT -q "create table 02900_destination (k Int8, v String) engine Memory" +$CLICKHOUSE_CLIENT -q "create table 02900_buffer (k Int8) engine Buffer(currentDatabase(), '02900_destination', 1, 1000, 1000, 10000, 10000, 1000000, 1000000)" + +$CLICKHOUSE_CLIENT -q "insert into 02900_buffer (k) select 0" + +# Start a long-running INSERT that uses the old schema. +$CLICKHOUSE_CLIENT -q "insert into 02900_buffer (k) select sleepEachRow(1)+1 from numbers(5) settings max_block_size=1, max_insert_block_size=1, min_insert_block_size_rows=0, min_insert_block_size_bytes=0" & + +sleep 1 + +$CLICKHOUSE_CLIENT -q "alter table 02900_buffer add column v String" + +$CLICKHOUSE_CLIENT -q "insert into 02900_buffer (k, v) select 2, 'bobr'" + +wait + +# The data produced by the long-running INSERT after the ALTER is not visible until flushed. +$CLICKHOUSE_CLIENT -q "select k, any(v) from 02900_buffer group by k order by k" + +$CLICKHOUSE_CLIENT -q "optimize table 02900_buffer" +$CLICKHOUSE_CLIENT -q "select * from 02900_buffer order by k" + +$CLICKHOUSE_CLIENT -q "drop table 02900_buffer" +$CLICKHOUSE_CLIENT -q "drop table 02900_destination"