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/202] 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/202] 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 cffc1e058cbbfdff965d8089a25fc726947fbf3b Mon Sep 17 00:00:00 2001 From: Alexander Nikolaev Date: Thu, 8 Jun 2023 19:51:50 +0300 Subject: [PATCH 003/202] 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 004/202] 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 adfdba4b83eee834beb69d4c0323128ca27db77c Mon Sep 17 00:00:00 2001 From: helifu Date: Thu, 3 Aug 2023 14:33:33 +0800 Subject: [PATCH 005/202] 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 006/202] 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 007/202] 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 008/202] 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 009/202] 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 010/202] 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 483184cbdbd4b415568e215fcd848074e4ca1adf Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 7 Sep 2023 16:02:11 +0000 Subject: [PATCH 011/202] 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 434e2d4b57d8fb428f4764f857885aca50bc70be Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Sep 2023 18:57:41 +0000 Subject: [PATCH 012/202] 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 013/202] 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 014/202] 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 d4027d835e0d12afcd751ceb82d7636a3f5c2069 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Sep 2023 18:40:05 +0000 Subject: [PATCH 015/202] 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 016/202] 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 f81f0b6e3d96843205c1422cb0902f6e8bfb87c5 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 20 Sep 2023 17:42:25 +0800 Subject: [PATCH 017/202] 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 018/202] 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 f27a76b92317c2f44ac863135d8fcf02cb8ced42 Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Sun, 24 Sep 2023 20:58:00 +0100 Subject: [PATCH 019/202] 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 5da03e8ba5c8812050e595171188f6383f46b647 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 27 Sep 2023 10:46:39 +0800 Subject: [PATCH 020/202] 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 021/202] 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 a7f389a7601021137ed9cf32290935ff5ecfedba Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 29 Sep 2023 15:14:25 +0200 Subject: [PATCH 022/202] 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 c64e5e282c3e8d9fc91acb87d63955677046875b Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 29 Sep 2023 17:44:32 +0000 Subject: [PATCH 023/202] 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 46fbe00c9a32c828078b304f0f4a8e23c3d6fc3f Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Mon, 2 Oct 2023 18:50:56 +0100 Subject: [PATCH 024/202] 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 540d966033515e3365837675e46600473ded285b Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Mon, 2 Oct 2023 22:51:58 +0100 Subject: [PATCH 025/202] 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 406dfe277c18b9c7b18dc1ef0011e0d627266699 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 5 Oct 2023 11:58:00 +0200 Subject: [PATCH 026/202] 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 027/202] 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 028/202] 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 302291e17b39aa443882975b3851e321b0879415 Mon Sep 17 00:00:00 2001 From: Priyansh Agrawal Date: Sat, 7 Oct 2023 23:05:07 +0000 Subject: [PATCH 029/202] 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 030/202] 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 031/202] 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 032/202] 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 033/202] 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 034/202] 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 035/202] 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 036/202] 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 31ae84604f2ea987523c24fc526418a5a543378b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 10 Oct 2023 09:54:00 +0000 Subject: [PATCH 037/202] 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 b107712e0cf8e2671da7c1641b268df57e54bd7a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 9 Oct 2023 16:08:22 +0200 Subject: [PATCH 038/202] 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 96e3a4f9b441628f1ee6b9a226d9b15c4f49ef8a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 10 Oct 2023 23:45:23 +0200 Subject: [PATCH 039/202] 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 040/202] 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 3df0e77ea3d5f03823f6f4e4b2a42296eb4afcb5 Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Wed, 11 Oct 2023 10:28:28 +0800 Subject: [PATCH 041/202] 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 042/202] 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 043/202] 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 18a6e624c9f05934e928141840995d7955b88508 Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Wed, 11 Oct 2023 17:24:08 +0800 Subject: [PATCH 044/202] 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 045/202] 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 046/202] 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 047/202] 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 048/202] 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 049/202] [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 0859fc8de9e553be37cbfcd53def1ccad79e0c0e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 11 Oct 2023 14:02:35 +0300 Subject: [PATCH 050/202] 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 f7e64ff20098955da9edc22168a3861bfc6a1e90 Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Wed, 11 Oct 2023 20:45:38 +0800 Subject: [PATCH 051/202] 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 f9033bdf319a6086ebce463879d116cc7ec5555b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 11 Oct 2023 13:17:12 +0300 Subject: [PATCH 052/202] 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 053/202] 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 054/202] 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 055/202] 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 47ff13056f0a6dbff8c61389f840f4c55216b8a8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 11 Oct 2023 14:18:23 +0200 Subject: [PATCH 056/202] 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 8b47bf5e7ef41d2c4523f27901aa35a2b53cb679 Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Thu, 12 Oct 2023 09:58:42 +0800 Subject: [PATCH 057/202] 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 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 058/202] 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 059/202] 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 060/202] 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 5d5d4a8443897235a69fe6b90717e6ebb81400ac Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 12 Oct 2023 09:50:04 +0000 Subject: [PATCH 061/202] 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 062/202] 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 063/202] 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 064/202] 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 065/202] 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 066/202] 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 067/202] 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 068/202] 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 069/202] 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 070/202] 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 071/202] 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 072/202] 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 073/202] 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 22f5d6d7085ab9f1b0b8d005bd0d36887a610d49 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 12 Oct 2023 10:57:21 +0000 Subject: [PATCH 074/202] 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 6e927325e404084f44cd30805cbc5266f07cf8bb Mon Sep 17 00:00:00 2001 From: guoxiaolong <467887319@qq.com> Date: Thu, 12 Oct 2023 20:32:36 +0800 Subject: [PATCH 075/202] 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 076/202] 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 077/202] 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 078/202] 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 079/202] 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 080/202] 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 081/202] 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 a2ee30707df12b738546bc1f706cd832bc5c1c94 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Oct 2023 16:59:03 +0200 Subject: [PATCH 082/202] 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 083/202] 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 084/202] 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 085/202] 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 086/202] 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 087/202] 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 088/202] 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 3c6ddb5e1552d9d22a21247e7481914c0a8f81c6 Mon Sep 17 00:00:00 2001 From: Jihyuk Bok Date: Fri, 13 Oct 2023 00:37:55 +0200 Subject: [PATCH 089/202] 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 090/202] 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 091/202] 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 092/202] 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 093/202] 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 094/202] 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 095/202] 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 096/202] 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 097/202] 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 48933f52fe66c6217d09d8d5b0ced67dad7bb5a6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 13 Oct 2023 11:30:37 +0000 Subject: [PATCH 098/202] 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 099/202] 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 100/202] 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 101/202] 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 102/202] 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 52480fe092b11b4e634951c3813d4519ba21e805 Mon Sep 17 00:00:00 2001 From: grantovsky Date: Fri, 13 Oct 2023 15:36:34 +0300 Subject: [PATCH 103/202] 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 104/202] 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 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 105/202] 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 106/202] 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 107/202] 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 108/202] 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 109/202] 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 593c3114a152f9eeecede03a1b065cca9bca2ae1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 13 Oct 2023 13:24:26 +0000 Subject: [PATCH 110/202] 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 111/202] 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 67068849da835ae8221a131b9335ba380f7ff556 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 13 Oct 2023 15:59:37 +0200 Subject: [PATCH 112/202] 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 113/202] 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 114/202] 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 115/202] 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 116/202] 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 117/202] 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 118/202] 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 119/202] 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 120/202] 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 121/202] 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 122/202] 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 123/202] 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 124/202] 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 125/202] 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 126/202] 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 127/202] 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 128/202] 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 129/202] 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 130/202] 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 131/202] 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 132/202] 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 133/202] 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 134/202] 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 135/202] 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 1d65393bff45923cbd2ae488245a6868070636bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 13 Oct 2023 23:28:34 +0200 Subject: [PATCH 136/202] 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 137/202] 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 138/202] 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 139/202] 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 140/202] 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 141/202] 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 142/202] 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 143/202] 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 144/202] 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 145/202] 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 146/202] 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 147/202] 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 0fb6ed5af7610968b779d90fc9fd3a893f7db3a4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 14 Oct 2023 10:28:23 +0200 Subject: [PATCH 148/202] 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 149/202] 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 150/202] 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 151/202] 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 152/202] 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 25efd188d5968911774ce901838cd19927560c68 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 22:17:36 +0200 Subject: [PATCH 153/202] 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 154/202] 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 155/202] 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 156/202] 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 157/202] 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 41479b21e60f85f953b4b5b52697efd1238ead31 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 15 Oct 2023 15:33:33 +0200 Subject: [PATCH 158/202] 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 159/202] 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 160/202] 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 6ac790e91831ece6b34718cc717aef7bf645f7e4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 15 Oct 2023 20:17:46 +0200 Subject: [PATCH 161/202] 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 162/202] 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 163/202] 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 164/202] 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 165/202] 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 fb901c24a19a41de9bc1b27bcfc6bcdcc8b03057 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 16 Oct 2023 09:45:49 +0200 Subject: [PATCH 166/202] 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 d7c77420b6539e73946fd58f4591ae93a3cc1a9f Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 16 Oct 2023 10:33:04 +0000 Subject: [PATCH 167/202] 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 168/202] 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 169/202] 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 170/202] 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 171/202] 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 172/202] 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 173/202] 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 174/202] 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 175/202] 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 176/202] 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 177/202] 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 178/202] 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 179/202] 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 34d804686a4b5fd4cb546faef33346a92504a503 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 16 Oct 2023 13:03:12 +0200 Subject: [PATCH 180/202] 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 181/202] 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 182/202] 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 183/202] 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 184/202] 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 185/202] 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 4d1edc27804b4defefbe16c7166249074436d92e Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 16 Oct 2023 15:00:13 +0000 Subject: [PATCH 186/202] 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 187/202] 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 188/202] 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 189/202] 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 2da1ff4b0d33df0e061e2e627727046640236734 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 16 Oct 2023 19:07:11 +0300 Subject: [PATCH 190/202] 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 191/202] 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 f85e82931d009100bbd1dfe1998810c930aee765 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 16 Oct 2023 18:36:45 +0200 Subject: [PATCH 192/202] 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 193/202] 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 67b9407530bb15f7e6d49cd1c2bde7b6c441389b Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Oct 2023 22:01:44 +0200 Subject: [PATCH 194/202] 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 195/202] 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 196/202] 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 197/202] 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 198/202] 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 ce7eca0615966d4ca3b8fe2e8a6daee779ab2f69 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 16 Oct 2023 17:00:07 -0700 Subject: [PATCH 199/202] 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 200/202] 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 fc9a5a725c6cd661ea51bd7b4a9aa257ec60d0c8 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 17 Oct 2023 08:18:17 +0000 Subject: [PATCH 201/202] 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 202/202] 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:")