Skip to content

Commit d909e2e

Browse files
committed
remove cloud-specific MergeTreeSettings
1 parent 8a96014 commit d909e2e

File tree

3 files changed

+0
-221
lines changed

3 files changed

+0
-221
lines changed

src/Core/ServerSettings.cpp

Lines changed: 0 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -1040,15 +1040,6 @@ Whether background memory worker should correct internal memory tracker based on
10401040
DECLARE(String, reserved_replicated_database_prefixes, "", R"(Comma separated list of prohibited replicated database prefixes.)", 0) \
10411041
DECLARE(String, user_with_indirect_database_creation, "", R"(Database creation for this user is simplified by setting necessary parameters automatically and prohibiting dangerous behavoir.)", 0) \
10421042
DECLARE(String, cluster_database, "", R"(Database used for cluster creation.)", 0) \
1043-
DECLARE(UInt64, parts_kill_delay_period, 30, R"(
1044-
Period to completely remove parts for SharedMergeTree. Only available in ClickHouse Cloud
1045-
)", 0) \
1046-
DECLARE(UInt64, parts_kill_delay_period_random_add, 10, R"(
1047-
Add uniformly distributed value from 0 to x seconds to kill_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables. Only available in ClickHouse Cloud
1048-
)", 0) \
1049-
DECLARE(UInt64, parts_killer_pool_size, 128, R"(
1050-
Threads for cleanup of shared merge tree outdated threads. Only available in ClickHouse Cloud
1051-
)", 0) \
10521043
DECLARE(UInt64, keeper_multiread_batch_size, 10'000, R"(
10531044
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.
10541045
)", 0) \
@@ -1122,7 +1113,6 @@ Whether background memory worker should correct internal memory tracker based on
11221113
<process_query_plan_packet>true</process_query_plan_packet>
11231114
```
11241115
)", 0) \
1125-
DECLARE(Bool, storage_shared_set_join_use_inner_uuid, true, "If enabled, an inner UUID is generated during the creation of SharedSet and SharedJoin. ClickHouse Cloud only", 0) \
11261116
DECLARE(UInt64, startup_mv_delay_ms, 0, R"(Debug parameter to simulate materizlied view creation delay)", 0) \
11271117
DECLARE(UInt64, os_cpu_busy_time_threshold, 1'000'000, "Threshold of OS CPU busy time in microseconds (OSCPUVirtualTimeMicroseconds metric) to consider CPU doing some useful work, no CPU overload would be considered if busy time was below this value.", 0) \
11281118
DECLARE(Float, min_os_cpu_wait_time_ratio_to_drop_connection, 0, R"(

src/Parsers/CreateQueryUUIDs.cpp

Lines changed: 0 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -11,11 +11,6 @@
1111
namespace DB
1212
{
1313

14-
namespace ServerSetting
15-
{
16-
extern const ServerSettingsBool storage_shared_set_join_use_inner_uuid;
17-
}
18-
1914
CreateQueryUUIDs::CreateQueryUUIDs(const ASTCreateQuery & query, bool generate_random, bool force_random)
2015
{
2116
if (!generate_random || !force_random)
@@ -54,16 +49,6 @@ CreateQueryUUIDs::CreateQueryUUIDs(const ASTCreateQuery & query, bool generate_r
5449
if (query.is_materialized_view && !(query.refresh_strategy && !query.refresh_strategy->append))
5550
generate_target_uuid(ViewTarget::To);
5651

57-
58-
/// We should generate UUID of inner table for `SharedSet` or `SharedJoin` table here
59-
if (query.storage && query.storage->engine
60-
&& (query.storage->engine->name == "SharedSet" || query.storage->engine->name == "SharedJoin")
61-
&& Context::getGlobalContextInstance()->getServerSettings()[ServerSetting::storage_shared_set_join_use_inner_uuid])
62-
{
63-
if (query.getTargetInnerUUID(ViewTarget::To) == UUIDHelpers::Nil)
64-
setTargetInnerUUID(ViewTarget::To, UUIDHelpers::generateV4());
65-
}
66-
6752
if (query.is_time_series_table)
6853
{
6954
generate_target_uuid(ViewTarget::Data);

src/Storages/MergeTree/MergeTreeSettings.cpp

Lines changed: 0 additions & 196 deletions
Original file line numberDiff line numberDiff line change
@@ -201,34 +201,6 @@ namespace ErrorCodes
201201
it is recommended to set it below the maximum filename length (usually 255
202202
bytes) with some gap to avoid filesystem errors.
203203
)", 0) \
204-
DECLARE(UInt64, min_bytes_for_full_part_storage, 0, R"(
205-
Only available in ClickHouse Cloud. Minimal uncompressed size in bytes to
206-
use full type of storage for data part instead of packed
207-
)", 0) \
208-
DECLARE(UInt64, min_rows_for_full_part_storage, 0, R"(
209-
Only available in ClickHouse Cloud. Minimal number of rows to use full type
210-
of storage for data part instead of packed
211-
)", 0) \
212-
DECLARE(UInt64, compact_parts_max_bytes_to_buffer, 128 * 1024 * 1024, R"(
213-
Only available in ClickHouse Cloud. Maximal number of bytes to write in a
214-
single stripe in compact parts
215-
)", 0) \
216-
DECLARE(UInt64, compact_parts_max_granules_to_buffer, 128, R"(
217-
Only available in ClickHouse Cloud. Maximal number of granules to write in a
218-
single stripe in compact parts
219-
)", 0) \
220-
DECLARE(UInt64, compact_parts_merge_max_bytes_to_prefetch_part, 16 * 1024 * 1024, R"(
221-
Only available in ClickHouse Cloud. Maximal size of compact part to read it
222-
in a whole to memory during merge.
223-
)", 0) \
224-
DECLARE(UInt64, merge_max_bytes_to_prewarm_cache, 1ULL * 1024 * 1024 * 1024, R"(
225-
Only available in ClickHouse Cloud. Maximal size of part (compact or packed)
226-
to prewarm cache during merge.
227-
)", 0) \
228-
DECLARE(UInt64, merge_total_max_bytes_to_prewarm_cache, 15ULL * 1024 * 1024 * 1024, R"(
229-
Only available in ClickHouse Cloud. Maximal size of parts in total to prewarm
230-
cache during merge.
231-
)", 0) \
232204
DECLARE(Bool, load_existing_rows_count_for_old_parts, false, R"(
233205
If enabled along with [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge),
234206
deleted rows count for existing data parts will be calculated during table
@@ -892,12 +864,6 @@ namespace ErrorCodes
892864
`simultaneous_parts_removal_limit` parts during one iteration.
893865
`simultaneous_parts_removal_limit` set to `0` means unlimited.
894866
)", 0) \
895-
DECLARE(UInt64, reduce_blocking_parts_sleep_ms, 5000, R"(
896-
Only available in ClickHouse Cloud. Minimum time to wait before trying to
897-
reduce blocking parts again after no ranges were dropped/replaced. A lower
898-
setting will trigger tasks in background_schedule_pool frequently which
899-
results in large amount of requests to zookeeper in large-scale clusters
900-
)", 0) \
901867
\
902868
/** Replication settings. */ \
903869
DECLARE(UInt64, replicated_deduplication_window, 1000, R"(
@@ -1068,11 +1034,6 @@ namespace ErrorCodes
10681034
Possible values:
10691035
- true, false
10701036
)", 0) \
1071-
DECLARE(UInt64, number_of_partitions_to_consider_for_merge, 10, R"(
1072-
Only available in ClickHouse Cloud. Up to top N partitions which we will
1073-
consider for merge. Partitions picked in a random weighted way where weight
1074-
is amount of data parts which can be merged in this partition.
1075-
)", 0) \
10761037
DECLARE(UInt64, max_suspicious_broken_parts, 100, R"(
10771038
If the number of broken parts in a single partition exceeds the
10781039
`max_suspicious_broken_parts` value, automatic deletion is denied.
@@ -1086,12 +1047,6 @@ namespace ErrorCodes
10861047
Possible values:
10871048
- Any positive integer.
10881049
)", 0) \
1089-
DECLARE(UInt64, shared_merge_tree_max_suspicious_broken_parts, 0, R"(
1090-
Max broken parts for SMT, if more - deny automatic detach.
1091-
)", 0) \
1092-
DECLARE(UInt64, shared_merge_tree_max_suspicious_broken_parts_bytes, 0, R"(
1093-
Max size of all broken parts for SMT, if more - deny automatic detach.
1094-
)", 0) \
10951050
DECLARE(UInt64, max_files_to_modify_in_alter_columns, 75, R"(
10961051
Do not apply ALTER if number of files for modification(deletion, addition)
10971052
is greater than this setting.
@@ -1215,117 +1170,6 @@ namespace ErrorCodes
12151170
DECLARE(Float, fault_probability_after_part_commit, 0, R"(
12161171
For testing. Do not change it.
12171172
)", 0) \
1218-
DECLARE(Bool, shared_merge_tree_disable_merges_and_mutations_assignment, false, R"(
1219-
Stop merges assignment for shared merge tree. Only available in ClickHouse
1220-
Cloud
1221-
)", 0) \
1222-
DECLARE(Bool, shared_merge_tree_enable_outdated_parts_check, true, R"(
1223-
Enable outdated parts check. Only available in ClickHouse Cloud
1224-
)", 0) \
1225-
DECLARE(Float, shared_merge_tree_partitions_hint_ratio_to_reload_merge_pred_for_mutations, 0.5, R"(
1226-
Will reload merge predicate in merge/mutate selecting task when `<candidate
1227-
partitions for mutations only (partitions that cannot be merged)>/<candidate
1228-
partitions for mutations>` ratio is higher than the setting. Only available
1229-
in ClickHouse Cloud
1230-
)", 0) \
1231-
DECLARE(UInt64, shared_merge_tree_parts_load_batch_size, 32, R"(
1232-
Amount of fetch parts metadata jobs to schedule at once. Only available in
1233-
ClickHouse Cloud
1234-
)", 0) \
1235-
DECLARE(UInt64, shared_merge_tree_max_parts_update_leaders_in_total, 6, R"(
1236-
Maximum number of parts update leaders. Only available in ClickHouse Cloud
1237-
)", 0) \
1238-
DECLARE(UInt64, shared_merge_tree_max_parts_update_leaders_per_az, 2, R"(
1239-
Maximum number of parts update leaders. Only available in ClickHouse Cloud
1240-
)", 0) \
1241-
DECLARE(UInt64, shared_merge_tree_leader_update_period_seconds, 30, R"(
1242-
Maximum period to recheck leadership for parts update. Only available in
1243-
ClickHouse Cloud
1244-
)", 0) \
1245-
DECLARE(UInt64, shared_merge_tree_leader_update_period_random_add_seconds, 10, R"(
1246-
Add uniformly distributed value from 0 to x seconds to
1247-
shared_merge_tree_leader_update_period to avoid thundering
1248-
herd effect. Only available in ClickHouse Cloud
1249-
)", 0) \
1250-
DECLARE(Bool, shared_merge_tree_read_virtual_parts_from_leader, true, R"(
1251-
Read virtual parts from leader when possible. Only available in ClickHouse
1252-
Cloud
1253-
)", 0) \
1254-
DECLARE(UInt64, shared_merge_tree_initial_parts_update_backoff_ms, 50, R"(
1255-
Initial backoff for parts update. Only available in ClickHouse Cloud
1256-
)", 0) \
1257-
DECLARE(UInt64, shared_merge_tree_max_parts_update_backoff_ms, 5000, R"(
1258-
Max backoff for parts update. Only available in ClickHouse Cloud
1259-
)", 0) \
1260-
DECLARE(UInt64, shared_merge_tree_interserver_http_connection_timeout_ms, 100, R"(
1261-
Timeouts for interserver HTTP connection. Only available in ClickHouse Cloud
1262-
)", 0) \
1263-
DECLARE(UInt64, shared_merge_tree_interserver_http_timeout_ms, 10000, R"(
1264-
Timeouts for interserver HTTP communication. Only available in ClickHouse
1265-
Cloud
1266-
)", 0) \
1267-
DECLARE(UInt64, shared_merge_tree_max_replicas_for_parts_deletion, 10, R"(
1268-
Max replicas which will participate in parts deletion (killer thread). Only
1269-
available in ClickHouse Cloud
1270-
)", 0) \
1271-
DECLARE(UInt64, shared_merge_tree_max_replicas_to_merge_parts_for_each_parts_range, 5, R"(
1272-
Max replicas which will try to assign potentially conflicting merges (allow
1273-
to avoid redundant conflicts in merges assignment). 0 means disabled. Only
1274-
available in ClickHouse Cloud
1275-
)", 0) \
1276-
DECLARE(Bool, shared_merge_tree_use_outdated_parts_compact_format, false, R"(
1277-
Use compact format for outdated parts: reduces load to Keeper, improves
1278-
outdated parts processing. Only available in ClickHouse Cloud
1279-
)", 0) \
1280-
DECLARE(Int64, shared_merge_tree_memo_ids_remove_timeout_seconds, 1800, R"(
1281-
How long we store insert memoization ids to avoid wrong actions during
1282-
insert retries. Only available in ClickHouse Cloud
1283-
)", 0) \
1284-
DECLARE(UInt64, shared_merge_tree_idle_parts_update_seconds, 3600, R"(
1285-
Interval in seconds for parts update without being triggered by ZooKeeper
1286-
watch in the shared merge tree. Only available in ClickHouse Cloud
1287-
)", 0) \
1288-
DECLARE(UInt64, shared_merge_tree_max_outdated_parts_to_process_at_once, 1000, R"(
1289-
Maximum amount of outdated parts leader will try to confirm for removal at
1290-
one HTTP request. Only available in ClickHouse Cloud.
1291-
)", 0) \
1292-
DECLARE(UInt64, shared_merge_tree_postpone_next_merge_for_locally_merged_parts_rows_threshold, 1000000, R"(
1293-
Minimum size of part (in rows) to postpone assigning a next merge just after
1294-
merging it locally. Only available in ClickHouse Cloud.
1295-
)", 0) \
1296-
DECLARE(UInt64, shared_merge_tree_postpone_next_merge_for_locally_merged_parts_ms, 0, R"(
1297-
Time to keep a locally merged part without starting a new merge containing
1298-
this part. Gives other replicas a chance fetch the part and start this merge.
1299-
Only available in ClickHouse Cloud.
1300-
)", 0) \
1301-
DECLARE(UInt64, shared_merge_tree_range_for_merge_window_size, 10, R"(
1302-
Time to keep a locally merged part without starting a new merge containing
1303-
this part. Gives other replicas a chance fetch the part and start this merge.
1304-
Only available in ClickHouse Cloud
1305-
)", 0) \
1306-
DECLARE(Bool, shared_merge_tree_use_too_many_parts_count_from_virtual_parts, 0, R"(
1307-
If enabled too many parts counter will rely on shared data in Keeper, not on
1308-
local replica state. Only available in ClickHouse Cloud
1309-
)", 0) \
1310-
DECLARE(Bool, shared_merge_tree_create_per_replica_metadata_nodes, true, R"(
1311-
Enables creation of per-replica /metadata and /columns nodes in ZooKeeper.
1312-
Only available in ClickHouse Cloud
1313-
)", 0) \
1314-
DECLARE(Bool, shared_merge_tree_use_metadata_hints_cache, true, R"(
1315-
Enables requesting FS cache hints from in-memory
1316-
cache on other replicas. Only available in ClickHouse Cloud
1317-
)", 0) \
1318-
DECLARE(Bool, shared_merge_tree_try_fetch_part_in_memory_data_from_replicas, false, R"(
1319-
If enabled all the replicas try to fetch part in memory data (like primary
1320-
key, partition info and so on) from other replicas where it already exists.
1321-
)", 0) \
1322-
DECLARE(Milliseconds, shared_merge_tree_update_replica_flags_delay_ms, 30000, R"(
1323-
How often replica will try to reload it's flags according to background schedule.
1324-
)", 0) \
1325-
DECLARE(Bool, allow_reduce_blocking_parts_task, true, R"(
1326-
Background task which reduces blocking parts for shared merge tree tables.
1327-
Only in ClickHouse Cloud
1328-
)", 0) \
13291173
DECLARE(Seconds, refresh_parts_interval, 0, R"(
13301174
If it is greater than zero - refresh the list of data parts from the underlying filesystem to check if the data was updated under the hood.
13311175
It can be set only if the table is located on readonly disks (which means that this is a readonly replica, while data is being written by another replica).
@@ -1743,45 +1587,6 @@ namespace ErrorCodes
17431587
17441588
**Default Value:** false
17451589
)", EXPERIMENTAL) \
1746-
DECLARE(Bool, notify_newest_block_number, false, R"(
1747-
Notify newest block number to SharedJoin or SharedSet. Only in ClickHouse Cloud.
1748-
)", EXPERIMENTAL) \
1749-
DECLARE(Bool, shared_merge_tree_enable_keeper_parts_extra_data, false, R"(
1750-
Enables writing attributes into virtual parts and committing blocks in keeper
1751-
)", EXPERIMENTAL) \
1752-
DECLARE(Bool, shared_merge_tree_enable_coordinated_merges, false, R"(
1753-
Enables coordinated merges strategy
1754-
)", EXPERIMENTAL) \
1755-
DECLARE(UInt64, shared_merge_tree_merge_coordinator_merges_prepare_count, 100, R"(
1756-
Number of merge entries that coordinator should prepare and distribute across workers
1757-
)", EXPERIMENTAL) \
1758-
DECLARE(Milliseconds, shared_merge_tree_merge_coordinator_fetch_fresh_metadata_period_ms, 10000, R"(
1759-
How often merge coordinator should sync with zookeeper to take fresh metadata
1760-
)", EXPERIMENTAL) \
1761-
DECLARE(UInt64, shared_merge_tree_merge_coordinator_max_merge_request_size, 20, R"(
1762-
Number of merges that coordinator can request from MergerMutator at once
1763-
)", EXPERIMENTAL) \
1764-
DECLARE(Milliseconds, shared_merge_tree_merge_coordinator_election_check_period_ms, 30000, R"(
1765-
Time between runs of merge coordinator election thread
1766-
)", EXPERIMENTAL) \
1767-
DECLARE(Milliseconds, shared_merge_tree_merge_coordinator_min_period_ms, 1, R"(
1768-
Minimum time between runs of merge coordinator thread
1769-
)", EXPERIMENTAL) \
1770-
DECLARE(Milliseconds, shared_merge_tree_merge_coordinator_max_period_ms, 10000, R"(
1771-
Maximum time between runs of merge coordinator thread
1772-
)", EXPERIMENTAL) \
1773-
DECLARE(UInt64, shared_merge_tree_merge_coordinator_factor, 2, R"(
1774-
Time changing factor for delay of coordinator thread
1775-
)", EXPERIMENTAL) \
1776-
DECLARE(Milliseconds, shared_merge_tree_merge_worker_fast_timeout_ms, 100, R"(
1777-
Timeout that merge worker thread will use if it is needed to update it's state after immediate action
1778-
)", EXPERIMENTAL) \
1779-
DECLARE(Milliseconds, shared_merge_tree_merge_worker_regular_timeout_ms, 10000, R"(
1780-
Time between runs of merge worker thread
1781-
)", EXPERIMENTAL) \
1782-
DECLARE(UInt64, shared_merge_tree_virtual_parts_discovery_batch, 1, R"(
1783-
How many partition discoveries should be packed into batch
1784-
)", EXPERIMENTAL) \
17851590
\
17861591
/** Compress marks and primary key. */ \
17871592
DECLARE(Bool, compress_marks, true, R"(
@@ -1929,7 +1734,6 @@ namespace ErrorCodes
19291734
MAKE_OBSOLETE_MERGE_TREE_SETTING(M, UInt64, kill_delay_period, 30) \
19301735
MAKE_OBSOLETE_MERGE_TREE_SETTING(M, UInt64, kill_delay_period_random_add, 10) \
19311736
MAKE_OBSOLETE_MERGE_TREE_SETTING(M, UInt64, kill_threads, 128) \
1932-
MAKE_OBSOLETE_MERGE_TREE_SETTING(M, UInt64, cleanup_threads, 128) \
19331737

19341738
/// Settings that should not change after the creation of a table.
19351739
/// NOLINTNEXTLINE

0 commit comments

Comments
 (0)