mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
move max_entries_for_hash_table_stats to server settings
This commit is contained in:
parent
985a4badd8
commit
44e4495e5b
@ -146,6 +146,7 @@ namespace DB
|
||||
M(UInt64, global_profiler_real_time_period_ns, 0, "Period for real clock timer of global profiler (in nanoseconds). Set 0 value to turn off the real clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
|
||||
M(UInt64, global_profiler_cpu_time_period_ns, 0, "Period for CPU clock timer of global profiler (in nanoseconds). Set 0 value to turn off the CPU clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
|
||||
M(Bool, enable_azure_sdk_logging, false, "Enables logging from Azure sdk", 0) \
|
||||
M(UInt64, max_entries_for_hash_table_stats, 10'000, "How many entries hash table statistics collected during aggregation is allowed to have", 0) \
|
||||
|
||||
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp
|
||||
|
||||
|
@ -699,7 +699,6 @@ class IColumn;
|
||||
M(UInt64, insert_shard_id, 0, "If non zero, when insert into a distributed table, the data will be inserted into the shard `insert_shard_id` synchronously. Possible values range from 1 to `shards_number` of corresponding distributed table", 0) \
|
||||
\
|
||||
M(Bool, collect_hash_table_stats_during_aggregation, true, "Enable collecting hash table statistics to optimize memory allocation", 0) \
|
||||
M(UInt64, max_entries_for_hash_table_stats, 10'000, "How many entries hash table statistics collected during aggregation is allowed to have", 0) \
|
||||
M(UInt64, max_size_to_preallocate_for_aggregation, 100'000'000, "For how many elements it is allowed to preallocate space in all hash tables in total before aggregation", 0) \
|
||||
\
|
||||
M(Bool, kafka_disable_num_consumers_limit, false, "Disable limit on kafka_num_consumers that depends on the number of available CPU cores", 0) \
|
||||
@ -976,6 +975,7 @@ class IColumn;
|
||||
MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, async_insert_threads, 16) \
|
||||
MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_replicated_fetches_network_bandwidth_for_server, 0) \
|
||||
MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_replicated_sends_network_bandwidth_for_server, 0) \
|
||||
MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_entries_for_hash_table_stats, 10'000) \
|
||||
/* ---- */ \
|
||||
MAKE_OBSOLETE(M, DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic) \
|
||||
MAKE_OBSOLETE(M, UInt64, max_pipeline_depth, 0) \
|
||||
|
@ -68,8 +68,8 @@ std::optional<HashTablesCacheStatistics> HashTablesStatistics::getCacheStats() c
|
||||
|
||||
HashTablesStatistics::CachePtr HashTablesStatistics::getHashTableStatsCache(const Params & params, const std::lock_guard<std::mutex> &)
|
||||
{
|
||||
if (!hash_table_stats || hash_table_stats->maxSizeInBytes() != params.max_entries_for_hash_table_stats)
|
||||
hash_table_stats = std::make_shared<Cache>(params.max_entries_for_hash_table_stats);
|
||||
if (!hash_table_stats)
|
||||
hash_table_stats = std::make_shared<Cache>(params.max_entries_for_hash_table_stats * sizeof(Entry));
|
||||
return hash_table_stats;
|
||||
}
|
||||
|
||||
|
@ -2668,7 +2668,7 @@ static Aggregator::Params getAggregatorParams(
|
||||
const auto stats_collecting_params = StatsCollectingParams(
|
||||
calculateCacheKey(query_ptr),
|
||||
settings.collect_hash_table_stats_during_aggregation,
|
||||
settings.max_entries_for_hash_table_stats,
|
||||
context.getServerSettings().max_entries_for_hash_table_stats,
|
||||
settings.max_size_to_preallocate_for_aggregation);
|
||||
|
||||
return Aggregator::Params
|
||||
|
@ -366,7 +366,7 @@ Aggregator::Params getAggregatorParams(const PlannerContextPtr & planner_context
|
||||
const auto stats_collecting_params = StatsCollectingParams(
|
||||
calculateCacheKey(select_query_info.query),
|
||||
settings.collect_hash_table_stats_during_aggregation,
|
||||
settings.max_entries_for_hash_table_stats,
|
||||
query_context->getServerSettings().max_entries_for_hash_table_stats,
|
||||
settings.max_size_to_preallocate_for_aggregation);
|
||||
|
||||
auto aggregate_descriptions = aggregation_analysis_result.aggregate_descriptions;
|
||||
|
@ -807,7 +807,7 @@ static std::shared_ptr<IJoin> tryCreateJoin(JoinAlgorithm algorithm,
|
||||
StatsCollectingParams params{
|
||||
calculateCacheKey(table_join, right_table_expression),
|
||||
settings.collect_hash_table_stats_during_aggregation,
|
||||
settings.max_entries_for_hash_table_stats,
|
||||
query_context->getServerSettings().max_entries_for_hash_table_stats,
|
||||
settings.max_size_to_preallocate_for_aggregation};
|
||||
return std::make_shared<ConcurrentHashJoin>(
|
||||
query_context, table_join, query_context->getSettings().max_threads, right_table_expression_header, params);
|
||||
|
Loading…
Reference in New Issue
Block a user