From 0818092ae8d49f2e7f87fed6c8703374384719fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 19:45:57 +0200 Subject: [PATCH 001/133] Enable Sparse columns by default --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 5416b77a97e3..27f482d79ba9 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -37,7 +37,7 @@ struct Settings; M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ - M(Float, ratio_of_defaults_for_sparse_serialization, 1.0, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ + M(Float, ratio_of_defaults_for_sparse_serialization, 0.95, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, 8192, "How many rows in blocks should be formed for merge operations. By default has the same value as `index_granularity`.", 0) \ From 7ec98205b58ab36eb28b2f46348dfcfe22215a3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 22:54:14 +0300 Subject: [PATCH 002/133] Update MergeTreeSettings.h --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 27f482d79ba9..caac86c67062 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -37,7 +37,7 @@ struct Settings; M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ - M(Float, ratio_of_defaults_for_sparse_serialization, 0.95, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ + M(Float, ratio_of_defaults_for_sparse_serialization, 0.9375f, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, 8192, "How many rows in blocks should be formed for merge operations. By default has the same value as `index_granularity`.", 0) \ From f3f6ccd7733aa4946c339b4973210f85243e44d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 00:28:54 +0200 Subject: [PATCH 003/133] Update tests --- .../0_stateless/00443_preferred_block_size_bytes.sh | 6 +++--- ...0484_preferred_max_column_in_block_size_bytes.sql | 8 ++++---- .../00804_test_delta_codec_compression.sql | 12 ++++++------ .../0_stateless/00950_test_double_delta_codec.sql | 2 +- ...00961_checksums_in_system_parts_columns_table.sql | 2 +- .../0_stateless/01055_compact_parts_granularity.sh | 2 +- .../queries/0_stateless/01786_explain_merge_tree.sh | 4 ++-- tests/queries/0_stateless/02263_lazy_mark_load.sh | 2 +- .../0_stateless/02293_selected_rows_and_merges.sh | 8 +++----- .../0_stateless/02361_fsync_profile_events.sh | 7 ++++--- .../02381_compress_marks_and_primary_key.sql | 4 ++-- 11 files changed, 28 insertions(+), 29 deletions(-) diff --git a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh index c184b58bf535..27b9f5c00c7f 100755 --- a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh +++ b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" -$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "INSERT INTO preferred_block_size_bytes (s) SELECT '16_bytes_-_-_-_' AS s FROM system.numbers LIMIT 10, 90" $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE preferred_block_size_bytes" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=26 -q "SELECT DISTINCT blockSize(), ignore(p, s) FROM preferred_block_size_bytes" @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" # PREWHERE using empty column $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS pbs" -$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "INSERT INTO pbs (p, i, sa) SELECT toDate(i % 30) AS p, number AS i, ['a'] AS sa FROM system.numbers LIMIT 1000" $CLICKHOUSE_CLIENT -q "ALTER TABLE pbs ADD COLUMN s UInt8 DEFAULT 0" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=100000 -q "SELECT count() FROM pbs PREWHERE s = 0" @@ -30,7 +30,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE pbs" # Nullable PREWHERE $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" -$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "INSERT INTO nullable_prewhere SELECT toDate(0) AS p, if(number % 2 = 0, CAST(number AS Nullable(UInt64)), CAST(NULL AS Nullable(UInt64))) AS f, number as d FROM system.numbers LIMIT 1001" $CLICKHOUSE_CLIENT -q "SELECT sum(d), sum(f), max(d) FROM nullable_prewhere PREWHERE NOT isNull(f)" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" diff --git a/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql b/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql index 470bca70e061..be4af2221a54 100644 --- a/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql +++ b/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql @@ -1,7 +1,7 @@ -- Tags: no-random-settings drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 8192; set preferred_block_size_bytes = 2000000; @@ -17,19 +17,19 @@ set preferred_max_column_in_block_size_bytes = 4194304; select max(blockSize()), min(blockSize()), any(ignore(*)) from tab_00484; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 47; set preferred_max_column_in_block_size_bytes = 1152; select blockSize(), * from tab_00484 where x = 1 or x > 36 format Null; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 10; set preferred_max_column_in_block_size_bytes = 128; select s from tab_00484 where s == '' format Null; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s String) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; +create table tab_00484 (date Date, x UInt64, s String) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; insert into tab_00484 select today(), number, 'abc' from system.numbers limit 81920; set preferred_block_size_bytes = 0; select count(*) from tab_00484 prewhere s != 'abc' format Null; diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql index 25988f6474b6..01a2f53bf93b 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql @@ -9,12 +9,12 @@ DROP TABLE IF EXISTS default_codec_synthetic; CREATE TABLE delta_codec_synthetic ( id UInt64 Codec(Delta, ZSTD(3)) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; CREATE TABLE default_codec_synthetic ( id UInt64 Codec(ZSTD(3)) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO delta_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000; INSERT INTO default_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000; @@ -47,12 +47,12 @@ DROP TABLE IF EXISTS default_codec_float; CREATE TABLE delta_codec_float ( id Float64 Codec(Delta, LZ4HC) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; CREATE TABLE default_codec_float ( id Float64 Codec(LZ4HC) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO delta_codec_float SELECT number FROM numbers(1547510400, 500000) WHERE number % 3 == 0 OR number % 5 == 0 OR number % 7 == 0 OR number % 11 == 0; INSERT INTO default_codec_float SELECT * from delta_codec_float; @@ -85,12 +85,12 @@ DROP TABLE IF EXISTS default_codec_string; CREATE TABLE delta_codec_string ( id Float64 Codec(Delta, LZ4) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; CREATE TABLE default_codec_string ( id Float64 Codec(LZ4) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO delta_codec_string SELECT concat(toString(number), toString(number % 100)) FROM numbers(1547510400, 500000); INSERT INTO default_codec_string SELECT * from delta_codec_string; diff --git a/tests/queries/0_stateless/00950_test_double_delta_codec.sql b/tests/queries/0_stateless/00950_test_double_delta_codec.sql index f6199a6e4ec0..58cf35b52487 100644 --- a/tests/queries/0_stateless/00950_test_double_delta_codec.sql +++ b/tests/queries/0_stateless/00950_test_double_delta_codec.sql @@ -24,7 +24,7 @@ CREATE TABLE codecTest ( valueI8 Int8 CODEC(DoubleDelta), valueDT DateTime CODEC(DoubleDelta), valueD Date CODEC(DoubleDelta) -) Engine = MergeTree ORDER BY key SETTINGS min_bytes_for_wide_part = 0; +) Engine = MergeTree ORDER BY key SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; -- checking for overflow diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql index 43b7775e8162..8df7d728560d 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS test_00961; CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree PARTITION BY d ORDER BY (a, b) - SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0, compress_marks = 0, compress_primary_key = 0, index_granularity_bytes = '10Mi'; + SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0, compress_marks = 0, compress_primary_key = 0, index_granularity_bytes = '10Mi', ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789); diff --git a/tests/queries/0_stateless/01055_compact_parts_granularity.sh b/tests/queries/0_stateless/01055_compact_parts_granularity.sh index f3da33f6ccfd..3e5da1e6f90d 100755 --- a/tests/queries/0_stateless/01055_compact_parts_granularity.sh +++ b/tests/queries/0_stateless/01055_compact_parts_granularity.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mt_compact" $CLICKHOUSE_CLIENT -q "CREATE TABLE mt_compact(a Int, s String) ENGINE = MergeTree ORDER BY a SETTINGS min_rows_for_wide_part = 1000, - index_granularity = 14;" + index_granularity = 14, ratio_of_defaults_for_sparse_serialization = 1;" $CLICKHOUSE_CLIENT -q "SYSTEM STOP MERGES mt_compact" diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.sh b/tests/queries/0_stateless/01786_explain_merge_tree.sh index 15f8821d80d7..0d4acba338af 100755 --- a/tests/queries/0_stateless/01786_explain_merge_tree.sh +++ b/tests/queries/0_stateless/01786_explain_merge_tree.sh @@ -10,7 +10,7 @@ CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --optimize_move_to_prewhere=1 --convert_qu $CLICKHOUSE_CLIENT -q "drop table if exists test_index" $CLICKHOUSE_CLIENT -q "drop table if exists idx" -$CLICKHOUSE_CLIENT -q "create table test_index (x UInt32, y UInt32, z UInt32, t UInt32, index t_minmax t % 20 TYPE minmax GRANULARITY 2, index t_set t % 19 type set(4) granularity 2) engine = MergeTree order by (x, y) partition by (y, bitAnd(z, 3), intDiv(t, 15)) settings index_granularity = 2, min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "create table test_index (x UInt32, y UInt32, z UInt32, t UInt32, index t_minmax t % 20 TYPE minmax GRANULARITY 2, index t_set t % 19 type set(4) granularity 2) engine = MergeTree order by (x, y) partition by (y, bitAnd(z, 3), intDiv(t, 15)) settings index_granularity = 2, min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "insert into test_index select number, number > 3 ? 3 : number, number = 1 ? 1 : 0, number from numbers(20)" $CLICKHOUSE_CLIENT -q " @@ -35,7 +35,7 @@ $CLICKHOUSE_CLIENT -q " explain actions = 1 select x from test_index where x > 15 order by x desc; " | grep -A 100 "ReadFromMergeTree" -$CLICKHOUSE_CLIENT -q "CREATE TABLE idx (x UInt32, y UInt32, z UInt32) ENGINE = MergeTree ORDER BY (x, x + y) settings min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE idx (x UInt32, y UInt32, z UInt32) ENGINE = MergeTree ORDER BY (x, x + y) settings min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "insert into idx select number, number, number from numbers(10)" $CLICKHOUSE_CLIENT -q " diff --git a/tests/queries/0_stateless/02263_lazy_mark_load.sh b/tests/queries/0_stateless/02263_lazy_mark_load.sh index bf37556bfa60..35a1b4a44dd1 100755 --- a/tests/queries/0_stateless/02263_lazy_mark_load.sh +++ b/tests/queries/0_stateless/02263_lazy_mark_load.sh @@ -24,7 +24,7 @@ CREATE TABLE lazy_mark_test n9 UInt64 ) ENGINE = MergeTree -ORDER BY n0 SETTINGS min_bytes_for_wide_part = 0; +ORDER BY n0 SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; EOF ${CLICKHOUSE_CLIENT} -q "SYSTEM STOP MERGES lazy_mark_test" diff --git a/tests/queries/0_stateless/02293_selected_rows_and_merges.sh b/tests/queries/0_stateless/02293_selected_rows_and_merges.sh index 9d1483f5bf79..76c562c97443 100755 --- a/tests/queries/0_stateless/02293_selected_rows_and_merges.sh +++ b/tests/queries/0_stateless/02293_selected_rows_and_merges.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) query_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4()))))") -${CLICKHOUSE_CLIENT} -q "create table tt (x UInt32, y UInt32) engine = MergeTree order by x" +${CLICKHOUSE_CLIENT} -q "create table tt (x UInt32, y UInt32) engine = MergeTree order by x SETTINGS ratio_of_defaults_for_sparse_serialization = 1" ${CLICKHOUSE_CLIENT} -q "insert into tt select number, 0 from numbers(1e6)" ${CLICKHOUSE_CLIENT} -q "insert into tt select number, 1 from numbers(1e6)" @@ -17,13 +17,11 @@ ${CLICKHOUSE_CLIENT} --optimize_throw_if_noop 1 -q "optimize table tt final" "-- # Here SelectRows and SelectBytes should be zero, MergedRows is 2m and MergedUncompressedBytes is 16m ${CLICKHOUSE_CLIENT} -q "system flush logs" -${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'], ProfileEvents['SelecteBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'optimize%' and current_database = currentDatabase()" +${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'], ProfileEvents['SelectedBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'optimize%' and current_database = currentDatabase()" ${CLICKHOUSE_CLIENT} --mutations_sync 1 -q "alter table tt update y = y + 1 where 1" "--query_id=$query_id" ${CLICKHOUSE_CLIENT} -q "system flush logs" # Here for mutation all values are 0, cause mutation is executed async. # It's pretty hard to write a test with total counter. -${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'] > 10, ProfileEvents['SelecteBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'alter%' and current_database = currentDatabase()" - - +${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'] > 10, ProfileEvents['SelectedBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'alter%' and current_database = currentDatabase()" diff --git a/tests/queries/0_stateless/02361_fsync_profile_events.sh b/tests/queries/0_stateless/02361_fsync_profile_events.sh index 5b603133f6c7..e150d70b896d 100755 --- a/tests/queries/0_stateless/02361_fsync_profile_events.sh +++ b/tests/queries/0_stateless/02361_fsync_profile_events.sh @@ -12,9 +12,10 @@ $CLICKHOUSE_CLIENT -nm -q " create table data_fsync_pe (key Int) engine=MergeTree() order by key settings - min_rows_for_wide_part=2, - fsync_after_insert=1, - fsync_part_directory=1; + min_rows_for_wide_part = 2, + fsync_after_insert = 1, + fsync_part_directory = 1, + ratio_of_defaults_for_sparse_serialization = 1; " ret=1 diff --git a/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql b/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql index 842e22ba87dc..2fe0943745db 100644 --- a/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql +++ b/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql @@ -1,12 +1,12 @@ -- Tags: no-upgrade-check, no-random-merge-tree-settings drop table if exists test_02381; -create table test_02381(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b) SETTINGS compress_marks=false, compress_primary_key=false; +create table test_02381(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b) SETTINGS compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; insert into test_02381 select number, number * 10 from system.numbers limit 1000000; drop table if exists test_02381_compress; create table test_02381_compress(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b) - SETTINGS compress_marks=true, compress_primary_key=true, marks_compression_codec='ZSTD(3)', primary_key_compression_codec='ZSTD(3)', marks_compress_block_size=65536, primary_key_compress_block_size=65536; + SETTINGS compress_marks = true, compress_primary_key = true, marks_compression_codec = 'ZSTD(3)', primary_key_compression_codec = 'ZSTD(3)', marks_compress_block_size = 65536, primary_key_compress_block_size = 65536, ratio_of_defaults_for_sparse_serialization = 1; insert into test_02381_compress select number, number * 10 from system.numbers limit 1000000; select * from test_02381_compress where a = 1000 limit 1; From e8f7a84ca6c4e00f6f9ddbf282b109f491244c4c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 00:37:10 +0200 Subject: [PATCH 004/133] Update a few tests --- tests/queries/0_stateless/01375_compact_parts_codecs.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01375_compact_parts_codecs.sql b/tests/queries/0_stateless/01375_compact_parts_codecs.sql index 1dd39e678766..1c89eb09d0b4 100644 --- a/tests/queries/0_stateless/01375_compact_parts_codecs.sql +++ b/tests/queries/0_stateless/01375_compact_parts_codecs.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS codecs; CREATE TABLE codecs (id UInt32, val UInt32, s String) ENGINE = MergeTree ORDER BY id - SETTINGS min_rows_for_wide_part = 10000; + SETTINGS min_rows_for_wide_part = 10000, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) FROM system.parts @@ -21,7 +21,7 @@ DROP TABLE codecs; CREATE TABLE codecs (id UInt32 CODEC(NONE), val UInt32 CODEC(NONE), s String CODEC(NONE)) ENGINE = MergeTree ORDER BY id - SETTINGS min_rows_for_wide_part = 10000; + SETTINGS min_rows_for_wide_part = 10000, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) FROM system.parts @@ -38,7 +38,7 @@ DROP TABLE codecs; CREATE TABLE codecs (id UInt32, val UInt32 CODEC(Delta, ZSTD), s String CODEC(ZSTD)) ENGINE = MergeTree ORDER BY id - SETTINGS min_rows_for_wide_part = 10000; + SETTINGS min_rows_for_wide_part = 10000, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) FROM system.parts From a25de5fb4186fbe103f916b07aa8bd89975048b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 00:55:44 +0200 Subject: [PATCH 005/133] Update a test --- .../02530_dictionaries_update_field.reference | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.reference b/tests/queries/0_stateless/02530_dictionaries_update_field.reference index 40f2c0ee4005..88c910e03133 100644 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.reference +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.reference @@ -4,13 +4,13 @@ flat SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 SecondUpdated @@ -21,13 +21,13 @@ flat/custom SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -38,13 +38,13 @@ hashed SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -55,13 +55,13 @@ hashed/custom SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -72,13 +72,13 @@ complex_key_hashed SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -89,13 +89,13 @@ complex_key_hashed/custom SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated From 6b0bd698d36014a5eac052857bac2185a1f45f41 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 11 May 2023 04:17:53 +0200 Subject: [PATCH 006/133] Fix mistake --- .../02530_dictionaries_update_field.reference | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.reference b/tests/queries/0_stateless/02530_dictionaries_update_field.reference index 88c910e03133..40f2c0ee4005 100644 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.reference +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.reference @@ -4,13 +4,13 @@ flat SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 SecondUpdated @@ -21,13 +21,13 @@ flat/custom SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -38,13 +38,13 @@ hashed SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -55,13 +55,13 @@ hashed/custom SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -72,13 +72,13 @@ complex_key_hashed SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -89,13 +89,13 @@ complex_key_hashed/custom SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated From 65d28a959ff5b21199c2b20d8dcb7c7b399f314d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 11 May 2023 04:26:29 +0200 Subject: [PATCH 007/133] Update integration tests (1/2) --- .../configs/config.d/storage_conf.xml | 1 + .../test_merge_tree_hdfs/configs/config.d/storage_conf.xml | 1 + .../test_merge_tree_s3_failover/configs/config.xml | 4 ++++ .../test_s3_zero_copy_replication/configs/config.d/s3.xml | 1 + 4 files changed, 7 insertions(+) diff --git a/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml b/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml index cb87abcc693d..d69fe96a3e2f 100644 --- a/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml +++ b/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml @@ -45,5 +45,6 @@ true + 1.0 diff --git a/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml index 890c396ed95c..7d59081486bc 100644 --- a/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml @@ -28,5 +28,6 @@ 0 + 1.0 diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.xml index feb537ebbceb..743d75d9a21f 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.xml @@ -15,4 +15,8 @@ 500 ./clickhouse/ users.xml + + + 1.0 + diff --git a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml index f7d9efc2caec..55c359997034 100644 --- a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml +++ b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml @@ -70,6 +70,7 @@ 1024 1 true + 1.0 From 4a7761c16210c7e2eccc0b26e172ec8dc7e6c183 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 10 Jun 2023 08:26:32 +0000 Subject: [PATCH 008/133] Add column is_obsolete for system.settings table and related system tables --- src/Interpreters/Context.cpp | 6 ++++-- src/Storages/System/StorageSystemMergeTreeSettings.cpp | 2 ++ src/Storages/System/StorageSystemServerSettings.cpp | 2 ++ src/Storages/System/StorageSystemSettings.cpp | 2 ++ tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- tests/queries/0_stateless/01945_system_warnings.reference | 2 +- tests/queries/0_stateless/01945_system_warnings.sh | 4 ++-- 7 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 995e78d8f0b2..a12117b7677c 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -781,8 +781,10 @@ Strings Context::getWarnings() const { if (setting.isValueChanged() && setting.isObsolete()) { - common_warnings.emplace_back("Some obsolete setting is changed. " - "Check 'select * from system.settings where changed' and read the changelog."); + common_warnings.emplace_back( + "Obsolete setting `" + setting.getName() + + "` is changed. " + "Check 'select * from system.settings where changed' and read the changelog."); break; } } diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.cpp b/src/Storages/System/StorageSystemMergeTreeSettings.cpp index 6de3fb800f4e..0ddd45462084 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.cpp +++ b/src/Storages/System/StorageSystemMergeTreeSettings.cpp @@ -21,6 +21,7 @@ NamesAndTypesList SystemMergeTreeSettings::getNamesAndTypes() {"max", std::make_shared(std::make_shared())}, {"readonly", std::make_shared()}, {"type", std::make_shared()}, + {"is_obsolete", std::make_shared()}, }; } @@ -52,6 +53,7 @@ void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, res_columns[5]->insert(max); res_columns[6]->insert(writability == SettingConstraintWritability::CONST); res_columns[7]->insert(setting.getTypeName()); + res_columns[8]->insert(setting.isObsolete()); } } diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index ad52c6896ac1..290b575465c5 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -15,6 +15,7 @@ NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() {"changed", std::make_shared()}, {"description", std::make_shared()}, {"type", std::make_shared()}, + {"is_obsolete", std::make_shared()}, }; } @@ -33,6 +34,7 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context res_columns[3]->insert(setting.isValueChanged()); res_columns[4]->insert(setting.getDescription()); res_columns[5]->insert(setting.getTypeName()); + res_columns[6]->insert(setting.isObsolete()); } } diff --git a/src/Storages/System/StorageSystemSettings.cpp b/src/Storages/System/StorageSystemSettings.cpp index c54f7eef25ff..dcb54eac0a0f 100644 --- a/src/Storages/System/StorageSystemSettings.cpp +++ b/src/Storages/System/StorageSystemSettings.cpp @@ -21,6 +21,7 @@ NamesAndTypesList StorageSystemSettings::getNamesAndTypes() {"type", std::make_shared()}, {"default", std::make_shared()}, {"alias_for", std::make_shared()}, + {"is_obsolete", std::make_shared()}, }; } @@ -51,6 +52,7 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, ContextPtr co res_columns[6]->insert(writability == SettingConstraintWritability::CONST); res_columns[7]->insert(setting.getTypeName()); res_columns[8]->insert(setting.getDefaultValueString()); + res_columns[10]->insert(setting.isObsolete()); }; const auto & settings_to_aliases = Settings::Traits::settingsToAliases(); diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 4e6dd3e1b0fd..3d5b1ca99a51 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Some obsolete setting is changed." +expect " * Obsolete setting `max_memory_usage_for_all_queries` is changed." expect ":) " send -- "q\r" expect eof diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index 296a03447dbd..d6ae567289c8 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Some obsolete setting is changed. Check \'select * from system.settings where changed\' and read the changelog. +Obsolete setting `multiple_joins_rewriter_version` is changed. Check \'select * from system.settings where changed\' and read the changelog. 1 1 diff --git a/tests/queries/0_stateless/01945_system_warnings.sh b/tests/queries/0_stateless/01945_system_warnings.sh index bf11cee2911c..112baab614e5 100755 --- a/tests/queries/0_stateless/01945_system_warnings.sh +++ b/tests/queries/0_stateless/01945_system_warnings.sh @@ -14,8 +14,8 @@ else echo "Server was built in debug mode. It will work slowly." fi -${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.warnings WHERE message LIKE '%obsolete setting%'" -${CLICKHOUSE_CLIENT} --multiple_joins_rewriter_version=42 -q "SELECT message FROM system.warnings WHERE message LIKE '%obsolete setting%'" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.warnings WHERE message LIKE '%Obsolete setting%'" +${CLICKHOUSE_CLIENT} --multiple_joins_rewriter_version=42 -q "SELECT message FROM system.warnings WHERE message LIKE '%Obsolete setting%'" # Avoid duplicated warnings ${CLICKHOUSE_CLIENT} -q "SELECT count() = countDistinct(message) FROM system.warnings" From 820673a5cf3f3f1c17b781496b3ab56f72f72c08 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 10 Jun 2023 10:16:53 +0000 Subject: [PATCH 009/133] update test --- .../queries/0_stateless/01221_system_settings.reference | 4 ++-- .../0_stateless/02117_show_create_table_system.reference | 9 ++++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01221_system_settings.reference b/tests/queries/0_stateless/01221_system_settings.reference index 399b3778b668..e9c2f3fec320 100644 --- a/tests/queries/0_stateless/01221_system_settings.reference +++ b/tests/queries/0_stateless/01221_system_settings.reference @@ -1,4 +1,4 @@ -send_timeout 300 0 Timeout for sending data to network, in seconds. If client needs to sent some data, but it did not able to send any bytes in this interval, exception is thrown. If you set this setting on client, the \'receive_timeout\' for the socket will be also set on the corresponding connection end on the server. \N \N 0 Seconds 300 -storage_policy default 0 Name of storage disk policy \N \N 0 String +send_timeout 300 0 Timeout for sending data to network, in seconds. If client needs to sent some data, but it did not able to send any bytes in this interval, exception is thrown. If you set this setting on client, the \'receive_timeout\' for the socket will be also set on the corresponding connection end on the server. \N \N 0 Seconds 300 0 +storage_policy default 0 Name of storage disk policy \N \N 0 String 0 1 1 diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index e864ba850188..38d00c157252 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -346,7 +346,8 @@ CREATE TABLE system.merge_tree_settings `min` Nullable(String), `max` Nullable(String), `readonly` UInt8, - `type` String + `type` String, + `is_obsolete` UInt8 ) ENGINE = SystemMergeTreeSettings COMMENT 'SYSTEM TABLE is built on the fly.' @@ -918,7 +919,8 @@ CREATE TABLE system.replicated_merge_tree_settings `min` Nullable(String), `max` Nullable(String), `readonly` UInt8, - `type` String + `type` String, + `is_obsolete` UInt8 ) ENGINE = SystemReplicatedMergeTreeSettings COMMENT 'SYSTEM TABLE is built on the fly.' @@ -993,7 +995,8 @@ CREATE TABLE system.settings `readonly` UInt8, `type` String, `default` String, - `alias_for` String + `alias_for` String, + `is_obsolete` UInt8 ) ENGINE = SystemSettings COMMENT 'SYSTEM TABLE is built on the fly.' From e9763caa0eb7078cd28e3765d0da1e0a9b4a204b Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 12 Jun 2023 14:21:58 +0000 Subject: [PATCH 010/133] fix --- src/Interpreters/Context.cpp | 18 +++++++++++++----- .../01945_system_warnings.reference | 2 +- 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a12117b7677c..823c3d678df3 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -777,17 +777,25 @@ Strings Context::getWarnings() const auto lock = getLock(); common_warnings = shared->warnings; } + String res = "Obsolete settings ["; + size_t obsolete_settings_count = 0; for (const auto & setting : settings) { if (setting.isValueChanged() && setting.isObsolete()) { - common_warnings.emplace_back( - "Obsolete setting `" + setting.getName() - + "` is changed. " - "Check 'select * from system.settings where changed' and read the changelog."); - break; + res += (obsolete_settings_count ? ", `" : "`") + setting.getName() + "`"; + ++obsolete_settings_count; } } + + if (obsolete_settings_count) + { + res = res + "]" + (obsolete_settings_count == 1 ? " is" : " are") + + " changed. " + "Please check 'select * from system.settings where changed and is_obsolete' and read the changelog."; + common_warnings.emplace_back(res); + } + return common_warnings; } diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index d6ae567289c8..3e7edacd2756 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Obsolete setting `multiple_joins_rewriter_version` is changed. Check \'select * from system.settings where changed\' and read the changelog. +Obsolete settings [`multiple_joins_rewriter_version`] is changed. Check \'select * from system.settings where changed\' and read the changelog. 1 1 From 18f4f1a5238c64f3b45e1d6781ef2c7104ab842d Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 12 Jun 2023 15:11:19 +0000 Subject: [PATCH 011/133] udpate test --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- tests/queries/0_stateless/01945_system_warnings.reference | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 3d5b1ca99a51..f0c97acb1f55 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete setting `max_memory_usage_for_all_queries` is changed." +expect " * Obsolete settings [`max_memory_usage_for_all_queries`] is changed." expect ":) " send -- "q\r" expect eof diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index 3e7edacd2756..0c05d5d7049f 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Obsolete settings [`multiple_joins_rewriter_version`] is changed. Check \'select * from system.settings where changed\' and read the changelog. +Obsolete settings [`multiple_joins_rewriter_version`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. 1 1 From b76ba13250ad5b0abe728875be0e41667450cd5f Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 12 Jun 2023 15:41:46 +0000 Subject: [PATCH 012/133] fix --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index f0c97acb1f55..617e54a375e8 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings [`max_memory_usage_for_all_queries`] is changed." +expect " * Obsolete settings [`max_memory_usage_for_all_queries`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." expect ":) " send -- "q\r" expect eof From 2148f29a40f44f387b2cfbd9d3496bf9bc0b7e8d Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 8 Jun 2023 20:29:27 -0400 Subject: [PATCH 013/133] More accurate DNS resolve for the keeper connection --- src/Common/ZooKeeper/ZooKeeper.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index a587ad6caf49..e078470476ab 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -13,6 +13,7 @@ #include #include #include "Common/ZooKeeper/IKeeper.h" +#include #include #include #include @@ -80,8 +81,12 @@ void ZooKeeper::init(ZooKeeperArgs args_) if (secure) host_string.erase(0, strlen("secure://")); - LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, Poco::Net::SocketAddress{host_string}.toString()); - nodes.emplace_back(Coordination::ZooKeeper::Node{Poco::Net::SocketAddress{host_string}, secure}); + /// We want to resolve all hosts without DNS cache for keeper connection. + Coordination::DNSResolver::instance().removeHostFromCache(host_string); + + auto address = Coordination::DNSResolver::instance().resolveAddress(host_string); + LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, address.toString()); + nodes.emplace_back(Coordination::ZooKeeper::Node{address, secure}); } catch (const Poco::Net::HostNotFoundException & e) { From bbf0548007432dc5482cd28fda4c31e57dd5c24f Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 13 Jun 2023 02:48:28 +0000 Subject: [PATCH 014/133] fix test --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 617e54a375e8..9a8e22aa26f5 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings [`max_memory_usage_for_all_queries`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." +expect " * Obsolete settings [\`max_memory_usage_for_all_queries\`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." expect ":) " send -- "q\r" expect eof From 6ad6c6afa3bdf1cd95e1454bad9e7eb75db7b0ab Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 13 Jun 2023 04:13:16 +0000 Subject: [PATCH 015/133] fix --- src/Interpreters/Context.cpp | 2 +- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- tests/queries/0_stateless/01945_system_warnings.reference | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 823c3d678df3..1b8c52ee06b7 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -783,7 +783,7 @@ Strings Context::getWarnings() const { if (setting.isValueChanged() && setting.isObsolete()) { - res += (obsolete_settings_count ? ", `" : "`") + setting.getName() + "`"; + res += (obsolete_settings_count ? ", '" : "'") + setting.getName() + "'"; ++obsolete_settings_count; } } diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 9a8e22aa26f5..5315c56bde85 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings [\`max_memory_usage_for_all_queries\`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." +expect " * Obsolete settings [\'max_memory_usage_for_all_queries\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." expect ":) " send -- "q\r" expect eof diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index 0c05d5d7049f..dcb296c61aa2 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Obsolete settings [`multiple_joins_rewriter_version`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. +Obsolete settings [\'multiple_joins_rewriter_version\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. 1 1 From 404bfe773ef726b63e944b70a8b4253907637b8c Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 13 Jun 2023 06:28:47 +0000 Subject: [PATCH 016/133] fix --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 5315c56bde85..9be0eb6e3992 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings [\'max_memory_usage_for_all_queries\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." +expect " * Obsolete settings" expect ":) " send -- "q\r" expect eof From e7d1dfb704caa283174823ba8ff59b6c10ae0e1d Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 13 Jun 2023 08:30:07 +0000 Subject: [PATCH 017/133] fix --- tests/queries/0_stateless/01945_system_warnings.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01945_system_warnings.sh b/tests/queries/0_stateless/01945_system_warnings.sh index 112baab614e5..e44fe0ad6b50 100755 --- a/tests/queries/0_stateless/01945_system_warnings.sh +++ b/tests/queries/0_stateless/01945_system_warnings.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 733e2905c74a85903ba0c50a9c5e20a002ab8c9a Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 23 Jun 2023 14:29:49 +0000 Subject: [PATCH 018/133] update docs --- .../system-tables/merge_tree_settings.md | 68 ++++++++++++------- .../system-tables/server_settings.md | 25 ++++--- docs/en/operations/system-tables/settings.md | 14 ++-- 3 files changed, 71 insertions(+), 36 deletions(-) diff --git a/docs/en/operations/system-tables/merge_tree_settings.md b/docs/en/operations/system-tables/merge_tree_settings.md index d8539908bf74..b90026154f84 100644 --- a/docs/en/operations/system-tables/merge_tree_settings.md +++ b/docs/en/operations/system-tables/merge_tree_settings.md @@ -7,11 +7,17 @@ Contains information about settings for `MergeTree` tables. Columns: -- `name` (String) — Setting name. -- `value` (String) — Setting value. -- `description` (String) — Setting description. -- `type` (String) — Setting type (implementation specific string value). -- `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. +- `name` ([String](../../sql-reference/data-types/string.md)) — Setting name. +- `value` ([String](../../sql-reference/data-types/string.md)) — Setting value. +- `changed` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Whether the setting was explicitly defined in the config or explicitly changed. +- `description` ([String](../../sql-reference/data-types/string.md)) — Setting description. +- `min` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — Minimum value of the setting, if any is set via [constraints](../../operations/settings/constraints-on-settings.md#constraints-on-settings). If the setting has no minimum value, contains [NULL](../../sql-reference/syntax.md#null-literal). +- `max` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — Maximum value of the setting, if any is set via [constraints](../../operations/settings/constraints-on-settings.md#constraints-on-settings). If the setting has no maximum value, contains [NULL](../../sql-reference/syntax.md#null-literal). +- `readonly` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether the current user can change the setting: + - `0` — Current user can change the setting. + - `1` — Current user can’t change the setting. +- `type` ([String](../../sql-reference/data-types/string.md)) — Setting type (implementation specific string value). +- `is_obsolete` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) _ Shows whether a setting is obsolete. **Example** ```sql @@ -19,37 +25,53 @@ SELECT * FROM system.merge_tree_settings LIMIT 4 FORMAT Vertical; ``` ```response -Row 1: -────── -name: index_granularity -value: 8192 +Row 1: +────── +name: min_compress_block_size +value: 0 changed: 0 -description: How many rows correspond to one primary key value. -type: SettingUInt64 +description: When granule is written, compress the data in buffer if the size of pending uncompressed data is larger or equal than the specified threshold. If this setting is not set, the corresponding global setting is used. +min: ____ +max: ____ +readonly: 0 +type: UInt64 +is_obsolete: 0 Row 2: ────── -name: min_bytes_for_wide_part +name: max_compress_block_size value: 0 changed: 0 -description: Minimal uncompressed size in bytes to create part in wide format instead of compact -type: SettingUInt64 +description: Compress the pending uncompressed data in buffer if its size is larger or equal than the specified threshold. Block of data will be compressed even if the current granule is not finished. If this setting is not set, the corresponding global setting is used. +min: ____ +max: ____ +readonly: 0 +type: UInt64 +is_obsolete: 0 Row 3: ────── -name: min_rows_for_wide_part -value: 0 +name: index_granularity +value: 8192 changed: 0 -description: Minimal number of rows to create part in wide format instead of compact -type: SettingUInt64 +description: How many rows correspond to one primary key value. +min: ____ +max: ____ +readonly: 0 +type: UInt64 +is_obsolete: 0 Row 4: ────── -name: merge_max_block_size -value: 8192 +name: max_digestion_size_per_segment +value: 268435456 changed: 0 -description: How many rows in blocks should be formed for merge operations. -type: SettingUInt64 +description: Max number of bytes to digest per segment to build GIN index. +min: ____ +max: ____ +readonly: 0 +type: UInt64 +is_obsolete: 0 -4 rows in set. Elapsed: 0.001 sec. +4 rows in set. Elapsed: 0.009 sec. ``` diff --git a/docs/en/operations/system-tables/server_settings.md b/docs/en/operations/system-tables/server_settings.md index 3085b1acaf40..df482261ae82 100644 --- a/docs/en/operations/system-tables/server_settings.md +++ b/docs/en/operations/system-tables/server_settings.md @@ -14,6 +14,7 @@ Columns: - `changed` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether a setting was specified in `config.xml` - `description` ([String](../../sql-reference/data-types/string.md)) — Short server setting description. - `type` ([String](../../sql-reference/data-types/string.md)) — Server setting value type. +- `is_obsolete` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) _ Shows whether a setting is obsolete. **Example** @@ -26,14 +27,22 @@ WHERE name LIKE '%thread_pool%' ``` ``` text -┌─name─────────────────────────┬─value─┬─default─┬─changed─┬─description─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─type───┐ -│ max_thread_pool_size │ 5000 │ 10000 │ 1 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │ -│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │ -│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │ -│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │ -│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │ -│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │ -└──────────────────────────────┴───────┴─────────┴─────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴────────┘ +┌─name────────────────────────────────────────_─value─_─default─_─changed─_─description────────────────────────────────────────────────────────────────────────────────────────────────────── +───────────────────────────────────_─type───_─is_obsolete─┐ +│ max_thread_pool_size │ 10000 │ 10000 │ 1 │ The maximum number of threads that could be allocated from the OS and used for query execution and background operations. │ UInt64 │ 0 │ +│ max_thread_pool_free_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks. │ UInt64 │ 0 │ +│ thread_pool_queue_size │ 10000 │ 10000 │ 0 │ The maximum number of tasks that will be placed in a queue and wait for execution. │ UInt64 │ 0 │ +│ max_io_thread_pool_size │ 100 │ 100 │ 0 │ The maximum number of threads that would be used for IO operations │ UInt64 │ 0 │ +│ max_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for IO thread pool. │ UInt64 │ 0 │ +│ io_thread_pool_queue_size │ 10000 │ 10000 │ 0 │ Queue size for IO thread pool. │ UInt64 │ 0 │ +│ max_active_parts_loading_thread_pool_size │ 64 │ 64 │ 0 │ The number of threads to load active set of data parts (Active ones) at startup. │ UInt64 │ 0 │ +│ max_outdated_parts_loading_thread_pool_size │ 32 │ 32 │ 0 │ The number of threads to load inactive set of data parts (Outdated ones) at startup. │ UInt64 │ 0 │ +│ max_parts_cleaning_thread_pool_size │ 128 │ 128 │ 0 │ The number of threads for concurrent removal of inactive data parts. │ UInt64 │ 0 │ +│ max_backups_io_thread_pool_size │ 1000 │ 1000 │ 0 │ The maximum number of threads that would be used for IO operations for BACKUP queries │ UInt64 │ 0 │ +│ max_backups_io_thread_pool_free_size │ 0 │ 0 │ 0 │ Max free size for backups IO thread pool. │ UInt64 │ 0 │ +│ backups_io_thread_pool_queue_size │ 0 │ 0 │ 0 │ Queue size for backups IO thread pool. │ UInt64 │ 0 │ +└─────────────────────────────────────────────┴───────┴─────────┴─────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────── +───────────────────────────────────┴────────┴─────────────┘ ``` Using of `WHERE changed` can be useful, for example, when you want to check diff --git a/docs/en/operations/system-tables/settings.md b/docs/en/operations/system-tables/settings.md index afae45077cc1..7dd2345a2d0f 100644 --- a/docs/en/operations/system-tables/settings.md +++ b/docs/en/operations/system-tables/settings.md @@ -17,6 +17,7 @@ Columns: - `0` — Current user can change the setting. - `1` — Current user can’t change the setting. - `default` ([String](../../sql-reference/data-types/string.md)) — Setting default value. +- `is_obsolete` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) _ Shows whether a setting is obsolete. **Example** @@ -29,11 +30,14 @@ WHERE name LIKE '%min_i%' ``` ``` text -┌─name────────────────────────────────────────┬─value─────┬─changed─┬─description───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─min──┬─max──┬─readonly─┐ -│ min_insert_block_size_rows │ 1048576 │ 0 │ Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough. │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ -│ min_insert_block_size_bytes │ 268435456 │ 0 │ Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough. │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ -│ read_backoff_min_interval_between_events_ms │ 1000 │ 0 │ Settings to reduce the number of threads in case of slow reads. Do not pay attention to the event, if the previous one has passed less than a certain amount of time. │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ -└─────────────────────────────────────────────┴───────────┴─────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴──────┴──────┴──────────┘ +┌─name───────────────────────────────────────────────_─value─────_─changed─_─description───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────_─min──_─max──_─readonly─_─type─────────_─default───_─alias_for─_─is_obsolete─┐ +│ min_insert_block_size_rows │ 1048449 │ 0 │ Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough. │ ____ │ ____ │ 0 │ UInt64 │ 1048449 │ │ 0 │ +│ min_insert_block_size_bytes │ 268402944 │ 0 │ Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough. │ ____ │ ____ │ 0 │ UInt64 │ 268402944 │ │ 0 │ +│ min_insert_block_size_rows_for_materialized_views │ 0 │ 0 │ Like min_insert_block_size_rows, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_rows) │ ____ │ ____ │ 0 │ UInt64 │ 0 │ │ 0 │ +│ min_insert_block_size_bytes_for_materialized_views │ 0 │ 0 │ Like min_insert_block_size_bytes, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_bytes) │ ____ │ ____ │ 0 │ UInt64 │ 0 │ │ 0 │ +│ read_backoff_min_interval_between_events_ms │ 1000 │ 0 │ Settings to reduce the number of threads in case of slow reads. Do not pay attention to the event, if the previous one has passed less than a certain amount of time. │ ____ │ ____ │ 0 │ Milliseconds │ 1000 │ │ 0 │ +└────────────────────────────────────────────────────┴───────────┴─────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────────────── +──────────────────────────────────────────────────────┴──────┴──────┴──────────┴──────────────┴───────────┴───────────┴─────────────┘ ``` Using of `WHERE changed` can be useful, for example, when you want to check: From 991d95b1e1cd86da4b78fa4c0976b80bcd3800ac Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 23 Jun 2023 14:32:08 +0000 Subject: [PATCH 019/133] update docs --- docs/en/operations/system-tables/merge_tree_settings.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/system-tables/merge_tree_settings.md b/docs/en/operations/system-tables/merge_tree_settings.md index b90026154f84..557835ce3b6b 100644 --- a/docs/en/operations/system-tables/merge_tree_settings.md +++ b/docs/en/operations/system-tables/merge_tree_settings.md @@ -25,9 +25,9 @@ SELECT * FROM system.merge_tree_settings LIMIT 4 FORMAT Vertical; ``` ```response -Row 1: -────── -name: min_compress_block_size +Row 1: +────── +name: min_compress_block_size value: 0 changed: 0 description: When granule is written, compress the data in buffer if the size of pending uncompressed data is larger or equal than the specified threshold. If this setting is not set, the corresponding global setting is used. @@ -73,5 +73,5 @@ readonly: 0 type: UInt64 is_obsolete: 0 -4 rows in set. Elapsed: 0.009 sec. +4 rows in set. Elapsed: 0.009 sec. ``` From 57ca8b60e0e060fdaedfd6617de1b5b6e05b11e4 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 24 Jun 2023 03:06:20 +0000 Subject: [PATCH 020/133] Fix --- src/Interpreters/Context.cpp | 54 +++++++++++++++++++----------------- 1 file changed, 29 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1b8c52ee06b7..2b0cc4e70d2a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -777,20 +777,27 @@ Strings Context::getWarnings() const auto lock = getLock(); common_warnings = shared->warnings; } - String res = "Obsolete settings ["; - size_t obsolete_settings_count = 0; + /// Make setting's name ordered + std::set obsolete_settings; for (const auto & setting : settings) { if (setting.isValueChanged() && setting.isObsolete()) - { - res += (obsolete_settings_count ? ", '" : "'") + setting.getName() + "'"; - ++obsolete_settings_count; - } + obsolete_settings.emplace(setting.getName()); } - if (obsolete_settings_count) + if (!obsolete_settings.empty()) { - res = res + "]" + (obsolete_settings_count == 1 ? " is" : " are") + bool single_element = obsolete_settings.size() == 1; + String res = single_element ? "Obsolete setting [" : "Obsolete settings ["; + + bool first = true; + for (const auto & setting : obsolete_settings) + { + res += first ? "" : ", "; + res += "'" + setting + "'"; + first = false; + } + res = res + "]" + (single_element ? " is" : " are") + " changed. " "Please check 'select * from system.settings where changed and is_obsolete' and read the changelog."; common_warnings.emplace_back(res); @@ -2156,9 +2163,9 @@ BackupsWorker & Context::getBackupsWorker() const const bool allow_concurrent_restores = this->getConfigRef().getBool("backups.allow_concurrent_restores", true); const auto & config = getConfigRef(); - const auto & settings_ = getSettingsRef(); - UInt64 backup_threads = config.getUInt64("backup_threads", settings_.backup_threads); - UInt64 restore_threads = config.getUInt64("restore_threads", settings_.restore_threads); + const auto & settings_ref = getSettingsRef(); + UInt64 backup_threads = config.getUInt64("backup_threads", settings_ref.backup_threads); + UInt64 restore_threads = config.getUInt64("restore_threads", settings_ref.restore_threads); if (!shared->backups_worker) shared->backups_worker.emplace(backup_threads, restore_threads, allow_concurrent_backups, allow_concurrent_restores); @@ -4296,10 +4303,10 @@ ReadSettings Context::getReadSettings() const ReadSettings Context::getBackupReadSettings() const { - ReadSettings settings_ = getReadSettings(); - settings_.remote_throttler = getBackupsThrottler(); - settings_.local_throttler = getBackupsThrottler(); - return settings_; + ReadSettings read_settings = getReadSettings(); + read_settings.remote_throttler = getBackupsThrottler(); + read_settings.local_throttler = getBackupsThrottler(); + return read_settings; } WriteSettings Context::getWriteSettings() const @@ -4328,14 +4335,13 @@ std::shared_ptr Context::getAsyncReadCounters() const Context::ParallelReplicasMode Context::getParallelReplicasMode() const { - const auto & settings_ = getSettingsRef(); + const auto & settings_ref = getSettingsRef(); using enum Context::ParallelReplicasMode; - if (!settings_.parallel_replicas_custom_key.value.empty()) + if (!settings_ref.parallel_replicas_custom_key.value.empty()) return CUSTOM_KEY; - if (settings_.allow_experimental_parallel_reading_from_replicas > 0 - && !settings_.use_hedged_requests) + if (settings_ref.allow_experimental_parallel_reading_from_replicas > 0 && !settings_ref.use_hedged_requests) return READ_TASKS; return SAMPLE_KEY; @@ -4343,17 +4349,15 @@ Context::ParallelReplicasMode Context::getParallelReplicasMode() const bool Context::canUseParallelReplicasOnInitiator() const { - const auto & settings_ = getSettingsRef(); - return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS - && settings_.max_parallel_replicas > 1 + const auto & settings_ref = getSettingsRef(); + return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS && settings_ref.max_parallel_replicas > 1 && !getClientInfo().collaborate_with_initiator; } bool Context::canUseParallelReplicasOnFollower() const { - const auto & settings_ = getSettingsRef(); - return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS - && settings_.max_parallel_replicas > 1 + const auto & settings_ref = getSettingsRef(); + return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS && settings_ref.max_parallel_replicas > 1 && getClientInfo().collaborate_with_initiator; } From f52a1159a2748e73dda2c6f897b3b1ee06d5a147 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 24 Jun 2023 03:52:32 +0000 Subject: [PATCH 021/133] fix test --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- tests/queries/0_stateless/01945_system_warnings.reference | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 9be0eb6e3992..28b114b5af49 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings" +expect " * Obsolete setting" expect ":) " send -- "q\r" expect eof diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index dcb296c61aa2..cfec2f638160 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Obsolete settings [\'multiple_joins_rewriter_version\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. +Obsolete setting [\'multiple_joins_rewriter_version\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. 1 1 From 550907c432c86f35d9d2dc56e6fefe1d43717c13 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 24 Jun 2023 05:14:01 +0000 Subject: [PATCH 022/133] see output of flaky test fix fix --- tests/queries/0_stateless/01945_system_warnings.reference | 1 + tests/queries/0_stateless/01945_system_warnings.sh | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index cfec2f638160..ce6195d52771 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -3,3 +3,4 @@ Server was built in debug mode. It will work slowly. Obsolete setting [\'multiple_joins_rewriter_version\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. 1 1 +3 diff --git a/tests/queries/0_stateless/01945_system_warnings.sh b/tests/queries/0_stateless/01945_system_warnings.sh index e44fe0ad6b50..26f04fbe31d4 100755 --- a/tests/queries/0_stateless/01945_system_warnings.sh +++ b/tests/queries/0_stateless/01945_system_warnings.sh @@ -23,4 +23,5 @@ ${CLICKHOUSE_CLIENT} -q "SELECT count() = countDistinct(message) FROM system.war # Avoid too many warnings, especially in CI ${CLICKHOUSE_CLIENT} -q "SELECT count() < 5 FROM system.warnings" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.warnings" From 8a483f3347a1c39c8a8ee8dfb340501005e8a698 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 24 Jun 2023 13:08:58 +0000 Subject: [PATCH 023/133] fix --- tests/queries/0_stateless/01945_system_warnings.reference | 1 - tests/queries/0_stateless/01945_system_warnings.sh | 3 +-- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index ce6195d52771..cfec2f638160 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -3,4 +3,3 @@ Server was built in debug mode. It will work slowly. Obsolete setting [\'multiple_joins_rewriter_version\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. 1 1 -3 diff --git a/tests/queries/0_stateless/01945_system_warnings.sh b/tests/queries/0_stateless/01945_system_warnings.sh index 26f04fbe31d4..4f11d63d3d09 100755 --- a/tests/queries/0_stateless/01945_system_warnings.sh +++ b/tests/queries/0_stateless/01945_system_warnings.sh @@ -22,6 +22,5 @@ ${CLICKHOUSE_CLIENT} --multiple_joins_rewriter_version=42 -q "SELECT message FRO ${CLICKHOUSE_CLIENT} -q "SELECT count() = countDistinct(message) FROM system.warnings" # Avoid too many warnings, especially in CI -${CLICKHOUSE_CLIENT} -q "SELECT count() < 5 FROM system.warnings" -${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.warnings" +${CLICKHOUSE_CLIENT} -q "SELECT count() <= 5 FROM system.warnings" From 27f41869a92037efb3f5c8868d8c0710cfef5fdd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 09:11:42 +0200 Subject: [PATCH 024/133] Remove code that I don't like --- src/Formats/ReadSchemaUtils.h | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 30 +++++---- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 38 +++++------ .../MergeTree/MergeTreeDataMergerMutator.h | 10 +-- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 63 +++++++----------- .../MergeTree/ReplicatedMergeTreeQueue.h | 14 ++-- src/Storages/StorageMergeTree.cpp | 65 ++++++++----------- src/Storages/StorageMergeTree.h | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 13 ++-- 10 files changed, 106 insertions(+), 139 deletions(-) diff --git a/src/Formats/ReadSchemaUtils.h b/src/Formats/ReadSchemaUtils.h index 82fbb3f7c460..6b4c78a4ff64 100644 --- a/src/Formats/ReadSchemaUtils.h +++ b/src/Formats/ReadSchemaUtils.h @@ -9,14 +9,14 @@ namespace DB using ReadBufferIterator = std::function(ColumnsDescription &)>; -/// Try to determine the schema of the data in specifying format. +/// Try to determine the schema of the data in the specified format. /// For formats that have an external schema reader, it will /// use it and won't create a read buffer. /// For formats that have a schema reader from the data, /// read buffer will be created by the provided iterator and /// the schema will be extracted from the data. If schema reader /// couldn't determine the schema we will try the next read buffer -/// from provided iterator if it makes sense. If format doesn't +/// from the provided iterator if it makes sense. If the format doesn't /// have any schema reader or we couldn't determine the schema, /// an exception will be thrown. ColumnsDescription readSchemaFromFormat( diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e9c3a7f66aed..9dfb1b3371da 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -382,7 +382,7 @@ MergeTreeData::MergeTreeData( checkTTLExpressions(metadata_, metadata_); String reason; - if (!canUsePolymorphicParts(*settings, &reason) && !reason.empty()) + if (!canUsePolymorphicParts(*settings, reason) && !reason.empty()) LOG_WARNING(log, "{} Settings 'min_rows_for_wide_part'and 'min_bytes_for_wide_part' will be ignored.", reason); #if !USE_ROCKSDB @@ -3319,7 +3319,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context MergeTreeSettings copy = *getSettings(); copy.applyChange(changed_setting); String reason; - if (!canUsePolymorphicParts(copy, &reason) && !reason.empty()) + if (!canUsePolymorphicParts(copy, reason) && !reason.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Can't change settings. Reason: {}", reason); } @@ -3344,7 +3344,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context auto copy = getDefaultSettings(); copy->applyChanges(new_changes); String reason; - if (!canUsePolymorphicParts(*copy, &reason) && !reason.empty()) + if (!canUsePolymorphicParts(*copy, reason) && !reason.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Can't change settings. Reason: {}", reason); } @@ -3386,8 +3386,9 @@ MergeTreeDataPartFormat MergeTreeData::choosePartFormat(size_t bytes_uncompresse using PartType = MergeTreeDataPartType; using PartStorageType = MergeTreeDataPartStorageType; - const auto settings = getSettings(); - if (!canUsePolymorphicParts(*settings)) + String out_reason; + const auto settings = getSettings(); + if (!canUsePolymorphicParts(*settings, out_reason)) return {PartType::Wide, PartStorageType::Full}; auto satisfies = [&](const auto & min_bytes_for, const auto & min_rows_for) @@ -7976,22 +7977,23 @@ bool MergeTreeData::partsContainSameProjections(const DataPartPtr & left, const bool MergeTreeData::canUsePolymorphicParts() const { - return canUsePolymorphicParts(*getSettings(), nullptr); + String unused; + return canUsePolymorphicParts(*getSettings(), unused); } -bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, String * out_reason) const +bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, String & out_reason) const { if (!canUseAdaptiveGranularity()) { - if (out_reason && (settings.min_rows_for_wide_part != 0 || settings.min_bytes_for_wide_part != 0 + if ((settings.min_rows_for_wide_part != 0 || settings.min_bytes_for_wide_part != 0 || settings.min_rows_for_compact_part != 0 || settings.min_bytes_for_compact_part != 0)) { - *out_reason = fmt::format( - "Table can't create parts with adaptive granularity, but settings" - " min_rows_for_wide_part = {}" - ", min_bytes_for_wide_part = {}" - ". Parts with non-adaptive granularity can be stored only in Wide (default) format.", - settings.min_rows_for_wide_part, settings.min_bytes_for_wide_part); + out_reason = fmt::format( + "Table can't create parts with adaptive granularity, but settings" + " min_rows_for_wide_part = {}" + ", min_bytes_for_wide_part = {}" + ". Parts with non-adaptive granularity can be stored only in Wide (default) format.", + settings.min_rows_for_wide_part, settings.min_bytes_for_wide_part); } return false; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b27392b355bf..cefebe8fc585 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1494,7 +1494,7 @@ class MergeTreeData : public IStorage, public WithMutableContext /// Check selected parts for movements. Used by ALTER ... MOVE queries. CurrentlyMovingPartsTaggerPtr checkPartsForMove(const DataPartsVector & parts, SpacePtr space); - bool canUsePolymorphicParts(const MergeTreeSettings & settings, String * out_reason = nullptr) const; + bool canUsePolymorphicParts(const MergeTreeSettings & settings, String & out_reason) const; std::mutex write_ahead_log_mutex; WriteAheadLogPtr write_ahead_log; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 29a1574b66e6..5a9a00f10591 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -136,7 +136,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( const AllowedMergingPredicate & can_merge_callback, bool merge_with_ttl_allowed, const MergeTreeTransactionPtr & txn, - String * out_disable_reason, + String & out_disable_reason, const PartitionIdsHint * partitions_hint) { MergeTreeData::DataPartsVector data_parts = getDataPartsToSelectMergeFrom(txn, partitions_hint); @@ -145,8 +145,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( if (data_parts.empty()) { - if (out_disable_reason) - *out_disable_reason = "There are no parts in the table"; + out_disable_reason = "There are no parts in the table"; return SelectPartsDecision::CANNOT_SELECT; } @@ -154,8 +153,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( if (info.parts_selected_precondition == 0) { - if (out_disable_reason) - *out_disable_reason = "No parts satisfy preconditions for merge"; + out_disable_reason = "No parts satisfy preconditions for merge"; return SelectPartsDecision::CANNOT_SELECT; } @@ -179,8 +177,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( /*optimize_skip_merged_partitions=*/true); } - if (out_disable_reason) - *out_disable_reason = "There is no need to merge parts according to merge selector algorithm"; + out_disable_reason = "There is no need to merge parts according to merge selector algorithm"; return SelectPartsDecision::CANNOT_SELECT; } @@ -197,7 +194,8 @@ MergeTreeDataMergerMutator::PartitionIdsHint MergeTreeDataMergerMutator::getPart auto metadata_snapshot = data.getInMemoryMetadataPtr(); - MergeSelectingInfo info = getPossibleMergeRanges(data_parts, can_merge_callback, txn); + String out_reason; + MergeSelectingInfo info = getPossibleMergeRanges(data_parts, can_merge_callback, txn, out_reason); if (info.parts_selected_precondition == 0) return res; @@ -227,7 +225,7 @@ MergeTreeDataMergerMutator::PartitionIdsHint MergeTreeDataMergerMutator::getPart /// This method should have been const, but something went wrong... it's const with dry_run = true auto status = const_cast(this)->selectPartsToMergeFromRanges( future_part, /*aggressive*/ false, max_total_size_to_merge, merge_with_ttl_allowed, - metadata_snapshot, ranges_per_partition[i], info.current_time, &out_disable_reason, + metadata_snapshot, ranges_per_partition[i], info.current_time, out_disable_reason, /* dry_run */ true); if (status == SelectPartsDecision::SELECTED) res.insert(all_partition_ids[i]); @@ -330,7 +328,7 @@ MergeTreeDataMergerMutator::MergeSelectingInfo MergeTreeDataMergerMutator::getPo const MergeTreeData::DataPartsVector & data_parts, const AllowedMergingPredicate & can_merge_callback, const MergeTreeTransactionPtr & txn, - String * out_disable_reason) const + String & out_disable_reason) const { MergeSelectingInfo res; @@ -443,7 +441,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( const StorageMetadataPtr & metadata_snapshot, const IMergeSelector::PartsRanges & parts_ranges, const time_t & current_time, - String * out_disable_reason, + String & out_disable_reason, bool dry_run) { const auto data_settings = data.getSettings(); @@ -514,8 +512,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( if (parts_to_merge.empty()) { - if (out_disable_reason) - *out_disable_reason = "Did not find any parts to merge (with usual merge selectors)"; + out_disable_reason = "Did not find any parts to merge (with usual merge selectors)"; return SelectPartsDecision::CANNOT_SELECT; } } @@ -562,22 +559,20 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti bool final, const StorageMetadataPtr & metadata_snapshot, const MergeTreeTransactionPtr & txn, - String * out_disable_reason, + String & out_disable_reason, bool optimize_skip_merged_partitions) { MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); if (parts.empty()) { - if (out_disable_reason) - *out_disable_reason = "There are no parts inside partition"; + out_disable_reason = "There are no parts inside partition"; return SelectPartsDecision::CANNOT_SELECT; } if (!final && parts.size() == 1) { - if (out_disable_reason) - *out_disable_reason = "There is only one part inside partition"; + out_disable_reason = "There is only one part inside partition"; return SelectPartsDecision::CANNOT_SELECT; } @@ -586,8 +581,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti if (final && optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 && (!metadata_snapshot->hasAnyTTL() || parts[0]->checkAllTTLCalculated(metadata_snapshot))) { - if (out_disable_reason) - *out_disable_reason = "Partition skipped due to optimize_skip_merged_partitions"; + out_disable_reason = "Partition skipped due to optimize_skip_merged_partitions"; return SelectPartsDecision::NOTHING_TO_MERGE; } @@ -628,9 +622,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti static_cast((DISK_USAGE_COEFFICIENT_TO_SELECT - 1.0) * 100)); } - if (out_disable_reason) - *out_disable_reason = fmt::format("Insufficient available disk space, required {}", ReadableSize(required_disk_space)); - + out_disable_reason = fmt::format("Insufficient available disk space, required {}", ReadableSize(required_disk_space)); return SelectPartsDecision::CANNOT_SELECT; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 428161ea71ee..6eab0ee0c371 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -43,7 +43,7 @@ class MergeTreeDataMergerMutator using AllowedMergingPredicate = std::function; + String &)>; explicit MergeTreeDataMergerMutator(MergeTreeData & data_); @@ -92,7 +92,7 @@ class MergeTreeDataMergerMutator const MergeTreeData::DataPartsVector & data_parts, const AllowedMergingPredicate & can_merge_callback, const MergeTreeTransactionPtr & txn, - String * out_disable_reason = nullptr) const; + String & out_disable_reason) const; /// The third step of selecting parts to merge: takes ranges that we can merge, and selects parts that we want to merge SelectPartsDecision selectPartsToMergeFromRanges( @@ -103,7 +103,7 @@ class MergeTreeDataMergerMutator const StorageMetadataPtr & metadata_snapshot, const IMergeSelector::PartsRanges & parts_ranges, const time_t & current_time, - String * out_disable_reason = nullptr, + String & out_disable_reason, bool dry_run = false); String getBestPartitionToOptimizeEntire(const PartitionsInfo & partitions_info) const; @@ -129,7 +129,7 @@ class MergeTreeDataMergerMutator const AllowedMergingPredicate & can_merge, bool merge_with_ttl_allowed, const MergeTreeTransactionPtr & txn, - String * out_disable_reason = nullptr, + String & out_disable_reason, const PartitionIdsHint * partitions_hint = nullptr); /** Select all the parts in the specified partition for merge, if possible. @@ -144,7 +144,7 @@ class MergeTreeDataMergerMutator bool final, const StorageMetadataPtr & metadata_snapshot, const MergeTreeTransactionPtr & txn, - String * out_disable_reason = nullptr, + String & out_disable_reason, bool optimize_skip_merged_partitions = false); /** Creates a task to merge parts. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 03ded2ef2604..295e717c08c3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -2197,7 +2197,7 @@ bool LocalMergePredicate::operator()( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, const MergeTreeTransaction *, - String * out_reason) const + String & out_reason) const { if (left) return canMergeTwoParts(left, right, out_reason); @@ -2209,7 +2209,7 @@ bool ReplicatedMergeTreeMergePredicate::operator()( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, const MergeTreeTransaction *, - String * out_reason) const + String & out_reason) const { if (left) return canMergeTwoParts(left, right, out_reason); @@ -2221,7 +2221,7 @@ bool ReplicatedMergeTreeMergePredicate::operator()( bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, - String * out_reason) const + String & out_reason) const { /// A sketch of a proof of why this method actually works: /// @@ -2265,22 +2265,19 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( { if (pinned_part_uuids.part_uuids.contains(part->uuid)) { - if (out_reason) - *out_reason = "Part " + part->name + " has uuid " + toString(part->uuid) + " which is currently pinned"; + out_reason = "Part " + part->name + " has uuid " + toString(part->uuid) + " which is currently pinned"; return false; } if (part->name == inprogress_quorum_part) { - if (out_reason) - *out_reason = "Quorum insert for part " + part->name + " is currently in progress"; + out_reason = "Quorum insert for part " + part->name + " is currently in progress"; return false; } if (prev_virtual_parts.getContainingPart(part->info).empty()) { - if (out_reason) - *out_reason = "Entry for part " + part->name + " hasn't been read from the replication log yet"; + out_reason = "Entry for part " + part->name + " hasn't been read from the replication log yet"; return false; } } @@ -2294,8 +2291,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( { if (partition_ids_hint && !partition_ids_hint->contains(left->info.partition_id)) { - if (out_reason) - *out_reason = fmt::format("Uncommitted block were not loaded for unexpected partition {}", left->info.partition_id); + out_reason = fmt::format("Uncommitted block were not loaded for unexpected partition {}", left->info.partition_id); return false; } @@ -2307,10 +2303,8 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( auto block_it = block_numbers.upper_bound(left_max_block); if (block_it != block_numbers.end() && *block_it < right_min_block) { - if (out_reason) - *out_reason = "Block number " + toString(*block_it) + " is still being inserted between parts " - + left->name + " and " + right->name; - + out_reason = "Block number " + toString(*block_it) + " is still being inserted between parts " + + left->name + " and " + right->name; return false; } } @@ -2322,7 +2316,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( bool LocalMergePredicate::canMergeTwoParts( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, - String * out_reason) const + String & out_reason) const { Int64 left_max_block = left->info.max_block; Int64 right_min_block = right->info.min_block; @@ -2336,8 +2330,7 @@ bool LocalMergePredicate::canMergeTwoParts( String containing_part = queue.virtual_parts.getContainingPart(part->info); if (containing_part != part->name) { - if (out_reason) - *out_reason = "Part " + part->name + " has already been assigned a merge into " + containing_part; + out_reason = "Part " + part->name + " has already been assigned a merge into " + containing_part; return false; } } @@ -2354,10 +2347,9 @@ bool LocalMergePredicate::canMergeTwoParts( Strings covered = queue.virtual_parts.getPartsCoveredBy(gap_part_info); if (!covered.empty()) { - if (out_reason) - *out_reason = "There are " + toString(covered.size()) + " parts (from " + covered.front() - + " to " + covered.back() + ") that are still not present or being processed by " - + " other background process on this replica between " + left->name + " and " + right->name; + out_reason = "There are " + toString(covered.size()) + " parts (from " + covered.front() + + " to " + covered.back() + ") that are still not present or being processed by " + + " other background process on this replica between " + left->name + " and " + right->name; return false; } } @@ -2370,9 +2362,8 @@ bool LocalMergePredicate::canMergeTwoParts( if (left_mutation_ver != right_mutation_ver) { - if (out_reason) - *out_reason = "Current mutation versions of parts " + left->name + " and " + right->name + " differ: " - + toString(left_mutation_ver) + " and " + toString(right_mutation_ver) + " respectively"; + out_reason = "Current mutation versions of parts " + left->name + " and " + right->name + " differ: " + + toString(left_mutation_ver) + " and " + toString(right_mutation_ver) + " respectively"; return false; } @@ -2381,33 +2372,30 @@ bool LocalMergePredicate::canMergeTwoParts( bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( const MergeTreeData::DataPartPtr & part, - String * out_reason) const + String & out_reason) const { if (pinned_part_uuids.part_uuids.contains(part->uuid)) { - if (out_reason) - *out_reason = fmt::format("Part {} has uuid {} which is currently pinned", part->name, part->uuid); + out_reason = fmt::format("Part {} has uuid {} which is currently pinned", part->name, part->uuid); return false; } if (part->name == inprogress_quorum_part) { - if (out_reason) - *out_reason = fmt::format("Quorum insert for part {} is currently in progress", part->name); + out_reason = fmt::format("Quorum insert for part {} is currently in progress", part->name); return false; } if (prev_virtual_parts.getContainingPart(part->info).empty()) { - if (out_reason) - *out_reason = fmt::format("Entry for part {} hasn't been read from the replication log yet", part->name); + out_reason = fmt::format("Entry for part {} hasn't been read from the replication log yet", part->name); return false; } return nested_pred.canMergeSinglePart(part, out_reason); } -bool LocalMergePredicate::canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const +bool LocalMergePredicate::canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String & out_reason) const { std::lock_guard lock(queue.state_mutex); @@ -2416,8 +2404,7 @@ bool LocalMergePredicate::canMergeSinglePart(const MergeTreeData::DataPartPtr & String containing_part = queue.virtual_parts.getContainingPart(part->info); if (containing_part != part->name) { - if (out_reason) - *out_reason = fmt::format("Part {} has already been assigned a merge into {}", part->name, containing_part); + out_reason = fmt::format("Part {} has already been assigned a merge into {}", part->name, containing_part); return false; } @@ -2425,7 +2412,7 @@ bool LocalMergePredicate::canMergeSinglePart(const MergeTreeData::DataPartPtr & } -bool ReplicatedMergeTreeMergePredicate::partParticipatesInReplaceRange(const MergeTreeData::DataPartPtr & part, String * out_reason) const +bool ReplicatedMergeTreeMergePredicate::partParticipatesInReplaceRange(const MergeTreeData::DataPartPtr & part, String & out_reason) const { std::lock_guard lock(queue.state_mutex); for (const auto & entry : queue.queue) @@ -2438,9 +2425,7 @@ bool ReplicatedMergeTreeMergePredicate::partParticipatesInReplaceRange(const Mer if (part->info.isDisjoint(MergeTreePartInfo::fromPartName(part_name, queue.format_version))) continue; - if (out_reason) - *out_reason = fmt::format("Part {} participates in REPLACE_RANGE {} ({})", part_name, entry->new_part_name, entry->znode_name); - + out_reason = fmt::format("Part {} participates in REPLACE_RANGE {} ({})", part_name, entry->new_part_name, entry->znode_name); return true; } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 79572e139632..26db5a05b45c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -501,13 +501,13 @@ class LocalMergePredicate bool operator()(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, const MergeTreeTransaction * txn, - String * out_reason = nullptr) const; + String & out_reason) const; bool canMergeTwoParts(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, - String * out_reason = nullptr) const; + String & out_reason) const; - bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const; + bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String & out_reason) const; private: const ReplicatedMergeTreeQueue & queue; @@ -523,23 +523,23 @@ class ReplicatedMergeTreeMergePredicate bool operator()(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, const MergeTreeTransaction * txn, - String * out_reason = nullptr) const; + String & out_reason) const; /// Can we assign a merge with these two parts? /// (assuming that no merge was assigned after the predicate was constructed) /// If we can't and out_reason is not nullptr, set it to the reason why we can't merge. bool canMergeTwoParts(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, - String * out_reason = nullptr) const; + String & out_reason) const; /// Can we assign a merge this part and some other part? /// For example a merge of a part and itself is needed for TTL. /// This predicate is checked for the first part of each range. - bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const; + bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String & out_reason) const; /// Returns true if part is needed for some REPLACE_RANGE entry. /// We should not drop part in this case, because replication queue may stuck without that part. - bool partParticipatesInReplaceRange(const MergeTreeData::DataPartPtr & part, String * out_reason) const; + bool partParticipatesInReplaceRange(const MergeTreeData::DataPartPtr & part, String & out_reason) const; /// Return nonempty optional of desired mutation version and alter version. /// If we have no alter (modify/drop) mutations in mutations queue, than we return biggest possible diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c02c96f62be4..c21b87c223df 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -839,7 +839,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( bool aggressive, const String & partition_id, bool final, - String * out_disable_reason, + String & out_disable_reason, TableLockHolder & /* table_lock_holder */, std::unique_lock & lock, const MergeTreeTransactionPtr & txn, @@ -857,7 +857,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( CurrentlyMergingPartsTaggerPtr merging_tagger; MergeList::EntryPtr merge_entry; - auto can_merge = [this, &lock](const DataPartPtr & left, const DataPartPtr & right, const MergeTreeTransaction * tx, String * disable_reason) -> bool + auto can_merge = [this, &lock](const DataPartPtr & left, const DataPartPtr & right, const MergeTreeTransaction * tx, String & disable_reason) -> bool { if (tx) { @@ -866,8 +866,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( if ((left && !left->version.isVisible(tx->getSnapshot(), Tx::EmptyTID)) || (right && !right->version.isVisible(tx->getSnapshot(), Tx::EmptyTID))) { - if (disable_reason) - *disable_reason = "Some part is not visible in transaction"; + disable_reason = "Some part is not visible in transaction"; return false; } @@ -875,8 +874,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( if ((left && left->version.isRemovalTIDLocked()) || (right && right->version.isRemovalTIDLocked())) { - if (disable_reason) - *disable_reason = "Some part is locked for removal in another cuncurrent transaction"; + disable_reason = "Some part is locked for removal in another cuncurrent transaction"; return false; } } @@ -887,8 +885,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( { if (currently_merging_mutating_parts.contains(right)) { - if (disable_reason) - *disable_reason = "Some part currently in a merging or mutating process"; + disable_reason = "Some part currently in a merging or mutating process"; return false; } else @@ -897,30 +894,26 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( if (currently_merging_mutating_parts.contains(left) || currently_merging_mutating_parts.contains(right)) { - if (disable_reason) - *disable_reason = "Some part currently in a merging or mutating process"; + disable_reason = "Some part currently in a merging or mutating process"; return false; } if (getCurrentMutationVersion(left, lock) != getCurrentMutationVersion(right, lock)) { - if (disable_reason) - *disable_reason = "Some parts have differ mmutatuon version"; + disable_reason = "Some parts have differ mmutatuon version"; return false; } if (!partsContainSameProjections(left, right)) { - if (disable_reason) - *disable_reason = "Some parts contains differ projections"; + disable_reason = "Some parts contains differ projections"; return false; } auto max_possible_level = getMaxLevelInBetween(left, right); if (max_possible_level > std::max(left->info.level, right->info.level)) { - if (disable_reason) - *disable_reason = fmt::format("There is an outdated part in a gap between two active parts ({}, {}) with merge level {} higher than these active parts have", left->name, right->name, max_possible_level); + disable_reason = fmt::format("There is an outdated part in a gap between two active parts ({}, {}) with merge level {} higher than these active parts have", left->name, right->name, max_possible_level); return false; } @@ -931,10 +924,9 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( if (!canEnqueueBackgroundTask()) { - if (out_disable_reason) - *out_disable_reason = fmt::format("Current background tasks memory usage ({}) is more than the limit ({})", - formatReadableSizeWithBinarySuffix(background_memory_tracker.get()), - formatReadableSizeWithBinarySuffix(background_memory_tracker.getSoftLimit())); + out_disable_reason = fmt::format("Current background tasks memory usage ({}) is more than the limit ({})", + formatReadableSizeWithBinarySuffix(background_memory_tracker.get()), + formatReadableSizeWithBinarySuffix(background_memory_tracker.getSoftLimit())); } else if (partition_id.empty()) { @@ -955,8 +947,8 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( txn, out_disable_reason); } - else if (out_disable_reason) - *out_disable_reason = "Current value of max_source_parts_size is zero"; + else + out_disable_reason = "Current value of max_source_parts_size is zero"; } else { @@ -970,15 +962,14 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( /// If final - we will wait for currently processing merges to finish and continue. if (final && select_decision != SelectPartsDecision::SELECTED - && !currently_merging_mutating_parts.empty() - && out_disable_reason) + && !currently_merging_mutating_parts.empty()) { LOG_DEBUG(log, "Waiting for currently running merges ({} parts are merging right now) to perform OPTIMIZE FINAL", currently_merging_mutating_parts.size()); if (std::cv_status::timeout == currently_processing_in_background_condition.wait_for(lock, timeout)) { - *out_disable_reason = fmt::format("Timeout ({} ms) while waiting for already running merges before running OPTIMIZE with FINAL", timeout_ms); + out_disable_reason = fmt::format("Timeout ({} ms) while waiting for already running merges before running OPTIMIZE with FINAL", timeout_ms); break; } } @@ -994,14 +985,9 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( if (select_decision != SelectPartsDecision::SELECTED) { - if (out_disable_reason) - { - if (!out_disable_reason->empty()) - { - *out_disable_reason += ". "; - } - *out_disable_reason += "Cannot select parts for optimization"; - } + if (!out_disable_reason.empty()) + out_disable_reason += ". "; + out_disable_reason += "Cannot select parts for optimization"; return {}; } @@ -1022,7 +1008,7 @@ bool StorageMergeTree::merge( const Names & deduplicate_by_columns, bool cleanup, const MergeTreeTransactionPtr & txn, - String * out_disable_reason, + String & out_disable_reason, bool optimize_skip_merged_partitions) { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -1077,7 +1063,7 @@ bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & p } MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( - const StorageMetadataPtr & metadata_snapshot, String * /* disable_reason */, TableLockHolder & /* table_lock_holder */, + const StorageMetadataPtr & metadata_snapshot, String & /* disable_reason */, TableLockHolder & /* table_lock_holder */, std::unique_lock & /*currently_processing_in_background_mutex_lock*/) { if (current_mutations_by_version.empty()) @@ -1278,10 +1264,11 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign if (merger_mutator.merges_blocker.isCancelled()) return false; - merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, shared_lock, lock, txn); + String out_reason; + merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, out_reason, shared_lock, lock, txn); if (!merge_entry && !current_mutations_by_version.empty()) - mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr, shared_lock, lock); + mutate_entry = selectPartsToMutate(metadata_snapshot, out_reason, shared_lock, lock); has_mutations = !current_mutations_by_version.empty(); } @@ -1484,7 +1471,7 @@ bool StorageMergeTree::optimize( deduplicate_by_columns, cleanup, txn, - &disable_reason, + disable_reason, local_context->getSettingsRef().optimize_skip_merged_partitions)) { constexpr auto message = "Cannot OPTIMIZE table: {}"; @@ -1512,7 +1499,7 @@ bool StorageMergeTree::optimize( deduplicate_by_columns, cleanup, txn, - &disable_reason, + disable_reason, local_context->getSettingsRef().optimize_skip_merged_partitions)) { constexpr auto message = "Cannot OPTIMIZE table: {}"; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 8099f9c16aa1..370283e650cc 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -176,7 +176,7 @@ class StorageMergeTree final : public MergeTreeData const Names & deduplicate_by_columns, bool cleanup, const MergeTreeTransactionPtr & txn, - String * out_disable_reason = nullptr, + String & out_disable_reason, bool optimize_skip_merged_partitions = false); void renameAndCommitEmptyParts(MutableDataPartsVector & new_parts, Transaction & transaction); @@ -203,7 +203,7 @@ class StorageMergeTree final : public MergeTreeData bool aggressive, const String & partition_id, bool final, - String * disable_reason, + String & disable_reason, TableLockHolder & table_lock_holder, std::unique_lock & lock, const MergeTreeTransactionPtr & txn, @@ -212,7 +212,7 @@ class StorageMergeTree final : public MergeTreeData MergeMutateSelectedEntryPtr selectPartsToMutate( - const StorageMetadataPtr & metadata_snapshot, String * disable_reason, + const StorageMetadataPtr & metadata_snapshot, String & disable_reason, TableLockHolder & table_lock_holder, std::unique_lock & currently_processing_in_background_mutex_lock); /// For current mutations queue, returns maximum version of mutation for a part, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b1ba06c77f9f..4298deb06dd1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3438,9 +3438,10 @@ void StorageReplicatedMergeTree::mergeSelectingTask() merge_pred.emplace(queue.getMergePredicate(zookeeper, partitions_to_merge_in)); } + String out_reason; if (can_assign_merge && merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, *merge_pred, - merge_with_ttl_allowed, NO_TRANSACTION_PTR, nullptr, &partitions_to_merge_in) == SelectPartsDecision::SELECTED) + merge_with_ttl_allowed, NO_TRANSACTION_PTR, out_reason, &partitions_to_merge_in) == SelectPartsDecision::SELECTED) { create_result = createLogEntryToMergeParts( zookeeper, @@ -5169,13 +5170,13 @@ bool StorageReplicatedMergeTree::optimize( { select_decision = merger_mutator.selectPartsToMerge( future_merged_part, /* aggressive */ true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, - can_merge, /* merge_with_ttl_allowed */ false, NO_TRANSACTION_PTR, &disable_reason); + can_merge, /* merge_with_ttl_allowed */ false, NO_TRANSACTION_PTR, disable_reason); } else { select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( future_merged_part, can_merge, partition_id, final, metadata_snapshot, NO_TRANSACTION_PTR, - &disable_reason, query_context->getSettingsRef().optimize_skip_merged_partitions); + disable_reason, query_context->getSettingsRef().optimize_skip_merged_partitions); } /// If there is nothing to merge then we treat this merge as successful (needed for optimize final optimization) @@ -7711,7 +7712,7 @@ void StorageReplicatedMergeTree::movePartitionToShard( /// canMergeSinglePart is overlapping with dropPart, let's try to use the same code. String out_reason; - if (!merge_pred.canMergeSinglePart(part, &out_reason)) + if (!merge_pred.canMergeSinglePart(part, out_reason)) throw Exception(ErrorCodes::PART_IS_TEMPORARILY_LOCKED, "Part is busy, reason: {}", out_reason); } @@ -7959,14 +7960,14 @@ bool StorageReplicatedMergeTree::dropPartImpl( /// There isn't a lot we can do otherwise. Can't cancel merges because it is possible that a replica already /// finished the merge. String out_reason; - if (!merge_pred.canMergeSinglePart(part, &out_reason)) + if (!merge_pred.canMergeSinglePart(part, out_reason)) { if (throw_if_noop) throw Exception::createDeprecated(out_reason, ErrorCodes::PART_IS_TEMPORARILY_LOCKED); return false; } - if (merge_pred.partParticipatesInReplaceRange(part, &out_reason)) + if (merge_pred.partParticipatesInReplaceRange(part, out_reason)) { if (throw_if_noop) throw Exception::createDeprecated(out_reason, ErrorCodes::PART_IS_TEMPORARILY_LOCKED); From dee71d2e2f8cdd6be4a82f26e7af9b8a75453091 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 6 Jul 2023 13:16:31 +0000 Subject: [PATCH 025/133] Add first version of hasSubsequence() --- src/Functions/HasSubsequenceImpl.h | 131 ++++++++++++++++++ src/Functions/hasSubsequence.cpp | 29 ++++ .../hasSubsequenceCaseInsensitive.cpp | 28 ++++ src/Functions/like.cpp | 1 - .../02809_has_subsequence.reference | 16 +++ .../0_stateless/02809_has_subsequence.sql | 19 +++ 6 files changed, 223 insertions(+), 1 deletion(-) create mode 100644 src/Functions/HasSubsequenceImpl.h create mode 100644 src/Functions/hasSubsequence.cpp create mode 100644 src/Functions/hasSubsequenceCaseInsensitive.cpp create mode 100644 tests/queries/0_stateless/02809_has_subsequence.reference create mode 100644 tests/queries/0_stateless/02809_has_subsequence.sql diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h new file mode 100644 index 000000000000..3a29ef68b0bf --- /dev/null +++ b/src/Functions/HasSubsequenceImpl.h @@ -0,0 +1,131 @@ +#pragma once + + +namespace DB +{ +namespace +{ + +template +struct HasSubsequenceImpl +{ + using ResultType = UInt8; + + static constexpr bool use_default_implementation_for_constants = false; + static constexpr bool supports_start_pos = false; + static constexpr auto name = Name::name; + + static ColumnNumbers getArgumentsThatAreAlwaysConstant() { return {};} + + /// Find one substring in many strings. + static void vectorConstant( + const ColumnString::Chars & /*haystack_data*/, + const ColumnString::Offsets & /*haystack_offsets*/, + const std::string & /*needle*/, + const ColumnPtr & /*start_pos*/, + PaddedPODArray & res, + [[maybe_unused]] ColumnUInt8 * /*res_null*/) + { + size_t size = res.size(); + for (size_t i = 0; i < size; ++i) + { + res[i] = 0; + } + } + + /// Search each time for a different single substring inside each time different string. + static void vectorVector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnString::Chars & needle_data, + const ColumnString::Offsets & needle_offsets, + const ColumnPtr & /*start_pos*/, + PaddedPODArray & res, + ColumnUInt8 * /*res_null*/) + { + ColumnString::Offset prev_haystack_offset = 0; + ColumnString::Offset prev_needle_offset = 0; + + size_t size = haystack_offsets.size(); + + for (size_t i = 0; i < size; ++i) + { + size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; + + if (0 == needle_size) + { + res[i] = 1; + } + else + { + const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); + const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); + res[i] = impl(haystack, haystack_size, needle, needle_size); + } + + prev_haystack_offset = haystack_offsets[i]; + prev_needle_offset = needle_offsets[i]; + } + } + + /// Find many substrings in single string. + static void constantVector( + const String & /*haystack*/, + const ColumnString::Chars & /*needle_data*/, + const ColumnString::Offsets & needle_offsets, + const ColumnPtr & /*start_pos*/, + PaddedPODArray & res, + ColumnUInt8 * /*res_null*/) + { + size_t size = needle_offsets.size(); + + for (size_t i = 0; i < size; ++i) + { + res[i] = 0; + } + } + + static UInt8 impl(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size) + { + size_t j = 0; + for (size_t i = 0; (i < haystack_size) && (j < needle_size); i++) + if (needle[j] == haystack[i]) + ++j; + return j == needle_size; + } + + static void constantConstant( + std::string haystack, + std::string needle, + const ColumnPtr & /*start_pos*/, + PaddedPODArray & res, + ColumnUInt8 * /*res_null*/) + { + size_t size = res.size(); + Impl::toLowerIfNeed(haystack); + Impl::toLowerIfNeed(needle); + + UInt8 result = impl(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); + + for (size_t i = 0; i < size; ++i) + { + res[i] = result; + } + } + template + static void vectorFixedConstant(Args &&...) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); + } + + template + static void vectorFixedVector(Args &&...) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); + } +}; + +} + +} diff --git a/src/Functions/hasSubsequence.cpp b/src/Functions/hasSubsequence.cpp new file mode 100644 index 000000000000..da2aaddcf50e --- /dev/null +++ b/src/Functions/hasSubsequence.cpp @@ -0,0 +1,29 @@ +#include +#include +#include + + +namespace DB +{ +namespace +{ + +struct HasSubsequenceCaseSensitiveASCII +{ + static void toLowerIfNeed(std::string & /*s*/) { } +}; + +struct NameHasSubsequence +{ + static constexpr auto name = "hasSubsequence"; +}; + +using FunctionHasSubsequence = FunctionsStringSearch>; +} + +REGISTER_FUNCTION(hasSubsequence) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/hasSubsequenceCaseInsensitive.cpp b/src/Functions/hasSubsequenceCaseInsensitive.cpp new file mode 100644 index 000000000000..f5c13a7cf8c7 --- /dev/null +++ b/src/Functions/hasSubsequenceCaseInsensitive.cpp @@ -0,0 +1,28 @@ +#include +#include +#include + +namespace DB +{ +namespace +{ + +struct HasSubsequenceCaseInsensitiveASCII +{ + static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } +}; + +struct NameHasSubsequenceCaseInsensitive +{ + static constexpr auto name = "hasSubsequenceCaseInsensitive"; +}; + +using FunctionHasSubsequenceCaseInsensitive = FunctionsStringSearch>; +} + +REGISTER_FUNCTION(hasSubsequenceCaseInsensitive) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/like.cpp b/src/Functions/like.cpp index 3a3345051d42..5a86e37a92d8 100644 --- a/src/Functions/like.cpp +++ b/src/Functions/like.cpp @@ -1,4 +1,3 @@ -#include "FunctionsStringSearch.h" #include "FunctionFactory.h" #include "like.h" diff --git a/tests/queries/0_stateless/02809_has_subsequence.reference b/tests/queries/0_stateless/02809_has_subsequence.reference new file mode 100644 index 000000000000..827caa105d0b --- /dev/null +++ b/tests/queries/0_stateless/02809_has_subsequence.reference @@ -0,0 +1,16 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +0 +0 +0 +1 +1 +1 +0 \ No newline at end of file diff --git a/tests/queries/0_stateless/02809_has_subsequence.sql b/tests/queries/0_stateless/02809_has_subsequence.sql new file mode 100644 index 000000000000..63ffb49dc540 --- /dev/null +++ b/tests/queries/0_stateless/02809_has_subsequence.sql @@ -0,0 +1,19 @@ +select hasSubsequence('garbage', ''); +select hasSubsequence('garbage', 'g'); +select hasSubsequence('garbage', 'a'); +select hasSubsequence('garbage', 'e'); +select hasSubsequence('garbage', 'gr'); +select hasSubsequence('garbage', 'ab'); +select hasSubsequence('garbage', 'be'); +select hasSubsequence('garbage', 'arg'); +select hasSubsequence('garbage', 'garbage'); + +select hasSubsequence('garbage', 'garbage1'); +select hasSubsequence('garbage', 'arbw'); +select hasSubsequence('garbage', 'ARG'); + +select hasSubsequenceCaseInsensitive('garbage', 'ARG'); + +select hasSubsequence(materialize('garbage'), materialize('')); +select hasSubsequence(materialize('garbage'), materialize('arg')); +select hasSubsequence(materialize('garbage'), materialize('garbage1')); \ No newline at end of file From 67e2dee7e2ea926d6a0a6ab35b31b2515f518426 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 6 Jul 2023 14:29:58 +0000 Subject: [PATCH 026/133] Allow SETTINGS before FORMAT in DESCRIBE TABLE query --- src/Parsers/ParserDescribeTableQuery.cpp | 20 +++++++++++++++---- src/Parsers/ParserQueryWithOutput.cpp | 2 +- src/Parsers/ParserTablePropertiesQuery.cpp | 2 -- src/Storages/StorageDistributed.cpp | 1 - src/Storages/getStructureOfRemoteTable.cpp | 1 - .../02789_describe_table_settings.reference | 10 ++++++++++ .../02789_describe_table_settings.sql | 3 +++ 7 files changed, 30 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02789_describe_table_settings.reference create mode 100644 tests/queries/0_stateless/02789_describe_table_settings.sql diff --git a/src/Parsers/ParserDescribeTableQuery.cpp b/src/Parsers/ParserDescribeTableQuery.cpp index ad6d2c5bcc67..fcfc4799dbec 100644 --- a/src/Parsers/ParserDescribeTableQuery.cpp +++ b/src/Parsers/ParserDescribeTableQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -16,8 +17,10 @@ bool ParserDescribeTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ex ParserKeyword s_describe("DESCRIBE"); ParserKeyword s_desc("DESC"); ParserKeyword s_table("TABLE"); + ParserKeyword s_settings("SETTINGS"); ParserToken s_dot(TokenType::Dot); ParserIdentifier name_p; + ParserSetQuery parser_settings(true); ASTPtr database; ASTPtr table; @@ -29,12 +32,21 @@ bool ParserDescribeTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ex s_table.ignore(pos, expected); - ASTPtr table_expression; - if (!ParserTableExpression().parse(pos, table_expression, expected)) + if (!ParserTableExpression().parse(pos, query->table_expression, expected)) return false; - query->children.push_back(std::move(table_expression)); - query->table_expression = query->children.back(); + /// For compatibility with SELECTs, where SETTINGS can be in front of FORMAT + ASTPtr settings; + if (s_settings.ignore(pos, expected)) + { + if (!parser_settings.parse(pos, query->settings_ast, expected)) + return false; + } + + query->children.push_back(query->table_expression); + + if (query->settings_ast) + query->children.push_back(query->settings_ast); node = query; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 6796f4528c4d..5dc713ca8c67 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -150,7 +150,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec // SETTINGS key1 = value1, key2 = value2, ... ParserKeyword s_settings("SETTINGS"); - if (s_settings.ignore(pos, expected)) + if (!query_with_output.settings_ast && s_settings.ignore(pos, expected)) { ParserSetQuery parser_settings(true); if (!parser_settings.parse(pos, query_with_output.settings_ast, expected)) diff --git a/src/Parsers/ParserTablePropertiesQuery.cpp b/src/Parsers/ParserTablePropertiesQuery.cpp index b73ce8de3598..94f264fcc89e 100644 --- a/src/Parsers/ParserTablePropertiesQuery.cpp +++ b/src/Parsers/ParserTablePropertiesQuery.cpp @@ -14,8 +14,6 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & { ParserKeyword s_exists("EXISTS"); ParserKeyword s_temporary("TEMPORARY"); - ParserKeyword s_describe("DESCRIBE"); - ParserKeyword s_desc("DESC"); ParserKeyword s_show("SHOW"); ParserKeyword s_create("CREATE"); ParserKeyword s_database("DATABASE"); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b91ad0b963a2..b6359bbb251e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -60,7 +60,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index e5fc01be9f47..ec8f27feedae 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include diff --git a/tests/queries/0_stateless/02789_describe_table_settings.reference b/tests/queries/0_stateless/02789_describe_table_settings.reference new file mode 100644 index 000000000000..c2bf9219f4de --- /dev/null +++ b/tests/queries/0_stateless/02789_describe_table_settings.reference @@ -0,0 +1,10 @@ +"id","Nullable(Int64)","","","","","" +"age","LowCardinality(UInt8)","","","","","" +"name","Nullable(String)","","","","","" +"status","Nullable(String)","","","","","" +"hobbies","Array(Nullable(String))","","","","","" +"id","Nullable(Int64)","","","","","" +"age","LowCardinality(UInt8)","","","","","" +"name","Nullable(String)","","","","","" +"status","Nullable(String)","","","","","" +"hobbies","Array(Nullable(String))","","","","","" diff --git a/tests/queries/0_stateless/02789_describe_table_settings.sql b/tests/queries/0_stateless/02789_describe_table_settings.sql new file mode 100644 index 000000000000..64b5b21fea80 --- /dev/null +++ b/tests/queries/0_stateless/02789_describe_table_settings.sql @@ -0,0 +1,3 @@ +DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}') SETTINGS schema_inference_hints = 'age LowCardinality(UInt8), status Nullable(String)', allow_suspicious_low_cardinality_types=1 FORMAT CSV; +DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}') FORMAT CSV SETTINGS schema_inference_hints = 'age LowCardinality(UInt8), status Nullable(String)', allow_suspicious_low_cardinality_types=1; +DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}') FORMAT CSV SETTINGS schema_inference_hints = 'age LowCardinality(UInt8), status Nullable(String)', allow_suspicious_low_cardinality_types=1 SETTINGS max_threads=0; -- { clientError SYNTAX_ERROR } From 7255c35edcefe03a39ad7bcf460d9dca5670ca3b Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 6 Jul 2023 19:43:37 +0000 Subject: [PATCH 027/133] Add more tests --- .../functions/string-search-functions.md | 50 +++++++++++++ .../functions/string-search-functions.md | 52 +++++++++++++ src/Functions/HasSubsequenceImpl.h | 74 ++++++++++++------- src/Functions/hasSubsequence.cpp | 2 +- .../hasSubsequenceCaseInsensitive.cpp | 2 +- .../hasSubsequenceCaseInsensitiveUTF8.cpp | 28 +++++++ src/Functions/hasSubsequenceUTF8.cpp | 29 ++++++++ .../02809_has_subsequence.reference | 13 +++- .../0_stateless/02809_has_subsequence.sql | 20 ++++- 9 files changed, 237 insertions(+), 33 deletions(-) create mode 100644 src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp create mode 100644 src/Functions/hasSubsequenceUTF8.cpp diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 3d8f89f72954..04ad64743108 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -631,3 +631,53 @@ Result: │ 100 │ 200 │ 100-200 │ 100 │ └──────────────────────────────────────────────┴──────────────────────────────────────────────┴──────────────────────────────────────────────┴───────────────────────────────────────────┘ ``` + +## hasSubsequence + +Returns 1 if needle is a subsequence of haystack, or 0 otherwise. +A subsequence of a string is a sequence that can be derived from the given string by deleting zero or more elements without changing the order of the remaining elements. + + +**Syntax** + +``` sql +hasSubsequence(haystack, needle) +``` + +**Arguments** + +- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal). + +**Returned values** + +- 1, if needle is a subsequence of haystack. +- 0, otherwise. + +Type: `UInt8`. + +**Examples** + +``` sql +SELECT hasSubsequence('garbage', 'arg') ; +``` + +Result: + +``` text +┌─hasSubsequence('garbage', 'arg')─┐ +│ 1 │ +└──────────────────────────────────┘ +``` + +## hasSubsequenceCaseInsensitive + +Like [hasSubsequence](#hasSubsequence) but searches case-insensitively. + +## hasSubsequenceUTF8 + +Like [hasSubsequence](#hasSubsequence) but assumes `haystack` and `needle` are UTF-8 encoded strings. + +## hasSubsequenceCaseInsensitiveUTF8 + +Like [hasSubsequenceUTF8](#hasSubsequenceUTF8) but searches case-insensitively. \ No newline at end of file diff --git a/docs/ru/sql-reference/functions/string-search-functions.md b/docs/ru/sql-reference/functions/string-search-functions.md index ea4f90d4f66e..21989e882b6d 100644 --- a/docs/ru/sql-reference/functions/string-search-functions.md +++ b/docs/ru/sql-reference/functions/string-search-functions.md @@ -801,3 +801,55 @@ SELECT countSubstringsCaseInsensitiveUTF8('аБв__АбВ__абв', 'Абв'); │ 3 │ └────────────────────────────────────────────────────────────┘ ``` + +## hasSubsequence(haystack, needle) {#hasSubsequence} + +Возвращает 1 если needle является подпоследовательностью haystack, иначе 0. + + +**Синтаксис** + +``` sql +hasSubsequence(haystack, needle) +``` + +**Аргументы** + +- `haystack` — строка, по которой выполняется поиск. [Строка](../syntax.md#syntax-string-literal). +- `needle` — подстрока, которую необходимо найти. [Строка](../syntax.md#syntax-string-literal). + +**Возвращаемые значения** + +- 1, если +- 0, если подстрока не найдена. + +Тип: `UInt8`. + +**Примеры** + +Запрос: + +``` sql +SELECT hasSubsequence('garbage', 'arg') ; +``` + +Результат: + +``` text +┌─hasSubsequence('garbage', 'arg')─┐ +│ 1 │ +└──────────────────────────────────┘ +``` + + +## hasSubsequenceCaseInsensitive + +Такая же, как и [hasSubsequence](#hasSubsequence), но работает без учета регистра. + +## hasSubsequenceUTF8 + +Такая же, как и [hasSubsequence](#hasSubsequence) при допущении что `haystack` и `needle` содержат набор кодовых точек, представляющий текст в кодировке UTF-8. + +## hasSubsequenceCaseInsensitiveUTF8 + +Такая же, как и [hasSubsequenceUTF8](#hasSubsequenceUTF8), но работает без учета регистра. diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h index 3a29ef68b0bf..bcb8e8e99e6d 100644 --- a/src/Functions/HasSubsequenceImpl.h +++ b/src/Functions/HasSubsequenceImpl.h @@ -1,11 +1,8 @@ #pragma once - - namespace DB { namespace { - template struct HasSubsequenceImpl { @@ -17,23 +14,31 @@ struct HasSubsequenceImpl static ColumnNumbers getArgumentsThatAreAlwaysConstant() { return {};} - /// Find one substring in many strings. static void vectorConstant( - const ColumnString::Chars & /*haystack_data*/, - const ColumnString::Offsets & /*haystack_offsets*/, - const std::string & /*needle*/, + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const String & needle, const ColumnPtr & /*start_pos*/, PaddedPODArray & res, [[maybe_unused]] ColumnUInt8 * /*res_null*/) { - size_t size = res.size(); - for (size_t i = 0; i < size; ++i) + if (needle.empty()) + { + for (auto & r : res) + r = 1; + return; + } + + ColumnString::Offset prev_haystack_offset = 0; + for (size_t i = 0; i < haystack_offsets.size(); ++i) { - res[i] = 0; + size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; + const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); + res[i] = hasSubsequence(haystack, haystack_size, needle.c_str(), needle.size()); + prev_haystack_offset = haystack_offsets[i]; } } - /// Search each time for a different single substring inside each time different string. static void vectorVector( const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, @@ -61,7 +66,7 @@ struct HasSubsequenceImpl { const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); - res[i] = impl(haystack, haystack_size, needle, needle_size); + res[i] = hasSubsequence(haystack, haystack_size, needle, needle_size); } prev_haystack_offset = haystack_offsets[i]; @@ -69,35 +74,38 @@ struct HasSubsequenceImpl } } - /// Find many substrings in single string. static void constantVector( - const String & /*haystack*/, - const ColumnString::Chars & /*needle_data*/, + const String & haystack, + const ColumnString::Chars & needle_data, const ColumnString::Offsets & needle_offsets, const ColumnPtr & /*start_pos*/, PaddedPODArray & res, ColumnUInt8 * /*res_null*/) { + ColumnString::Offset prev_needle_offset = 0; + size_t size = needle_offsets.size(); for (size_t i = 0; i < size; ++i) { - res[i] = 0; - } - } + size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; - static UInt8 impl(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size) - { - size_t j = 0; - for (size_t i = 0; (i < haystack_size) && (j < needle_size); i++) - if (needle[j] == haystack[i]) - ++j; - return j == needle_size; + if (0 == needle_size) + { + res[i] = 1; + } + else + { + const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); + res[i] = hasSubsequence(haystack.c_str(), haystack.size(), needle, needle_size); + } + prev_needle_offset = needle_offsets[i]; + } } static void constantConstant( - std::string haystack, - std::string needle, + String haystack, + String needle, const ColumnPtr & /*start_pos*/, PaddedPODArray & res, ColumnUInt8 * /*res_null*/) @@ -106,13 +114,23 @@ struct HasSubsequenceImpl Impl::toLowerIfNeed(haystack); Impl::toLowerIfNeed(needle); - UInt8 result = impl(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); + UInt8 result = hasSubsequence(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); for (size_t i = 0; i < size; ++i) { res[i] = result; } } + + static UInt8 hasSubsequence(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size) + { + size_t j = 0; + for (size_t i = 0; (i < haystack_size) && (j < needle_size); i++) + if (needle[j] == haystack[i]) + ++j; + return j == needle_size; + } + template static void vectorFixedConstant(Args &&...) { diff --git a/src/Functions/hasSubsequence.cpp b/src/Functions/hasSubsequence.cpp index da2aaddcf50e..bb1f295cee43 100644 --- a/src/Functions/hasSubsequence.cpp +++ b/src/Functions/hasSubsequence.cpp @@ -10,7 +10,7 @@ namespace struct HasSubsequenceCaseSensitiveASCII { - static void toLowerIfNeed(std::string & /*s*/) { } + static void toLowerIfNeed(String & /*s*/) { } }; struct NameHasSubsequence diff --git a/src/Functions/hasSubsequenceCaseInsensitive.cpp b/src/Functions/hasSubsequenceCaseInsensitive.cpp index f5c13a7cf8c7..fe50ada9be96 100644 --- a/src/Functions/hasSubsequenceCaseInsensitive.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitive.cpp @@ -9,7 +9,7 @@ namespace struct HasSubsequenceCaseInsensitiveASCII { - static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } + static void toLowerIfNeed(String & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } }; struct NameHasSubsequenceCaseInsensitive diff --git a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp new file mode 100644 index 000000000000..2908c284a25d --- /dev/null +++ b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp @@ -0,0 +1,28 @@ +#include +#include +#include + +namespace DB +{ +namespace +{ + +struct HasSubsequenceCaseInsensitiveUTF8 +{ + static void toLowerIfNeed(String & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } +}; + +struct NameHasSubsequenceCaseInsensitiveUTF8 +{ + static constexpr auto name = "hasSubsequenceCaseInsensitiveUTF8"; +}; + +using FunctionHasSubsequenceCaseInsensitiveUTF8 = FunctionsStringSearch>; +} + +REGISTER_FUNCTION(hasSubsequenceCaseInsensitiveUTF8) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/hasSubsequenceUTF8.cpp b/src/Functions/hasSubsequenceUTF8.cpp new file mode 100644 index 000000000000..c0811de65751 --- /dev/null +++ b/src/Functions/hasSubsequenceUTF8.cpp @@ -0,0 +1,29 @@ +#include +#include +#include + + +namespace DB +{ +namespace +{ + +struct HasSubsequenceCaseSensitiveUTF8 +{ + static void toLowerIfNeed(String & /*s*/) { } +}; + +struct NameHasSubsequenceUTF8 +{ + static constexpr auto name = "hasSubsequenceUTF8"; +}; + +using FunctionHasSubsequenceUTF8 = FunctionsStringSearch>; +} + +REGISTER_FUNCTION(hasSubsequenceUTF8) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/tests/queries/0_stateless/02809_has_subsequence.reference b/tests/queries/0_stateless/02809_has_subsequence.reference index 827caa105d0b..d12c0ba9fb36 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.reference +++ b/tests/queries/0_stateless/02809_has_subsequence.reference @@ -1,3 +1,4 @@ +hasSubsequence / const / const 1 1 1 @@ -10,7 +11,17 @@ 0 0 0 +hasSubsequence / const / string 1 1 +0 +hasSubsequence / string / const +1 +1 +0 +hasSubsequence / string / string +1 +1 +0 +hasSubsequenceCaseInsensitive / const / const 1 -0 \ No newline at end of file diff --git a/tests/queries/0_stateless/02809_has_subsequence.sql b/tests/queries/0_stateless/02809_has_subsequence.sql index 63ffb49dc540..64f3fd8dc770 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.sql +++ b/tests/queries/0_stateless/02809_has_subsequence.sql @@ -1,3 +1,4 @@ +select 'hasSubsequence / const / const'; select hasSubsequence('garbage', ''); select hasSubsequence('garbage', 'g'); select hasSubsequence('garbage', 'a'); @@ -12,8 +13,23 @@ select hasSubsequence('garbage', 'garbage1'); select hasSubsequence('garbage', 'arbw'); select hasSubsequence('garbage', 'ARG'); -select hasSubsequenceCaseInsensitive('garbage', 'ARG'); +select 'hasSubsequence / const / string'; +select hasSubsequence('garbage', materialize('')); +select hasSubsequence('garbage', materialize('arg')); +select hasSubsequence('garbage', materialize('arbw')); + +select 'hasSubsequence / string / const'; +select hasSubsequence(materialize('garbage'), ''); +select hasSubsequence(materialize('garbage'), 'arg'); +select hasSubsequence(materialize('garbage'), 'arbw'); + +select 'hasSubsequence / string / string'; select hasSubsequence(materialize('garbage'), materialize('')); select hasSubsequence(materialize('garbage'), materialize('arg')); -select hasSubsequence(materialize('garbage'), materialize('garbage1')); \ No newline at end of file +select hasSubsequence(materialize('garbage'), materialize('garbage1')); + +select 'hasSubsequenceCaseInsensitive / const / const'; + +select hasSubsequenceCaseInsensitive('garbage', 'ARG'); + From ef3551fea00b6eeaa76884880a977e9a0768bb82 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Jul 2023 03:54:10 +0200 Subject: [PATCH 028/133] Maybe better tests --- tests/queries/0_stateless/00995_exception_while_insert.sh | 3 +-- tests/queries/0_stateless/01030_limit_by_with_ties_error.sh | 3 +-- tests/queries/0_stateless/01187_set_profile_as_setting.sh | 2 +- tests/queries/0_stateless/01442_merge_detach_attach_long.sh | 3 +-- tests/queries/0_stateless/01515_logtrace_function.sh | 3 +-- .../01583_parallel_parsing_exception_with_offset.sh | 3 +-- tests/queries/0_stateless/02359_send_logs_source_regexp.sh | 2 +- .../0_stateless/02360_rename_table_along_with_log_name.sh | 2 +- 8 files changed, 8 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/00995_exception_while_insert.sh b/tests/queries/0_stateless/00995_exception_while_insert.sh index e0cd264a2b7d..927ac6a54e56 100755 --- a/tests/queries/0_stateless/00995_exception_while_insert.sh +++ b/tests/queries/0_stateless/00995_exception_while_insert.sh @@ -1,11 +1,10 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS check;" $CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64, y UInt64 DEFAULT throwIf(x > 1500000)) ENGINE = Memory;" diff --git a/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh b/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh index 711a015f0449..c34148387898 100755 --- a/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh +++ b/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh @@ -1,11 +1,10 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - $CLICKHOUSE_CLIENT --query=""" SELECT * FROM (SELECT number % 5 AS a, count() AS b, c FROM numbers(10) ARRAY JOIN [1,2] AS c GROUP BY a,c) AS table diff --git a/tests/queries/0_stateless/01187_set_profile_as_setting.sh b/tests/queries/0_stateless/01187_set_profile_as_setting.sh index dacb609d790d..fccac57aea8a 100755 --- a/tests/queries/0_stateless/01187_set_profile_as_setting.sh +++ b/tests/queries/0_stateless/01187_set_profile_as_setting.sh @@ -4,13 +4,13 @@ unset CLICKHOUSE_LOG_COMMENT CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -n -m -q "select value, changed from system.settings where name='readonly';" $CLICKHOUSE_CLIENT -n -m -q "set profile='default'; select value, changed from system.settings where name='readonly';" $CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" 2>&1| grep -Fa "Cannot modify 'send_logs_level' setting in readonly mode" > /dev/null && echo "OK" -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=fatal/g') $CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=select+value,changed+from+system.settings+where+name='readonly'" diff --git a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh index c080dded1c8c..acb2550d48c3 100755 --- a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh +++ b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh @@ -4,11 +4,10 @@ set -e CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE t (x Int8) ENGINE = MergeTree ORDER BY tuple()" diff --git a/tests/queries/0_stateless/01515_logtrace_function.sh b/tests/queries/0_stateless/01515_logtrace_function.sh index 131ec0edb9ed..4ebecd0cc186 100755 --- a/tests/queries/0_stateless/01515_logtrace_function.sh +++ b/tests/queries/0_stateless/01515_logtrace_function.sh @@ -2,9 +2,8 @@ # Tags: race CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=debug # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') - ${CLICKHOUSE_CLIENT} --query="SELECT logTrace('logTrace Function Test');" 2>&1 | grep -q "logTrace Function Test" && echo "OK" || echo "FAIL" diff --git a/tests/queries/0_stateless/01583_parallel_parsing_exception_with_offset.sh b/tests/queries/0_stateless/01583_parallel_parsing_exception_with_offset.sh index aa3a25096c00..00d22cb8e83d 100755 --- a/tests/queries/0_stateless/01583_parallel_parsing_exception_with_offset.sh +++ b/tests/queries/0_stateless/01583_parallel_parsing_exception_with_offset.sh @@ -1,11 +1,10 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS check;" $CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64) ENGINE = Memory;" diff --git a/tests/queries/0_stateless/02359_send_logs_source_regexp.sh b/tests/queries/0_stateless/02359_send_logs_source_regexp.sh index d3b60bc59f44..f287e323ca7e 100755 --- a/tests/queries/0_stateless/02359_send_logs_source_regexp.sh +++ b/tests/queries/0_stateless/02359_send_logs_source_regexp.sh @@ -1,11 +1,11 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh [ ! -z "$CLICKHOUSE_CLIENT_REDEFINED" ] && CLICKHOUSE_CLIENT=$CLICKHOUSE_CLIENT_REDEFINED -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') regexp="executeQuery|InterpreterSelectQuery" $CLICKHOUSE_CLIENT --send_logs_source_regexp "$regexp" -q "SELECT 1;" 2> >(grep -v -E "$regexp" 1>&2) diff --git a/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh index e8c7f844b5cc..c07dcdd549bf 100755 --- a/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh +++ b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh @@ -11,7 +12,6 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS y;" $CLICKHOUSE_CLIENT -q "CREATE TABLE x(i int) ENGINE MergeTree ORDER BY i;" $CLICKHOUSE_CLIENT -q "RENAME TABLE x TO y;" -CLICKHOUSE_CLIENT_WITH_LOG=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') regexp="${CLICKHOUSE_DATABASE}\\.x" # Check if there are still log entries with old table name $CLICKHOUSE_CLIENT_WITH_LOG --send_logs_source_regexp "$regexp" -q "INSERT INTO y VALUES(1);" From 39d0b309bd730748b52acfb32de729e8f8496f83 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 7 Jul 2023 13:15:26 +0000 Subject: [PATCH 029/133] Make own function with slices --- src/Functions/HasSubsequenceImpl.h | 191 ++++++++---------- src/Functions/hasSubsequence.cpp | 2 +- .../hasSubsequenceCaseInsensitive.cpp | 2 +- .../hasSubsequenceCaseInsensitiveUTF8.cpp | 2 +- src/Functions/hasSubsequenceUTF8.cpp | 2 +- 5 files changed, 86 insertions(+), 113 deletions(-) diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h index bcb8e8e99e6d..1396e64ade51 100644 --- a/src/Functions/HasSubsequenceImpl.h +++ b/src/Functions/HasSubsequenceImpl.h @@ -1,124 +1,109 @@ #pragma once + +#include +#include +#include +#include +#include namespace DB { namespace { + +using namespace GatherUtils; + template -struct HasSubsequenceImpl +class FunctionsHasSubsequenceImpl : public IFunction { - using ResultType = UInt8; - - static constexpr bool use_default_implementation_for_constants = false; - static constexpr bool supports_start_pos = false; +public: static constexpr auto name = Name::name; - static ColumnNumbers getArgumentsThatAreAlwaysConstant() { return {};} + static FunctionPtr create(ContextPtr) { return std::make_shared(); } - static void vectorConstant( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const String & needle, - const ColumnPtr & /*start_pos*/, - PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * /*res_null*/) - { - if (needle.empty()) - { - for (auto & r : res) - r = 1; - return; - } + String getName() const override { return name; } - ColumnString::Offset prev_haystack_offset = 0; - for (size_t i = 0; i < haystack_offsets.size(); ++i) - { - size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; - const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); - res[i] = hasSubsequence(haystack, haystack_size, needle.c_str(), needle.size()); - prev_haystack_offset = haystack_offsets[i]; - } - } + bool isVariadic() const override { return false; } - static void vectorVector( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const ColumnString::Chars & needle_data, - const ColumnString::Offsets & needle_offsets, - const ColumnPtr & /*start_pos*/, - PaddedPODArray & res, - ColumnUInt8 * /*res_null*/) - { - ColumnString::Offset prev_haystack_offset = 0; - ColumnString::Offset prev_needle_offset = 0; + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - size_t size = haystack_offsets.size(); + size_t getNumberOfArguments() const override { return 2; } - for (size_t i = 0; i < size; ++i) - { - size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; - size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; - - if (0 == needle_size) - { - res[i] = 1; - } - else - { - const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); - const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); - res[i] = hasSubsequence(haystack, haystack_size, needle, needle_size); - } - - prev_haystack_offset = haystack_offsets[i]; - prev_needle_offset = needle_offsets[i]; - } - } + bool useDefaultImplementationForConstants() const override { return false; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {};} - static void constantVector( - const String & haystack, - const ColumnString::Chars & needle_data, - const ColumnString::Offsets & needle_offsets, - const ColumnPtr & /*start_pos*/, - PaddedPODArray & res, - ColumnUInt8 * /*res_null*/) + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - ColumnString::Offset prev_needle_offset = 0; + if (!isString(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[0]->getName(), getName()); - size_t size = needle_offsets.size(); + if (!isString(arguments[1])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[1]->getName(), getName()); - for (size_t i = 0; i < size; ++i) - { - size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; - - if (0 == needle_size) - { - res[i] = 1; - } - else - { - const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); - res[i] = hasSubsequence(haystack.c_str(), haystack.size(), needle, needle_size); - } - prev_needle_offset = needle_offsets[i]; - } + return std::make_shared>(); } - static void constantConstant( - String haystack, - String needle, - const ColumnPtr & /*start_pos*/, - PaddedPODArray & res, - ColumnUInt8 * /*res_null*/) + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override { - size_t size = res.size(); - Impl::toLowerIfNeed(haystack); - Impl::toLowerIfNeed(needle); + const ColumnPtr & column_haystack = arguments[0].column; + const ColumnPtr & column_needle = arguments[1].column; + + const ColumnConst * haystack_const_string = checkAndGetColumnConst(column_haystack.get()); + const ColumnConst * needle_const_string = checkAndGetColumnConst(column_needle.get()); + const ColumnString * haystack_string = checkAndGetColumn(&*column_haystack); + const ColumnString * needle_string = checkAndGetColumn(&*column_needle); + + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_res = col_res->getData(); + vec_res.resize(input_rows_count); + + if (haystack_string && needle_string) + execute(StringSource{*haystack_string}, StringSource{*needle_string}, vec_res); + else if (haystack_string && needle_const_string) + execute(StringSource{*haystack_string}, ConstSource{*needle_const_string}, vec_res); + else if (haystack_const_string && needle_string) + execute(ConstSource{*haystack_const_string}, StringSource{*needle_string}, vec_res); + else if (haystack_const_string && needle_const_string) + execute(ConstSource{*haystack_const_string}, ConstSource{*needle_const_string}, vec_res); + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {}, first argument of function {} must be a string", + arguments[0].column->getName(), + getName()); + + return col_res; + } - UInt8 result = hasSubsequence(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); +private: - for (size_t i = 0; i < size; ++i) + template + void execute( + SourceHaystack && haystacks, + SourceNeedle && needles, + PaddedPODArray & res_data) const + { + size_t row_num = 0; + + while (!haystacks.isEnd()) { - res[i] = result; + [[maybe_unused]] auto haystack_slice = haystacks.getWhole(); + [[maybe_unused]] auto needle_slice = needles.getWhole(); + + auto haystack = std::string(reinterpret_cast(haystack_slice.data), haystack_slice.size); + auto needle = std::string(reinterpret_cast(needle_slice.data), needle_slice.size); + + Impl::toLowerIfNeed(haystack); + Impl::toLowerIfNeed(needle); + + res_data[row_num] = hasSubsequence(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); + haystacks.next(); + needles.next(); + ++row_num; } } @@ -130,18 +115,6 @@ struct HasSubsequenceImpl ++j; return j == needle_size; } - - template - static void vectorFixedConstant(Args &&...) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); - } - - template - static void vectorFixedVector(Args &&...) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); - } }; } diff --git a/src/Functions/hasSubsequence.cpp b/src/Functions/hasSubsequence.cpp index bb1f295cee43..900e80f55241 100644 --- a/src/Functions/hasSubsequence.cpp +++ b/src/Functions/hasSubsequence.cpp @@ -18,7 +18,7 @@ struct NameHasSubsequence static constexpr auto name = "hasSubsequence"; }; -using FunctionHasSubsequence = FunctionsStringSearch>; +using FunctionHasSubsequence = FunctionsHasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequence) diff --git a/src/Functions/hasSubsequenceCaseInsensitive.cpp b/src/Functions/hasSubsequenceCaseInsensitive.cpp index fe50ada9be96..dbac62d7f095 100644 --- a/src/Functions/hasSubsequenceCaseInsensitive.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitive.cpp @@ -17,7 +17,7 @@ struct NameHasSubsequenceCaseInsensitive static constexpr auto name = "hasSubsequenceCaseInsensitive"; }; -using FunctionHasSubsequenceCaseInsensitive = FunctionsStringSearch>; +using FunctionHasSubsequenceCaseInsensitive = FunctionsHasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequenceCaseInsensitive) diff --git a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp index 2908c284a25d..c104ff52857f 100644 --- a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp @@ -17,7 +17,7 @@ struct NameHasSubsequenceCaseInsensitiveUTF8 static constexpr auto name = "hasSubsequenceCaseInsensitiveUTF8"; }; -using FunctionHasSubsequenceCaseInsensitiveUTF8 = FunctionsStringSearch>; +using FunctionHasSubsequenceCaseInsensitiveUTF8 = FunctionsHasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequenceCaseInsensitiveUTF8) diff --git a/src/Functions/hasSubsequenceUTF8.cpp b/src/Functions/hasSubsequenceUTF8.cpp index c0811de65751..c67ce7d9c74e 100644 --- a/src/Functions/hasSubsequenceUTF8.cpp +++ b/src/Functions/hasSubsequenceUTF8.cpp @@ -18,7 +18,7 @@ struct NameHasSubsequenceUTF8 static constexpr auto name = "hasSubsequenceUTF8"; }; -using FunctionHasSubsequenceUTF8 = FunctionsStringSearch>; +using FunctionHasSubsequenceUTF8 = FunctionsHasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequenceUTF8) From 8e4c8f118cf64fcd77524439508b838c05a58fcf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Jul 2023 09:07:05 +0200 Subject: [PATCH 030/133] Fix disaster in integration tests, part 2 --- tests/integration/ci-runner.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index d6d17abe725e..43184574e6e3 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -406,9 +406,9 @@ def _get_all_tests(self, repo_path): out_file_full = os.path.join(self.result_path, "runner_get_all_tests.log") cmd = ( "cd {repo_path}/tests/integration && " - "timeout -s 9 1h ./runner {runner_opts} {image_cmd} ' --setup-plan' " - "| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' " - "| grep -v 'SKIPPED' | sort -u > {out_file}".format( + "timeout --signal=KILL 1h ./runner {runner_opts} {image_cmd} ' --setup-plan' " + "| tee '{out_file_full}' | grep -F '::' | sed -r 's/ \(fixtures used:.*//g; s/^ *//g; s/ *$//g' " + "| grep -v -F 'SKIPPED' | sort --unique > {out_file}".format( repo_path=repo_path, runner_opts=self._get_runner_opts(), image_cmd=image_cmd, @@ -626,7 +626,7 @@ def run_test_group( info_basename = test_group_str + "_" + str(i) + ".nfo" info_path = os.path.join(repo_path, "tests/integration", info_basename) - test_cmd = " ".join([test for test in sorted(test_names)]) + test_cmd = " ".join([f"'{test}'" for test in sorted(test_names)]) parallel_cmd = ( " --parallel {} ".format(num_workers) if num_workers > 0 else "" ) From 62bfa4ed93fb3796eccb0df041a9dfa057583c9b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Jul 2023 02:21:48 +0200 Subject: [PATCH 031/133] Fix performance test for regexp cache --- src/Functions/Regexps.h | 4 +++- tests/performance/re2_regex_caching.xml | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Functions/Regexps.h b/src/Functions/Regexps.h index 4bfd10bdbf56..aa8ae5b40540 100644 --- a/src/Functions/Regexps.h +++ b/src/Functions/Regexps.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -21,6 +22,7 @@ # include #endif + namespace ProfileEvents { extern const Event RegexpCreated; @@ -86,7 +88,7 @@ class LocalCacheTable private: constexpr static size_t CACHE_SIZE = 100; /// collision probability - std::hash hasher; + DefaultHash hasher; struct Bucket { String pattern; /// key diff --git a/tests/performance/re2_regex_caching.xml b/tests/performance/re2_regex_caching.xml index 6edc83097bae..9778a8d4c0c5 100644 --- a/tests/performance/re2_regex_caching.xml +++ b/tests/performance/re2_regex_caching.xml @@ -24,8 +24,8 @@ '.*' || toString(number) || '.' '.*' || toString(number % 10) || '.' - - '([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number) + + '([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number % 10)