From 434458cc830d2ced68f1f96dcfa13f967c9bc74e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 16 Aug 2024 11:22:22 +0000 Subject: [PATCH 01/22] Remove -n / --multiquery --- .../00115_shard_in_incomplete_result.sh | 2 +- .../0_stateless/00550_join_insert_select.sh | 2 +- .../0_stateless/01086_window_view_cleanup.sh | 2 +- ...396_inactive_replica_cleanup_nodes_zookeeper.sh | 4 ++-- .../0_stateless/01455_opentelemetry_distributed.sh | 2 +- .../0_stateless/01509_format_raw_blob.reference | 4 ++-- tests/queries/0_stateless/01509_format_raw_blob.sh | 4 ++-- .../01565_query_loop_after_client_error.expect | 2 +- .../01811_storage_buffer_flush_parameters.sh | 6 +++--- ...3_correct_block_size_prediction_with_default.sh | 4 ++-- .../02020_alter_table_modify_comment.sh | 2 +- ...e_sorting_by_input_stream_properties_explain.sh | 4 ++-- .../0_stateless/02383_join_and_filtering_set.sh | 2 +- .../0_stateless/02423_ddl_for_opentelemetry.sh | 2 +- tests/queries/0_stateless/02539_settings_alias.sh | 4 ++-- .../02697_stop_reading_on_first_cancel.sh | 2 +- .../0_stateless/02703_row_policies_for_asterisk.sh | 2 +- .../02703_row_policies_for_database_combination.sh | 2 +- .../0_stateless/02703_row_policy_for_database.sh | 4 ++-- tests/queries/0_stateless/02724_delay_mutations.sh | 6 +++--- ...02765_queries_with_subqueries_profile_events.sh | 14 +++++++------- .../queries/0_stateless/02841_not_ready_set_bug.sh | 4 ++-- .../0_stateless/02871_peak_threads_usage.sh | 14 +++++++------- .../0_stateless/02911_backup_restore_keeper_map.sh | 6 +++--- .../0_stateless/02968_file_log_multiple_read.sh | 4 ++-- .../03002_part_log_rmt_fetch_merge_error.sh | 8 ++++---- .../03002_part_log_rmt_fetch_mutate_error.sh | 10 +++++----- .../03164_selects_with_pk_usage_profile_event.sh | 8 ++++---- .../0_stateless/03172_system_detached_tables.sh | 4 ++-- .../03173_parallel_replicas_join_bug.sh | 2 +- 30 files changed, 68 insertions(+), 68 deletions(-) diff --git a/tests/queries/0_stateless/00115_shard_in_incomplete_result.sh b/tests/queries/0_stateless/00115_shard_in_incomplete_result.sh index 5c3918dea9f..4916721764c 100755 --- a/tests/queries/0_stateless/00115_shard_in_incomplete_result.sh +++ b/tests/queries/0_stateless/00115_shard_in_incomplete_result.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -o errexit set -o pipefail -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" DROP TABLE IF EXISTS users; CREATE TABLE users (UserID UInt64) ENGINE = Log; INSERT INTO users VALUES (1468013291393583084); diff --git a/tests/queries/0_stateless/00550_join_insert_select.sh b/tests/queries/0_stateless/00550_join_insert_select.sh index bfaccb613ca..ee2f3ab286b 100755 --- a/tests/queries/0_stateless/00550_join_insert_select.sh +++ b/tests/queries/0_stateless/00550_join_insert_select.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n --ignore-error --query=" +$CLICKHOUSE_CLIENT --ignore-error --query=" DROP TABLE IF EXISTS test1_00550; DROP TABLE IF EXISTS test2_00550; DROP TABLE IF EXISTS test3_00550; diff --git a/tests/queries/0_stateless/01086_window_view_cleanup.sh b/tests/queries/0_stateless/01086_window_view_cleanup.sh index 8b8e794c8ff..1bfa3c50869 100755 --- a/tests/queries/0_stateless/01086_window_view_cleanup.sh +++ b/tests/queries/0_stateless/01086_window_view_cleanup.sh @@ -13,7 +13,7 @@ opts=( DATABASE_ORDINARY="${CLICKHOUSE_DATABASE}_ordinary" -$CLICKHOUSE_CLIENT "${opts[@]}" --allow_deprecated_database_ordinary=1 --multiquery " +$CLICKHOUSE_CLIENT "${opts[@]}" --allow_deprecated_database_ordinary=1 " SET allow_experimental_window_view = 1; SET window_view_clean_interval = 1; diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index bff85b3e29f..4a0b6a8c93c 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -13,7 +13,7 @@ REPLICA=$($CLICKHOUSE_CLIENT --query "Select getMacro('replica')") SCALE=1000 -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS r1; DROP TABLE IF EXISTS r2; CREATE TABLE r1 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}', '1{replica}') ORDER BY x @@ -46,7 +46,7 @@ $CLICKHOUSE_CLIENT --receive_timeout 600 --query "SYSTEM SYNC REPLICA r2" # Need $CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/$SHARD/replicas/2$REPLICA' AND name = 'is_lost'"; -$CLICKHOUSE_CLIENT -n --query " +$CLICKHOUSE_CLIENT --query " DROP TABLE IF EXISTS r1; DROP TABLE IF EXISTS r2; " diff --git a/tests/queries/0_stateless/01455_opentelemetry_distributed.sh b/tests/queries/0_stateless/01455_opentelemetry_distributed.sh index 2b6da6132ed..30940f93a56 100755 --- a/tests/queries/0_stateless/01455_opentelemetry_distributed.sh +++ b/tests/queries/0_stateless/01455_opentelemetry_distributed.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function check_log { -${CLICKHOUSE_CLIENT} --format=JSONEachRow -nq " +${CLICKHOUSE_CLIENT} --format=JSONEachRow -q " set enable_analyzer = 1; system flush logs; diff --git a/tests/queries/0_stateless/01509_format_raw_blob.reference b/tests/queries/0_stateless/01509_format_raw_blob.reference index 05014001bd9..eb074457e07 100644 --- a/tests/queries/0_stateless/01509_format_raw_blob.reference +++ b/tests/queries/0_stateless/01509_format_raw_blob.reference @@ -1,2 +1,2 @@ -9fd46251e5574c633cbfbb9293671888 - -9fd46251e5574c633cbfbb9293671888 - +48fad37bc89fc3bcc29c4750897b6709 - +48fad37bc89fc3bcc29c4750897b6709 - diff --git a/tests/queries/0_stateless/01509_format_raw_blob.sh b/tests/queries/0_stateless/01509_format_raw_blob.sh index 3d1d3fbb17b..355928014e8 100755 --- a/tests/queries/0_stateless/01509_format_raw_blob.sh +++ b/tests/queries/0_stateless/01509_format_raw_blob.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS t; CREATE TABLE t (a LowCardinality(Nullable(String))) ENGINE = Memory; " @@ -12,7 +12,7 @@ CREATE TABLE t (a LowCardinality(Nullable(String))) ENGINE = Memory; ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT RawBLOB" < ${BASH_SOURCE[0]} cat ${BASH_SOURCE[0]} | md5sum -${CLICKHOUSE_CLIENT} -n --query "SELECT * FROM t FORMAT RawBLOB" | md5sum +${CLICKHOUSE_CLIENT} --query "SELECT * FROM t FORMAT RawBLOB" | md5sum ${CLICKHOUSE_CLIENT} --query " DROP TABLE t; diff --git a/tests/queries/0_stateless/01565_query_loop_after_client_error.expect b/tests/queries/0_stateless/01565_query_loop_after_client_error.expect index 6253840c63c..f08ef911da4 100755 --- a/tests/queries/0_stateless/01565_query_loop_after_client_error.expect +++ b/tests/queries/0_stateless/01565_query_loop_after_client_error.expect @@ -24,7 +24,7 @@ expect_after { -i $any_spawn_id timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion -mn --history_file=$history_file --highlight 0" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion -m --history_file=$history_file --highlight 0" expect "\n:) " send -- "DROP TABLE IF EXISTS t01565;\r" diff --git a/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sh b/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sh index 6a5949741ab..7878867e159 100755 --- a/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sh +++ b/tests/queries/0_stateless/01811_storage_buffer_flush_parameters.sh @@ -17,7 +17,7 @@ function wait_with_limit() done } -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists data_01811; drop table if exists buffer_01811; @@ -39,9 +39,9 @@ $CLICKHOUSE_CLIENT -nm -q " # wait for background buffer flush wait_with_limit 30 '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01811") -gt 0 ]]' -$CLICKHOUSE_CLIENT -nm -q "select count() from data_01811" +$CLICKHOUSE_CLIENT -m -q "select count() from data_01811" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table buffer_01811; drop table data_01811; " diff --git a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh index 1482730af2c..57f9b5595de 100755 --- a/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh +++ b/tests/queries/0_stateless/01903_correct_block_size_prediction_with_default.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) sql="toUInt16OrNull(arrayFirst((v, k) -> (k = '4Id'), arr[2], arr[1]))" # Create the table and fill it -$CLICKHOUSE_CLIENT -n --query=" +$CLICKHOUSE_CLIENT --query=" CREATE TABLE test_extract(str String, arr Array(Array(String)) ALIAS extractAllGroupsHorizontal(str, '\\W(\\w+)=(\"[^\"]*?\"|[^\",}]*)')) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY tuple(); INSERT INTO test_extract (str) WITH range(8) as range_arr, arrayMap(x-> concat(toString(x),'Id'), range_arr) as key, arrayMap(x -> rand() % 8, range_arr) as val, arrayStringConcat(arrayMap((x,y) -> concat(x,'=',toString(y)), key, val),',') as str SELECT str FROM numbers(500000); ALTER TABLE test_extract ADD COLUMN 15Id Nullable(UInt16) DEFAULT $sql;" @@ -24,7 +24,7 @@ function test() $CLICKHOUSE_CLIENT --query="SELECT uniq(15Id) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_1 uuid_2=$(cat /proc/sys/kernel/random/uuid) $CLICKHOUSE_CLIENT --query="SELECT uniq($sql) FROM test_extract $where SETTINGS max_threads=1" --query_id=$uuid_2 - $CLICKHOUSE_CLIENT -n --query=" + $CLICKHOUSE_CLIENT --query=" SYSTEM FLUSH LOGS; WITH memory_1 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_1' AND type = 'QueryFinish' as memory_1), memory_2 AS (SELECT memory_usage FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$uuid_2' AND type = 'QueryFinish' as memory_2) diff --git a/tests/queries/0_stateless/02020_alter_table_modify_comment.sh b/tests/queries/0_stateless/02020_alter_table_modify_comment.sh index 3448f052f51..fa2d84e131a 100755 --- a/tests/queries/0_stateless/02020_alter_table_modify_comment.sh +++ b/tests/queries/0_stateless/02020_alter_table_modify_comment.sh @@ -16,7 +16,7 @@ function test_table_comments() local ENGINE_NAME="$1" echo "engine : ${ENGINE_NAME}" - $CLICKHOUSE_CLIENT -nm <&1 | grep -q "SYNTAX_ERROR" -$CLICKHOUSE_CLIENT --multiquery " +$CLICKHOUSE_CLIENT " CREATE TABLE 02703_rqtable_default (x UInt8) ENGINE = MergeTree ORDER BY x; CREATE ROW POLICY ${CLICKHOUSE_DATABASE}_filter_11_db_policy ON * USING x=1 AS permissive TO ALL; diff --git a/tests/queries/0_stateless/02724_delay_mutations.sh b/tests/queries/0_stateless/02724_delay_mutations.sh index f349e29253a..7843e692822 100755 --- a/tests/queries/0_stateless/02724_delay_mutations.sh +++ b/tests/queries/0_stateless/02724_delay_mutations.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=./mergetree_mutations.lib . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " DROP TABLE IF EXISTS t_delay_mutations SYNC; CREATE TABLE t_delay_mutations (id UInt64, v UInt64) @@ -36,14 +36,14 @@ SELECT count() FROM system.mutations WHERE database = currentDatabase() AND tabl ${CLICKHOUSE_CLIENT} --query "SYSTEM START MERGES t_delay_mutations" wait_for_mutation "t_delay_mutations" "mutation_5.txt" -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SELECT * FROM t_delay_mutations ORDER BY id; SELECT count() FROM system.mutations WHERE database = currentDatabase() AND table = 't_delay_mutations' AND NOT is_done; DROP TABLE IF EXISTS t_delay_mutations SYNC; " -${CLICKHOUSE_CLIENT} -n --query " +${CLICKHOUSE_CLIENT} --query " SYSTEM FLUSH LOGS; SELECT diff --git a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh index b7d93b5396c..fd64e8d8cb8 100755 --- a/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh +++ b/tests/queries/0_stateless/02765_queries_with_subqueries_profile_events.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -n -q " +$CLICKHOUSE_CLIENT -q " DROP TABLE IF EXISTS mv; DROP TABLE IF EXISTS output; DROP TABLE IF EXISTS input; @@ -17,7 +17,7 @@ $CLICKHOUSE_CLIENT -n -q " for enable_analyzer in 0 1; do query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "INSERT INTO input SELECT * FROM numbers(1)" - $CLICKHOUSE_CLIENT -mn -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT 1 view, @@ -35,7 +35,7 @@ for enable_analyzer in 0 1; do query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "SELECT * FROM system.one WHERE dummy IN (SELECT * FROM system.one) FORMAT Null" - $CLICKHOUSE_CLIENT -mn -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT 1 subquery, @@ -52,7 +52,7 @@ for enable_analyzer in 0 1; do query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "WITH (SELECT * FROM system.one) AS x SELECT x FORMAT Null" - $CLICKHOUSE_CLIENT -mn -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT 1 CSE, @@ -69,7 +69,7 @@ for enable_analyzer in 0 1; do query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "WITH (SELECT * FROM system.one) AS x SELECT x, x FORMAT Null" - $CLICKHOUSE_CLIENT -mn -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT 1 CSE_Multi, @@ -86,7 +86,7 @@ for enable_analyzer in 0 1; do query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "WITH x AS (SELECT * FROM system.one) SELECT * FROM x FORMAT Null" - $CLICKHOUSE_CLIENT -mn -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT 1 CTE, @@ -103,7 +103,7 @@ for enable_analyzer in 0 1; do query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --enable_analyzer "$enable_analyzer" --query_id "$query_id" -q "WITH x AS (SELECT * FROM system.one) SELECT * FROM x UNION ALL SELECT * FROM x FORMAT Null" - $CLICKHOUSE_CLIENT -mn -q " + $CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT 1 CTE_Multi, diff --git a/tests/queries/0_stateless/02841_not_ready_set_bug.sh b/tests/queries/0_stateless/02841_not_ready_set_bug.sh index 556e2f52de2..d5a2d034014 100755 --- a/tests/queries/0_stateless/02841_not_ready_set_bug.sh +++ b/tests/queries/0_stateless/02841_not_ready_set_bug.sh @@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -q "SELECT * FROM system.tables WHERE 1 in (SELECT number fro $CLICKHOUSE_CLIENT -q "SELECT xor(1, 0) FROM system.parts WHERE 1 IN (SELECT 1) FORMAT Null" # (Not all of these tests are effective because some of these tables are empty.) -$CLICKHOUSE_CLIENT -nq " +$CLICKHOUSE_CLIENT -q " select * from system.columns where table in (select '123'); select * from system.replicas where database in (select '123'); select * from system.data_skipping_indices where database in (select '123'); @@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT -nq " select * from system.replication_queue where database in (select '123'); select * from system.distribution_queue where database in (select '123'); " -$CLICKHOUSE_CLIENT -nq " +$CLICKHOUSE_CLIENT -q " create table a (x Int8) engine MergeTree order by x; insert into a values (1); select * from mergeTreeIndex(currentDatabase(), 'a') where part_name in (select '123'); diff --git a/tests/queries/0_stateless/02871_peak_threads_usage.sh b/tests/queries/0_stateless/02871_peak_threads_usage.sh index dfb3e665020..0f0473bbb47 100755 --- a/tests/queries/0_stateless/02871_peak_threads_usage.sh +++ b/tests/queries/0_stateless/02871_peak_threads_usage.sh @@ -26,7 +26,7 @@ ${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_6" --query='SELECT * FROM nu ${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_7" --query='SELECT * FROM numbers_mt(5000), numbers(5000) SETTINGS max_threads = 1, joined_subquery_requires_alias=0' "${QUERY_OPTIONS[@]}" ${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_8" --query='SELECT * FROM numbers_mt(5000), numbers(5000) SETTINGS max_threads = 4, joined_subquery_requires_alias=0' "${QUERY_OPTIONS[@]}" -${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_9" -mn --query=""" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_9" -m --query=""" SELECT count() FROM (SELECT number FROM numbers_mt(1,100000) UNION ALL SELECT number FROM numbers_mt(10000, 200000) @@ -38,7 +38,7 @@ SELECT count() FROM UNION ALL SELECT number FROM numbers_mt(300000, 4000000) ) SETTINGS max_threads = 1""" "${QUERY_OPTIONS[@]}" -${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_10" -mn --query=""" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_10" -m --query=""" SELECT count() FROM (SELECT number FROM numbers_mt(1,100000) UNION ALL SELECT number FROM numbers_mt(10000, 2000) @@ -50,7 +50,7 @@ SELECT count() FROM UNION ALL SELECT number FROM numbers_mt(300000, 4000000) ) SETTINGS max_threads = 4""" "${QUERY_OPTIONS[@]}" -${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_11" -mn --query=""" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_11" -m --query=""" SELECT count() FROM (SELECT number FROM numbers_mt(1,100000) UNION ALL SELECT number FROM numbers_mt(1, 1) @@ -62,20 +62,20 @@ SELECT count() FROM UNION ALL SELECT number FROM numbers_mt(1, 4000000) ) SETTINGS max_threads = 4""" "${QUERY_OPTIONS[@]}" -${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_12" -mn --query=""" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_12" -m --query=""" SELECT sum(number) FROM numbers_mt(100000) GROUP BY number % 2 WITH TOTALS ORDER BY number % 2 SETTINGS max_threads = 4""" "${QUERY_OPTIONS[@]}" -${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_13" -mn --query="SELECT * FROM numbers(100000) SETTINGS max_threads = 1" "${QUERY_OPTIONS[@]}" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_13" -m --query="SELECT * FROM numbers(100000) SETTINGS max_threads = 1" "${QUERY_OPTIONS[@]}" -${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_14" -mn --query="SELECT * FROM numbers(100000) SETTINGS max_threads = 4" "${QUERY_OPTIONS[@]}" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_14" -m --query="SELECT * FROM numbers(100000) SETTINGS max_threads = 4" "${QUERY_OPTIONS[@]}" ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" for i in {1..14} do - ${CLICKHOUSE_CLIENT} -mn --query=""" + ${CLICKHOUSE_CLIENT} -m --query=""" SELECT '${i}', peak_threads_usage, (select count() from system.query_thread_log WHERE system.query_thread_log.query_id = '${UNIQUE_QUERY_ID}_${i}' AND current_database = currentDatabase()) = length(thread_ids), diff --git a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh index c04667505c3..01aba244a02 100755 --- a/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh +++ b/tests/queries/0_stateless/02911_backup_restore_keeper_map.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh database_name="$CLICKHOUSE_DATABASE"_02911_keeper_map -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " DROP DATABASE IF EXISTS $database_name; CREATE DATABASE $database_name; CREATE TABLE $database_name.02911_backup_restore_keeper_map1 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911') PRIMARY KEY key; @@ -13,9 +13,9 @@ $CLICKHOUSE_CLIENT -nm -q " CREATE TABLE $database_name.02911_backup_restore_keeper_map3 (key UInt64, value String) Engine=KeeperMap('/' || currentDatabase() || '/test02911_different') PRIMARY KEY key; " -$CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000;" +$CLICKHOUSE_CLIENT -m -q "INSERT INTO $database_name.02911_backup_restore_keeper_map2 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 5000;" -$CLICKHOUSE_CLIENT -nm -q "INSERT INTO $database_name.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000;" +$CLICKHOUSE_CLIENT -m -q "INSERT INTO $database_name.02911_backup_restore_keeper_map3 SELECT number, 'test' || toString(number) FROM system.numbers LIMIT 3000;" backup_path="$database_name" for i in $(seq 1 3); do diff --git a/tests/queries/0_stateless/02968_file_log_multiple_read.sh b/tests/queries/0_stateless/02968_file_log_multiple_read.sh index d9bae05270a..0879bf02d60 100755 --- a/tests/queries/0_stateless/02968_file_log_multiple_read.sh +++ b/tests/queries/0_stateless/02968_file_log_multiple_read.sh @@ -15,7 +15,7 @@ do echo $i >> ${logs_dir}/a.txt done -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" DROP TABLE IF EXISTS file_log; DROP TABLE IF EXISTS table_to_store_data; DROP TABLE IF EXISTS file_log_mv; @@ -69,7 +69,7 @@ done ${CLICKHOUSE_CLIENT} --query "SELECT * FROM table_to_store_data ORDER BY id;" -${CLICKHOUSE_CLIENT} -n --query=" +${CLICKHOUSE_CLIENT} --query=" DROP TABLE file_log; DROP TABLE table_to_store_data; DROP TABLE file_log_mv; diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh index e58c542b8ac..185e46a2eac 100755 --- a/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_merge_error.sh @@ -11,13 +11,13 @@ set -e function wait_until() { local q=$1 && shift - while [ "$($CLICKHOUSE_CLIENT -nm -q "$q")" != "1" ]; do + while [ "$($CLICKHOUSE_CLIENT -m -q "$q")" != "1" ]; do # too frequent FLUSH LOGS is too costly sleep 2 done } -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists rmt_master; drop table if exists rmt_slave; @@ -33,7 +33,7 @@ $CLICKHOUSE_CLIENT -nm -q " optimize table rmt_master final settings alter_sync=1, optimize_throw_if_noop=1; " -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system flush logs; select 'before'; select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; @@ -42,7 +42,7 @@ $CLICKHOUSE_CLIENT -nm -q " " # wait until rmt_slave will fetch the part and reflect this error in system.part_log wait_until "system flush logs; select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system sync replica rmt_slave; system flush logs; diff --git a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh index cc8f53aafb9..e731d51e7e3 100755 --- a/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh +++ b/tests/queries/0_stateless/03002_part_log_rmt_fetch_mutate_error.sh @@ -14,13 +14,13 @@ set -e function wait_until() { local q=$1 && shift - while [ "$($CLICKHOUSE_CLIENT -nm -q "$q")" != "1" ]; do + while [ "$($CLICKHOUSE_CLIENT -m -q "$q")" != "1" ]; do # too frequent FLUSH LOGS is too costly sleep 2 done } -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " drop table if exists rmt_master; drop table if exists rmt_slave; @@ -41,10 +41,10 @@ $CLICKHOUSE_CLIENT -nm -q " # the part, and rmt_slave will consider it instead of performing mutation on # it's own, otherwise prefer_fetch_merged_part_*_threshold will be simply ignored wait_for_mutation rmt_master 0000000000 -$CLICKHOUSE_CLIENT -nm -q "system start pulling replication log rmt_slave" +$CLICKHOUSE_CLIENT -m -q "system start pulling replication log rmt_slave" # and wait until rmt_slave to fetch the part and reflect this error in system.part_log wait_until "system flush logs; select count()>0 from system.part_log where table = 'rmt_slave' and database = '$CLICKHOUSE_DATABASE' and error > 0" -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system flush logs; select 'before'; select table, event_type, error>0, countIf(error=0) from system.part_log where database = currentDatabase() group by 1, 2, 3 order by 1, 2, 3; @@ -52,7 +52,7 @@ $CLICKHOUSE_CLIENT -nm -q " system start replicated sends rmt_master; " wait_for_mutation rmt_slave 0000000000 -$CLICKHOUSE_CLIENT -nm -q " +$CLICKHOUSE_CLIENT -m -q " system sync replica rmt_slave; system flush logs; diff --git a/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.sh b/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.sh index 29d4c877909..75efc3f057a 100755 --- a/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.sh +++ b/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.sh @@ -33,7 +33,7 @@ $CLICKHOUSE_CLIENT -q " query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --query_id "$query_id" -q " SELECT count(*) FROM table_$table_id FORMAT Null;" -$CLICKHOUSE_CLIENT -mn -q " +$CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] AS selects_with_pk_usage @@ -50,7 +50,7 @@ $CLICKHOUSE_CLIENT -mn -q " query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --query_id "$query_id" -q " SELECT count(*) FROM table_$table_id WHERE col2 >= 50000 FORMAT Null;" -$CLICKHOUSE_CLIENT -mn -q " +$CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] AS selects_with_pk_usage @@ -67,7 +67,7 @@ $CLICKHOUSE_CLIENT -mn -q " query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --query_id "$query_id" -q " SELECT count(*) FROM table_$table_id WHERE pk >= 50000 FORMAT Null;" -$CLICKHOUSE_CLIENT -mn -q " +$CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] AS selects_with_pk_usage @@ -84,7 +84,7 @@ $CLICKHOUSE_CLIENT -mn -q " query_id="$(random_str 10)" $CLICKHOUSE_CLIENT --query_id "$query_id" -q " SELECT count(*) FROM table_$table_id WHERE col1 >= 50000 FORMAT Null;" -$CLICKHOUSE_CLIENT -mn -q " +$CLICKHOUSE_CLIENT -m -q " SYSTEM FLUSH LOGS; SELECT ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] AS selects_with_pk_usage diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sh b/tests/queries/0_stateless/03172_system_detached_tables.sh index 47775abcc45..60e913b62a8 100755 --- a/tests/queries/0_stateless/03172_system_detached_tables.sh +++ b/tests/queries/0_stateless/03172_system_detached_tables.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATABASE_ATOMIC="${CLICKHOUSE_DATABASE}_atomic" DATABASE_LAZY="${CLICKHOUSE_DATABASE}_lazy" -$CLICKHOUSE_CLIENT --multiquery " +$CLICKHOUSE_CLIENT " SELECT 'database atomic tests'; DROP DATABASE IF EXISTS ${DATABASE_ATOMIC}; @@ -36,7 +36,7 @@ DROP DATABASE ${DATABASE_ATOMIC} SYNC; " -$CLICKHOUSE_CLIENT --multiquery " +$CLICKHOUSE_CLIENT " SELECT '-----------------------'; SELECT 'database lazy tests'; diff --git a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh index af702569794..d2be9899f86 100755 --- a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh +++ b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh @@ -16,7 +16,7 @@ $CLICKHOUSE_CLIENT -q " INSERT INTO data2 VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-02', 'CREATED'); " -$CLICKHOUSE_CLIENT -nq " +$CLICKHOUSE_CLIENT -q " SET enable_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 10, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; SELECT From 915daafd3a0c9f1539dad75dc3805e740f0bc75a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 21 Aug 2024 10:45:48 +0000 Subject: [PATCH 02/22] Fix 01086_window_view_cleanup.sh --- tests/queries/0_stateless/01086_window_view_cleanup.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01086_window_view_cleanup.sh b/tests/queries/0_stateless/01086_window_view_cleanup.sh index 1bfa3c50869..2e6cc7e2520 100755 --- a/tests/queries/0_stateless/01086_window_view_cleanup.sh +++ b/tests/queries/0_stateless/01086_window_view_cleanup.sh @@ -13,7 +13,8 @@ opts=( DATABASE_ORDINARY="${CLICKHOUSE_DATABASE}_ordinary" -$CLICKHOUSE_CLIENT "${opts[@]}" --allow_deprecated_database_ordinary=1 " +$CLICKHOUSE_CLIENT "${opts[@]}" --query " + SET allow_deprecated_database_ordinary = 1; SET allow_experimental_window_view = 1; SET window_view_clean_interval = 1; @@ -28,8 +29,7 @@ $CLICKHOUSE_CLIENT "${opts[@]}" --allow_deprecated_database_ordinary=1 " INSERT INTO ${DATABASE_ORDINARY}.mt VALUES (1, 2, toDateTime('1990/01/01 12:00:01', 'US/Samoa')); INSERT INTO ${DATABASE_ORDINARY}.mt VALUES (1, 3, toDateTime('1990/01/01 12:00:02', 'US/Samoa')); INSERT INTO ${DATABASE_ORDINARY}.mt VALUES (1, 4, toDateTime('1990/01/01 12:00:05', 'US/Samoa')); - INSERT INTO ${DATABASE_ORDINARY}.mt VALUES (1, 5, toDateTime('1990/01/01 12:00:06', 'US/Samoa')); -" + INSERT INTO ${DATABASE_ORDINARY}.mt VALUES (1, 5, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));" while true; do $CLICKHOUSE_CLIENT "${opts[@]}" --query="SELECT count(*) FROM ${DATABASE_ORDINARY}.\`.inner.wv\`" | grep -q "5" && break || sleep .5 ||: From e87de3cfcd22870bf7aea3dfaf1607b180b2b1d8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 21 Aug 2024 15:19:29 +0000 Subject: [PATCH 03/22] return back virtual columns to distributed tables --- src/Storages/StorageDistributed.cpp | 4 ++++ .../03228_virtual_column_merge_dist.reference | 8 +++++++ .../03228_virtual_column_merge_dist.sql | 24 +++++++++++++++++++ 3 files changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/03228_virtual_column_merge_dist.reference create mode 100644 tests/queries/0_stateless/03228_virtual_column_merge_dist.sql diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index e146e95f89f..c4668159759 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -290,6 +290,10 @@ VirtualColumnsDescription StorageDistributed::createVirtuals() desc.addEphemeral("_shard_num", std::make_shared(), "Deprecated. Use function shardNum instead"); + /// Add virtual columns from table of storage Merges. + desc.addEphemeral("_database", std::make_shared(std::make_shared()), "The name of database which the row comes from"); + desc.addEphemeral("_table", std::make_shared(std::make_shared()), "The name of table which the row comes from"); + return desc; } diff --git a/tests/queries/0_stateless/03228_virtual_column_merge_dist.reference b/tests/queries/0_stateless/03228_virtual_column_merge_dist.reference new file mode 100644 index 00000000000..28f00bafdfe --- /dev/null +++ b/tests/queries/0_stateless/03228_virtual_column_merge_dist.reference @@ -0,0 +1,8 @@ +1 t_local_1 +2 t_local_2 +1 t_local_1 +2 t_local_2 +1 1 +2 1 +1 1 +2 1 diff --git a/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql b/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql new file mode 100644 index 00000000000..caf00a2e407 --- /dev/null +++ b/tests/queries/0_stateless/03228_virtual_column_merge_dist.sql @@ -0,0 +1,24 @@ +DROP TABLE IF EXISTS t_local_1; +DROP TABLE IF EXISTS t_local_2; +DROP TABLE IF EXISTS t_merge; +DROP TABLE IF EXISTS t_distr; + +CREATE TABLE t_local_1 (a UInt32) ENGINE = MergeTree ORDER BY a; +CREATE TABLE t_local_2 (a UInt32) ENGINE = MergeTree ORDER BY a; + +INSERT INTO t_local_1 VALUES (1); +INSERT INTO t_local_2 VALUES (2); + +CREATE TABLE t_merge AS t_local_1 ENGINE = Merge(currentDatabase(), '^(t_local_1|t_local_2)$'); +CREATE TABLE t_distr AS t_local_1 engine=Distributed('test_shard_localhost', currentDatabase(), t_merge, rand()); + +SELECT a, _table FROM t_merge ORDER BY a; +SELECT a, _table FROM t_distr ORDER BY a; + +SELECT a, _database = currentDatabase() FROM t_merge ORDER BY a; +SELECT a, _database = currentDatabase() FROM t_distr ORDER BY a; + +DROP TABLE IF EXISTS t_local_1; +DROP TABLE IF EXISTS t_local_2; +DROP TABLE IF EXISTS t_merge; +DROP TABLE IF EXISTS t_distr; From 6e5465ae5126f3281d81172e952b6811f8946f2d Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 21 Aug 2024 15:47:08 +0200 Subject: [PATCH 04/22] CI: SQLLogix job fix --- docker/test/sqllogic/Dockerfile | 3 --- tests/ci/sqllogic_test.py | 10 ++++------ .../docker_scripts/sqllogic_runner.sh | 18 +++++++++--------- 3 files changed, 13 insertions(+), 18 deletions(-) rename docker/test/sqllogic/run.sh => tests/docker_scripts/sqllogic_runner.sh (87%) diff --git a/docker/test/sqllogic/Dockerfile b/docker/test/sqllogic/Dockerfile index 6397526388e..0d21a2da44e 100644 --- a/docker/test/sqllogic/Dockerfile +++ b/docker/test/sqllogic/Dockerfile @@ -40,6 +40,3 @@ RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ARG sqllogic_test_repo="https://github.com/gregrahn/sqllogictest.git" RUN git clone --recursive ${sqllogic_test_repo} - -COPY run.sh / -CMD ["/bin/bash", "/run.sh"] diff --git a/tests/ci/sqllogic_test.py b/tests/ci/sqllogic_test.py index 63880f07e92..7fe44c235c7 100755 --- a/tests/ci/sqllogic_test.py +++ b/tests/ci/sqllogic_test.py @@ -31,7 +31,7 @@ IMAGE_NAME = "clickhouse/sqllogic-test" def get_run_command( builds_path: Path, - repo_tests_path: Path, + repo_path: Path, result_path: Path, server_log_path: Path, image: DockerImage, @@ -39,11 +39,11 @@ def get_run_command( return ( f"docker run " f"--volume={builds_path}:/package_folder " - f"--volume={repo_tests_path}:/clickhouse-tests " + f"--volume={repo_path}:/repo " f"--volume={result_path}:/test_output " f"--volume={server_log_path}:/var/log/clickhouse-server " "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls - f"--cap-add=SYS_PTRACE {image}" + f"--cap-add=SYS_PTRACE {image} /repo/tests/docker_scripts/sqllogic_runner.sh" ) @@ -94,8 +94,6 @@ def main(): docker_image = pull_image(get_docker_image(IMAGE_NAME)) - repo_tests_path = repo_path / "tests" - packages_path = temp_path / "packages" packages_path.mkdir(parents=True, exist_ok=True) @@ -111,7 +109,7 @@ def main(): run_command = get_run_command( # run script inside docker packages_path, - repo_tests_path, + repo_path, result_path, server_log_path, docker_image, diff --git a/docker/test/sqllogic/run.sh b/tests/docker_scripts/sqllogic_runner.sh similarity index 87% rename from docker/test/sqllogic/run.sh rename to tests/docker_scripts/sqllogic_runner.sh index 32368980f9b..8b8f1e7aec7 100755 --- a/docker/test/sqllogic/run.sh +++ b/tests/docker_scripts/sqllogic_runner.sh @@ -15,10 +15,10 @@ echo "Files in current directory" ls -la ./ echo "Files in root directory" ls -la / -echo "Files in /clickhouse-tests directory" -ls -la /clickhouse-tests -echo "Files in /clickhouse-tests/sqllogic directory" -ls -la /clickhouse-tests/sqllogic +echo "Files in /repo/tests directory" +ls -la /repo/tests +echo "Files in /repo/tests/sqllogic directory" +ls -la /repo/tests/sqllogic echo "Files in /package_folder directory" ls -la /package_folder echo "Files in /test_output" @@ -45,13 +45,13 @@ function run_tests() cd /test_output - /clickhouse-tests/sqllogic/runner.py --help 2>&1 \ + /repo/tests/sqllogic/runner.py --help 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' mkdir -p /test_output/self-test - /clickhouse-tests/sqllogic/runner.py --log-file /test_output/runner-self-test.log \ + /repo/tests/sqllogic/runner.py --log-file /test_output/runner-self-test.log \ self-test \ - --self-test-dir /clickhouse-tests/sqllogic/self-test \ + --self-test-dir /repo/tests/sqllogic/self-test \ --out-dir /test_output/self-test \ 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' @@ -63,7 +63,7 @@ function run_tests() if [ -d /sqllogictest ] then mkdir -p /test_output/statements-test - /clickhouse-tests/sqllogic/runner.py \ + /repo/tests/sqllogic/runner.py \ --log-file /test_output/runner-statements-test.log \ --log-level info \ statements-test \ @@ -77,7 +77,7 @@ function run_tests() tar -zcvf statements-check.tar.gz statements-test 1>/dev/null mkdir -p /test_output/complete-test - /clickhouse-tests/sqllogic/runner.py \ + /repo/tests/sqllogic/runner.py \ --log-file /test_output/runner-complete-test.log \ --log-level info \ complete-test \ From 8a89cd31a1e7770479af6eaf1b4211ef4ece1795 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 22 Aug 2024 00:29:32 +0200 Subject: [PATCH 05/22] Fix Upgrade Check: move some settings to 24.9 section --- src/Core/SettingsChangesHistory.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index fb59577b0f0..5e831c6301c 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -72,11 +72,13 @@ static std::initializer_list Date: Thu, 22 Aug 2024 13:42:30 +0000 Subject: [PATCH 06/22] fix virtual columns in Merge engine --- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageMerge.cpp | 14 +++++++------- .../02890_describe_table_options.reference | 8 ++++++++ 3 files changed, 16 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c4668159759..0b80858800b 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -290,7 +290,7 @@ VirtualColumnsDescription StorageDistributed::createVirtuals() desc.addEphemeral("_shard_num", std::make_shared(), "Deprecated. Use function shardNum instead"); - /// Add virtual columns from table of storage Merges. + /// Add virtual columns from table with Merge engine. desc.addEphemeral("_database", std::make_shared(std::make_shared()), "The name of database which the row comes from"); desc.addEphemeral("_table", std::make_shared(std::make_shared()), "The name of table which the row comes from"); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e88844e2d31..0827321e296 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -642,10 +642,6 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); } } - else - { - - } auto child = createPlanForTable( nested_storage_snaphsot, @@ -657,6 +653,7 @@ std::vector ReadFromMerge::createChildrenPlans(SelectQ row_policy_data_opt, modified_context, current_streams); + child.plan.addInterpreterContext(modified_context); if (child.plan.isInitialized()) @@ -914,12 +911,14 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo modified_query_info.table_expression = replacement_table_expression; modified_query_info.planner_context->getOrCreateTableExpressionData(replacement_table_expression); - auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); - if (storage_snapshot_->storage.supportsSubcolumns()) - get_column_options.withSubcolumns(); + auto get_column_options = GetColumnsOptions(GetColumnsOptions::All) + .withExtendedObjects() + .withSubcolumns(storage_snapshot_->storage.supportsSubcolumns()); std::unordered_map column_name_to_node; + /// Consider only non-virtual columns of storage while checking for _table and _database columns. + /// I.e. always override virtual columns with these names from underlying table (if any). if (!storage_snapshot_->tryGetColumn(get_column_options, "_table")) { auto table_name_node = std::make_shared(current_storage_id.table_name); @@ -946,6 +945,7 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo column_name_to_node.emplace("_database", function_node); } + get_column_options.withVirtuals(); auto storage_columns = storage_snapshot_->metadata->getColumns(); bool with_aliases = /* common_processed_stage == QueryProcessingStage::FetchColumns && */ !storage_columns.getAliases().empty(); diff --git a/tests/queries/0_stateless/02890_describe_table_options.reference b/tests/queries/0_stateless/02890_describe_table_options.reference index 9181cb27cb0..b77ef4a0fdf 100644 --- a/tests/queries/0_stateless/02890_describe_table_options.reference +++ b/tests/queries/0_stateless/02890_describe_table_options.reference @@ -54,6 +54,8 @@ _row_exists UInt8 Persisted mask created by lightweight delete that show wheth _block_number UInt64 Persisted original number of block that was assigned at insert Delta, LZ4 1 _block_offset UInt64 Persisted original number of row in block that was assigned at insert Delta, LZ4 1 _shard_num UInt32 Deprecated. Use function shardNum instead 1 +_database LowCardinality(String) The name of database which the row comes from 1 +_table LowCardinality(String) The name of table which the row comes from 1 SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options; id UInt64 index column 0 0 @@ -87,6 +89,8 @@ _row_exists UInt8 Persisted mask created by lightweight delete that show wheth _block_number UInt64 Persisted original number of block that was assigned at insert Delta, LZ4 0 1 _block_offset UInt64 Persisted original number of row in block that was assigned at insert Delta, LZ4 0 1 _shard_num UInt32 Deprecated. Use function shardNum instead 0 1 +_database LowCardinality(String) The name of database which the row comes from 0 1 +_table LowCardinality(String) The name of table which the row comes from 0 1 arr.size0 UInt64 1 0 t.a String ZSTD(1) 1 0 t.b UInt64 ZSTD(1) 1 0 @@ -144,6 +148,8 @@ _row_exists UInt8 1 _block_number UInt64 1 _block_offset UInt64 1 _shard_num UInt32 1 +_database LowCardinality(String) 1 +_table LowCardinality(String) 1 SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options; id UInt64 0 0 @@ -177,6 +183,8 @@ _row_exists UInt8 0 1 _block_number UInt64 0 1 _block_offset UInt64 0 1 _shard_num UInt32 0 1 +_database LowCardinality(String) 0 1 +_table LowCardinality(String) 0 1 arr.size0 UInt64 1 0 t.a String 1 0 t.b UInt64 1 0 From a9e793532ae308767da3bc4da74d9631cd85eb90 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 22 Aug 2024 16:34:14 +0200 Subject: [PATCH 07/22] fix shutdown for PeriodicLog --- src/Interpreters/PeriodicLog.cpp | 3 ++- src/Interpreters/PeriodicLog.h | 7 ++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/PeriodicLog.cpp b/src/Interpreters/PeriodicLog.cpp index 9d2891e11eb..15970ca5b81 100644 --- a/src/Interpreters/PeriodicLog.cpp +++ b/src/Interpreters/PeriodicLog.cpp @@ -1,6 +1,7 @@ #include #include #include +#include "Functions/DateTimeTransforms.h" namespace DB { @@ -27,7 +28,7 @@ template void PeriodicLog::shutdown() { stopCollect(); - this->stopFlushThread(); + Base::shutdown(); } template diff --git a/src/Interpreters/PeriodicLog.h b/src/Interpreters/PeriodicLog.h index 08c3f7eb23f..ceac8088d40 100644 --- a/src/Interpreters/PeriodicLog.h +++ b/src/Interpreters/PeriodicLog.h @@ -17,6 +17,7 @@ template class PeriodicLog : public SystemLog { using SystemLog::SystemLog; + using Base = SystemLog; public: using TimePoint = std::chrono::system_clock::time_point; @@ -24,12 +25,12 @@ public: /// Launches a background thread to collect metrics with interval void startCollect(size_t collect_interval_milliseconds_); - /// Stop background thread - void stopCollect(); - void shutdown() final; protected: + /// Stop background thread + void stopCollect(); + virtual void stepFunction(TimePoint current_time) = 0; private: From 1ea0163dfe6b3278d8a5e8d86c31b3d63d7a3780 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 22 Aug 2024 16:42:14 +0200 Subject: [PATCH 08/22] Fix issue with maps with arrays as keys --- src/Functions/FunctionsHashing.h | 4 ++-- tests/queries/0_stateless/02534_keyed_siphash.reference | 7 ++++++- tests/queries/0_stateless/02534_keyed_siphash.sql | 5 ++++- 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 0cf4246fd66..3da0b2cd9be 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -93,9 +93,9 @@ namespace impl if (is_const) i = 0; assert(key0->size() == key1->size()); - if (offsets != nullptr) + if (offsets != nullptr && i > 0) { - const auto * const begin = offsets->begin(); + const auto * const begin = std::upper_bound(offsets->begin(), offsets->end(), i - 1); const auto * upper = std::upper_bound(begin, offsets->end(), i); if (upper != offsets->end()) i = upper - begin; diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 31c0cae8981..8b147025a05 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -244,5 +244,10 @@ Test emtpy arrays and maps 0AD04BFD000000000000000000000000 4761183170873013810 0AD04BFD000000000000000000000000 +Test maps with arrays as keys 16734549324845627102 -D675BB3D687973A238AB891DD99C7047 +1D03941D808D04810D2363A6C107D622 +16734549324845627102 +16734549324845627102 +1D03941D808D04810D2363A6C107D622 +1D03941D808D04810D2363A6C107D622 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index b499d8ef02b..ba3c4a9156d 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -351,5 +351,8 @@ SELECT sipHash64Keyed((1::UInt64, 2::UInt64), []); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), [])); SELECT sipHash64Keyed((1::UInt64, 2::UInt64), mapFromArrays([], [])); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), mapFromArrays([], []))); +SELECT 'Test maps with arrays as keys'; SELECT sipHash64Keyed((1::UInt64, 2::UInt64), map([0], 1, [2], 3)); -SELECT hex(sipHash128Keyed((0::UInt64, 0::UInt64), map([0], 1, [2], 3))); +SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), map([0], 1, [2], 3))); +SELECT sipHash64Keyed((materialize(1::UInt64), 2::UInt64), map([0], 1, [2], 3)) FROM numbers(2); +SELECT hex(sipHash128Keyed((materialize(1::UInt64), 2::UInt64), map([0], 1, [2], 3))) FROM numbers(2); From a93d1919804d1c8dc7760f20084ade9a09710a47 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 22 Aug 2024 16:43:38 +0200 Subject: [PATCH 09/22] Fix typo in test case --- tests/queries/0_stateless/02534_keyed_siphash.reference | 2 +- tests/queries/0_stateless/02534_keyed_siphash.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 8b147025a05..a05446a494e 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -239,7 +239,7 @@ Check bug found fuzzing Test arrays and maps 608E1FF030C9E206185B112C2A25F1A7 ABB65AE97711A2E053E324ED88B1D08B -Test emtpy arrays and maps +Test empty arrays and maps 4761183170873013810 0AD04BFD000000000000000000000000 4761183170873013810 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index ba3c4a9156d..7cfc82512bd 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -346,7 +346,7 @@ INSERT INTO sipHashKeyed_keys FORMAT VALUES ({'a':'b', 'c':'d'}), ({'e':'f', 'g' SELECT hex(sipHash128ReferenceKeyed((0::UInt64, materialize(0::UInt64)), a)) FROM sipHashKeyed_keys ORDER BY a; DROP TABLE sipHashKeyed_keys; -SELECT 'Test emtpy arrays and maps'; +SELECT 'Test empty arrays and maps'; SELECT sipHash64Keyed((1::UInt64, 2::UInt64), []); SELECT hex(sipHash128Keyed((1::UInt64, 2::UInt64), [])); SELECT sipHash64Keyed((1::UInt64, 2::UInt64), mapFromArrays([], [])); From 0b9c24f31d548c87deca3334282c14fc78a295ba Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 15 Aug 2024 12:09:50 +0000 Subject: [PATCH 10/22] write metadata to disk and keeper in the same format --- src/Storages/ColumnsDescription.cpp | 30 +++++--- src/Storages/ColumnsDescription.h | 6 +- .../__init__.py | 0 .../config/enable_keeper.xml | 26 +++++++ .../config/users.xml | 8 +++ .../test.py | 71 +++++++++++++++++++ 6 files changed, 128 insertions(+), 13 deletions(-) create mode 100644 tests/integration/test_aliases_in_default_expr_not_break_table_structure/__init__.py create mode 100644 tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/enable_keeper.xml create mode 100644 tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/users.xml create mode 100644 tests/integration/test_aliases_in_default_expr_not_break_table_structure/test.py diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 0d724245b49..0212bbd6fff 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -113,7 +113,15 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const && ast_to_str(ttl) == ast_to_str(other.ttl); } -void ColumnDescription::writeText(WriteBuffer & buf) const +String formatASTStateAware(IAST & ast, IAST::FormatState & state) +{ + WriteBufferFromOwnString buf; + IAST::FormatSettings settings(buf, true, false); + ast.formatImpl(settings, state, IAST::FormatStateStacked()); + return buf.str(); +} + +void ColumnDescription::writeText(WriteBuffer & buf, IAST::FormatState & state, bool include_comment) const { /// NOTE: Serialization format is insane. @@ -126,20 +134,21 @@ void ColumnDescription::writeText(WriteBuffer & buf) const writeChar('\t', buf); DB::writeText(DB::toString(default_desc.kind), buf); writeChar('\t', buf); - writeEscapedString(queryToString(default_desc.expression), buf); + writeEscapedString(formatASTStateAware(*default_desc.expression, state), buf); } - if (!comment.empty()) + if (!comment.empty() && include_comment) { writeChar('\t', buf); DB::writeText("COMMENT ", buf); - writeEscapedString(queryToString(ASTLiteral(Field(comment))), buf); + auto ast = ASTLiteral(Field(comment)); + writeEscapedString(formatASTStateAware(ast, state), buf); } if (codec) { writeChar('\t', buf); - writeEscapedString(queryToString(codec), buf); + writeEscapedString(formatASTStateAware(*codec, state), buf); } if (!settings.empty()) @@ -150,21 +159,21 @@ void ColumnDescription::writeText(WriteBuffer & buf) const ASTSetQuery ast; ast.is_standalone = false; ast.changes = settings; - writeEscapedString(queryToString(ast), buf); + writeEscapedString(formatASTStateAware(ast, state), buf); DB::writeText(")", buf); } if (!statistics.empty()) { writeChar('\t', buf); - writeEscapedString(queryToString(statistics.getAST()), buf); + writeEscapedString(formatASTStateAware(*statistics.getAST(), state), buf); } if (ttl) { writeChar('\t', buf); DB::writeText("TTL ", buf); - writeEscapedString(queryToString(ttl), buf); + writeEscapedString(formatASTStateAware(*ttl, state), buf); } writeChar('\n', buf); @@ -895,16 +904,17 @@ void ColumnsDescription::resetColumnTTLs() } -String ColumnsDescription::toString() const +String ColumnsDescription::toString(bool include_comments) const { WriteBufferFromOwnString buf; + IAST::FormatState ast_format_state; writeCString("columns format version: 1\n", buf); DB::writeText(columns.size(), buf); writeCString(" columns:\n", buf); for (const ColumnDescription & column : columns) - column.writeText(buf); + column.writeText(buf, ast_format_state, include_comments); return buf.str(); } diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index f0760160f0a..c89c26501e8 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -104,7 +104,7 @@ struct ColumnDescription bool operator==(const ColumnDescription & other) const; bool operator!=(const ColumnDescription & other) const { return !(*this == other); } - void writeText(WriteBuffer & buf) const; + void writeText(WriteBuffer & buf, IAST::FormatState & state, bool include_comment) const; void readText(ReadBuffer & buf); }; @@ -137,7 +137,7 @@ public: /// NOTE Must correspond with Nested::flatten function. void flattenNested(); /// TODO: remove, insert already flattened Nested columns. - bool operator==(const ColumnsDescription & other) const { return columns == other.columns; } + bool operator==(const ColumnsDescription & other) const { return toString(false) == other.toString(false); } bool operator!=(const ColumnsDescription & other) const { return !(*this == other); } auto begin() const { return columns.begin(); } @@ -221,7 +221,7 @@ public: /// Does column has non default specified compression codec bool hasCompressionCodec(const String & column_name) const; - String toString() const; + String toString(bool include_comments = true) const; static ColumnsDescription parse(const String & str); size_t size() const diff --git a/tests/integration/test_aliases_in_default_expr_not_break_table_structure/__init__.py b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/enable_keeper.xml b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/enable_keeper.xml new file mode 100644 index 00000000000..4ca4f604ec3 --- /dev/null +++ b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/enable_keeper.xml @@ -0,0 +1,26 @@ + + + 2181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + 20000 + + + + 1 + localhost + 9444 + + + + + + + localhost + 2181 + + 20000 + + \ No newline at end of file diff --git a/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/users.xml b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/users.xml new file mode 100644 index 00000000000..c5de0b6819c --- /dev/null +++ b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/config/users.xml @@ -0,0 +1,8 @@ + + + + default + + + + \ No newline at end of file diff --git a/tests/integration/test_aliases_in_default_expr_not_break_table_structure/test.py b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/test.py new file mode 100644 index 00000000000..e0c15e18c23 --- /dev/null +++ b/tests/integration/test_aliases_in_default_expr_not_break_table_structure/test.py @@ -0,0 +1,71 @@ +import pytest +import random +import string + +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=[ + "config/enable_keeper.xml", + "config/users.xml", + ], + stay_alive=True, + with_minio=True, + macros={"shard": 1, "replica": 1}, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def randomize_table_name(table_name, random_suffix_length=10): + letters = string.ascii_letters + string.digits + return f"{table_name}_{''.join(random.choice(letters) for _ in range(random_suffix_length))}" + + +@pytest.mark.parametrize("engine", ["ReplicatedMergeTree"]) +def test_aliases_in_default_expr_not_break_table_structure(start_cluster, engine): + """ + Making sure that using aliases in columns' default expressions does not lead to having different columns metadata in ZooKeeper and on disk. + Issue: https://github.com/ClickHouse/clickhouse-private/issues/5150 + """ + + data = '{"event": {"col1-key": "col1-val", "col2-key": "col2-val"}}' + + table_name = randomize_table_name("t") + + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ( + `data` String, + `col1` String DEFAULT JSONExtractString(JSONExtractString(data, 'event') AS event, 'col1-key'), + `col2` String MATERIALIZED JSONExtractString(JSONExtractString(data, 'event') AS event, 'col2-key') + ) + ENGINE = {engine}('/test/{table_name}', '{{replica}}') + ORDER BY col1 + """ + ) + + node.restart_clickhouse() + + node.query( + f""" + INSERT INTO {table_name} (data) VALUES ('{data}'); + """ + ) + assert node.query(f"SELECT data FROM {table_name}").strip() == data + assert node.query(f"SELECT col1 FROM {table_name}").strip() == "col1-val" + assert node.query(f"SELECT col2 FROM {table_name}").strip() == "col2-val" + + node.query(f"DROP TABLE {table_name}") From 859d2bfe273f571458be6f007761bc8c743d589a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 22 Aug 2024 17:18:06 +0200 Subject: [PATCH 11/22] move stopFlushThread to SystemLogBase --- src/Common/SystemLogBase.cpp | 19 +++++++++++++++++++ src/Common/SystemLogBase.h | 2 ++ src/Interpreters/PeriodicLog.cpp | 6 +++--- src/Interpreters/PeriodicLog.h | 2 +- src/Interpreters/SystemLog.cpp | 21 +-------------------- src/Interpreters/SystemLog.h | 7 +------ 6 files changed, 27 insertions(+), 30 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 127c8862a35..45f4eb1c5a6 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -273,6 +273,25 @@ void SystemLogBase::startup() saving_thread = std::make_unique([this] { savingThreadFunction(); }); } +template +void SystemLogBase::stopFlushThread() +{ + { + std::lock_guard lock(thread_mutex); + + if (!saving_thread || !saving_thread->joinable()) + return; + + if (is_shutdown) + return; + + is_shutdown = true; + queue->shutdown(); + } + + saving_thread->join(); +} + template void SystemLogBase::add(LogElement element) { diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 0d7b04d5c57..0942e920a42 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -216,6 +216,8 @@ public: static consteval bool shouldTurnOffLogger() { return false; } protected: + void stopFlushThread() final; + std::shared_ptr> queue; }; } diff --git a/src/Interpreters/PeriodicLog.cpp b/src/Interpreters/PeriodicLog.cpp index 15970ca5b81..1b285aad3ff 100644 --- a/src/Interpreters/PeriodicLog.cpp +++ b/src/Interpreters/PeriodicLog.cpp @@ -11,7 +11,7 @@ void PeriodicLog::startCollect(size_t collect_interval_milliseconds_ { collect_interval_milliseconds = collect_interval_milliseconds_; is_shutdown_metric_thread = false; - flush_thread = std::make_unique([this] { threadFunction(); }); + collecting_thread = std::make_unique([this] { threadFunction(); }); } template @@ -20,8 +20,8 @@ void PeriodicLog::stopCollect() bool old_val = false; if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true)) return; - if (flush_thread) - flush_thread->join(); + if (collecting_thread) + collecting_thread->join(); } template diff --git a/src/Interpreters/PeriodicLog.h b/src/Interpreters/PeriodicLog.h index ceac8088d40..8254a02434a 100644 --- a/src/Interpreters/PeriodicLog.h +++ b/src/Interpreters/PeriodicLog.h @@ -36,7 +36,7 @@ protected: private: void threadFunction(); - std::unique_ptr flush_thread; + std::unique_ptr collecting_thread; size_t collect_interval_milliseconds; std::atomic is_shutdown_metric_thread{false}; }; diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 832c39bfaf8..6a3ec197c6e 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -402,32 +402,13 @@ SystemLog::SystemLog( template void SystemLog::shutdown() { - stopFlushThread(); + Base::stopFlushThread(); auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); if (table) table->flushAndShutdown(); } -template -void SystemLog::stopFlushThread() -{ - { - std::lock_guard lock(thread_mutex); - - if (!saving_thread || !saving_thread->joinable()) - return; - - if (is_shutdown) - return; - - is_shutdown = true; - queue->shutdown(); - } - - saving_thread->join(); -} - template void SystemLog::savingThreadFunction() diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 9e1af3578bd..31652c1af67 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -125,8 +125,6 @@ public: void shutdown() override; - void stopFlushThread() override; - /** Creates new table if it does not exist. * Renames old table if its structure is not suitable. * This cannot be done in constructor to avoid deadlock while renaming a table under locked Context when SystemLog object is created. @@ -136,10 +134,7 @@ public: protected: LoggerPtr log; - using ISystemLog::is_shutdown; - using ISystemLog::saving_thread; - using ISystemLog::thread_mutex; - using Base::queue; + using Base::queue; StoragePtr getStorage() const; From 9c0e1df1663dd5c56066dd615fc3cafe6408d308 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Aug 2024 17:58:15 +0200 Subject: [PATCH 12/22] Fix flaky test 00989_parallel_parts_loading --- tests/queries/0_stateless/00989_parallel_parts_loading.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/00989_parallel_parts_loading.sql b/tests/queries/0_stateless/00989_parallel_parts_loading.sql index 407e124f137..dc074241ff6 100644 --- a/tests/queries/0_stateless/00989_parallel_parts_loading.sql +++ b/tests/queries/0_stateless/00989_parallel_parts_loading.sql @@ -1,3 +1,5 @@ +-- Tags: no-random-settings, no-random-merge-tree-settings +-- small insert block size can make insert terribly slow, especially with some build like msan DROP TABLE IF EXISTS mt; CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; From 52cdd88eb6d7bbb5d395dd80445655ad47c83c92 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Aug 2024 17:59:10 +0200 Subject: [PATCH 13/22] Better comment --- tests/queries/0_stateless/00989_parallel_parts_loading.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00989_parallel_parts_loading.sql b/tests/queries/0_stateless/00989_parallel_parts_loading.sql index dc074241ff6..3b73e6a0e3c 100644 --- a/tests/queries/0_stateless/00989_parallel_parts_loading.sql +++ b/tests/queries/0_stateless/00989_parallel_parts_loading.sql @@ -1,5 +1,5 @@ -- Tags: no-random-settings, no-random-merge-tree-settings --- small insert block size can make insert terribly slow, especially with some build like msan +-- small number of insert threads can make insert terribly slow, especially with some build like msan DROP TABLE IF EXISTS mt; CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; From e7b89537bf1bb760c6082f04de4668bd1c00f33a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 22 Aug 2024 18:02:42 +0200 Subject: [PATCH 14/22] fix style --- src/Interpreters/PeriodicLog.cpp | 1 - src/Interpreters/SystemLog.h | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/PeriodicLog.cpp b/src/Interpreters/PeriodicLog.cpp index 1b285aad3ff..22bc14856c4 100644 --- a/src/Interpreters/PeriodicLog.cpp +++ b/src/Interpreters/PeriodicLog.cpp @@ -1,7 +1,6 @@ #include #include #include -#include "Functions/DateTimeTransforms.h" namespace DB { diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 31652c1af67..c03f9370068 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -134,7 +134,7 @@ public: protected: LoggerPtr log; - using Base::queue; + using Base::queue; StoragePtr getStorage() const; From fa453c3664b18da7a6945e662b881f80fedadc5b Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 22 Aug 2024 18:13:45 +0200 Subject: [PATCH 15/22] Disable SqlLogic job --- tests/ci/ci_config.py | 7 ++++--- tests/ci/ci_definitions.py | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 58de25f039f..0885f1d9ec2 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -498,9 +498,10 @@ class CI: JobNames.SQLANCER_DEBUG: CommonJobConfigs.SQLLANCER_TEST.with_properties( required_builds=[BuildNames.PACKAGE_DEBUG], ), - JobNames.SQL_LOGIC_TEST: CommonJobConfigs.SQLLOGIC_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_RELEASE], - ), + # TODO: job does not work at all, uncomment and fix + # JobNames.SQL_LOGIC_TEST: CommonJobConfigs.SQLLOGIC_TEST.with_properties( + # required_builds=[BuildNames.PACKAGE_RELEASE], + # ), JobNames.SQLTEST: CommonJobConfigs.SQL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], ), diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 1cdb3f1487e..9d95a19790f 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -204,7 +204,7 @@ class JobNames(metaclass=WithIter): PERFORMANCE_TEST_AMD64 = "Performance Comparison (release)" PERFORMANCE_TEST_ARM64 = "Performance Comparison (aarch64)" - SQL_LOGIC_TEST = "Sqllogic test (release)" + # SQL_LOGIC_TEST = "Sqllogic test (release)" SQLANCER = "SQLancer (release)" SQLANCER_DEBUG = "SQLancer (debug)" From 4200b3d5cbbfe065073c40f1e122c44189f3554f Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 22 Aug 2024 14:02:25 +0200 Subject: [PATCH 16/22] CI: Stress test fix --- tests/clickhouse-test | 2 +- tests/docker_scripts/stress_runner.sh | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 4f9380d6f20..ad6173065fe 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -3567,7 +3567,7 @@ if __name__ == "__main__": f"Cannot access the specified directory with queries ({args.queries})", file=sys.stderr, ) - sys.exit(1) + assert False, "No --queries provided" CAPTURE_CLIENT_STACKTRACE = args.capture_client_stacktrace diff --git a/tests/docker_scripts/stress_runner.sh b/tests/docker_scripts/stress_runner.sh index 7666398e10b..039c60c8e4e 100755 --- a/tests/docker_scripts/stress_runner.sh +++ b/tests/docker_scripts/stress_runner.sh @@ -10,8 +10,7 @@ dmesg --clear # shellcheck disable=SC1091 source /setup_export_logs.sh -ln -s /repo/tests/clickhouse-test/ci/stress.py /usr/bin/stress -ln -s /repo/tests/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test +ln -s /repo/tests/clickhouse-test /usr/bin/clickhouse-test # Stress tests and upgrade check uses similar code that was placed # in a separate bash library. See tests/ci/stress_tests.lib @@ -266,6 +265,7 @@ fi start_server +cd /repo/tests/ || exit 1 # clickhouse-test can find queries dir from there python3 /repo/tests/ci/stress.py --hung-check --drop-databases --output-folder /test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \ && echo -e "Test script exit code$OK" >> /test_output/test_results.tsv \ || echo -e "Test script failed$FAIL script exit code: $?" >> /test_output/test_results.tsv From 4c790999eb6ad74e3a8f99c072dcc12c956a63d8 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 23 Aug 2024 02:18:26 +0200 Subject: [PATCH 17/22] CI: Force package_debug build on release branches --- .github/workflows/release_branches.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 82826794ea3..ec119b6ff95 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -130,6 +130,7 @@ jobs: with: build_name: package_debug data: ${{ needs.RunConfig.outputs.data }} + force: true BuilderBinDarwin: needs: [RunConfig, BuildDockers] if: ${{ !failure() && !cancelled() }} From f5739dfe06db8610818fafb5c3a2c33f59fd0a8d Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 23 Aug 2024 02:51:27 +0200 Subject: [PATCH 18/22] CI: Make job rerun possible if triggered manually --- tests/ci/ci.py | 7 +++++-- tests/ci/ci_utils.py | 5 +++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index a9ae078b449..d201b6602f5 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -333,7 +333,10 @@ def _pre_action(s3, job_name, batch, indata, pr_info): CI.JobNames.BUILD_CHECK, ): # we might want to rerun build report job rerun_helper = RerunHelper(commit, _get_ext_check_name(job_name)) - if rerun_helper.is_already_finished_by_status(): + if ( + rerun_helper.is_already_finished_by_status() + and not Utils.is_job_triggered_manually() + ): print("WARNING: Rerunning job with GH status ") status = rerun_helper.get_finished_status() assert status @@ -344,7 +347,7 @@ def _pre_action(s3, job_name, batch, indata, pr_info): skip_status = status.state # ci cache check - if not to_be_skipped and not no_cache: + if not to_be_skipped and not no_cache and not Utils.is_job_triggered_manually(): ci_cache = CiCache(s3, indata["jobs_data"]["digests"]).update() job_config = CI.get_job_config(job_name) if ci_cache.is_successful( diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index a4c0977f47c..e8d9e7dc254 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -18,6 +18,7 @@ class Envs: ) S3_BUILDS_BUCKET = os.getenv("S3_BUILDS_BUCKET", "clickhouse-builds") GITHUB_WORKFLOW = os.getenv("GITHUB_WORKFLOW", "") + GITHUB_ACTOR = os.getenv("GITHUB_ACTOR", "") class WithIter(type): @@ -282,3 +283,7 @@ class Utils: ): res = res.replace(*r) return res + + @staticmethod + def is_job_triggered_manually(): + return "robot" not in Envs.GITHUB_ACTOR From 60e4bcbbf0b1991b42bcab4b83e55be344e8a659 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Thu, 22 Aug 2024 20:45:28 -0700 Subject: [PATCH 19/22] Update README.md Update Raleigh meetup link --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 83a5c05c667..546f08afd3d 100644 --- a/README.md +++ b/README.md @@ -44,7 +44,7 @@ The following upcoming meetups are featuring creator of ClickHouse & CTO, Alexey * [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 * [San Francisco Meetup (Cloudflare)](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/302540575) - September 5 -* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/clickhouse-nc-meetup-group/events/302557230) - September 9 +* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/triangletechtalks/events/302723486/) - September 9 * [New York Meetup (Rokt)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 * [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12 From e5380806653f8d391c6e88664b0096c3c51240f5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 23 Aug 2024 07:09:03 +0000 Subject: [PATCH 20/22] Update version_date.tsv and changelogs after v24.5.6.45-stable --- docs/changelogs/v24.5.6.45-stable.md | 33 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 +- 2 files changed, 34 insertions(+), 1 deletion(-) create mode 100644 docs/changelogs/v24.5.6.45-stable.md diff --git a/docs/changelogs/v24.5.6.45-stable.md b/docs/changelogs/v24.5.6.45-stable.md new file mode 100644 index 00000000000..b329ebab27b --- /dev/null +++ b/docs/changelogs/v24.5.6.45-stable.md @@ -0,0 +1,33 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.5.6.45-stable (bdca8604c29) FIXME as compared to v24.5.5.78-stable (0138248cb62) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#67902](https://github.com/ClickHouse/ClickHouse/issues/67902): Fixing the `Not-ready Set` error after the `PREWHERE` optimization for StorageMerge. [#65057](https://github.com/ClickHouse/ClickHouse/pull/65057) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#68252](https://github.com/ClickHouse/ClickHouse/issues/68252): Fixed `Not-ready Set` in some system tables when filtering using subqueries. [#66018](https://github.com/ClickHouse/ClickHouse/pull/66018) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#68064](https://github.com/ClickHouse/ClickHouse/issues/68064): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68158](https://github.com/ClickHouse/ClickHouse/issues/68158): Fix cluster() for inter-server secret (preserve initial user as before). [#66364](https://github.com/ClickHouse/ClickHouse/pull/66364) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68115](https://github.com/ClickHouse/ClickHouse/issues/68115): Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67886](https://github.com/ClickHouse/ClickHouse/issues/67886): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#68272](https://github.com/ClickHouse/ClickHouse/issues/68272): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67807](https://github.com/ClickHouse/ClickHouse/issues/67807): Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. [#67665](https://github.com/ClickHouse/ClickHouse/pull/67665) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#67836](https://github.com/ClickHouse/ClickHouse/issues/67836): Fix potential stack overflow in `JSONMergePatch` function. Renamed this function from `jsonMergePatch` to `JSONMergePatch` because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes [#67304](https://github.com/ClickHouse/ClickHouse/issues/67304). [#67756](https://github.com/ClickHouse/ClickHouse/pull/67756) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#67991](https://github.com/ClickHouse/ClickHouse/issues/67991): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68207](https://github.com/ClickHouse/ClickHouse/issues/68207): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68091](https://github.com/ClickHouse/ClickHouse/issues/68091): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68122](https://github.com/ClickHouse/ClickHouse/issues/68122): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68171](https://github.com/ClickHouse/ClickHouse/issues/68171): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68337](https://github.com/ClickHouse/ClickHouse/issues/68337): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68667](https://github.com/ClickHouse/ClickHouse/issues/68667): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Update version after release. [#67862](https://github.com/ClickHouse/ClickHouse/pull/67862) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68077](https://github.com/ClickHouse/ClickHouse/issues/68077): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). +* Backported in [#68756](https://github.com/ClickHouse/ClickHouse/issues/68756): To make patch release possible from every commit on release branch, package_debug build is required and must not be skipped. [#68750](https://github.com/ClickHouse/ClickHouse/pull/68750) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0e25f8d3b62..57a59d7ac49 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -6,8 +6,8 @@ v24.7.1.2915-stable 2024-07-30 v24.6.3.95-stable 2024-08-06 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 +v24.5.6.45-stable 2024-08-23 v24.5.5.78-stable 2024-08-05 -v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 From e1a7bd9163bebf0aeab12d8dd46c729f73b068be Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 23 Aug 2024 07:37:32 +0000 Subject: [PATCH 21/22] Update version_date.tsv and changelogs after v24.6.4.42-stable --- docs/changelogs/v24.6.4.42-stable.md | 33 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 3 ++- 2 files changed, 35 insertions(+), 1 deletion(-) create mode 100644 docs/changelogs/v24.6.4.42-stable.md diff --git a/docs/changelogs/v24.6.4.42-stable.md b/docs/changelogs/v24.6.4.42-stable.md new file mode 100644 index 00000000000..29b6ba095af --- /dev/null +++ b/docs/changelogs/v24.6.4.42-stable.md @@ -0,0 +1,33 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.6.4.42-stable (c534bb4b4dd) FIXME as compared to v24.6.3.95-stable (8325c920d11) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#68066](https://github.com/ClickHouse/ClickHouse/issues/68066): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68566](https://github.com/ClickHouse/ClickHouse/issues/68566): Fix indexHint function case found by fuzzer. [#66286](https://github.com/ClickHouse/ClickHouse/pull/66286) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68159](https://github.com/ClickHouse/ClickHouse/issues/68159): Fix cluster() for inter-server secret (preserve initial user as before). [#66364](https://github.com/ClickHouse/ClickHouse/pull/66364) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68116](https://github.com/ClickHouse/ClickHouse/issues/68116): Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn. [#66659](https://github.com/ClickHouse/ClickHouse/pull/66659) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#67887](https://github.com/ClickHouse/ClickHouse/issues/67887): Correctly parse file name/URI containing `::` if it's not an archive. [#67433](https://github.com/ClickHouse/ClickHouse/pull/67433) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#68611](https://github.com/ClickHouse/ClickHouse/issues/68611): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68275](https://github.com/ClickHouse/ClickHouse/issues/68275): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#67993](https://github.com/ClickHouse/ClickHouse/issues/67993): Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. [#67911](https://github.com/ClickHouse/ClickHouse/pull/67911) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68208](https://github.com/ClickHouse/ClickHouse/issues/68208): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68093](https://github.com/ClickHouse/ClickHouse/issues/68093): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68124](https://github.com/ClickHouse/ClickHouse/issues/68124): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68221](https://github.com/ClickHouse/ClickHouse/issues/68221): Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. [#68098](https://github.com/ClickHouse/ClickHouse/pull/68098) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Backported in [#68173](https://github.com/ClickHouse/ClickHouse/issues/68173): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68339](https://github.com/ClickHouse/ClickHouse/issues/68339): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68396](https://github.com/ClickHouse/ClickHouse/issues/68396): Fix missing sync replica mode in query `SYSTEM SYNC REPLICA`. [#68326](https://github.com/ClickHouse/ClickHouse/pull/68326) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#68668](https://github.com/ClickHouse/ClickHouse/issues/68668): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Update version after release. [#67909](https://github.com/ClickHouse/ClickHouse/pull/67909) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68079](https://github.com/ClickHouse/ClickHouse/issues/68079): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). +* Backported in [#68758](https://github.com/ClickHouse/ClickHouse/issues/68758): To make patch release possible from every commit on release branch, package_debug build is required and must not be skipped. [#68750](https://github.com/ClickHouse/ClickHouse/pull/68750) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0e25f8d3b62..8ce510f110d 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -3,11 +3,12 @@ v24.8.1.2684-lts 2024-08-21 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 +v24.6.4.42-stable 2024-08-23 v24.6.3.95-stable 2024-08-06 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 +v24.5.6.45-stable 2024-08-23 v24.5.5.78-stable 2024-08-05 -v24.5.5.41-stable 2024-08-22 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 v24.5.2.34-stable 2024-06-13 From eec720dab60ea63b033919bbc4c1f6837920a42d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 23 Aug 2024 08:05:27 +0000 Subject: [PATCH 22/22] Update version_date.tsv and changelogs after v24.7.4.51-stable --- docs/changelogs/v24.7.4.51-stable.md | 36 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 2 files changed, 38 insertions(+) create mode 100644 docs/changelogs/v24.7.4.51-stable.md diff --git a/docs/changelogs/v24.7.4.51-stable.md b/docs/changelogs/v24.7.4.51-stable.md new file mode 100644 index 00000000000..a7cf9790383 --- /dev/null +++ b/docs/changelogs/v24.7.4.51-stable.md @@ -0,0 +1,36 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.7.4.51-stable (70fe2f6fa52) FIXME as compared to v24.7.3.42-stable (63730bc4293) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#68232](https://github.com/ClickHouse/ClickHouse/issues/68232): Fixed `Not-ready Set` in some system tables when filtering using subqueries. [#66018](https://github.com/ClickHouse/ClickHouse/pull/66018) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#68068](https://github.com/ClickHouse/ClickHouse/issues/68068): Fix boolean literals in query sent to external database (for engines like `PostgreSQL`). [#66282](https://github.com/ClickHouse/ClickHouse/pull/66282) ([vdimir](https://github.com/vdimir)). +* Backported in [#68613](https://github.com/ClickHouse/ClickHouse/issues/68613): Fixes [#66026](https://github.com/ClickHouse/ClickHouse/issues/66026). Avoid unresolved table function arguments traversal in `ReplaceTableNodeToDummyVisitor`. [#67522](https://github.com/ClickHouse/ClickHouse/pull/67522) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#68278](https://github.com/ClickHouse/ClickHouse/issues/68278): Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. [#67554](https://github.com/ClickHouse/ClickHouse/pull/67554) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68040](https://github.com/ClickHouse/ClickHouse/issues/68040): Fix creation of view with recursive CTE. [#67587](https://github.com/ClickHouse/ClickHouse/pull/67587) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#68038](https://github.com/ClickHouse/ClickHouse/issues/68038): Fix crash on `percent_rank`. `percent_rank`'s default frame type is changed to `range unbounded preceding and unbounded following`. `IWindowFunction`'s default window frame is considered and now window functions without window frame definition in sql can be put into different `WindowTransfomer`s properly. [#67661](https://github.com/ClickHouse/ClickHouse/pull/67661) ([lgbo](https://github.com/lgbo-ustc)). +* Backported in [#68224](https://github.com/ClickHouse/ClickHouse/issues/68224): Fix wrong `count()` result when there is non-deterministic function in predicate. [#67922](https://github.com/ClickHouse/ClickHouse/pull/67922) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#68095](https://github.com/ClickHouse/ClickHouse/issues/68095): Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. [#67963](https://github.com/ClickHouse/ClickHouse/pull/67963) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#68126](https://github.com/ClickHouse/ClickHouse/issues/68126): Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. [#68052](https://github.com/ClickHouse/ClickHouse/pull/68052) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#68223](https://github.com/ClickHouse/ClickHouse/issues/68223): Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. [#68098](https://github.com/ClickHouse/ClickHouse/pull/68098) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Backported in [#68175](https://github.com/ClickHouse/ClickHouse/issues/68175): Removes an incorrect optimization to remove sorting in subqueries that use `OFFSET`. Fixes [#67906](https://github.com/ClickHouse/ClickHouse/issues/67906). [#68099](https://github.com/ClickHouse/ClickHouse/pull/68099) ([Graham Campbell](https://github.com/GrahamCampbell)). +* Backported in [#68341](https://github.com/ClickHouse/ClickHouse/issues/68341): Try fix postgres crash when query is cancelled. [#68288](https://github.com/ClickHouse/ClickHouse/pull/68288) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#68398](https://github.com/ClickHouse/ClickHouse/issues/68398): Fix missing sync replica mode in query `SYSTEM SYNC REPLICA`. [#68326](https://github.com/ClickHouse/ClickHouse/pull/68326) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#68669](https://github.com/ClickHouse/ClickHouse/issues/68669): Fix `LOGICAL_ERROR`s when functions `sipHash64Keyed`, `sipHash128Keyed`, or `sipHash128ReferenceKeyed` are applied to empty arrays or tuples. [#68630](https://github.com/ClickHouse/ClickHouse/pull/68630) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#67803](https://github.com/ClickHouse/ClickHouse/issues/67803): Disable some Dynamic tests under sanitizers, rewrite 03202_dynamic_null_map_subcolumn to sql. [#67359](https://github.com/ClickHouse/ClickHouse/pull/67359) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#68081](https://github.com/ClickHouse/ClickHouse/issues/68081): Add an explicit error for `ALTER MODIFY SQL SECURITY` on non-view tables. [#67953](https://github.com/ClickHouse/ClickHouse/pull/67953) ([pufit](https://github.com/pufit)). +* Update version after release. [#68044](https://github.com/ClickHouse/ClickHouse/pull/68044) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Backported in [#68269](https://github.com/ClickHouse/ClickHouse/issues/68269): [Green CI] Fix test 01903_correct_block_size_prediction_with_default. [#68203](https://github.com/ClickHouse/ClickHouse/pull/68203) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#68432](https://github.com/ClickHouse/ClickHouse/issues/68432): tests: make 01600_parts_states_metrics_long better. [#68265](https://github.com/ClickHouse/ClickHouse/pull/68265) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#68538](https://github.com/ClickHouse/ClickHouse/issues/68538): CI: Native build for package_aarch64. [#68457](https://github.com/ClickHouse/ClickHouse/pull/68457) ([Max K.](https://github.com/maxknv)). +* Backported in [#68555](https://github.com/ClickHouse/ClickHouse/issues/68555): CI: Minor release workflow fix. [#68536](https://github.com/ClickHouse/ClickHouse/pull/68536) ([Max K.](https://github.com/maxknv)). +* Backported in [#68760](https://github.com/ClickHouse/ClickHouse/issues/68760): To make patch release possible from every commit on release branch, package_debug build is required and must not be skipped. [#68750](https://github.com/ClickHouse/ClickHouse/pull/68750) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 57a59d7ac49..d9674ed2366 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,8 +1,10 @@ v24.8.2.3-lts 2024-08-22 v24.8.1.2684-lts 2024-08-21 +v24.7.4.51-stable 2024-08-23 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 +v24.6.4.42-stable 2024-08-23 v24.6.3.95-stable 2024-08-06 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01