From 32d892c9db424b7df8616ca5d6459cc78ba0cfde Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Tue, 15 Oct 2024 15:06:57 +0000 Subject: [PATCH 01/35] tests fix --- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- tests/integration/test_drop_replica/test.py | 19 ++++++++++--------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index b743095e6f6..3016f62f20d 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1010,7 +1010,7 @@ void InterpreterSystemQuery::dropReplica(ASTSystemQuery & query) { ReplicatedTableStatus status; storage_replicated->getStatus(status); - if (status.zookeeper_info.path == query.replica_zk_path) + if (status.replica_path == remote_replica_path) throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED, "There is a local table {}, which has the same table path in ZooKeeper. " "Please check the path in query. " diff --git a/tests/integration/test_drop_replica/test.py b/tests/integration/test_drop_replica/test.py index e0928c6ab08..b959e80fc19 100644 --- a/tests/integration/test_drop_replica/test.py +++ b/tests/integration/test_drop_replica/test.py @@ -141,11 +141,7 @@ def test_drop_replica(start_cluster): shard=1 ) ) - assert "There is a local table" in node_1_2.query_and_get_error( - "SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test/{shard}/replicated/test_table'".format( - shard=1 - ) - ) + assert "There is a local table" in node_1_1.query_and_get_error( "SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test/{shard}/replicated/test_table'".format( shard=1 @@ -221,11 +217,16 @@ def test_drop_replica(start_cluster): ) assert exists_replica_1_1 == None - node_1_2.query("SYSTEM DROP REPLICA 'node_1_1'") - exists_replica_1_1 = check_exists( + node_1_2.query("DETACH TABLE test4.test_table") + node_1_1.query( + "SYSTEM DROP REPLICA 'node_1_2' FROM ZKPATH '/clickhouse/tables/test4/{shard}/replicated/test_table'".format( + shard=1 + ) + ) + exists_replica_1_2 = check_exists( zk, "/clickhouse/tables/test4/{shard}/replicated/test_table/replicas/{replica}".format( - shard=1, replica="node_1_1" + shard=1, replica="node_1_2" ), ) - assert exists_replica_1_1 == None + assert exists_replica_1_2 == None From 42dd97b78c189ffdd1dcba13c5bc3ca84e1388f6 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Tue, 15 Oct 2024 15:23:47 +0000 Subject: [PATCH 02/35] Empty From d1b3f364fb55404427c756dafe959b8d05b31c99 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 16 Oct 2024 08:59:11 +0000 Subject: [PATCH 03/35] Fix flaky check --- tests/integration/test_drop_replica/test.py | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_drop_replica/test.py b/tests/integration/test_drop_replica/test.py index b959e80fc19..b70a0725039 100644 --- a/tests/integration/test_drop_replica/test.py +++ b/tests/integration/test_drop_replica/test.py @@ -8,6 +8,7 @@ def fill_nodes(nodes, shard): for node in nodes: node.query( """ + DROP DATABASE IF EXISTS test SYNC; CREATE DATABASE test; CREATE TABLE test.test_table(date Date, id UInt32) @@ -20,6 +21,7 @@ def fill_nodes(nodes, shard): node.query( """ + DROP DATABASE IF EXISTS test1 SYNC; CREATE DATABASE test1; CREATE TABLE test1.test_table(date Date, id UInt32) @@ -32,6 +34,7 @@ def fill_nodes(nodes, shard): node.query( """ + DROP DATABASE IF EXISTS test2 SYNC; CREATE DATABASE test2; CREATE TABLE test2.test_table(date Date, id UInt32) @@ -44,7 +47,8 @@ def fill_nodes(nodes, shard): node.query( """ - CREATE DATABASE test3; + DROP DATABASE IF EXISTS test3 SYNC; + CREATE DATABASE test3; CREATE TABLE test3.test_table(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test3/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) @@ -56,6 +60,7 @@ def fill_nodes(nodes, shard): node.query( """ + DROP DATABASE IF EXISTS test4 SYNC; CREATE DATABASE test4; CREATE TABLE test4.test_table(date Date, id UInt32) @@ -83,9 +88,6 @@ node_1_3 = cluster.add_instance( def start_cluster(): try: cluster.start() - - fill_nodes([node_1_1, node_1_2], 1) - yield cluster except Exception as ex: @@ -101,6 +103,8 @@ def check_exists(zk, path): def test_drop_replica(start_cluster): + fill_nodes([node_1_1, node_1_2], 1) + node_1_1.query( "INSERT INTO test.test_table SELECT number, toString(number) FROM numbers(100)" ) @@ -230,3 +234,7 @@ def test_drop_replica(start_cluster): ), ) assert exists_replica_1_2 == None + + node_1_1.query("ATTACH DATABASE test") + for i in range(1, 5): + node_1_1.query("ATTACH DATABASE test{}".format(i)) From 09a1f86db7d135d35b1dcb826776bca63f899abc Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Mon, 21 Oct 2024 08:36:08 +0000 Subject: [PATCH 04/35] Test fix --- tests/integration/test_drop_replica/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_drop_replica/test.py b/tests/integration/test_drop_replica/test.py index b70a0725039..201507a1734 100644 --- a/tests/integration/test_drop_replica/test.py +++ b/tests/integration/test_drop_replica/test.py @@ -221,6 +221,8 @@ def test_drop_replica(start_cluster): ) assert exists_replica_1_1 == None + node_1_1.query("ATTACH DATABASE test4") + node_1_2.query("DETACH TABLE test4.test_table") node_1_1.query( "SYSTEM DROP REPLICA 'node_1_2' FROM ZKPATH '/clickhouse/tables/test4/{shard}/replicated/test_table'".format( @@ -236,5 +238,5 @@ def test_drop_replica(start_cluster): assert exists_replica_1_2 == None node_1_1.query("ATTACH DATABASE test") - for i in range(1, 5): + for i in range(1, 4): node_1_1.query("ATTACH DATABASE test{}".format(i)) From 03e0e9a14b6099bf41f402b57065aedfc6515f0b Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 24 Oct 2024 16:05:19 -0300 Subject: [PATCH 05/35] draft --- .../Impl/Parquet/ParquetDataValuesReader.cpp | 43 +++++++++++++++++++ .../Impl/Parquet/ParquetDataValuesReader.h | 21 +++++++++ .../Impl/Parquet/ParquetLeafColReader.cpp | 23 ++++++++++ .../Impl/Parquet/ParquetRecordReader.cpp | 2 +- 4 files changed, 88 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index b8e4db8700c..977f2ad298b 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -296,6 +296,40 @@ void ParquetPlainValuesReader::readBatch( ); } +template <> +void ParquetBitPlainReader::readBatch( + MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) +{ + auto & column = *assert_cast(col_ptr.get()); + auto cursor = column.size(); + + auto & container = column.getData(); + + container.resize(cursor + num_values); + + def_level_reader->visitNullableValues( + cursor, + num_values, + max_def_level, + null_map, + /* individual_visitor */ [&](size_t nest_cursor) + { + uint8_t byte; + bit_reader->GetValue(1, &byte); + container[nest_cursor] = byte; + }, + /* repeated_visitor */ [&](size_t nest_cursor, UInt32 count) + { + for (UInt32 i = 0; i < count; i++) + { + uint8_t byte; + bit_reader->GetValue(1, &byte); + container[nest_cursor++] = byte; + } + } + ); +} + template <> void ParquetPlainValuesReader, ParquetReaderTypes::TimestampInt96>::readBatch( @@ -515,6 +549,13 @@ void ParquetRleDictReader::readBatch( ); } +template <> +void ParquetRleDictReader::readBatch( + MutableColumnPtr & , LazyNullMap &, UInt32) +{ + assert(false); +} + template void ParquetRleDictReader::readBatch( MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) @@ -561,6 +602,7 @@ template class ParquetPlainValuesReader>; template class ParquetPlainValuesReader>; template class ParquetPlainValuesReader>; template class ParquetPlainValuesReader; +template class ParquetPlainValuesReader; template class ParquetFixedLenPlainReader>; template class ParquetFixedLenPlainReader>; @@ -569,6 +611,7 @@ template class ParquetRleLCReader; template class ParquetRleLCReader; template class ParquetRleLCReader; +template class ParquetRleDictReader; template class ParquetRleDictReader; template class ParquetRleDictReader; template class ParquetRleDictReader; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h index fbccb612b3c..db55f7e2d6a 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h @@ -172,6 +172,27 @@ private: ParquetDataBuffer plain_data_buffer; }; +template +class ParquetBitPlainReader : public ParquetDataValuesReader +{ +public: + ParquetBitPlainReader( + Int32 max_def_level_, + std::unique_ptr def_level_reader_, + std::unique_ptr bit_reader_) + : max_def_level(max_def_level_) + , def_level_reader(std::move(def_level_reader_)) + , bit_reader(std::move(bit_reader_)) + {} + + void readBatch(MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) override; + +private: + Int32 max_def_level; + std::unique_ptr def_level_reader; + std::unique_ptr bit_reader; +}; + /** * The data and definition level encoding are same as ParquetPlainValuesReader. * But the element size is const and bigger than primitive data type. diff --git a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp index 4b5880eba37..f32d7e61062 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp @@ -463,6 +463,28 @@ void ParquetLeafColReader::initDataReader( } } +template <> +void ParquetLeafColReader::initDataReader( + parquet::Encoding::type enconding_type, + const uint8_t * buffer, + std::size_t max_size, + std::unique_ptr && def_level_reader) +{ + switch (enconding_type) + { + case parquet::Encoding::PLAIN: + { + auto bit_reader = std::make_unique(buffer, max_size); + data_values_reader = std::make_unique>(col_descriptor.max_definition_level(), + std::move(def_level_reader), + std::move(bit_reader)); + break; + } + default: + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "Unknown encoding type: {}", enconding_type); + } +} + template void ParquetLeafColReader::readPageV1(const parquet::DataPageV1 & page) { @@ -620,6 +642,7 @@ std::unique_ptr ParquetLeafColReader::createDi } +template class ParquetLeafColReader; template class ParquetLeafColReader; template class ParquetLeafColReader; template class ParquetLeafColReader; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index acf11a30162..971bb9e1be5 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -263,7 +263,7 @@ std::unique_ptr ColReaderFactory::makeReader() switch (col_descriptor.physical_type()) { case parquet::Type::BOOLEAN: - break; + return makeLeafReader(); case parquet::Type::INT32: return fromInt32(); case parquet::Type::INT64: From 2da1926338bd0f7347ef6f2ac3768984597c895f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 25 Oct 2024 09:37:22 -0300 Subject: [PATCH 06/35] tests --- ...03254_parquet_bool_native_reader.reference | 20 +++++++++++++++++ .../03254_parquet_bool_native_reader.sh | 21 ++++++++++++++++++ .../0_stateless/data_parquet/nullbool.parquet | Bin 0 -> 508 bytes 3 files changed, 41 insertions(+) create mode 100644 tests/queries/0_stateless/03254_parquet_bool_native_reader.reference create mode 100755 tests/queries/0_stateless/03254_parquet_bool_native_reader.sh create mode 100644 tests/queries/0_stateless/data_parquet/nullbool.parquet diff --git a/tests/queries/0_stateless/03254_parquet_bool_native_reader.reference b/tests/queries/0_stateless/03254_parquet_bool_native_reader.reference new file mode 100644 index 00000000000..0c7e55ad234 --- /dev/null +++ b/tests/queries/0_stateless/03254_parquet_bool_native_reader.reference @@ -0,0 +1,20 @@ +0 false +1 \N +2 false +3 \N +4 false +5 \N +6 false +7 \N +8 true +9 \N +0 false +1 \N +2 false +3 \N +4 false +5 \N +6 false +7 \N +8 true +9 \N diff --git a/tests/queries/0_stateless/03254_parquet_bool_native_reader.sh b/tests/queries/0_stateless/03254_parquet_bool_native_reader.sh new file mode 100755 index 00000000000..c28523b3c54 --- /dev/null +++ b/tests/queries/0_stateless/03254_parquet_bool_native_reader.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash +# Tags: no-ubsan, no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +WORKING_DIR="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}" + +mkdir -p "${WORKING_DIR}" + +DATA_FILE="${CUR_DIR}/data_parquet/nullbool.parquet" + +DATA_FILE_USER_PATH="${WORKING_DIR}/nullbool.parquet" + +cp ${DATA_FILE} ${DATA_FILE_USER_PATH} + +${CLICKHOUSE_CLIENT} --query="select id, bool from file('${DATA_FILE_USER_PATH}', Parquet) order by id SETTINGS input_format_parquet_use_native_reader=false;" +${CLICKHOUSE_CLIENT} --query="select id, bool from file('${DATA_FILE_USER_PATH}', Parquet) order by id SETTINGS input_format_parquet_use_native_reader=true;" diff --git a/tests/queries/0_stateless/data_parquet/nullbool.parquet b/tests/queries/0_stateless/data_parquet/nullbool.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d9b365bbe75bcd69ccee52c5eddffd3aa17b62cb GIT binary patch literal 508 zcmZ8e%}T>S5T1>-#va5%cE~~wy)?8?OJizlBow@fry}?OsadelSWT@zm!8Cz@Z<}4 z_TWi;1>eEDM?w5HxtZN>Co|v7>^JQA@Fb8V$5=m456@ekbl}?3rs5MgEnp3(0P8>% z*Z@q*CV>2HtnaVa&&{&DYRXN?`l;AfbxXOXmSBo}OED>Su&E#g7$GFWH52x0HBuhi za>b^|<3}M!<`)_9k)7Qy&dzm~$O-~Ya;<3!2~EqbOy=_$p@FA5zU?8qNwzY)M3h&& z^6j{kQ0if76@p3+H(?U=q_|y*BYv@@!yiH(kpCz^t39fsWpPu{bi^YtG32{~xYjD{ zzfNWz;&)3j{|eb7eiQ8YHjHD&bL+SH^jhcLY@X^__ae!(yP@xr>~f~bJ-$rxtEIl) z$@0dH&KJ}9MI-12{cg}`O_tMH+K7fa)zM%-91i=vUK*ssNQXf*7=(jK5~ezgcHvEu Qw(599*mQ(f9pl~q08?vBH2?qr literal 0 HcmV?d00001 From 5e9aa01f33a2a5745e4d4a131f3d3ddbe84a5808 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 17:25:52 +0100 Subject: [PATCH 07/35] Introduce Setting tiers --- docs/en/operations/system-tables/settings.md | 105 ++++++++++++++++-- src/Core/BaseSettings.cpp | 9 ++ src/Core/BaseSettings.h | 59 +++++----- src/Core/ServerSettings.cpp | 2 +- src/Core/Settings.cpp | 18 +-- src/Core/SettingsObsoleteMacros.h | 4 +- src/Core/SettingsTierType.cpp | 19 ++++ src/Core/SettingsTierType.h | 26 +++++ src/Storages/MergeTree/MergeTreeSettings.cpp | 4 +- src/Storages/System/StorageSystemSettings.cpp | 10 ++ 10 files changed, 206 insertions(+), 50 deletions(-) create mode 100644 src/Core/SettingsTierType.cpp create mode 100644 src/Core/SettingsTierType.h diff --git a/docs/en/operations/system-tables/settings.md b/docs/en/operations/system-tables/settings.md index a04e095e990..1cfee0ba5f4 100644 --- a/docs/en/operations/system-tables/settings.md +++ b/docs/en/operations/system-tables/settings.md @@ -18,6 +18,11 @@ Columns: - `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. +- `tier` ([Enum8](../../sql-reference/data-types/enum.md)) — Support level for this feature. ClickHouse features are organized in tiers, varying depending on the current status of their development and the expectations one might have when using them. Values: + - `'Production'` — The feature is stable, safe to use and does not have issues interacting with other **production** features. . + - `'Beta'` — The feature is stable and safe. The outcome of using it together with other features is unknown and correctness is not guaranteed. Testing and reports are welcome. + - `'Experimental'` — The feature is under development. Only intended for developers and ClickHouse enthusiasts. The feature might or might not work and could be removed at any time. + - `'Obsolete'` — No longer supported. Either it is already removed or it will be removed in future releases. **Example** @@ -26,19 +31,99 @@ The following example shows how to get information about settings which name con ``` sql SELECT * FROM system.settings -WHERE name LIKE '%min_i%' +WHERE name LIKE '%min_insert_block_size_%' +FORMAT Vertical ``` ``` text -┌─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 │ -└────────────────────────────────────────────────────┴───────────┴─────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────────────── -──────────────────────────────────────────────────────┴──────┴──────┴──────────┴──────────────┴───────────┴───────────┴─────────────┘ -``` +Row 1: +────── +name: min_insert_block_size_rows +value: 1048449 +changed: 0 +description: Sets the minimum number of rows in the block that can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. + +Possible values: + +- Positive integer. +- 0 — Squashing disabled. +min: ᴺᵁᴸᴸ +max: ᴺᵁᴸᴸ +readonly: 0 +type: UInt64 +default: 1048449 +alias_for: +is_obsolete: 0 +tier: Production + +Row 2: +────── +name: min_insert_block_size_bytes +value: 268402944 +changed: 0 +description: Sets the minimum number of bytes in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. + +Possible values: + +- Positive integer. +- 0 — Squashing disabled. +min: ᴺᵁᴸᴸ +max: ᴺᵁᴸᴸ +readonly: 0 +type: UInt64 +default: 268402944 +alias_for: +is_obsolete: 0 +tier: Production + +Row 3: +────── +name: min_insert_block_size_rows_for_materialized_views +value: 0 +changed: 0 +description: Sets the minimum number of rows in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. This setting is applied only for blocks inserted into [materialized view](../../sql-reference/statements/create/view.md). By adjusting this setting, you control blocks squashing while pushing to materialized view and avoid excessive memory usage. + +Possible values: + +- Any positive integer. +- 0 — Squashing disabled. + +**See Also** + +- [min_insert_block_size_rows](#min-insert-block-size-rows) +min: ᴺᵁᴸᴸ +max: ᴺᵁᴸᴸ +readonly: 0 +type: UInt64 +default: 0 +alias_for: +is_obsolete: 0 +tier: Production + +Row 4: +────── +name: min_insert_block_size_bytes_for_materialized_views +value: 0 +changed: 0 +description: Sets the minimum number of bytes in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. This setting is applied only for blocks inserted into [materialized view](../../sql-reference/statements/create/view.md). By adjusting this setting, you control blocks squashing while pushing to materialized view and avoid excessive memory usage. + +Possible values: + +- Any positive integer. +- 0 — Squashing disabled. + +**See also** + +- [min_insert_block_size_bytes](#min-insert-block-size-bytes) +min: ᴺᵁᴸᴸ +max: ᴺᵁᴸᴸ +readonly: 0 +type: UInt64 +default: 0 +alias_for: +is_obsolete: 0 +tier: Production + ``` Using of `WHERE changed` can be useful, for example, when you want to check: diff --git a/src/Core/BaseSettings.cpp b/src/Core/BaseSettings.cpp index c535b9ce65e..7bfa581598d 100644 --- a/src/Core/BaseSettings.cpp +++ b/src/Core/BaseSettings.cpp @@ -8,6 +8,7 @@ namespace DB { namespace ErrorCodes { + extern const int INCORRECT_DATA; extern const int UNKNOWN_SETTING; } @@ -38,6 +39,14 @@ BaseSettingsHelpers::Flags BaseSettingsHelpers::readFlags(ReadBuffer & in) return static_cast(res); } +SettingsTierType BaseSettingsHelpers::getTier(Flags flags) +{ + int8_t tier = (flags & Flags::TIER); + if (tier > SettingsTierType::OBSOLETE) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown tier value: '{}'", tier); + return SettingsTierType{tier}; +} + void BaseSettingsHelpers::throwSettingNotFound(std::string_view name) { diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 2a2e0bb334e..218460330f4 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -21,6 +22,27 @@ namespace DB class ReadBuffer; class WriteBuffer; +struct BaseSettingsHelpers +{ + [[noreturn]] static void throwSettingNotFound(std::string_view name); + static void warningSettingNotFound(std::string_view name); + + static void writeString(std::string_view str, WriteBuffer & out); + static String readString(ReadBuffer & in); + + enum Flags : UInt64 + { + IMPORTANT = 0x01, + CUSTOM = 0x02, + TIER = 0x0c, /// 0b1100 == 2 bits + /// If adding new flags, consider first if Tier might need more bits + }; + + static SettingsTierType getTier(Flags flags); + static void writeFlags(Flags flags, WriteBuffer & out); + static Flags readFlags(ReadBuffer & in); +}; + /** Template class to define collections of settings. * If you create a new setting, please also add it to ./utils/check-style/check-settings-style * for validation @@ -138,7 +160,7 @@ public: const char * getTypeName() const; const char * getDescription() const; bool isCustom() const; - bool isObsolete() const; + SettingsTierType getTier() const; bool operator==(const SettingFieldRef & other) const { return (getName() == other.getName()) && (getValue() == other.getValue()); } bool operator!=(const SettingFieldRef & other) const { return !(*this == other); } @@ -225,24 +247,6 @@ private: std::conditional_t custom_settings_map; }; -struct BaseSettingsHelpers -{ - [[noreturn]] static void throwSettingNotFound(std::string_view name); - static void warningSettingNotFound(std::string_view name); - - static void writeString(std::string_view str, WriteBuffer & out); - static String readString(ReadBuffer & in); - - enum Flags : UInt64 - { - IMPORTANT = 0x01, - CUSTOM = 0x02, - OBSOLETE = 0x04, - }; - static void writeFlags(Flags flags, WriteBuffer & out); - static Flags readFlags(ReadBuffer & in); -}; - template void BaseSettings::set(std::string_view name, const Field & value) { @@ -797,14 +801,14 @@ bool BaseSettings::SettingFieldRef::isCustom() const } template -bool BaseSettings::SettingFieldRef::isObsolete() const +SettingsTierType BaseSettings::SettingFieldRef::getTier() const { if constexpr (Traits::allow_custom_settings) { if (custom_setting) - return false; + return SettingsTierType::PRODUCTION; } - return accessor->isObsolete(index); + return accessor->getTier(index); } using AliasMap = std::unordered_map; @@ -835,8 +839,8 @@ using AliasMap = std::unordered_map; const String & getName(size_t index) const { return field_infos[index].name; } \ const char * getTypeName(size_t index) const { return field_infos[index].type; } \ const char * getDescription(size_t index) const { return field_infos[index].description; } \ - bool isImportant(size_t index) const { return field_infos[index].is_important; } \ - bool isObsolete(size_t index) const { return field_infos[index].is_obsolete; } \ + bool isImportant(size_t index) const { return field_infos[index].flags & BaseSettingsHelpers::Flags::IMPORTANT; } \ + SettingsTierType getTier(size_t index) const { return BaseSettingsHelpers::getTier(field_infos[index].flags); } \ Field castValueUtil(size_t index, const Field & value) const { return field_infos[index].cast_value_util_function(value); } \ String valueToStringUtil(size_t index, const Field & value) const { return field_infos[index].value_to_string_util_function(value); } \ Field stringToValueUtil(size_t index, const String & str) const { return field_infos[index].string_to_value_util_function(str); } \ @@ -856,8 +860,7 @@ using AliasMap = std::unordered_map; String name; \ const char * type; \ const char * description; \ - bool is_important; \ - bool is_obsolete; \ + BaseSettingsHelpers::Flags flags; \ Field (*cast_value_util_function)(const Field &); \ String (*value_to_string_util_function)(const Field &); \ Field (*string_to_value_util_function)(const String &); \ @@ -968,8 +971,8 @@ struct DefineAliases /// NOLINTNEXTLINE #define IMPLEMENT_SETTINGS_TRAITS_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ res.field_infos.emplace_back( \ - FieldInfo{#NAME, #TYPE, DESCRIPTION, (FLAGS) & IMPORTANT, \ - static_cast((FLAGS) & BaseSettingsHelpers::Flags::OBSOLETE), \ + FieldInfo{#NAME, #TYPE, DESCRIPTION, \ + static_cast(FLAGS), \ [](const Field & value) -> Field { return static_cast(SettingField##TYPE{value}); }, \ [](const Field & value) -> String { return SettingField##TYPE{value}.toString(); }, \ [](const String & str) -> Field { SettingField##TYPE temp; temp.parseFromString(str); return static_cast(temp); }, \ diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 7c2cb49a2ba..326f151b12f 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -337,7 +337,7 @@ void ServerSettings::dumpToSystemServerSettingsColumns(ServerSettingColumnsParam res_columns[4]->insert(setting.getDescription()); res_columns[5]->insert(setting.getTypeName()); res_columns[6]->insert(is_changeable ? changeable_settings_it->second.second : ChangeableWithoutRestart::No); - res_columns[7]->insert(setting.isObsolete()); + res_columns[7]->insert(setting.getTier() == SettingsTierType::OBSOLETE); } } } diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 0aecb7cf941..54cd3ad9a4f 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1,7 +1,5 @@ -#include #include #include -#include #include #include #include @@ -40,10 +38,15 @@ namespace ErrorCodes * Note: as an alternative, we could implement settings to be completely dynamic in the form of the map: String -> Field, * but we are not going to do it, because settings are used everywhere as static struct fields. * - * `flags` can be either 0 or IMPORTANT. + * `flags` can be either 0 or IMPORTANT + a Tier (PRODUCTION | BETA | EXPERIMENTAL) * A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions. + * Tiers: + * EXPERIMENTAL: The feature is in active development stage. Mostly for developers or for ClickHouse enthusiasts. + * BETA: There are no known bugs problems in the functionality, but the outcome of using it together with other + * features/components is unknown and correctness is not guaranteed. + * PRODUCTION (Default): The feature is safe to use along with other features from the PRODUCTION tier. * - * When adding new or changing existing settings add them to the settings changes history in SettingsChangesHistory.h + * When adding new or changing existing settings add them to the settings changes history in SettingsChangesHistory.cpp * for tracking settings changes in different versions and for special `compatibility` settings to work correctly. */ @@ -6007,7 +6010,7 @@ void SettingsImpl::checkNoSettingNamesAtTopLevel(const Poco::Util::AbstractConfi { const auto & name = setting.getName(); bool should_skip_check = name == "max_table_size_to_drop" || name == "max_partition_size_to_drop"; - if (config.has(name) && !setting.isObsolete() && !should_skip_check) + if (config.has(name) && (setting.getTier() != SettingsTierType::OBSOLETE) && !should_skip_check) { throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "A setting '{}' appeared at top level in config {}." " But it is user-level setting that should be located in users.xml inside section for specific profile." @@ -6183,7 +6186,7 @@ std::vector Settings::getChangedAndObsoleteNames() const std::vector setting_names; for (const auto & setting : impl->allChanged()) { - if (setting.isObsolete()) + if (setting.getTier() == SettingsTierType::OBSOLETE) setting_names.emplace_back(setting.getName()); } return setting_names; @@ -6232,7 +6235,8 @@ void Settings::dumpToSystemSettingsColumns(MutableColumnsAndConstraints & params 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()); + res_columns[10]->insert(setting.getTier() == SettingsTierType::OBSOLETE); + res_columns[11]->insert(setting.getTier()); }; const auto & settings_to_aliases = SettingsImpl::Traits::settingsToAliases(); diff --git a/src/Core/SettingsObsoleteMacros.h b/src/Core/SettingsObsoleteMacros.h index 97db1def294..c680cdc45b6 100644 --- a/src/Core/SettingsObsoleteMacros.h +++ b/src/Core/SettingsObsoleteMacros.h @@ -2,8 +2,8 @@ // clang-format off #define MAKE_OBSOLETE(M, TYPE, NAME, DEFAULT) \ - M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", BaseSettingsHelpers::Flags::OBSOLETE) + M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", SettingsTierType::OBSOLETE) /// NOTE: ServerSettings::loadSettingsFromConfig() should be updated to include this settings #define MAKE_DEPRECATED_BY_SERVER_CONFIG(M, TYPE, NAME, DEFAULT) \ - M(TYPE, NAME, DEFAULT, "User-level setting is deprecated, and it must be defined in the server configuration instead.", BaseSettingsHelpers::Flags::OBSOLETE) + M(TYPE, NAME, DEFAULT, "User-level setting is deprecated, and it must be defined in the server configuration instead.", SettingsTierType::OBSOLETE) diff --git a/src/Core/SettingsTierType.cpp b/src/Core/SettingsTierType.cpp new file mode 100644 index 00000000000..48090f26fae --- /dev/null +++ b/src/Core/SettingsTierType.cpp @@ -0,0 +1,19 @@ +#include +#include + +namespace DB +{ + +std::shared_ptr getSettingsTierEnum() +{ + return std::make_shared( + DataTypeEnum8::Values + { + {"Production", static_cast(SettingsTierType::PRODUCTION)}, + {"Obsolete", static_cast(SettingsTierType::OBSOLETE)}, + {"Experimental", static_cast(SettingsTierType::EXPERIMENTAL)}, + {"Beta", static_cast(SettingsTierType::BETA)} + }); +} + +} diff --git a/src/Core/SettingsTierType.h b/src/Core/SettingsTierType.h new file mode 100644 index 00000000000..d8bba89bc18 --- /dev/null +++ b/src/Core/SettingsTierType.h @@ -0,0 +1,26 @@ +#pragma once + +#include + +#include +#include + +namespace DB +{ + +template +class DataTypeEnum; +using DataTypeEnum8 = DataTypeEnum; + +// Make it signed for compatibility with DataTypeEnum8 +enum SettingsTierType : int8_t +{ + PRODUCTION = 0b0000, + OBSOLETE = 0b0100, + EXPERIMENTAL = 0b1000, + BETA = 0b1100 +}; + +std::shared_ptr getSettingsTierEnum(); + +} diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 8c6aafe48f2..b95b3a856de 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -238,7 +238,7 @@ namespace ErrorCodes DECLARE(DeduplicateMergeProjectionMode, deduplicate_merge_projection_mode, DeduplicateMergeProjectionMode::THROW, "Whether to allow create projection for the table with non-classic MergeTree. Ignore option is purely for compatibility which might result in incorrect answer. Otherwise, if allowed, what is the action when merge, drop or rebuild.", 0) \ #define MAKE_OBSOLETE_MERGE_TREE_SETTING(M, TYPE, NAME, DEFAULT) \ - M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", BaseSettingsHelpers::Flags::OBSOLETE) + M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", SettingsTierType::OBSOLETE) #define OBSOLETE_MERGE_TREE_SETTINGS(M, ALIAS) \ /** Obsolete settings that do nothing but left for compatibility reasons. */ \ @@ -648,7 +648,7 @@ void MergeTreeSettings::dumpToSystemMergeTreeSettingsColumns(MutableColumnsAndCo res_columns[5]->insert(max); res_columns[6]->insert(writability == SettingConstraintWritability::CONST); res_columns[7]->insert(setting.getTypeName()); - res_columns[8]->insert(setting.isObsolete()); + res_columns[8]->insert(setting.getTier() == SettingsTierType::OBSOLETE); } } diff --git a/src/Storages/System/StorageSystemSettings.cpp b/src/Storages/System/StorageSystemSettings.cpp index 9309f10378e..debd40386a6 100644 --- a/src/Storages/System/StorageSystemSettings.cpp +++ b/src/Storages/System/StorageSystemSettings.cpp @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include #include @@ -34,6 +36,14 @@ ColumnsDescription StorageSystemSettings::getColumnsDescription() {"default", std::make_shared(), "Setting default value."}, {"alias_for", std::make_shared(), "Flag that shows whether this name is an alias to another setting."}, {"is_obsolete", std::make_shared(), "Shows whether a setting is obsolete."}, + {"tier", getSettingsTierEnum(), R"( +Support level for this feature. ClickHouse features are organized in tiers, varying depending on the current status of their +development and the expectations one might have when using them: +* PRODUCTION: The feature is stable, safe to use and does not have issues interacting with other PRODUCTION features. +* BETA: The feature is stable and safe. The outcome of using it together with other features is unknown and correctness is not guaranteed. Testing and reports are welcome. +* EXPERIMENTAL: The feature is under development. Only intended for developers and ClickHouse enthusiasts. The feature might or might not work and could be removed at any time. +* OBSOLETE: No longer supported. Either it is already removed or it will be removed in future releases. +)"}, }; } From 309f18debef94455e1d50ca08fc9dbe3baa54796 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 17:26:20 +0100 Subject: [PATCH 08/35] Mark some NON-PRODUCTION settings --- src/Core/BaseSettings.cpp | 2 +- src/Core/Settings.cpp | 403 +++++++++--------- src/Storages/MergeTree/MergeTreeSettings.cpp | 19 +- .../System/StorageSystemMergeTreeSettings.cpp | 11 +- 4 files changed, 227 insertions(+), 208 deletions(-) diff --git a/src/Core/BaseSettings.cpp b/src/Core/BaseSettings.cpp index 7bfa581598d..51e99262bdb 100644 --- a/src/Core/BaseSettings.cpp +++ b/src/Core/BaseSettings.cpp @@ -42,7 +42,7 @@ BaseSettingsHelpers::Flags BaseSettingsHelpers::readFlags(ReadBuffer & in) SettingsTierType BaseSettingsHelpers::getTier(Flags flags) { int8_t tier = (flags & Flags::TIER); - if (tier > SettingsTierType::OBSOLETE) + if (tier > SettingsTierType::BETA) throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown tier value: '{}'", tier); return SettingsTierType{tier}; } diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 54cd3ad9a4f..4159758fe76 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5506,90 +5506,102 @@ For testing purposes. Replaces all external table functions to Null to not initi DECLARE(Bool, restore_replace_external_dictionary_source_to_null, false, R"( Replace external dictionary sources to Null on restore. Useful for testing purposes )", 0) \ - DECLARE(Bool, create_if_not_exists, false, R"( -Enable `IF NOT EXISTS` for `CREATE` statement by default. If either this setting or `IF NOT EXISTS` is specified and a table with the provided name already exists, no exception will be thrown. -)", 0) \ - DECLARE(Bool, enforce_strict_identifier_format, false, R"( -If enabled, only allow identifiers containing alphanumeric characters and underscores. -)", 0) \ - DECLARE(Bool, mongodb_throw_on_unsupported_query, true, R"( -If enabled, MongoDB tables will return an error when a MongoDB query cannot be built. Otherwise, ClickHouse reads the full table and processes it locally. This option does not apply to the legacy implementation or when 'allow_experimental_analyzer=0'. -)", 0) \ - \ - /* ###################################### */ \ - /* ######## EXPERIMENTAL FEATURES ####### */ \ - /* ###################################### */ \ - DECLARE(Bool, allow_experimental_materialized_postgresql_table, false, R"( -Allows to use the MaterializedPostgreSQL table engine. Disabled by default, because this feature is experimental -)", 0) \ - DECLARE(Bool, allow_experimental_funnel_functions, false, R"( -Enable experimental functions for funnel analysis. -)", 0) \ - DECLARE(Bool, allow_experimental_nlp_functions, false, R"( -Enable experimental functions for natural language processing. -)", 0) \ - DECLARE(Bool, allow_experimental_hash_functions, false, R"( -Enable experimental hash functions -)", 0) \ - DECLARE(Bool, allow_experimental_object_type, false, R"( -Allow Object and JSON data types -)", 0) \ - DECLARE(Bool, allow_experimental_time_series_table, false, R"( -Allows creation of tables with the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine. + /* Parallel replicas */ \ + DECLARE(UInt64, allow_experimental_parallel_reading_from_replicas, 0, R"( +Use up to `max_parallel_replicas` the number of replicas from each shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure +)", BETA) ALIAS(enable_parallel_replicas) \ + DECLARE(NonZeroUInt64, max_parallel_replicas, 1, R"( +The maximum number of replicas for each shard when executing a query. Possible values: -- 0 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is disabled. -- 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled. -)", 0) \ - DECLARE(Bool, allow_experimental_vector_similarity_index, false, R"( -Allow experimental vector similarity index -)", 0) \ - DECLARE(Bool, allow_experimental_variant_type, false, R"( -Allows creation of experimental [Variant](../../sql-reference/data-types/variant.md). -)", 0) \ - DECLARE(Bool, allow_experimental_dynamic_type, false, R"( -Allow Dynamic data type -)", 0) \ - DECLARE(Bool, allow_experimental_json_type, false, R"( -Allow JSON data type -)", 0) \ - DECLARE(Bool, allow_experimental_codecs, false, R"( -If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing). -)", 0) \ - DECLARE(Bool, allow_experimental_shared_set_join, true, R"( -Only in ClickHouse Cloud. Allow to create ShareSet and SharedJoin -)", 0) \ - DECLARE(UInt64, max_limit_for_ann_queries, 1'000'000, R"( -SELECT queries with LIMIT bigger than this setting cannot use vector similarity indexes. Helps to prevent memory overflows in vector similarity indexes. -)", 0) \ - DECLARE(UInt64, hnsw_candidate_list_size_for_search, 256, R"( -The size of the dynamic candidate list when searching the vector similarity index, also known as 'ef_search'. -)", 0) \ - DECLARE(Bool, throw_on_unsupported_query_inside_transaction, true, R"( -Throw exception if unsupported query is used inside transaction -)", 0) \ - DECLARE(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, R"( -Wait for committed changes to become actually visible in the latest snapshot -)", 0) \ - DECLARE(Bool, implicit_transaction, false, R"( -If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback) -)", 0) \ - DECLARE(UInt64, grace_hash_join_initial_buckets, 1, R"( -Initial number of grace hash join buckets -)", 0) \ - DECLARE(UInt64, grace_hash_join_max_buckets, 1024, R"( -Limit on the number of grace hash join buckets -)", 0) \ - DECLARE(UInt64, join_to_sort_minimum_perkey_rows, 40, R"( -The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys -)", 0) \ - DECLARE(UInt64, join_to_sort_maximum_table_rows, 10000, R"( -The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join. -)", 0) \ - DECLARE(Bool, allow_experimental_join_right_table_sorting, false, R"( -If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join. +- Positive integer. + +**Additional Info** + +This options will produce different results depending on the settings used. + +:::note +This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. +::: + +### Parallel processing using `SAMPLE` key + +A query may be processed faster if it is executed on several servers in parallel. But the query performance may degrade in the following cases: + +- The position of the sampling key in the partitioning key does not allow efficient range scans. +- Adding a sampling key to the table makes filtering by other columns less efficient. +- The sampling key is an expression that is expensive to calculate. +- The cluster latency distribution has a long tail, so that querying more servers increases the query overall latency. + +### Parallel processing using [parallel_replicas_custom_key](#parallel_replicas_custom_key) + +This setting is useful for any replicated table. )", 0) \ + DECLARE(ParallelReplicasMode, parallel_replicas_mode, ParallelReplicasMode::READ_TASKS, R"( +Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key. +)", BETA) \ + DECLARE(UInt64, parallel_replicas_count, 0, R"( +This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing. +)", BETA) \ + DECLARE(UInt64, parallel_replica_offset, 0, R"( +This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas. +)", BETA) \ + DECLARE(String, parallel_replicas_custom_key, "", R"( +An arbitrary integer expression that can be used to split work between replicas for a specific table. +The value can be any integer expression. + +Simple expressions using primary keys are preferred. + +If the setting is used on a cluster that consists of a single shard with multiple replicas, those replicas will be converted into virtual shards. +Otherwise, it will behave same as for `SAMPLE` key, it will use multiple replicas of each shard. +)", BETA) \ + DECLARE(UInt64, parallel_replicas_custom_key_range_lower, 0, R"( +Allows the filter type `range` to split the work evenly between replicas based on the custom range `[parallel_replicas_custom_key_range_lower, INT_MAX]`. + +When used in conjunction with [parallel_replicas_custom_key_range_upper](#parallel_replicas_custom_key_range_upper), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. + +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. +)", BETA) \ + DECLARE(UInt64, parallel_replicas_custom_key_range_upper, 0, R"( +Allows the filter type `range` to split the work evenly between replicas based on the custom range `[0, parallel_replicas_custom_key_range_upper]`. A value of 0 disables the upper bound, setting it the max value of the custom key expression. + +When used in conjunction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. + +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing +)", BETA) \ + DECLARE(String, cluster_for_parallel_replicas, "", R"( +Cluster for a shard in which current server is located +)", BETA) \ + DECLARE(Bool, parallel_replicas_allow_in_with_subquery, true, R"( +If true, subquery for IN will be executed on every follower replica. +)", BETA) \ + DECLARE(Float, parallel_replicas_single_task_marks_count_multiplier, 2, R"( +A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas. +)", BETA) \ + DECLARE(Bool, parallel_replicas_for_non_replicated_merge_tree, false, R"( +If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables +)", BETA) \ + DECLARE(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, R"( +Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas' +)", BETA) \ + DECLARE(Bool, parallel_replicas_prefer_local_join, true, R"( +If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN. +)", BETA) \ + DECLARE(UInt64, parallel_replicas_mark_segment_size, 0, R"( +Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing. Value should be in range [128; 16384] +)", BETA) \ + DECLARE(Bool, parallel_replicas_local_plan, false, R"( +Build local plan for local replica +)", BETA) \ + \ + DECLARE(Bool, allow_experimental_analyzer, true, R"( +Allow new query analyzer. +)", IMPORTANT | BETA) ALIAS(enable_analyzer) \ + DECLARE(Bool, analyzer_compatibility_join_using_top_level_identifier, false, R"( +Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`). +)", BETA) \ + \ DECLARE(Timezone, session_timezone, "", R"( Sets the implicit time zone of the current session or query. The implicit time zone is the time zone applied to values of type DateTime/DateTime64 which have no explicitly specified time zone. @@ -5649,126 +5661,121 @@ This happens due to different parsing pipelines: **See also** - [timezone](../server-configuration-parameters/settings.md#timezone) +)", BETA) \ +DECLARE(Bool, create_if_not_exists, false, R"( +Enable `IF NOT EXISTS` for `CREATE` statement by default. If either this setting or `IF NOT EXISTS` is specified and a table with the provided name already exists, no exception will be thrown. +)", 0) \ + DECLARE(Bool, enforce_strict_identifier_format, false, R"( +If enabled, only allow identifiers containing alphanumeric characters and underscores. +)", 0) \ + DECLARE(Bool, mongodb_throw_on_unsupported_query, true, R"( +If enabled, MongoDB tables will return an error when a MongoDB query cannot be built. Otherwise, ClickHouse reads the full table and processes it locally. This option does not apply to the legacy implementation or when 'allow_experimental_analyzer=0'. +)", 0) \ + DECLARE(Bool, implicit_select, false, R"( +Allow writing simple SELECT queries without the leading SELECT keyword, which makes it simple for calculator-style usage, e.g. `1 + 2` becomes a valid query. )", 0) \ - DECLARE(Bool, use_hive_partitioning, false, R"( -When enabled, ClickHouse will detect Hive-style partitioning in path (`/name=value/`) in file-like table engines [File](../../engines/table-engines/special/file.md#hive-style-partitioning)/[S3](../../engines/table-engines/integrations/s3.md#hive-style-partitioning)/[URL](../../engines/table-engines/special/url.md#hive-style-partitioning)/[HDFS](../../engines/table-engines/integrations/hdfs.md#hive-style-partitioning)/[AzureBlobStorage](../../engines/table-engines/integrations/azureBlobStorage.md#hive-style-partitioning) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. -)", 0)\ \ - DECLARE(Bool, allow_statistics_optimize, false, R"( -Allows using statistics to optimize queries -)", 0) ALIAS(allow_statistic_optimize) \ - DECLARE(Bool, allow_experimental_statistics, false, R"( -Allows defining columns with [statistics](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) and [manipulate statistics](../../engines/table-engines/mergetree-family/mergetree.md#column-statistics). -)", 0) ALIAS(allow_experimental_statistic) \ \ - /* Parallel replicas */ \ - DECLARE(UInt64, allow_experimental_parallel_reading_from_replicas, 0, R"( -Use up to `max_parallel_replicas` the number of replicas from each shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure -)", 0) ALIAS(enable_parallel_replicas) \ - DECLARE(NonZeroUInt64, max_parallel_replicas, 1, R"( -The maximum number of replicas for each shard when executing a query. + /* ####################################################### */ \ + /* ########### START OF EXPERIMENTAL FEATURES ############ */ \ + /* ## ADD PRODUCTION / BETA FEATURES BEFORE THIS BLOCK ## */ \ + /* ####################################################### */ \ + \ + DECLARE(Bool, allow_experimental_materialized_postgresql_table, false, R"( +Allows to use the MaterializedPostgreSQL table engine. Disabled by default, because this feature is experimental +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_funnel_functions, false, R"( +Enable experimental functions for funnel analysis. +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_nlp_functions, false, R"( +Enable experimental functions for natural language processing. +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_hash_functions, false, R"( +Enable experimental hash functions +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_object_type, false, R"( +Allow Object and JSON data types +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_time_series_table, false, R"( +Allows creation of tables with the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine. Possible values: -- Positive integer. - -**Additional Info** - -This options will produce different results depending on the settings used. - -:::note -This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. -::: - -### Parallel processing using `SAMPLE` key - -A query may be processed faster if it is executed on several servers in parallel. But the query performance may degrade in the following cases: - -- The position of the sampling key in the partitioning key does not allow efficient range scans. -- Adding a sampling key to the table makes filtering by other columns less efficient. -- The sampling key is an expression that is expensive to calculate. -- The cluster latency distribution has a long tail, so that querying more servers increases the query overall latency. - -### Parallel processing using [parallel_replicas_custom_key](#parallel_replicas_custom_key) - -This setting is useful for any replicated table. -)", 0) \ - DECLARE(ParallelReplicasMode, parallel_replicas_mode, ParallelReplicasMode::READ_TASKS, R"( -Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key. -)", 0) \ - DECLARE(UInt64, parallel_replicas_count, 0, R"( -This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing. -)", 0) \ - DECLARE(UInt64, parallel_replica_offset, 0, R"( -This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas. -)", 0) \ - DECLARE(String, parallel_replicas_custom_key, "", R"( -An arbitrary integer expression that can be used to split work between replicas for a specific table. -The value can be any integer expression. - -Simple expressions using primary keys are preferred. - -If the setting is used on a cluster that consists of a single shard with multiple replicas, those replicas will be converted into virtual shards. -Otherwise, it will behave same as for `SAMPLE` key, it will use multiple replicas of each shard. -)", 0) \ - DECLARE(UInt64, parallel_replicas_custom_key_range_lower, 0, R"( -Allows the filter type `range` to split the work evenly between replicas based on the custom range `[parallel_replicas_custom_key_range_lower, INT_MAX]`. - -When used in conjunction with [parallel_replicas_custom_key_range_upper](#parallel_replicas_custom_key_range_upper), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. - -Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. -)", 0) \ - DECLARE(UInt64, parallel_replicas_custom_key_range_upper, 0, R"( -Allows the filter type `range` to split the work evenly between replicas based on the custom range `[0, parallel_replicas_custom_key_range_upper]`. A value of 0 disables the upper bound, setting it the max value of the custom key expression. - -When used in conjunction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. - -Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing -)", 0) \ - DECLARE(String, cluster_for_parallel_replicas, "", R"( -Cluster for a shard in which current server is located -)", 0) \ - DECLARE(Bool, parallel_replicas_allow_in_with_subquery, true, R"( -If true, subquery for IN will be executed on every follower replica. -)", 0) \ - DECLARE(Float, parallel_replicas_single_task_marks_count_multiplier, 2, R"( -A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas. -)", 0) \ - DECLARE(Bool, parallel_replicas_for_non_replicated_merge_tree, false, R"( -If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables -)", 0) \ - DECLARE(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, R"( -Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas' -)", 0) \ - DECLARE(Bool, parallel_replicas_prefer_local_join, true, R"( -If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN. -)", 0) \ - DECLARE(UInt64, parallel_replicas_mark_segment_size, 0, R"( -Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing. Value should be in range [128; 16384] +- 0 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is disabled. +- 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled. )", 0) \ + DECLARE(Bool, allow_experimental_vector_similarity_index, false, R"( +Allow experimental vector similarity index +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_variant_type, false, R"( +Allows creation of experimental [Variant](../../sql-reference/data-types/variant.md). +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_dynamic_type, false, R"( +Allow Dynamic data type +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_json_type, false, R"( +Allow JSON data type +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_codecs, false, R"( +If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing). +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_shared_set_join, true, R"( +Only in ClickHouse Cloud. Allow to create ShareSet and SharedJoin +)", EXPERIMENTAL) \ + DECLARE(UInt64, max_limit_for_ann_queries, 1'000'000, R"( +SELECT queries with LIMIT bigger than this setting cannot use vector similarity indexes. Helps to prevent memory overflows in vector similarity indexes. +)", EXPERIMENTAL) \ + DECLARE(UInt64, hnsw_candidate_list_size_for_search, 256, R"( +The size of the dynamic candidate list when searching the vector similarity index, also known as 'ef_search'. +)", EXPERIMENTAL) \ + DECLARE(Bool, throw_on_unsupported_query_inside_transaction, true, R"( +Throw exception if unsupported query is used inside transaction +)", EXPERIMENTAL) \ + DECLARE(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, R"( +Wait for committed changes to become actually visible in the latest snapshot +)", EXPERIMENTAL) \ + DECLARE(Bool, implicit_transaction, false, R"( +If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback) +)", EXPERIMENTAL) \ + DECLARE(UInt64, grace_hash_join_initial_buckets, 1, R"( +Initial number of grace hash join buckets +)", EXPERIMENTAL) \ + DECLARE(UInt64, grace_hash_join_max_buckets, 1024, R"( +Limit on the number of grace hash join buckets +)", EXPERIMENTAL) \ + DECLARE(UInt64, join_to_sort_minimum_perkey_rows, 40, R"( +The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys +)", EXPERIMENTAL) \ + DECLARE(UInt64, join_to_sort_maximum_table_rows, 10000, R"( +The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join. +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_join_right_table_sorting, false, R"( +If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join. +)", EXPERIMENTAL) \ + DECLARE(Bool, use_hive_partitioning, false, R"( +When enabled, ClickHouse will detect Hive-style partitioning in path (`/name=value/`) in file-like table engines [File](../../engines/table-engines/special/file.md#hive-style-partitioning)/[S3](../../engines/table-engines/integrations/s3.md#hive-style-partitioning)/[URL](../../engines/table-engines/special/url.md#hive-style-partitioning)/[HDFS](../../engines/table-engines/integrations/hdfs.md#hive-style-partitioning)/[AzureBlobStorage](../../engines/table-engines/integrations/azureBlobStorage.md#hive-style-partitioning) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. +)", EXPERIMENTAL)\ + \ + DECLARE(Bool, allow_statistics_optimize, false, R"( +Allows using statistics to optimize queries +)", EXPERIMENTAL) ALIAS(allow_statistic_optimize) \ + DECLARE(Bool, allow_experimental_statistics, false, R"( +Allows defining columns with [statistics](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) and [manipulate statistics](../../engines/table-engines/mergetree-family/mergetree.md#column-statistics). +)", EXPERIMENTAL) ALIAS(allow_experimental_statistic) \ + \ DECLARE(Bool, allow_archive_path_syntax, true, R"( File/S3 engines/table function will parse paths with '::' as '\\ :: \\' if archive has correct extension -)", 0) \ - DECLARE(Bool, parallel_replicas_local_plan, false, R"( -Build local plan for local replica -)", 0) \ +)", EXPERIMENTAL) \ \ DECLARE(Bool, allow_experimental_inverted_index, false, R"( If it is set to true, allow to use experimental inverted index. -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_full_text_index, false, R"( If it is set to true, allow to use experimental full-text index. -)", 0) \ +)", EXPERIMENTAL) \ \ DECLARE(Bool, allow_experimental_join_condition, false, R"( Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y. -)", 0) \ - \ - DECLARE(Bool, allow_experimental_analyzer, true, R"( -Allow new query analyzer. -)", IMPORTANT) ALIAS(enable_analyzer) \ - DECLARE(Bool, analyzer_compatibility_join_using_top_level_identifier, false, R"( -Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`). )", 0) \ \ DECLARE(Bool, allow_experimental_live_view, false, R"( @@ -5781,43 +5788,45 @@ Possible values: )", 0) \ DECLARE(Seconds, live_view_heartbeat_interval, 15, R"( The heartbeat interval in seconds to indicate live query is alive. -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(UInt64, max_live_view_insert_blocks_before_refresh, 64, R"( Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed. -)", 0) \ +)", EXPERIMENTAL) \ \ DECLARE(Bool, allow_experimental_window_view, false, R"( Enable WINDOW VIEW. Not mature enough. -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Seconds, window_view_clean_interval, 60, R"( The clean interval of window view in seconds to free outdated data. -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Seconds, window_view_heartbeat_interval, 15, R"( The heartbeat interval in seconds to indicate watch query is alive. -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Seconds, wait_for_window_view_fire_signal_timeout, 10, R"( Timeout for waiting for window view fire signal in event time processing -)", 0) \ +)", EXPERIMENTAL) \ \ DECLARE(Bool, stop_refreshable_materialized_views_on_startup, false, R"( On server startup, prevent scheduling of refreshable materialized views, as if with SYSTEM STOP VIEWS. You can manually start them with SYSTEM START VIEWS or SYSTEM START VIEW \\ afterwards. Also applies to newly created views. Has no effect on non-refreshable materialized views. -)", 0) \ +)", EXPERIMENTAL) \ \ DECLARE(Bool, allow_experimental_database_materialized_mysql, false, R"( Allow to create database with Engine=MaterializedMySQL(...). -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_database_materialized_postgresql, false, R"( Allow to create database with Engine=MaterializedPostgreSQL(...). -)", 0) \ +)", EXPERIMENTAL) \ \ /** Experimental feature for moving data between shards. */ \ DECLARE(Bool, allow_experimental_query_deduplication, false, R"( Experimental data deduplication for SELECT queries based on part UUIDs -)", 0) \ - DECLARE(Bool, implicit_select, false, R"( -Allow writing simple SELECT queries without the leading SELECT keyword, which makes it simple for calculator-style usage, e.g. `1 + 2` becomes a valid query. -)", 0) - +)", EXPERIMENTAL) \ + \ + /* ####################################################### */ \ + /* ############ END OF EXPERIMENTAL FEATURES ############# */ \ + /* ## ADD PRODUCTION / BETA FEATURES BEFORE THIS BLOCK ## */ \ + /* ####################################################### */ \ + /* ####################################################### */ \ // End of COMMON_SETTINGS // Please add settings related to formats in Core/FormatFactorySettings.h, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index b95b3a856de..36e146f4624 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -88,7 +88,7 @@ namespace ErrorCodes DECLARE(UInt64, min_age_to_force_merge_seconds, 0, "If all parts in a certain range are older than this value, range will be always eligible for merging. Set to 0 to disable.", 0) \ DECLARE(Bool, min_age_to_force_merge_on_partition_only, false, "Whether min_age_to_force_merge_seconds should be applied only on the entire partition and not on subset.", false) \ DECLARE(UInt64, number_of_free_entries_in_pool_to_execute_optimize_entire_partition, 25, "When there is less than specified number of free entries in pool, do not try to execute optimize entire partition with a merge (this merge is created when set min_age_to_force_merge_seconds > 0 and min_age_to_force_merge_on_partition_only = true). This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ - DECLARE(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \ + DECLARE(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", EXPERIMENTAL) \ DECLARE(UInt64, replicated_max_mutations_in_one_entry, 10000, "Max number of mutation commands that can be merged together and executed in one MUTATE_PART entry (0 means unlimited)", 0) \ DECLARE(UInt64, number_of_mutations_to_delay, 500, "If table has at least that many unfinished mutations, artificially slow down mutations of table. Disabled if set to 0", 0) \ DECLARE(UInt64, number_of_mutations_to_throw, 1000, "If table has at least that many unfinished mutations, throw 'Too many mutations' exception. Disabled if set to 0", 0) \ @@ -214,14 +214,14 @@ namespace ErrorCodes DECLARE(Bool, enable_block_offset_column, false, "Enable persisting column _block_offset for each row.", 0) \ \ /** Experimental/work in progress feature. Unsafe for production. */ \ - DECLARE(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \ - DECLARE(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ - DECLARE(Bool, allow_remote_fs_zero_copy_replication, false, "Don't use this setting in production, because it is not ready.", 0) \ - DECLARE(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for zero-copy table-independent info.", 0) \ - DECLARE(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \ - DECLARE(Bool, cache_populated_by_fetch, false, "Only available in ClickHouse Cloud", 0) \ - DECLARE(Bool, force_read_through_cache_for_merges, false, "Force read-through filesystem cache for merges", 0) \ - DECLARE(Bool, allow_experimental_replacing_merge_with_cleanup, false, "Allow experimental CLEANUP merges for ReplacingMergeTree with is_deleted column.", 0) \ + DECLARE(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", EXPERIMENTAL) \ + DECLARE(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", EXPERIMENTAL) \ + DECLARE(Bool, allow_remote_fs_zero_copy_replication, false, "Don't use this setting in production, because it is not ready.", BETA) \ + DECLARE(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for zero-copy table-independent info.", EXPERIMENTAL) \ + DECLARE(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", EXPERIMENTAL) \ + DECLARE(Bool, cache_populated_by_fetch, false, "Only available in ClickHouse Cloud", EXPERIMENTAL) \ + DECLARE(Bool, force_read_through_cache_for_merges, false, "Force read-through filesystem cache for merges", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_replacing_merge_with_cleanup, false, "Allow experimental CLEANUP merges for ReplacingMergeTree with is_deleted column.", EXPERIMENTAL) \ \ /** Compress marks and primary key. */ \ DECLARE(Bool, compress_marks, true, "Marks support compression, reduce mark file size and speed up network transmission.", 0) \ @@ -649,6 +649,7 @@ void MergeTreeSettings::dumpToSystemMergeTreeSettingsColumns(MutableColumnsAndCo res_columns[6]->insert(writability == SettingConstraintWritability::CONST); res_columns[7]->insert(setting.getTypeName()); res_columns[8]->insert(setting.getTier() == SettingsTierType::OBSOLETE); + res_columns[9]->insert(setting.getTier()); } } diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.cpp b/src/Storages/System/StorageSystemMergeTreeSettings.cpp index 35d975216f6..1da4835dba5 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.cpp +++ b/src/Storages/System/StorageSystemMergeTreeSettings.cpp @@ -1,4 +1,5 @@ -#include +#include +#include #include #include #include @@ -30,6 +31,14 @@ ColumnsDescription SystemMergeTreeSettings::getColumnsDescription() }, {"type", std::make_shared(), "Setting type (implementation specific string value)."}, {"is_obsolete", std::make_shared(), "Shows whether a setting is obsolete."}, + {"tier", getSettingsTierEnum(), R"( +Support level for this feature. ClickHouse features are organized in tiers, varying depending on the current status of their +development and the expectations one might have when using them: +* PRODUCTION: The feature is stable, safe to use and does not have issues interacting with other PRODUCTION features. +* BETA: The feature is stable and safe. The outcome of using it together with other features is unknown and correctness is not guaranteed. Testing and reports are welcome. +* EXPERIMENTAL: The feature is under development. Only intended for developers and ClickHouse enthusiasts. The feature might or might not work and could be removed at any time. +* OBSOLETE: No longer supported. Either it is already removed or it will be removed in future releases. +)"}, }; } From 08d070d982ababa39f726480efc4ba76d85f365e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 17:46:11 +0100 Subject: [PATCH 09/35] Add basic test for setting tiers --- .../queries/0_stateless/03257_setting_tiers.reference | 10 ++++++++++ tests/queries/0_stateless/03257_setting_tiers.sql | 11 +++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/03257_setting_tiers.reference create mode 100644 tests/queries/0_stateless/03257_setting_tiers.sql diff --git a/tests/queries/0_stateless/03257_setting_tiers.reference b/tests/queries/0_stateless/03257_setting_tiers.reference new file mode 100644 index 00000000000..d3d171221e8 --- /dev/null +++ b/tests/queries/0_stateless/03257_setting_tiers.reference @@ -0,0 +1,10 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/03257_setting_tiers.sql b/tests/queries/0_stateless/03257_setting_tiers.sql new file mode 100644 index 00000000000..c7ffe87a80b --- /dev/null +++ b/tests/queries/0_stateless/03257_setting_tiers.sql @@ -0,0 +1,11 @@ +SELECT count() > 0 FROM system.settings WHERE tier = 'Production'; +SELECT count() > 0 FROM system.settings WHERE tier = 'Beta'; +SELECT count() > 0 FROM system.settings WHERE tier = 'Experimental'; +SELECT count() > 0 FROM system.settings WHERE tier = 'Obsolete'; +SELECT count() == countIf(tier IN ['Production', 'Beta', 'Experimental', 'Obsolete']) FROM system.settings; + +SELECT count() > 0 FROM system.merge_tree_settings WHERE tier = 'Production'; +SELECT count() > 0 FROM system.merge_tree_settings WHERE tier = 'Beta'; +SELECT count() > 0 FROM system.merge_tree_settings WHERE tier = 'Experimental'; +SELECT count() > 0 FROM system.merge_tree_settings WHERE tier = 'Obsolete'; +SELECT count() == countIf(tier IN ['Production', 'Beta', 'Experimental', 'Obsolete']) FROM system.merge_tree_settings; From f89887de6a6d5ffa7a5e8eec20a4a2358fed4410 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 18:18:09 +0100 Subject: [PATCH 10/35] Adjust existing tests --- .../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 32a0ed11b6c..821d2e386a9 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 the network, in seconds. If a client needs to send some data but is not able to send any bytes in this interval, the exception is thrown. If you set this setting on the client, the \'receive_timeout\' for the socket will also be 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 +send_timeout 300 0 Timeout for sending data to the network, in seconds. If a client needs to send some data but is not able to send any bytes in this interval, the exception is thrown. If you set this setting on the client, the \'receive_timeout\' for the socket will also be set on the corresponding connection end on the server. \N \N 0 Seconds 300 0 Production +storage_policy default 0 Name of storage disk policy \N \N 0 String 0 Production 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 b260e2dce6c..2ea62444cff 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -342,7 +342,8 @@ CREATE TABLE system.merge_tree_settings `max` Nullable(String), `readonly` UInt8, `type` String, - `is_obsolete` UInt8 + `is_obsolete` UInt8, + `tier` Enum8('Production' = 0, 'Obsolete' = 4, 'Experimental' = 8, 'Beta' = 12) ) ENGINE = SystemMergeTreeSettings COMMENT 'Contains a list of all MergeTree engine specific settings, their current and default values along with descriptions. You may change any of them in SETTINGS section in CREATE query.' @@ -932,7 +933,8 @@ CREATE TABLE system.replicated_merge_tree_settings `max` Nullable(String), `readonly` UInt8, `type` String, - `is_obsolete` UInt8 + `is_obsolete` UInt8, + `tier` Enum8('Production' = 0, 'Obsolete' = 4, 'Experimental' = 8, 'Beta' = 12) ) ENGINE = SystemReplicatedMergeTreeSettings COMMENT 'Contains a list of all ReplicatedMergeTree engine specific settings, their current and default values along with descriptions. You may change any of them in SETTINGS section in CREATE query. ' @@ -1009,7 +1011,8 @@ CREATE TABLE system.settings `type` String, `default` String, `alias_for` String, - `is_obsolete` UInt8 + `is_obsolete` UInt8, + `tier` Enum8('Production' = 0, 'Obsolete' = 4, 'Experimental' = 8, 'Beta' = 12) ) ENGINE = SystemSettings COMMENT 'Contains a list of all user-level settings (which can be modified in a scope of query or session), their current and default values along with descriptions.' From 49655e71f5dc6ca87a41ef30de6bd8b2b53be354 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 18:20:43 +0100 Subject: [PATCH 11/35] Update docs --- docs/en/operations/system-tables/merge_tree_settings.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/operations/system-tables/merge_tree_settings.md b/docs/en/operations/system-tables/merge_tree_settings.md index 48217d63f9d..473315d3941 100644 --- a/docs/en/operations/system-tables/merge_tree_settings.md +++ b/docs/en/operations/system-tables/merge_tree_settings.md @@ -18,6 +18,11 @@ Columns: - `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. +- `tier` ([Enum8](../../sql-reference/data-types/enum.md)) — Support level for this feature. ClickHouse features are organized in tiers, varying depending on the current status of their development and the expectations one might have when using them. Values: + - `'Production'` — The feature is stable, safe to use and does not have issues interacting with other **production** features. . + - `'Beta'` — The feature is stable and safe. The outcome of using it together with other features is unknown and correctness is not guaranteed. Testing and reports are welcome. + - `'Experimental'` — The feature is under development. Only intended for developers and ClickHouse enthusiasts. The feature might or might not work and could be removed at any time. + - `'Obsolete'` — No longer supported. Either it is already removed or it will be removed in future releases. **Example** ```sql From 5f140ea0a82601b83cc4aa253f9f794d3402fd00 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 28 Oct 2024 18:06:16 -0300 Subject: [PATCH 12/35] progress --- .../Impl/Parquet/ParquetDataValuesReader.cpp | 7 --- .../Impl/Parquet/ParquetLeafColReader.cpp | 57 +++++++++---------- 2 files changed, 27 insertions(+), 37 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index 977f2ad298b..9a79bcffad3 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -549,13 +549,6 @@ void ParquetRleDictReader::readBatch( ); } -template <> -void ParquetRleDictReader::readBatch( - MutableColumnPtr & , LazyNullMap &, UInt32) -{ - assert(false); -} - template void ParquetRleDictReader::readBatch( MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp index f32d7e61062..c3c7db510ed 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp @@ -425,16 +425,29 @@ void ParquetLeafColReader::initDataReader( degradeDictionary(); } - ParquetDataBuffer parquet_buffer = [&]() + if (col_descriptor.physical_type() == parquet::Type::BOOLEAN) { - if constexpr (!std::is_same_v, TColumn>) - return ParquetDataBuffer(buffer, max_size); + if constexpr (std::is_same_v) + { + auto bit_reader = std::make_unique(buffer, max_size); + data_values_reader = std::make_unique>(col_descriptor.max_definition_level(), + std::move(def_level_reader), + std::move(bit_reader)); + } + } + else + { + ParquetDataBuffer parquet_buffer = [&]() + { + if constexpr (!std::is_same_v, TColumn>) + return ParquetDataBuffer(buffer, max_size); - auto scale = assert_cast(*base_data_type).getScale(); - return ParquetDataBuffer(buffer, max_size, scale); - }(); - data_values_reader = createPlainReader( - col_descriptor, std::move(def_level_reader), std::move(parquet_buffer)); + auto scale = assert_cast(*base_data_type).getScale(); + return ParquetDataBuffer(buffer, max_size, scale); + }(); + data_values_reader = createPlainReader( + col_descriptor, std::move(def_level_reader), std::move(parquet_buffer)); + } break; } case parquet::Encoding::RLE_DICTIONARY: @@ -463,28 +476,6 @@ void ParquetLeafColReader::initDataReader( } } -template <> -void ParquetLeafColReader::initDataReader( - parquet::Encoding::type enconding_type, - const uint8_t * buffer, - std::size_t max_size, - std::unique_ptr && def_level_reader) -{ - switch (enconding_type) - { - case parquet::Encoding::PLAIN: - { - auto bit_reader = std::make_unique(buffer, max_size); - data_values_reader = std::make_unique>(col_descriptor.max_definition_level(), - std::move(def_level_reader), - std::move(bit_reader)); - break; - } - default: - throw Exception(ErrorCodes::PARQUET_EXCEPTION, "Unknown encoding type: {}", enconding_type); - } -} - template void ParquetLeafColReader::readPageV1(const parquet::DataPageV1 & page) { @@ -634,6 +625,12 @@ std::unique_ptr ParquetLeafColReader::createDi }); return res; } + + if (col_descriptor.physical_type() == parquet::Type::type::BOOLEAN) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Dictionary encoding for booleans is not supported"); + } + return std::make_unique>( col_descriptor.max_definition_level(), std::move(def_level_reader), From 4839c1d9cebd8e0a8f3221b250f4c90ae2910196 Mon Sep 17 00:00:00 2001 From: xmy Date: Tue, 29 Oct 2024 18:42:35 +0800 Subject: [PATCH 13/35] Support write hdfs files with space --- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 14 ++++++------- .../HDFS/WriteBufferFromHDFS.cpp | 21 ++++++++++--------- .../ObjectStorage/HDFS/WriteBufferFromHDFS.h | 3 ++- tests/integration/test_storage_hdfs/test.py | 15 +++++++++++++ 4 files changed, 35 insertions(+), 18 deletions(-) diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 182534529ea..7698193ee2f 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -103,15 +103,15 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects"); - std::string path = object.remote_path; - if (path.starts_with("/")) - path = path.substr(1); - if (!path.starts_with(url)) - path = fs::path(url) / path; - + auto path = extractObjectKeyFromURL(object); /// Single O_WRONLY in libhdfs adds O_TRUNC return std::make_unique( - path, config, settings->replication, patchSettings(write_settings), buf_size, + url_without_path, + fs::path(data_directory) / path, + config, + settings->replication, + patchSettings(write_settings), + buf_size, mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND); } diff --git a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp index 4f6f8c782f2..4879dc41d53 100644 --- a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp @@ -29,6 +29,7 @@ extern const int CANNOT_FSYNC; struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl { std::string hdfs_uri; + std::string hdfs_file_path; hdfsFile fout; HDFSBuilderWrapper builder; HDFSFSPtr fs; @@ -36,25 +37,24 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl WriteBufferFromHDFSImpl( const std::string & hdfs_uri_, + const std::string & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, int replication_, const WriteSettings & write_settings_, int flags) : hdfs_uri(hdfs_uri_) + , hdfs_file_path(hdfs_file_path_) , builder(createHDFSBuilder(hdfs_uri, config_)) , fs(createHDFSFS(builder.get())) , write_settings(write_settings_) { - const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2); - const String path = hdfs_uri.substr(begin_of_path); - /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here - fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, replication_, 0); + fout = hdfsOpenFile(fs.get(), hdfs_file_path.c_str(), flags, 0, replication_, 0); if (fout == nullptr) { throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Unable to open HDFS file: {} ({}) error: {}", - path, hdfs_uri, std::string(hdfsGetLastError())); + hdfs_file_path, hdfs_uri, std::string(hdfsGetLastError())); } } @@ -71,7 +71,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl rlock.unlock(std::max(0, bytes_written)); if (bytes_written < 0) - throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to write HDFS file: {} {}", hdfs_uri, std::string(hdfsGetLastError())); + throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to write HDFS file: {}, hdfs_uri: {}, {}", hdfs_file_path, hdfs_uri, std::string(hdfsGetLastError())); if (write_settings.remote_throttler) write_settings.remote_throttler->add(bytes_written, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); @@ -83,20 +83,21 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl { int result = hdfsSync(fs.get(), fout); if (result < 0) - throw ErrnoException(ErrorCodes::CANNOT_FSYNC, "Cannot HDFS sync {} {}", hdfs_uri, std::string(hdfsGetLastError())); + throw ErrnoException(ErrorCodes::CANNOT_FSYNC, "Cannot HDFS sync {}, hdfs_url: {}, {}", hdfs_file_path, hdfs_uri, std::string(hdfsGetLastError())); } }; WriteBufferFromHDFS::WriteBufferFromHDFS( - const std::string & hdfs_name_, + const std::string & hdfs_uri_, + const std::string & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, int replication_, const WriteSettings & write_settings_, size_t buf_size_, int flags_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) - , impl(std::make_unique(hdfs_name_, config_, replication_, write_settings_, flags_)) - , filename(hdfs_name_) + , impl(std::make_unique(hdfs_uri_, hdfs_file_path_, config_, replication_, write_settings_, flags_)) + , filename(hdfs_file_path_) { } diff --git a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h index e3f0ae96a8f..8166da92e16 100644 --- a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h @@ -22,7 +22,8 @@ class WriteBufferFromHDFS final : public WriteBufferFromFileBase public: WriteBufferFromHDFS( - const String & hdfs_name_, + const String & hdfs_uri_, + const String & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, int replication_, const WriteSettings & write_settings_ = {}, diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 362ea7d5bda..366bc28d2c9 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -396,6 +396,21 @@ def test_read_files_with_spaces(started_cluster): node1.query(f"drop table test") +def test_write_files_with_spaces(started_cluster): + fs = HdfsClient(hosts=started_cluster.hdfs_ip) + dir = "/itime=2024-10-24 10%3A02%3A04" + fs.mkdirs(dir) + + node1.query( + f"insert into function hdfs('hdfs://hdfs1:9000{dir}/test.csv', TSVRaw) select 123 settings hdfs_truncate_on_insert=1" + ) + result = node1.query( + f"select * from hdfs('hdfs://hdfs1:9000{dir}/test.csv', TSVRaw)" + ) + assert int(result) == 123 + fs.delete(dir, recursive=True) + + def test_truncate_table(started_cluster): hdfs_api = started_cluster.hdfs_api node1.query( From e2459c663deb7c1f573b0ee5418d0c5042193f16 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 29 Oct 2024 12:38:59 +0100 Subject: [PATCH 14/35] Fix tidy report --- src/Core/BaseSettings.cpp | 6 +++--- src/Core/BaseSettings.h | 10 +++++----- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Core/BaseSettings.cpp b/src/Core/BaseSettings.cpp index 51e99262bdb..9d55179a5db 100644 --- a/src/Core/BaseSettings.cpp +++ b/src/Core/BaseSettings.cpp @@ -32,14 +32,14 @@ void BaseSettingsHelpers::writeFlags(Flags flags, WriteBuffer & out) } -BaseSettingsHelpers::Flags BaseSettingsHelpers::readFlags(ReadBuffer & in) +UInt64 BaseSettingsHelpers::readFlags(ReadBuffer & in) { UInt64 res; readVarUInt(res, in); - return static_cast(res); + return res; } -SettingsTierType BaseSettingsHelpers::getTier(Flags flags) +SettingsTierType BaseSettingsHelpers::getTier(UInt64 flags) { int8_t tier = (flags & Flags::TIER); if (tier > SettingsTierType::BETA) diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 218460330f4..949b884636f 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -38,9 +38,9 @@ struct BaseSettingsHelpers /// If adding new flags, consider first if Tier might need more bits }; - static SettingsTierType getTier(Flags flags); + static SettingsTierType getTier(UInt64 flags); static void writeFlags(Flags flags, WriteBuffer & out); - static Flags readFlags(ReadBuffer & in); + static UInt64 readFlags(ReadBuffer & in); }; /** Template class to define collections of settings. @@ -481,7 +481,7 @@ void BaseSettings::read(ReadBuffer & in, SettingsWriteFormat format) size_t index = accessor.find(name); using Flags = BaseSettingsHelpers::Flags; - Flags flags{0}; + UInt64 flags{0}; if (format >= SettingsWriteFormat::STRINGS_WITH_FLAGS) flags = BaseSettingsHelpers::readFlags(in); bool is_important = (flags & Flags::IMPORTANT); @@ -860,7 +860,7 @@ using AliasMap = std::unordered_map; String name; \ const char * type; \ const char * description; \ - BaseSettingsHelpers::Flags flags; \ + UInt64 flags; \ Field (*cast_value_util_function)(const Field &); \ String (*value_to_string_util_function)(const Field &); \ Field (*string_to_value_util_function)(const String &); \ @@ -972,7 +972,7 @@ struct DefineAliases #define IMPLEMENT_SETTINGS_TRAITS_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ res.field_infos.emplace_back( \ FieldInfo{#NAME, #TYPE, DESCRIPTION, \ - static_cast(FLAGS), \ + static_cast(FLAGS), \ [](const Field & value) -> Field { return static_cast(SettingField##TYPE{value}); }, \ [](const Field & value) -> String { return SettingField##TYPE{value}.toString(); }, \ [](const String & str) -> Field { SettingField##TYPE temp; temp.parseFromString(str); return static_cast(temp); }, \ From 573204c3033a21f5dad745e946e102a596d26e6e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 29 Oct 2024 09:30:44 -0300 Subject: [PATCH 15/35] getbatch --- .../Formats/Impl/Parquet/ParquetDataValuesReader.cpp | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index 9a79bcffad3..fa38a24fd3c 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -320,12 +320,7 @@ void ParquetBitPlainReader::readBatch( }, /* repeated_visitor */ [&](size_t nest_cursor, UInt32 count) { - for (UInt32 i = 0; i < count; i++) - { - uint8_t byte; - bit_reader->GetValue(1, &byte); - container[nest_cursor++] = byte; - } + bit_reader->GetBatch(1, &container[nest_cursor], count); } ); } From 308763ce678076efdd6d3298f1eb78ba30b4276e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 29 Oct 2024 16:15:41 +0100 Subject: [PATCH 16/35] Improve error and log messages around memory usage --- src/Client/ClientApplicationBase.cpp | 2 +- src/Common/MemoryTracker.cpp | 60 +++++++++++-------- src/Common/ThreadStatus.cpp | 2 +- src/Interpreters/ProcessList.cpp | 4 +- src/Interpreters/ThreadStatusExt.cpp | 6 +- .../test.py | 2 +- tests/integration/test_storage_s3/test.py | 2 +- ...gate_state_exception_memory_leak.reference | 2 +- ...1_aggregate_state_exception_memory_leak.sh | 2 +- ...01514_distributed_cancel_query_on_error.sh | 2 +- 10 files changed, 47 insertions(+), 37 deletions(-) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index d26641fe5f9..f7d2d0035d9 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -418,7 +418,7 @@ void ClientApplicationBase::init(int argc, char ** argv) UInt64 max_client_memory_usage_int = parseWithSizeSuffix(max_client_memory_usage.c_str(), max_client_memory_usage.length()); total_memory_tracker.setHardLimit(max_client_memory_usage_int); - total_memory_tracker.setDescription("(total)"); + total_memory_tracker.setDescription("Global"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); } diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 3ed943f217d..f4af019605e 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -68,15 +68,15 @@ inline std::string_view toDescription(OvercommitResult result) case OvercommitResult::NONE: return ""; case OvercommitResult::DISABLED: - return "Memory overcommit isn't used. Waiting time or overcommit denominator are set to zero."; + return "Memory overcommit isn't used. Waiting time or overcommit denominator are set to zero"; case OvercommitResult::MEMORY_FREED: throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "OvercommitResult::MEMORY_FREED shouldn't be asked for description"); case OvercommitResult::SELECTED: - return "Query was selected to stop by OvercommitTracker."; + return "Query was selected to stop by OvercommitTracker"; case OvercommitResult::TIMEOUTED: - return "Waiting timeout for memory to be freed is reached."; + return "Waiting timeout for memory to be freed is reached"; case OvercommitResult::NOT_ENOUGH_FREED: - return "Memory overcommit has freed not enough memory."; + return "Memory overcommit has not freed enough memory"; } } @@ -150,15 +150,23 @@ void MemoryTracker::logPeakMemoryUsage() auto peak_bytes = peak.load(std::memory_order::relaxed); if (peak_bytes < 128 * 1024) return; - LOG_DEBUG(getLogger("MemoryTracker"), - "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak_bytes)); + LOG_DEBUG( + getLogger("MemoryTracker"), + "{}{} memory usage: {}.", + description ? std::string(description) : "", + description ? " peak" : "Peak", + ReadableSize(peak_bytes)); } void MemoryTracker::logMemoryUsage(Int64 current) const { const auto * description = description_ptr.load(std::memory_order_relaxed); - LOG_DEBUG(getLogger("MemoryTracker"), - "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current)); + LOG_DEBUG( + getLogger("MemoryTracker"), + "{}{} memory usage: {}.", + description ? std::string(description) : "", + description ? " current" : "Current", + ReadableSize(current)); } void MemoryTracker::injectFault() const @@ -178,9 +186,9 @@ void MemoryTracker::injectFault() const const auto * description = description_ptr.load(std::memory_order_relaxed); throw DB::Exception( DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, - "Memory tracker{}{}: fault injected (at specific point)", - description ? " " : "", - description ? description : ""); + "{}{}: fault injected (at specific point)", + description ? description : "", + description ? " memory tracker" : "Memory tracker"); } void MemoryTracker::debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused]]) @@ -282,9 +290,9 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed const auto * description = description_ptr.load(std::memory_order_relaxed); throw DB::Exception( DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, - "Memory tracker{}{}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}", - description ? " " : "", + "{}{}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}", description ? description : "", + description ? " memory tracker" : "Memory tracker", formatReadableSizeWithBinarySuffix(will_be), size, formatReadableSizeWithBinarySuffix(current_hard_limit)); @@ -305,6 +313,8 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed if (overcommit_result != OvercommitResult::MEMORY_FREED) { + bool overcommit_result_ignore + = overcommit_result == OvercommitResult::NONE || overcommit_result == OvercommitResult::DISABLED; /// Revert amount.fetch_sub(size, std::memory_order_relaxed); rss.fetch_sub(size, std::memory_order_relaxed); @@ -314,18 +324,18 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); const auto * description = description_ptr.load(std::memory_order_relaxed); throw DB::Exception( - DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, - "Memory limit{}{} exceeded: " - "would use {} (attempt to allocate chunk of {} bytes), current RSS {}, maximum: {}." - "{}{}", - description ? " " : "", - description ? description : "", - formatReadableSizeWithBinarySuffix(will_be), - size, - formatReadableSizeWithBinarySuffix(rss.load(std::memory_order_relaxed)), - formatReadableSizeWithBinarySuffix(current_hard_limit), - overcommit_result == OvercommitResult::NONE ? "" : " OvercommitTracker decision: ", - toDescription(overcommit_result)); + DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, + "{}{} exceeded: " + "would use {} (attempt to allocate chunk of {} bytes), current RSS {}, maximum: {}." + "{}{}", + description ? description : "", + description ? " memory limit" : "Memory limit", + formatReadableSizeWithBinarySuffix(will_be), + size, + formatReadableSizeWithBinarySuffix(rss.load(std::memory_order_relaxed)), + formatReadableSizeWithBinarySuffix(current_hard_limit), + overcommit_result_ignore ? "" : " OvercommitTracker decision: ", + overcommit_result_ignore ? "" : toDescription(overcommit_result)); } // If OvercommitTracker::needToStopQuery returned false, it guarantees that enough memory is freed. diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index e38d3480664..268d97e62ef 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -78,7 +78,7 @@ ThreadStatus::ThreadStatus(bool check_current_thread_on_destruction_) last_rusage = std::make_unique(); - memory_tracker.setDescription("(for thread)"); + memory_tracker.setDescription("Thread"); log = getLogger("ThreadStatus"); current_thread = this; diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 177468f1c8b..21c30a60617 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -276,7 +276,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q thread_group->performance_counters.setTraceProfileEvents(settings[Setting::trace_profile_events]); } - thread_group->memory_tracker.setDescription("(for query)"); + thread_group->memory_tracker.setDescription("Query"); if (settings[Setting::memory_tracker_fault_probability] > 0.0) thread_group->memory_tracker.setFaultProbability(settings[Setting::memory_tracker_fault_probability]); @@ -311,7 +311,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q /// Track memory usage for all simultaneously running queries from single user. user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings[Setting::max_memory_usage_for_user]); user_process_list.user_memory_tracker.setSoftLimit(settings[Setting::memory_overcommit_ratio_denominator_for_user]); - user_process_list.user_memory_tracker.setDescription("(for user)"); + user_process_list.user_memory_tracker.setDescription("User"); if (!total_network_throttler && settings[Setting::max_network_bandwidth_for_all_users]) { diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 0544bbcc92e..4d27a840d51 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -119,7 +119,7 @@ void ThreadGroup::unlinkThread() ThreadGroupPtr ThreadGroup::createForQuery(ContextPtr query_context_, std::function fatal_error_callback_) { auto group = std::make_shared(query_context_, std::move(fatal_error_callback_)); - group->memory_tracker.setDescription("(for query)"); + group->memory_tracker.setDescription("Query"); return group; } @@ -127,7 +127,7 @@ ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_contex { auto group = std::make_shared(storage_context); - group->memory_tracker.setDescription("background process to apply mutate/merge in table"); + group->memory_tracker.setDescription("Background process (mutate/merge)"); /// However settings from storage context have to be applied const Settings & settings = storage_context->getSettingsRef(); group->memory_tracker.setProfilerStep(settings[Setting::memory_profiler_step]); @@ -384,7 +384,7 @@ void ThreadStatus::initPerformanceCounters() /// TODO: make separate query_thread_performance_counters and thread_performance_counters performance_counters.resetCounters(); memory_tracker.resetCounters(); - memory_tracker.setDescription("(for thread)"); + memory_tracker.setDescription("Thread"); query_start_time.setUp(); diff --git a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py index 7a843a87ec2..74c35e7f4ea 100644 --- a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py +++ b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py @@ -78,7 +78,7 @@ def test_distributed_background_insert_split_batch_on_failure_OFF(started_cluste with pytest.raises( QueryRuntimeException, # no DOTALL in pytest.raises, use '(.|\n)' - match=r"DB::Exception: Received from.*Memory limit \(for query\) exceeded: (.|\n)*While sending a batch", + match=r"DB::Exception: Received from.*Query memory limit exceeded: (.|\n)*While sending a batch", ): node2.query("system flush distributed dist") assert int(node2.query("select count() from dist_data")) == 0 diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index ad1842f4509..d8326711d84 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1592,7 +1592,7 @@ def test_parallel_reading_with_memory_limit(started_cluster): f"select * from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_memory_limit.native') settings max_memory_usage=1000" ) - assert "Memory limit (for query) exceeded" in result + assert "Query memory limit exceeded" in result time.sleep(5) diff --git a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.reference b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.reference index 6282bf366d0..76c31901df7 100644 --- a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.reference +++ b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.reference @@ -1,2 +1,2 @@ -Memory limit exceeded +Query memory limit exceeded Ok diff --git a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh index 5b7cba77432..ceb7b60be0f 100755 --- a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh @@ -16,5 +16,5 @@ for _ in {1..1000}; do if [[ $elapsed -gt 30 ]]; then break fi -done 2>&1 | grep -o -P 'Memory limit .+ exceeded' | sed -r -e 's/(Memory limit)(.+)( exceeded)/\1\3/' | uniq +done 2>&1 | grep -o -P 'Query memory limit exceeded' | sed -r -e 's/(.*):([a-Z ]*)([mM]emory limit exceeded)(.*)/\2\3/' | uniq echo 'Ok' diff --git a/tests/queries/0_stateless/01514_distributed_cancel_query_on_error.sh b/tests/queries/0_stateless/01514_distributed_cancel_query_on_error.sh index edf3683ccba..245aa3ceb99 100755 --- a/tests/queries/0_stateless/01514_distributed_cancel_query_on_error.sh +++ b/tests/queries/0_stateless/01514_distributed_cancel_query_on_error.sh @@ -19,6 +19,6 @@ opts=( ) ${CLICKHOUSE_CLIENT} "${opts[@]}" -q "SELECT groupArray(repeat('a', if(_shard_num == 2, 100000, 1))), number%100000 k from remote('127.{2,3}', system.numbers) GROUP BY k LIMIT 10e6" |& { # the query should fail earlier on 127.3 and 127.2 should not even go to the memory limit exceeded error. - grep -F -q 'DB::Exception: Received from 127.3:9000. DB::Exception: Memory limit (for query) exceeded:' + grep -F -q "DB::Exception: Received from 127.3:${CLICKHOUSE_PORT_TCP}. DB::Exception: Query memory limit exceeded:" # while if this will not correctly then it will got the exception from the 127.2:9000 and fail } From 2d3f5fabda5aa46e0af4b102e2c809ffadb03dc0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 29 Oct 2024 17:08:32 +0100 Subject: [PATCH 17/35] Adjust some tests --- tests/integration/test_grpc_protocol/test.py | 2 +- tests/integration/test_peak_memory_usage/test.py | 11 ++++++----- tests/queries/0_stateless/01921_test_progress_bar.py | 2 +- .../03096_text_log_format_string_args_not_empty.sql | 2 +- 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 732907eed7a..561f5144aac 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -364,7 +364,7 @@ def test_logs(): ) assert query in logs assert "Read 1000000 rows" in logs - assert "Peak memory usage" in logs + assert "Query peak memory usage" in logs def test_progress(): diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py index 877cf97bb18..f5ebc8bd99c 100644 --- a/tests/integration/test_peak_memory_usage/test.py +++ b/tests/integration/test_peak_memory_usage/test.py @@ -68,7 +68,8 @@ def get_memory_usage_from_client_output_and_close(client_output): for line in client_output: print(f"'{line}'\n") if not peek_memory_usage_str_found: - peek_memory_usage_str_found = "Peak memory usage" in line + # Can be both Peak/peak + peek_memory_usage_str_found = "eak memory usage" in line if peek_memory_usage_str_found: search_obj = re.search(r"[+-]?[0-9]+\.[0-9]+", line) @@ -92,13 +93,13 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): client1.send( "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number", ) - client1.expect("Peak memory usage", timeout=60) + client1.expect("Query peak memory usage", timeout=60) client1.expect(prompt) peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) assert peak_memory_usage assert shard_2.contains_in_log( - f"Peak memory usage (for query): {peak_memory_usage}" + f"Query peak memory usage: {peak_memory_usage}" ) @@ -113,11 +114,11 @@ def test_clickhouse_client_max_peak_memory_single_node(started_cluster): client1.send( "SELECT COUNT(*) FROM (SELECT number FROM numbers(1,300000) INTERSECT SELECT number FROM numbers(10000,1200000))" ) - client1.expect("Peak memory usage", timeout=60) + client1.expect("Query peak memory usage", timeout=60) client1.expect(prompt) peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) assert peak_memory_usage assert shard_1.contains_in_log( - f"Peak memory usage (for query): {peak_memory_usage}" + f"Query peak memory usage: {peak_memory_usage}" ) diff --git a/tests/queries/0_stateless/01921_test_progress_bar.py b/tests/queries/0_stateless/01921_test_progress_bar.py index e686698ad9f..4199503ba4a 100755 --- a/tests/queries/0_stateless/01921_test_progress_bar.py +++ b/tests/queries/0_stateless/01921_test_progress_bar.py @@ -17,4 +17,4 @@ with client(name="client1>", log=log) as client1: client1.send("SELECT number FROM numbers(1000) FORMAT Null") client1.expect("Progress: 1\\.00 thousand rows, 8\\.00 KB .*" + end_of_block) client1.expect("0 rows in set. Elapsed: [\\w]{1}\\.[\\w]{3} sec.") - client1.expect("Peak memory usage: .*B" + end_of_block) + client1.expect("Query peak memory usage: .*B" + end_of_block) diff --git a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql index a08f35cfc1d..a4eef59f442 100644 --- a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql +++ b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql @@ -7,7 +7,7 @@ select conut(); -- { serverError UNKNOWN_FUNCTION } system flush logs; SET max_rows_to_read = 0; -- system.text_log can be really big -select count() > 0 from system.text_log where message_format_string = 'Peak memory usage{}: {}.' and value1 is not null and value2 like '% MiB'; +select count() > 0 from system.text_log where message_format_string = '{}{} memory usage: {}.' and not empty(value1) and value3 like '% MiB'; select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier {} in scope {}{}' and value1 = 'expression' and value3 = '`count`' and value4 = 'SELECT count'; From 50de2f4073aa13ac3b8130d065fe7fc5ea681bd9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 29 Oct 2024 20:15:11 +0100 Subject: [PATCH 18/35] Fix style --- tests/integration/test_peak_memory_usage/test.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py index f5ebc8bd99c..b4f33b54bbf 100644 --- a/tests/integration/test_peak_memory_usage/test.py +++ b/tests/integration/test_peak_memory_usage/test.py @@ -98,9 +98,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) assert peak_memory_usage - assert shard_2.contains_in_log( - f"Query peak memory usage: {peak_memory_usage}" - ) + assert shard_2.contains_in_log(f"Query peak memory usage: {peak_memory_usage}") def test_clickhouse_client_max_peak_memory_single_node(started_cluster): @@ -119,6 +117,4 @@ def test_clickhouse_client_max_peak_memory_single_node(started_cluster): peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) assert peak_memory_usage - assert shard_1.contains_in_log( - f"Query peak memory usage: {peak_memory_usage}" - ) + assert shard_1.contains_in_log(f"Query peak memory usage: {peak_memory_usage}") From bbbb81f43dfa09cc1727b8596685a6acfe57ea9f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 30 Oct 2024 13:23:48 +0100 Subject: [PATCH 19/35] Improvements based on review --- src/Core/BaseSettings.cpp | 2 +- src/Core/Settings.cpp | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Core/BaseSettings.cpp b/src/Core/BaseSettings.cpp index 9d55179a5db..2cce94f9d0a 100644 --- a/src/Core/BaseSettings.cpp +++ b/src/Core/BaseSettings.cpp @@ -41,7 +41,7 @@ UInt64 BaseSettingsHelpers::readFlags(ReadBuffer & in) SettingsTierType BaseSettingsHelpers::getTier(UInt64 flags) { - int8_t tier = (flags & Flags::TIER); + int8_t tier = static_cast(flags & Flags::TIER); if (tier > SettingsTierType::BETA) throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown tier value: '{}'", tier); return SettingsTierType{tier}; diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 4159758fe76..aa9b7fd817b 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -38,7 +38,9 @@ namespace ErrorCodes * Note: as an alternative, we could implement settings to be completely dynamic in the form of the map: String -> Field, * but we are not going to do it, because settings are used everywhere as static struct fields. * - * `flags` can be either 0 or IMPORTANT + a Tier (PRODUCTION | BETA | EXPERIMENTAL) + * `flags` can include a Tier (BETA | EXPERIMENTAL) and an optional bitwise AND with IMPORTANT. + * The default (0) means a PRODUCTION ready setting + * * A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions. * Tiers: * EXPERIMENTAL: The feature is in active development stage. Mostly for developers or for ClickHouse enthusiasts. @@ -5824,8 +5826,6 @@ Experimental data deduplication for SELECT queries based on part UUIDs \ /* ####################################################### */ \ /* ############ END OF EXPERIMENTAL FEATURES ############# */ \ - /* ## ADD PRODUCTION / BETA FEATURES BEFORE THIS BLOCK ## */ \ - /* ####################################################### */ \ /* ####################################################### */ \ // End of COMMON_SETTINGS From 4364be72f1983fc8306eb5e4e209c71d64a0e71a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 30 Oct 2024 13:27:12 +0100 Subject: [PATCH 20/35] Mark merge_selector_algorithm as experimental --- src/Core/Settings.cpp | 3 ++- src/Storages/MergeTree/MergeTreeSettings.cpp | 8 +++++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index aa9b7fd817b..1c392d2c547 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5905,13 +5905,14 @@ Experimental data deduplication for SELECT queries based on part UUIDs /** The section above is for obsolete settings. Do not add anything there. */ #endif /// __CLION_IDE__ - #define LIST_OF_SETTINGS(M, ALIAS) \ COMMON_SETTINGS(M, ALIAS) \ OBSOLETE_SETTINGS(M, ALIAS) \ FORMAT_FACTORY_SETTINGS(M, ALIAS) \ OBSOLETE_FORMAT_SETTINGS(M, ALIAS) \ +// clang-format on + DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(SettingsTraits, LIST_OF_SETTINGS) IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS) diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 36e146f4624..38c8f389fbe 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -30,10 +30,11 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +// clang-format off + /** These settings represent fine tunes for internal details of MergeTree storages * and should not be changed by the user without a reason. */ - #define MERGE_TREE_SETTINGS(DECLARE, ALIAS) \ DECLARE(UInt64, min_compress_block_size, 0, "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.", 0) \ DECLARE(UInt64, max_compress_block_size, 0, "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.", 0) \ @@ -98,7 +99,7 @@ namespace ErrorCodes DECLARE(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ DECLARE(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ DECLARE(Milliseconds, background_task_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge or mutation. Can be exceeded if one step takes longer time", 0) \ - DECLARE(MergeSelectorAlgorithm, merge_selector_algorithm, MergeSelectorAlgorithm::SIMPLE, "The algorithm to select parts for merges assignment", 0) \ + DECLARE(MergeSelectorAlgorithm, merge_selector_algorithm, MergeSelectorAlgorithm::SIMPLE, "The algorithm to select parts for merges assignment", EXPERIMENTAL) \ \ /** Inserts settings. */ \ DECLARE(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ @@ -276,8 +277,9 @@ namespace ErrorCodes MERGE_TREE_SETTINGS(M, ALIAS) \ OBSOLETE_MERGE_TREE_SETTINGS(M, ALIAS) -DECLARE_SETTINGS_TRAITS(MergeTreeSettingsTraits, LIST_OF_MERGE_TREE_SETTINGS) +// clang-format on +DECLARE_SETTINGS_TRAITS(MergeTreeSettingsTraits, LIST_OF_MERGE_TREE_SETTINGS) /** Settings for the MergeTree family of engines. * Could be loaded from config or from a CREATE TABLE query (SETTINGS clause). From a819cfa709f3e100e9ae139a81f16eb99e98eec8 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 30 Oct 2024 16:50:40 +0100 Subject: [PATCH 21/35] Read ECS token from file --- src/IO/S3/Credentials.cpp | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index a3f671e76d9..91571432840 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include namespace DB { @@ -693,6 +695,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( static const char AWS_ECS_CONTAINER_CREDENTIALS_RELATIVE_URI[] = "AWS_CONTAINER_CREDENTIALS_RELATIVE_URI"; static const char AWS_ECS_CONTAINER_CREDENTIALS_FULL_URI[] = "AWS_CONTAINER_CREDENTIALS_FULL_URI"; static const char AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN[] = "AWS_CONTAINER_AUTHORIZATION_TOKEN"; + static const char AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN_PATH[] = "AWS_CONTAINER_AUTHORIZATION_TOKEN_PATH"; static const char AWS_EC2_METADATA_DISABLED[] = "AWS_EC2_METADATA_DISABLED"; /// The only difference from DefaultAWSCredentialsProviderChain::DefaultAWSCredentialsProviderChain() @@ -750,7 +753,22 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( } else if (!absolute_uri.empty()) { - const auto token = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN); + auto token = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN); + const auto token_path = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN_PATH); + + if (!token_path.empty()) + { + LOG_INFO(logger, "The environment variable value {} is {}", AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN_PATH, token_path); + + String token_from_file; + + ReadBufferFromFile in(token_path); + readStringUntilEOF(token_from_file, in); + Poco::trimInPlace(token_from_file); + + token = token_from_file; + } + AddProvider(std::make_shared(absolute_uri.c_str(), token.c_str())); /// DO NOT log the value of the authorization token for security purposes. From 12e36c39fc823986e3aecb105773d18a9b4e601e Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 30 Oct 2024 16:52:59 +0100 Subject: [PATCH 22/35] Sort headers --- src/IO/S3/Credentials.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 91571432840..cde9a7a3662 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -1,7 +1,7 @@ -#include -#include #include #include +#include +#include namespace DB { From acdd9f37d210e4f51d24bbbdf1c34449c89a708c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 30 Oct 2024 17:12:48 +0100 Subject: [PATCH 23/35] Fix tests --- tests/queries/0_stateless/01921_test_progress_bar.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01921_test_progress_bar.py b/tests/queries/0_stateless/01921_test_progress_bar.py index 4199503ba4a..e686698ad9f 100755 --- a/tests/queries/0_stateless/01921_test_progress_bar.py +++ b/tests/queries/0_stateless/01921_test_progress_bar.py @@ -17,4 +17,4 @@ with client(name="client1>", log=log) as client1: client1.send("SELECT number FROM numbers(1000) FORMAT Null") client1.expect("Progress: 1\\.00 thousand rows, 8\\.00 KB .*" + end_of_block) client1.expect("0 rows in set. Elapsed: [\\w]{1}\\.[\\w]{3} sec.") - client1.expect("Query peak memory usage: .*B" + end_of_block) + client1.expect("Peak memory usage: .*B" + end_of_block) From 124736756f6b60f915c47e0844214f98590c8574 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Oct 2024 16:18:43 -0300 Subject: [PATCH 24/35] fix msan issue --- .../Impl/Parquet/ParquetDataValuesReader.cpp | 18 +++---- .../Impl/Parquet/ParquetFilterCondition.cpp | 5 ++ .../Impl/Parquet/ParquetFilterCondition.h | 49 +++++++++++++++++++ 3 files changed, 62 insertions(+), 10 deletions(-) create mode 100644 src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp create mode 100644 src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index fa38a24fd3c..b471989076b 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -296,16 +296,12 @@ void ParquetPlainValuesReader::readBatch( ); } -template <> -void ParquetBitPlainReader::readBatch( +template +void ParquetBitPlainReader::readBatch( MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) { - auto & column = *assert_cast(col_ptr.get()); - auto cursor = column.size(); - - auto & container = column.getData(); - - container.resize(cursor + num_values); + auto cursor = col_ptr->size(); + auto * column_data = getResizedPrimitiveData(*assert_cast(col_ptr.get()), cursor + num_values); def_level_reader->visitNullableValues( cursor, @@ -316,11 +312,11 @@ void ParquetBitPlainReader::readBatch( { uint8_t byte; bit_reader->GetValue(1, &byte); - container[nest_cursor] = byte; + column_data[nest_cursor] = byte; }, /* repeated_visitor */ [&](size_t nest_cursor, UInt32 count) { - bit_reader->GetBatch(1, &container[nest_cursor], count); + bit_reader->GetBatch(1, &column_data[nest_cursor], count); } ); } @@ -592,6 +588,8 @@ template class ParquetPlainValuesReader>; template class ParquetPlainValuesReader; template class ParquetPlainValuesReader; +template class ParquetBitPlainReader; + template class ParquetFixedLenPlainReader>; template class ParquetFixedLenPlainReader>; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp b/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp new file mode 100644 index 00000000000..27be594d3c2 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp @@ -0,0 +1,5 @@ +// +// Created by laptop on 10/29/24. +// + +#include "ParquetFilterCondition.h" diff --git a/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h b/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h new file mode 100644 index 00000000000..a09eaa9ced0 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h @@ -0,0 +1,49 @@ +#pragma once + +#include + +#if USE_PARQUET + +#include + +namespace DB +{ + +class ParquetFilterCondition +{ + struct ConditionElement + { + enum Function + { + /// Atoms of a Boolean expression. + FUNCTION_EQUALS, + FUNCTION_NOT_EQUALS, + FUNCTION_IN, + FUNCTION_NOT_IN, + /// Can take any value. + FUNCTION_UNKNOWN, + /// Operators of the logical expression. + FUNCTION_NOT, + FUNCTION_AND, + FUNCTION_OR, + /// Constants + ALWAYS_FALSE, + ALWAYS_TRUE, + }; + + using ColumnPtr = IColumn::Ptr; + using HashesForColumns = std::vector>; + using KeyColumns = std::vector; + + Function function; + // each entry represents a list of hashes per column + // suppose there are three columns with 2 rows each + // hashes_per_column.size() == 3 and hashes_per_column[0].size() == 2 + HashesForColumns hashes_per_column; + KeyColumns key_columns; + }; +}; + +} + +#endif From f70053d925a0f0980a0f57e9787b4a642f28da1b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 30 Oct 2024 20:14:31 +0100 Subject: [PATCH 25/35] Adapt another test to new error message --- tests/integration/test_peak_memory_usage/test.py | 4 ++-- tests/queries/0_stateless/01383_log_broken_table.sh | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py index b4f33b54bbf..51268dcf386 100644 --- a/tests/integration/test_peak_memory_usage/test.py +++ b/tests/integration/test_peak_memory_usage/test.py @@ -93,7 +93,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): client1.send( "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number", ) - client1.expect("Query peak memory usage", timeout=60) + client1.expect("Peak memory usage", timeout=60) client1.expect(prompt) peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) @@ -112,7 +112,7 @@ def test_clickhouse_client_max_peak_memory_single_node(started_cluster): client1.send( "SELECT COUNT(*) FROM (SELECT number FROM numbers(1,300000) INTERSECT SELECT number FROM numbers(10000,1200000))" ) - client1.expect("Query peak memory usage", timeout=60) + client1.expect("Peak memory usage", timeout=60) client1.expect(prompt) peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) diff --git a/tests/queries/0_stateless/01383_log_broken_table.sh b/tests/queries/0_stateless/01383_log_broken_table.sh index 997daf1bf2f..d3c5a2e9aad 100755 --- a/tests/queries/0_stateless/01383_log_broken_table.sh +++ b/tests/queries/0_stateless/01383_log_broken_table.sh @@ -24,7 +24,7 @@ function test_func() $CLICKHOUSE_CLIENT --query "INSERT INTO log SELECT number, number, number FROM numbers(1000000)" --max_memory_usage $MAX_MEM > "${CLICKHOUSE_TMP}"/insert_result 2>&1 RES=$? - grep -o -F 'Memory limit' "${CLICKHOUSE_TMP}"/insert_result || cat "${CLICKHOUSE_TMP}"/insert_result + grep -o -F 'emory limit' "${CLICKHOUSE_TMP}"/insert_result || cat "${CLICKHOUSE_TMP}"/insert_result $CLICKHOUSE_CLIENT --query "SELECT count(), sum(x + y + z) FROM log" > "${CLICKHOUSE_TMP}"/select_result 2>&1; @@ -36,9 +36,9 @@ function test_func() $CLICKHOUSE_CLIENT --query "DROP TABLE log"; } -test_func TinyLog | grep -v -P '^(Memory limit|0\t0|[1-9]000000\t)' -test_func StripeLog | grep -v -P '^(Memory limit|0\t0|[1-9]000000\t)' -test_func Log | grep -v -P '^(Memory limit|0\t0|[1-9]000000\t)' +test_func TinyLog | grep -v -P '^(emory limit|0\t0|[1-9]000000\t)' +test_func StripeLog | grep -v -P '^(emory limit|0\t0|[1-9]000000\t)' +test_func Log | grep -v -P '^(emory limit|0\t0|[1-9]000000\t)' rm "${CLICKHOUSE_TMP}/insert_result" rm "${CLICKHOUSE_TMP}/select_result" From 8d622000b05c7bb54d7e4587a0568bdba327d059 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Oct 2024 19:26:13 -0300 Subject: [PATCH 26/35] remove unrelated file --- .../Impl/Parquet/ParquetFilterCondition.h | 49 ------------------- 1 file changed, 49 deletions(-) delete mode 100644 src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h diff --git a/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h b/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h deleted file mode 100644 index a09eaa9ced0..00000000000 --- a/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h +++ /dev/null @@ -1,49 +0,0 @@ -#pragma once - -#include - -#if USE_PARQUET - -#include - -namespace DB -{ - -class ParquetFilterCondition -{ - struct ConditionElement - { - enum Function - { - /// Atoms of a Boolean expression. - FUNCTION_EQUALS, - FUNCTION_NOT_EQUALS, - FUNCTION_IN, - FUNCTION_NOT_IN, - /// Can take any value. - FUNCTION_UNKNOWN, - /// Operators of the logical expression. - FUNCTION_NOT, - FUNCTION_AND, - FUNCTION_OR, - /// Constants - ALWAYS_FALSE, - ALWAYS_TRUE, - }; - - using ColumnPtr = IColumn::Ptr; - using HashesForColumns = std::vector>; - using KeyColumns = std::vector; - - Function function; - // each entry represents a list of hashes per column - // suppose there are three columns with 2 rows each - // hashes_per_column.size() == 3 and hashes_per_column[0].size() == 2 - HashesForColumns hashes_per_column; - KeyColumns key_columns; - }; -}; - -} - -#endif From 33fdddf9d9327ccc62ac9e0eae3bc022c25f5975 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Oct 2024 19:26:37 -0300 Subject: [PATCH 27/35] remove unrelated file --- .../Formats/Impl/Parquet/ParquetFilterCondition.cpp | 5 ----- 1 file changed, 5 deletions(-) delete mode 100644 src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp diff --git a/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp b/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp deleted file mode 100644 index 27be594d3c2..00000000000 --- a/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp +++ /dev/null @@ -1,5 +0,0 @@ -// -// Created by laptop on 10/29/24. -// - -#include "ParquetFilterCondition.h" From e126092c1f4123f26caf7c7f29ef2ebded6434d3 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 31 Oct 2024 08:51:52 +0000 Subject: [PATCH 28/35] Improve system.query_metric_log to remove flakiness - Use an interval of 400ms instead of 1234ms for the 2500ms sleep to avoid having a last collection that may clash with the finish one. - Move the check for number of events to a separate check. This way we don't have to remove the first and last event, which makes the check less good the fewer events we have. - Add explicit comments of what each check does for readability. --- .../03203_system_query_metric_log.reference | 36 ++++++--- .../03203_system_query_metric_log.sh | 75 ++++++++++++------- 2 files changed, 74 insertions(+), 37 deletions(-) diff --git a/tests/queries/0_stateless/03203_system_query_metric_log.reference b/tests/queries/0_stateless/03203_system_query_metric_log.reference index d761659fce2..940b0c4e178 100644 --- a/tests/queries/0_stateless/03203_system_query_metric_log.reference +++ b/tests/queries/0_stateless/03203_system_query_metric_log.reference @@ -1,12 +1,30 @@ -number_of_metrics_1000_ok timestamp_diff_in_metrics_1000_ok -initial_data_1000_ok -data_1000_ok -number_of_metrics_1234_ok timestamp_diff_in_metrics_1234_ok -initial_data_1234_ok -data_1234_ok -number_of_metrics_123_ok timestamp_diff_in_metrics_123_ok -initial_data_123_ok -data_123_ok +--Interval 1000: check that amount of events is correct +1 +--Interval 1000: check that the delta/diff between the events is correct +1 +--Interval 1000: check that the Query, SelectQuery and InitialQuery values are correct for the first event +1 +--Interval 1000: check that the SleepFunctionCalls, SleepFunctionMilliseconds and ProfileEvent_SleepFunctionElapsedMicroseconds are correct +1 +--Interval 400: check that amount of events is correct +1 +--Interval 400: check that the delta/diff between the events is correct +1 +--Interval 400: check that the Query, SelectQuery and InitialQuery values are correct for the first event +1 +--Interval 400: check that the SleepFunctionCalls, SleepFunctionMilliseconds and ProfileEvent_SleepFunctionElapsedMicroseconds are correct +1 +--Interval 123: check that amount of events is correct +1 +--Interval 123: check that the delta/diff between the events is correct +1 +--Interval 123: check that the Query, SelectQuery and InitialQuery values are correct for the first event +1 +--Interval 123: check that the SleepFunctionCalls, SleepFunctionMilliseconds and ProfileEvent_SleepFunctionElapsedMicroseconds are correct +1 +--Check that a query_metric_log_interval=0 disables the collection 0 +-Check that a query which execution time is less than query_metric_log_interval is never collected 0 +--Check that there is a final event when queries finish 3 diff --git a/tests/queries/0_stateless/03203_system_query_metric_log.sh b/tests/queries/0_stateless/03203_system_query_metric_log.sh index 1c189c6ce41..b66e274df78 100755 --- a/tests/queries/0_stateless/03203_system_query_metric_log.sh +++ b/tests/queries/0_stateless/03203_system_query_metric_log.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) readonly query_prefix=$CLICKHOUSE_DATABASE $CLICKHOUSE_CLIENT --query-id="${query_prefix}_1000" -q "SELECT sleep(2.5) FORMAT Null" & -$CLICKHOUSE_CLIENT --query-id="${query_prefix}_1234" -q "SELECT sleep(2.5) SETTINGS query_metric_log_interval=1234 FORMAT Null" & +$CLICKHOUSE_CLIENT --query-id="${query_prefix}_400" -q "SELECT sleep(2.5) SETTINGS query_metric_log_interval=400 FORMAT Null" & $CLICKHOUSE_CLIENT --query-id="${query_prefix}_123" -q "SELECT sleep(2.5) SETTINGS query_metric_log_interval=123 FORMAT Null" & $CLICKHOUSE_CLIENT --query-id="${query_prefix}_0" -q "SELECT sleep(2.5) SETTINGS query_metric_log_interval=0 FORMAT Null" & $CLICKHOUSE_CLIENT --query-id="${query_prefix}_fast" -q "SELECT sleep(0.1) FORMAT Null" & @@ -20,32 +20,42 @@ function check_log() { interval=$1 + # Check that the amount of events collected is correct, leaving a 20% of margin. + $CLICKHOUSE_CLIENT -m -q """ + SELECT '--Interval $interval: check that amount of events is correct'; + SELECT + count() BETWEEN (ceil(2500 / $interval) * 0.8) AND (ceil(2500 / $interval) * 1.2) + FROM system.query_metric_log + WHERE event_date >= yesterday() AND query_id = '${query_prefix}_${interval}' + """ + # We calculate the diff of each row with its previous row to check whether the intervals at # which data is collected is right. The first row is always skipped because the diff with the # preceding one (itself) is 0. The last row is also skipped, because it doesn't contain a full # interval. $CLICKHOUSE_CLIENT --max_threads=1 -m -q """ - WITH diff AS ( - SELECT - row_number() OVER () AS row, - count() OVER () as total_rows, - event_time_microseconds, - first_value(event_time_microseconds) OVER (ORDER BY event_time_microseconds ROWS BETWEEN 1 PRECEDING AND 0 FOLLOWING) as prev, - dateDiff('ms', prev, event_time_microseconds) AS diff - FROM system.query_metric_log - WHERE event_date >= yesterday() AND query_id = '${query_prefix}_${interval}' - ORDER BY event_time_microseconds - OFFSET 1 - ) - SELECT if(count() BETWEEN ((ceil(2500 / $interval) - 2) * 0.8) AND ((ceil(2500 / $interval) - 2) * 1.2), 'number_of_metrics_${interval}_ok', 'number_of_metrics_${interval}_error'), - if(avg(diff) BETWEEN $interval * 0.8 AND $interval * 1.2, 'timestamp_diff_in_metrics_${interval}_ok', 'timestamp_diff_in_metrics_${interval}_error') - FROM diff WHERE row < total_rows + SELECT '--Interval $interval: check that the delta/diff between the events is correct'; + WITH diff AS ( + SELECT + row_number() OVER () AS row, + count() OVER () as total_rows, + event_time_microseconds, + first_value(event_time_microseconds) OVER (ORDER BY event_time_microseconds ROWS BETWEEN 1 PRECEDING AND 0 FOLLOWING) as prev, + dateDiff('ms', prev, event_time_microseconds) AS diff + FROM system.query_metric_log + WHERE event_date >= yesterday() AND query_id = '${query_prefix}_${interval}' + ORDER BY event_time_microseconds + OFFSET 1 + ) + SELECT avg(diff) BETWEEN $interval * 0.8 AND $interval * 1.2 + FROM diff WHERE row < total_rows """ # Check that the first event contains information from the beginning of the query. # Notice the rest of the events won't contain these because the diff will be 0. $CLICKHOUSE_CLIENT -m -q """ - SELECT if(ProfileEvent_Query = 1 AND ProfileEvent_SelectQuery = 1 AND ProfileEvent_InitialQuery = 1, 'initial_data_${interval}_ok', 'initial_data_${interval}_error') + SELECT '--Interval $interval: check that the Query, SelectQuery and InitialQuery values are correct for the first event'; + SELECT ProfileEvent_Query = 1 AND ProfileEvent_SelectQuery = 1 AND ProfileEvent_InitialQuery = 1 FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_${interval}' ORDER BY event_time_microseconds @@ -55,27 +65,36 @@ function check_log() # Also check that it contains some data that we know it's going to be there. # Notice the Sleep events can be in any of the rows, not only in the first one. $CLICKHOUSE_CLIENT -m -q """ - SELECT if(sum(ProfileEvent_SleepFunctionCalls) = 1 AND - sum(ProfileEvent_SleepFunctionMicroseconds) = 2500000 AND - sum(ProfileEvent_SleepFunctionElapsedMicroseconds) = 2500000 AND - sum(ProfileEvent_Query) = 1 AND - sum(ProfileEvent_SelectQuery) = 1 AND - sum(ProfileEvent_InitialQuery) = 1, - 'data_${interval}_ok', 'data_${interval}_error') + SELECT '--Interval $interval: check that the SleepFunctionCalls, SleepFunctionMilliseconds and ProfileEvent_SleepFunctionElapsedMicroseconds are correct'; + SELECT sum(ProfileEvent_SleepFunctionCalls) = 1 AND + sum(ProfileEvent_SleepFunctionMicroseconds) = 2500000 AND + sum(ProfileEvent_SleepFunctionElapsedMicroseconds) = 2500000 AND + sum(ProfileEvent_Query) = 1 AND + sum(ProfileEvent_SelectQuery) = 1 AND + sum(ProfileEvent_InitialQuery) = 1 FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_${interval}' """ } check_log 1000 -check_log 1234 +check_log 400 check_log 123 # query_metric_log_interval=0 disables the collection altogether -$CLICKHOUSE_CLIENT -m -q """SELECT count() FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_0'""" +$CLICKHOUSE_CLIENT -m -q """ + SELECT '--Check that a query_metric_log_interval=0 disables the collection'; + SELECT count() FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_0' +""" # a quick query that takes less than query_metric_log_interval is never collected -$CLICKHOUSE_CLIENT -m -q """SELECT count() FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_fast'""" +$CLICKHOUSE_CLIENT -m -q """ + SELECT '-Check that a query which execution time is less than query_metric_log_interval is never collected'; + SELECT count() FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_fast' +""" # a query that takes more than query_metric_log_interval is collected including the final row -$CLICKHOUSE_CLIENT -m -q """SELECT count() FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_1000'""" +$CLICKHOUSE_CLIENT -m -q """ + SELECT '--Check that there is a final event when queries finish'; + SELECT count() FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_1000' +""" From 9ea9e9422e478b84e8c750ba69e005a16d8ff30f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 31 Oct 2024 14:45:16 +0100 Subject: [PATCH 29/35] Fix bad cleanup of output format in client when an exception happens --- src/Client/ClientBase.cpp | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 73885ba522d..b6bf637ab44 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1454,8 +1454,22 @@ void ClientBase::resetOutput() /// Order is important: format, compression, file - if (output_format) - output_format->finalize(); + try + { + if (output_format) + output_format->finalize(); + } + catch (...) + { + /// We need to make sure we continue resetting output_format (will stop threads on parallel output) + /// as well as cleaning other output related setup + if (!have_error) + { + client_exception + = std::make_unique(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode()); + have_error = true; + } + } output_format.reset(); logs_out_stream.reset(); From 33cbc540d523888eea630f467718ac84f723f068 Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Thu, 31 Oct 2024 13:49:24 +0000 Subject: [PATCH 30/35] Update kill.md - remove ON CLUSTER for KILL MUTATION ON CLUSTER is not valid for KILL MUTATION, and will result in an exception. Correcting the docs for this syntax --- docs/en/sql-reference/statements/kill.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/kill.md b/docs/en/sql-reference/statements/kill.md index 667a5b51f5c..ff6f64a97fe 100644 --- a/docs/en/sql-reference/statements/kill.md +++ b/docs/en/sql-reference/statements/kill.md @@ -83,7 +83,7 @@ The presence of long-running or incomplete mutations often indicates that a Clic - Or manually kill some of these mutations by sending a `KILL` command. ``` sql -KILL MUTATION [ON CLUSTER cluster] +KILL MUTATION WHERE [TEST] [FORMAT format] @@ -135,7 +135,6 @@ KILL MUTATION WHERE database = 'default' AND table = 'table' -- Cancel the specific mutation: KILL MUTATION WHERE database = 'default' AND table = 'table' AND mutation_id = 'mutation_3.txt' ``` -:::tip If you are killing a mutation in ClickHouse Cloud or in a self-managed cluster, then be sure to use the ```ON CLUSTER [cluster-name]``` option, in order to ensure the mutation is killed on all replicas::: The query is useful when a mutation is stuck and cannot finish (e.g. if some function in the mutation query throws an exception when applied to the data contained in the table). From cdb479d10daeb0edd4bd1ff2c9e400b6cb77c07d Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 31 Oct 2024 14:37:37 +0000 Subject: [PATCH 31/35] Fix debug log timestamp Increase the error margin for the test to avoid flakiness in the intervals where the number of events is smaller. --- src/Interpreters/QueryMetricLog.cpp | 6 +++--- tests/queries/0_stateless/03203_system_query_metric_log.sh | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index 8a84c95a5a3..5ab3fe590e0 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -100,7 +100,7 @@ void QueryMetricLog::startQuery(const String & query_id, TimePoint start_time, U const auto query_info = process_list.getQueryInfo(query_id, false, true, false); if (!query_info) { - LOG_TRACE(logger, "Query {} is not running anymore, so we couldn't get its QueryInfo", query_id); + LOG_TRACE(logger, "Query {} is not running anymore, so we couldn't get its QueryStatusInfo", query_id); return; } @@ -156,8 +156,8 @@ std::optional QueryMetricLog::createLogMetricElement(cons { /// fmtlib supports subsecond formatting in 10.0.0. We're in 9.1.0, so we need to add the milliseconds ourselves. auto seconds = std::chrono::time_point_cast(query_info_time); - auto milliseconds = std::chrono::duration_cast(query_info_time - seconds).count(); - LOG_DEBUG(logger, "Collecting query_metric_log for query {} with QueryStatusInfo from {:%Y.%m.%d %H:%M:%S}.{:05}. Schedule next: {}", query_id, seconds, milliseconds, schedule_next); + auto microseconds = std::chrono::duration_cast(query_info_time - seconds).count(); + LOG_DEBUG(logger, "Collecting query_metric_log for query {} with QueryStatusInfo from {:%Y.%m.%d %H:%M:%S}.{:06}. Schedule next: {}", query_id, seconds, microseconds, schedule_next); std::unique_lock lock(queries_mutex); auto query_status_it = queries.find(query_id); diff --git a/tests/queries/0_stateless/03203_system_query_metric_log.sh b/tests/queries/0_stateless/03203_system_query_metric_log.sh index b66e274df78..bf94be79d7c 100755 --- a/tests/queries/0_stateless/03203_system_query_metric_log.sh +++ b/tests/queries/0_stateless/03203_system_query_metric_log.sh @@ -24,7 +24,7 @@ function check_log() $CLICKHOUSE_CLIENT -m -q """ SELECT '--Interval $interval: check that amount of events is correct'; SELECT - count() BETWEEN (ceil(2500 / $interval) * 0.8) AND (ceil(2500 / $interval) * 1.2) + count() BETWEEN ((ceil(2500 / $interval) - 1) * 0.8) AND ((ceil(2500 / $interval) + 1) * 1.2) FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_${interval}' """ From a57c64e6b01dde6084e40162142bf1325f59f11c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 31 Oct 2024 14:59:04 +0000 Subject: [PATCH 32/35] fix async inserts with empty blocks via native protocol --- src/Interpreters/AsynchronousInsertQueue.cpp | 7 +++++ ..._async_insert_native_empty_block.reference | 9 +++++++ .../03257_async_insert_native_empty_block.sh | 27 +++++++++++++++++++ 3 files changed, 43 insertions(+) create mode 100644 tests/queries/0_stateless/03257_async_insert_native_empty_block.reference create mode 100755 tests/queries/0_stateless/03257_async_insert_native_empty_block.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 5cc97effad6..8b8a6d4e9ef 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -1121,6 +1121,13 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( "Expected entry with data kind Preprocessed. Got: {}", entry->chunk.getDataKind()); Block block_to_insert = *block; + if (block_to_insert.rows() == 0) + { + add_to_async_insert_log(entry, /*parsing_exception=*/ "", block_to_insert.rows(), block_to_insert.bytes()); + entry->resetChunk(); + continue; + } + if (!isCompatibleHeader(block_to_insert, header)) convertBlockToHeader(block_to_insert, header); diff --git a/tests/queries/0_stateless/03257_async_insert_native_empty_block.reference b/tests/queries/0_stateless/03257_async_insert_native_empty_block.reference new file mode 100644 index 00000000000..6df2a541bff --- /dev/null +++ b/tests/queries/0_stateless/03257_async_insert_native_empty_block.reference @@ -0,0 +1,9 @@ +1 name1 +2 name2 +3 +4 +5 +Ok Preprocessed 2 +Ok Preprocessed 3 +Ok Preprocessed 0 +Ok Preprocessed 0 diff --git a/tests/queries/0_stateless/03257_async_insert_native_empty_block.sh b/tests/queries/0_stateless/03257_async_insert_native_empty_block.sh new file mode 100755 index 00000000000..43a5472914d --- /dev/null +++ b/tests/queries/0_stateless/03257_async_insert_native_empty_block.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query " + DROP TABLE IF EXISTS json_square_brackets; + CREATE TABLE json_square_brackets (id UInt32, name String) ENGINE = MergeTree ORDER BY tuple() +" + +MY_CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --async_insert 1 --wait_for_async_insert 1" + +echo '[{"id": 1, "name": "name1"}, {"id": 2, "name": "name2"}]' | $MY_CLICKHOUSE_CLIENT -q "INSERT INTO json_square_brackets FORMAT JSONEachRow" + +echo '[{"id": 3}, {"id": 4}, {"id": 5}]' | $MY_CLICKHOUSE_CLIENT -q "INSERT INTO json_square_brackets FORMAT JSONEachRow" + +echo '[]' | $MY_CLICKHOUSE_CLIENT -q "INSERT INTO json_square_brackets FORMAT JSONEachRow" + +echo '' | $MY_CLICKHOUSE_CLIENT -q "INSERT INTO json_square_brackets FORMAT JSONEachRow" + +$CLICKHOUSE_CLIENT --query " + SYSTEM FLUSH LOGS; + SELECT * FROM json_square_brackets ORDER BY id; + SELECT status, data_kind, rows FROM system.asynchronous_insert_log WHERE database = currentDatabase() AND table = 'json_square_brackets' ORDER BY event_time_microseconds; + DROP TABLE json_square_brackets; +" From 4784c3f0a3e15d908148878270ba7695cadb22c8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 31 Oct 2024 17:12:43 +0100 Subject: [PATCH 33/35] Better style for some sever-level settings --- src/Core/ServerSettings.cpp | 7 +++++++ src/Interpreters/Context.cpp | 26 +++++++++++++++----------- 2 files changed, 22 insertions(+), 11 deletions(-) diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index ead40061493..637c3196f33 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -192,6 +192,13 @@ namespace DB DECLARE(UInt64, parts_killer_pool_size, 128, "Threads for cleanup of shared merge tree outdated threads. Only available in ClickHouse Cloud", 0) \ DECLARE(UInt64, keeper_multiread_batch_size, 10'000, "Maximum size of batch for MultiRead request to [Zoo]Keeper that support batching. If set to 0, batching is disabled. Available only in ClickHouse Cloud.", 0) \ DECLARE(Bool, use_legacy_mongodb_integration, true, "Use the legacy MongoDB integration implementation. Note: it's highly recommended to set this option to false, since legacy implementation will be removed in the future. Please submit any issues you encounter with the new implementation.", 0) \ + \ + DECLARE(UInt64, prefetch_threadpool_pool_size, 100, "Size of background pool for prefetches for remote object storages", 0) \ + DECLARE(UInt64, prefetch_threadpool_queue_size, 1000000, "Number of tasks which is possible to push into prefetches pool", 0) \ + DECLARE(UInt64, load_marks_threadpool_pool_size, 50, "Size of background pool for marks loading", 0) \ + DECLARE(UInt64, load_marks_threadpool_queue_size, 1000000, "Number of tasks which is possible to push into prefetches pool", 0) \ + DECLARE(UInt64, threadpool_writer_pool_size, 100, "Size of background pool for write requests to object storages", 0) \ + DECLARE(UInt64, threadpool_writer_queue_size, 1000000, "Number of tasks which is possible to push into background pool for write requests to object storages", 0) /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fbf0cbd0eb7..4f82ed7b046 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -273,6 +273,13 @@ namespace ServerSetting extern const ServerSettingsUInt64 max_replicated_sends_network_bandwidth_for_server; extern const ServerSettingsUInt64 tables_loader_background_pool_size; extern const ServerSettingsUInt64 tables_loader_foreground_pool_size; + extern const ServerSettingsUInt64 prefetch_threadpool_pool_size; + extern const ServerSettingsUInt64 prefetch_threadpool_queue_size; + extern const ServerSettingsUInt64 load_marks_threadpool_pool_size; + extern const ServerSettingsUInt64 load_marks_threadpool_queue_size; + extern const ServerSettingsUInt64 threadpool_writer_pool_size; + extern const ServerSettingsUInt64 threadpool_writer_queue_size; + } namespace ErrorCodes @@ -3215,9 +3222,8 @@ void Context::clearMarkCache() const ThreadPool & Context::getLoadMarksThreadpool() const { callOnce(shared->load_marks_threadpool_initialized, [&] { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".load_marks_threadpool_pool_size", 50); - auto queue_size = config.getUInt(".load_marks_threadpool_queue_size", 1000000); + auto pool_size = shared->server_settings[ServerSetting::load_marks_threadpool_pool_size]; + auto queue_size = shared->server_settings[ServerSetting::load_marks_threadpool_queue_size]; shared->load_marks_threadpool = std::make_unique( CurrentMetrics::MarksLoaderThreads, CurrentMetrics::MarksLoaderThreadsActive, CurrentMetrics::MarksLoaderThreadsScheduled, pool_size, pool_size, queue_size); }); @@ -3410,9 +3416,9 @@ AsynchronousMetrics * Context::getAsynchronousMetrics() const ThreadPool & Context::getPrefetchThreadpool() const { callOnce(shared->prefetch_threadpool_initialized, [&] { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".prefetch_threadpool_pool_size", 100); - auto queue_size = config.getUInt(".prefetch_threadpool_queue_size", 1000000); + auto pool_size = shared->server_settings[ServerSetting::prefetch_threadpool_pool_size]; + auto queue_size = shared->server_settings[ServerSetting::prefetch_threadpool_queue_size]; + shared->prefetch_threadpool = std::make_unique( CurrentMetrics::IOPrefetchThreads, CurrentMetrics::IOPrefetchThreadsActive, CurrentMetrics::IOPrefetchThreadsScheduled, pool_size, pool_size, queue_size); }); @@ -3422,8 +3428,7 @@ ThreadPool & Context::getPrefetchThreadpool() const size_t Context::getPrefetchThreadpoolSize() const { - const auto & config = getConfigRef(); - return config.getUInt(".prefetch_threadpool_pool_size", 100); + return shared->server_settings[ServerSetting::prefetch_threadpool_pool_size]; } ThreadPool & Context::getBuildVectorSimilarityIndexThreadPool() const @@ -5696,9 +5701,8 @@ IOUringReader & Context::getIOUringReader() const ThreadPool & Context::getThreadPoolWriter() const { callOnce(shared->threadpool_writer_initialized, [&] { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100); - auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); + auto pool_size = shared->server_settings[ServerSetting::threadpool_writer_pool_size]; + auto queue_size = shared->server_settings[ServerSetting::threadpool_writer_queue_size]; shared->threadpool_writer = std::make_unique( CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, CurrentMetrics::IOWriterThreadsScheduled, pool_size, pool_size, queue_size); From e5be813de559b197e020d4a474fb0bed5d0a2637 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 31 Oct 2024 18:50:43 +0100 Subject: [PATCH 34/35] Sync --- src/Core/Settings.cpp | 3 +++ src/Core/SettingsChangesHistory.cpp | 1 + 2 files changed, 4 insertions(+) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 3b63d1231af..7ed24bb85fd 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5111,6 +5111,9 @@ Only in ClickHouse Cloud. A maximum number of unacknowledged in-flight packets i )", 0) \ DECLARE(UInt64, distributed_cache_data_packet_ack_window, DistributedCache::ACK_DATA_PACKET_WINDOW, R"( Only in ClickHouse Cloud. A window for sending ACK for DataPacket sequence in a single distributed cache read request +)", 0) \ + DECLARE(Bool, distributed_cache_discard_connection_if_unread_data, true, R"( +Only in ClickHouse Cloud. Discard connection if some data is unread. )", 0) \ \ DECLARE(Bool, parallelize_output_from_storages, true, R"( diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 3fe3e960dc6..7ea388f18dd 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -64,6 +64,7 @@ static std::initializer_list Date: Thu, 31 Oct 2024 19:46:35 +0100 Subject: [PATCH 35/35] add requirements and fix warning --- docker/test/style/Dockerfile | 2 +- docker/test/style/requirements.txt | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index fa6b087eb7d..564301f447c 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -28,7 +28,7 @@ COPY requirements.txt / RUN pip3 install --no-cache-dir -r requirements.txt RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8 -ENV LC_ALL en_US.UTF-8 +ENV LC_ALL=en_US.UTF-8 # Architecture of the image when BuildKit/buildx is used ARG TARGETARCH diff --git a/docker/test/style/requirements.txt b/docker/test/style/requirements.txt index cc87f6e548d..aab20b5bee0 100644 --- a/docker/test/style/requirements.txt +++ b/docker/test/style/requirements.txt @@ -12,6 +12,7 @@ charset-normalizer==3.3.2 click==8.1.7 codespell==2.2.1 cryptography==43.0.1 +datacompy==0.7.3 Deprecated==1.2.14 dill==0.3.8 flake8==4.0.1 @@ -23,6 +24,7 @@ mccabe==0.6.1 multidict==6.0.5 mypy==1.8.0 mypy-extensions==1.0.0 +pandas==2.2.3 packaging==24.1 pathspec==0.9.0 pip==24.1.1