Merge pull request #64969 from ClickHouse/revert-64423-precise-memtracking

Revert "Add dynamic untracked memory limits for more precise memory tracking"
This commit is contained in:
Sergei Trifonov 2024-06-07 16:51:36 +00:00 committed by GitHub
commit 686d6d676a
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 10 additions and 54 deletions

View File

@ -3188,7 +3188,7 @@ Default value: `0`.
## lightweight_deletes_sync {#lightweight_deletes_sync}
The same as 'mutation_sync', but controls only execution of lightweight deletes.
The same as 'mutation_sync', but controls only execution of lightweight deletes.
Possible values:
@ -4632,16 +4632,6 @@ Read more about [memory overcommit](memory-overcommit.md).
Default value: `1GiB`.
## max_untracked_memory {#max_untracked_memory}
Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.
Default value: `4MiB`.
## min_untracked_memory {#min_untracked_memory}
Lower bound for untracked memory limit which is applied to threads with low memory consumption. Untracked memory limit equals thread memory usage divided by 16 and clamped between `min_untracked_memory` and `max_untracked_memory` for every thread. It guarantees that total untracked memory does not exceed 10% of current memory footprint even with a lot of small threads. To disable dynamic limit for untracked memory set value `4MiB`.
Default value: `4KiB`.
## Schema Inference settings
See [schema inference](../../interfaces/schema-inference.md#schema-inference-modes) documentation for more details.

View File

@ -57,7 +57,6 @@ AllocationTrace CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory
{
auto res = memory_tracker->allocImpl(will_be, throw_if_memory_exceeded);
current_thread->untracked_memory = 0;
current_thread->updateUntrackedMemoryLimit(memory_tracker->get());
return res;
}
else
@ -85,13 +84,6 @@ void CurrentMemoryTracker::check()
std::ignore = memory_tracker->allocImpl(0, true);
}
Int64 CurrentMemoryTracker::get()
{
if (auto * memory_tracker = getMemoryTracker())
return memory_tracker->get();
return 0;
}
AllocationTrace CurrentMemoryTracker::alloc(Int64 size)
{
bool throw_if_memory_exceeded = true;
@ -111,12 +103,10 @@ AllocationTrace CurrentMemoryTracker::free(Int64 size)
if (current_thread)
{
current_thread->untracked_memory -= size;
// Note that we use `max_untracked_memory` and not `untracked_memory_limit` to create hysteresis to avoid track/untrack cycles
if (current_thread->untracked_memory < -current_thread->max_untracked_memory)
if (current_thread->untracked_memory < -current_thread->untracked_memory_limit)
{
Int64 untracked_memory = current_thread->untracked_memory;
current_thread->untracked_memory = 0;
current_thread->updateUntrackedMemoryLimit(memory_tracker->get() + untracked_memory);
return memory_tracker->free(-untracked_memory);
}
}

View File

@ -12,9 +12,7 @@ struct CurrentMemoryTracker
/// This function should be called after memory deallocation.
[[nodiscard]] static AllocationTrace free(Int64 size);
static void check();
[[nodiscard]] static Int64 get();
/// Throws MEMORY_LIMIT_EXCEEDED (if it's allowed to throw exceptions)
static void injectFault();

View File

@ -183,12 +183,6 @@ public:
Int64 untracked_memory = 0;
/// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters.
Int64 untracked_memory_limit = 4 * 1024 * 1024;
/// To keep total untracked memory limited to `untracked_memory_ratio * RSS` we have to account threads with small and large memory footprint differently.
/// For this purpose we dynamically change `untracked_memory_limit` after every tracking event using a simple formula:
/// untracked_memory_limit = clamp(untracked_memory_ratio * cur_memory_bytes, min_untracked_memory, max_untracked_memory)
/// Note that this values are updated when thread is attached to a group
Int64 min_untracked_memory = 4 * 1024 * 1024; // Default value is kept 4MB mostly for tests and client (should be changed to 4KB as default value a setting)
Int64 max_untracked_memory = 4 * 1024 * 1024;
/// Statistics of read and write rows/bytes
Progress progress_in;
@ -315,12 +309,6 @@ public:
void initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period);
void updateUntrackedMemoryLimit(Int64 current)
{
constexpr Int64 untracked_memory_ratio_bits = 4; // untracked_memory_ratio = 1.0 / (1 << untracked_memory_ratio_bits) = 1.0 / 16 = 6.25%
untracked_memory_limit = std::clamp<Int64>(current >> untracked_memory_ratio_bits, min_untracked_memory, max_untracked_memory);
}
private:
void applyGlobalSettings();
void applyQuerySettings();

View File

@ -493,7 +493,6 @@ class IColumn;
M(UInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \
M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \
M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \
M(UInt64, min_untracked_memory, (4 * 1024), "Lower bound for untracked memory limit which is applied to threads with low memory consumption. Untracked memory limit equals thread_memory_usage/16 and clamped between min_untracked_memory and max_untracked_memory for every thread.", 0) \
M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \
M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \

View File

@ -96,7 +96,6 @@ static const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges
{"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"},
{"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"},
{"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"},
{"min_untracked_memory", 4_MiB, 4_KiB, "A new setting to enable more accurate memory tracking."},
{"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"},
{"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."},
{"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."},

View File

@ -11,7 +11,6 @@
#include <Parsers/formatAST.h>
#include <Parsers/queryNormalization.h>
#include <Common/CurrentThread.h>
#include <Common/CurrentMemoryTracker.h>
#include <Common/Exception.h>
#include <Common/ProfileEvents.h>
#include <Common/QueryProfiler.h>
@ -211,12 +210,9 @@ void ThreadStatus::applyQuerySettings()
query_id_from_query_context = query_context_ptr->getCurrentQueryId();
initQueryProfiler();
max_untracked_memory = settings.max_untracked_memory;
if (settings.memory_profiler_step && settings.memory_profiler_step < static_cast<UInt64>(max_untracked_memory))
max_untracked_memory = settings.memory_profiler_step;
min_untracked_memory = std::min<Int64>(settings.min_untracked_memory, max_untracked_memory);
updateUntrackedMemoryLimit(CurrentMemoryTracker::get());
untracked_memory_limit = settings.max_untracked_memory;
if (settings.memory_profiler_step && settings.memory_profiler_step < static_cast<UInt64>(untracked_memory_limit))
untracked_memory_limit = settings.memory_profiler_step;
#if defined(OS_LINUX)
/// Set "nice" value if required.

View File

@ -45,7 +45,9 @@ def test_failed_async_inserts(started_cluster):
ignore_error=True,
)
select_query = "SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery' SETTINGS min_untracked_memory = '4Mi'"
select_query = (
"SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery'"
)
assert node.query(select_query) == "4\n"

View File

@ -136,10 +136,7 @@ def test_select_clamps_settings():
)
assert (
distributed.query(
query,
settings={"max_memory_usage": 1, "min_untracked_memory": 4 * 1024 * 1024},
)
distributed.query(query, settings={"max_memory_usage": 1})
== "node1\tmax_memory_usage\t11111111\n"
"node1\treadonly\t0\n"
"node2\tmax_memory_usage\t0\n"

View File

@ -7,8 +7,7 @@
-- sizeof(HLL) is (2^K * 6 / 8)
-- hence max_memory_usage for 100 rows = (96<<10)*100 = 9830400
SET use_uncompressed_cache = 0;
SET min_untracked_memory = '4Mi';
SET use_uncompressed_cache = 0;
-- HashTable for UInt32 (used until (1<<13) elements), hence 8192 elements
SELECT 'UInt32';

View File

@ -13,8 +13,6 @@ create table dist_out as data engine=Distributed(test_shard_localhost, currentDa
set prefer_localhost_replica=0;
set min_untracked_memory='4Mi'; -- Disable precise memory tracking
insert into dist_in select number/100, number from system.numbers limit 1e6 settings max_memory_usage='20Mi';
system flush distributed dist_in; -- { serverError MEMORY_LIMIT_EXCEEDED }
system flush distributed dist_in settings max_memory_usage=0;