2018-05-31 15:54:08 +00:00
|
|
|
#include "MemoryTracker.h"
|
2020-01-16 12:37:29 +00:00
|
|
|
|
|
|
|
#include <IO/WriteHelpers.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/Exception.h>
|
2023-11-02 16:14:15 +00:00
|
|
|
#include <Common/HashTable/Hash.h>
|
2022-01-10 19:39:10 +00:00
|
|
|
#include <Common/LockMemoryExceptionInThread.h>
|
|
|
|
#include <Common/MemoryTrackerBlockerInThread.h>
|
2021-10-22 12:56:09 +00:00
|
|
|
#include <Common/OvercommitTracker.h>
|
2023-11-02 16:14:15 +00:00
|
|
|
#include <Common/ProfileEvents.h>
|
2022-10-22 11:39:41 +00:00
|
|
|
#include <Common/Stopwatch.h>
|
2023-11-02 16:14:15 +00:00
|
|
|
#include <Common/ThreadStatus.h>
|
|
|
|
#include <Common/TraceSender.h>
|
|
|
|
#include <Common/VariableContext.h>
|
|
|
|
#include <Common/formatReadable.h>
|
2022-04-27 15:05:45 +00:00
|
|
|
#include <Common/logger_useful.h>
|
2023-11-02 16:14:15 +00:00
|
|
|
#include <Common/thread_local_rng.h>
|
2020-01-16 12:37:29 +00:00
|
|
|
|
2022-09-28 13:29:29 +00:00
|
|
|
#include "config.h"
|
2022-09-20 21:56:10 +00:00
|
|
|
|
|
|
|
#if USE_JEMALLOC
|
|
|
|
# include <jemalloc/jemalloc.h>
|
|
|
|
|
|
|
|
#define STRINGIFY_HELPER(x) #x
|
|
|
|
#define STRINGIFY(x) STRINGIFY_HELPER(x)
|
|
|
|
|
|
|
|
#endif
|
|
|
|
|
2020-01-21 13:53:30 +00:00
|
|
|
#include <atomic>
|
2020-01-22 15:20:19 +00:00
|
|
|
#include <cmath>
|
2020-04-30 13:25:17 +00:00
|
|
|
#include <random>
|
2020-01-16 12:37:29 +00:00
|
|
|
#include <cstdlib>
|
2022-05-23 14:35:09 +00:00
|
|
|
#include <string>
|
2014-05-03 22:57:43 +00:00
|
|
|
|
2021-01-27 00:54:57 +00:00
|
|
|
|
Use total_memory_tracker when there is no other MemoryTracker object.
This should significantly reduce the MemoryTracking drift, test shows
that there is 0 drift after query storm (100 queries, via http/tcp/tcp
in one session).
TL;DR;
To track memory, clickhouse creates memory tracker object for each
thread **explicitly**, but until it is not created the memory
allocations are not under account.
There should not be lot of allocations w/o memory tracker, since most of
the time it is created early enough, but even this maybe enough to
trigger some problems.
Plus sometimes it is not possible to create it, for example some 3d
party library does not allow to do this explicitly:
- for example before #15740 allocations from librdkafka threads,
- or even worse, poco threads, they don't have any routines to do this.
This won't be a problem for `MemoryTracking` metric if the deallocation
will be done from the same thread w/o memory tracker (or vise versa),
but this is not always true.
NOTE, that this will slow down per-thread allocations w/o memory
tracker, since before this patch there were no memory tracking for them
while now they will be accounted in total_memory_tracker, and for
total_memory_tracker max_untracked_memory is always reached.
But this should not be significant.
2020-10-18 07:32:49 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
2020-11-26 18:31:23 +00:00
|
|
|
/// MemoryTracker cannot throw MEMORY_LIMIT_EXCEEDED (either configured memory
|
|
|
|
/// limit reached or fault injected), in the following cases:
|
|
|
|
///
|
2020-11-26 18:31:23 +00:00
|
|
|
/// - when it is explicitly blocked with LockExceptionInThread
|
|
|
|
///
|
2021-04-14 20:32:13 +00:00
|
|
|
/// - when there are uncaught exceptions objects in the current thread
|
|
|
|
/// (to avoid std::terminate())
|
2020-11-26 18:31:23 +00:00
|
|
|
///
|
|
|
|
/// NOTE: that since C++11 destructor marked with noexcept by default, and
|
|
|
|
/// this means that any throw from destructor (that is not marked with
|
|
|
|
/// noexcept(false)) will cause std::terminate()
|
2020-12-05 08:23:12 +00:00
|
|
|
bool inline memoryTrackerCanThrow(VariableContext level, bool fault_injection)
|
2020-11-26 18:31:23 +00:00
|
|
|
{
|
2022-01-10 19:39:10 +00:00
|
|
|
return !LockMemoryExceptionInThread::isBlocked(level, fault_injection) && !std::uncaught_exceptions();
|
2020-11-26 18:31:23 +00:00
|
|
|
}
|
|
|
|
|
Use total_memory_tracker when there is no other MemoryTracker object.
This should significantly reduce the MemoryTracking drift, test shows
that there is 0 drift after query storm (100 queries, via http/tcp/tcp
in one session).
TL;DR;
To track memory, clickhouse creates memory tracker object for each
thread **explicitly**, but until it is not created the memory
allocations are not under account.
There should not be lot of allocations w/o memory tracker, since most of
the time it is created early enough, but even this maybe enough to
trigger some problems.
Plus sometimes it is not possible to create it, for example some 3d
party library does not allow to do this explicitly:
- for example before #15740 allocations from librdkafka threads,
- or even worse, poco threads, they don't have any routines to do this.
This won't be a problem for `MemoryTracking` metric if the deallocation
will be done from the same thread w/o memory tracker (or vise versa),
but this is not always true.
NOTE, that this will slow down per-thread allocations w/o memory
tracker, since before this patch there were no memory tracking for them
while now they will be accounted in total_memory_tracker, and for
total_memory_tracker max_untracked_memory is always reached.
But this should not be significant.
2020-10-18 07:32:49 +00:00
|
|
|
}
|
2014-05-03 22:57:43 +00:00
|
|
|
|
2016-10-24 04:06:27 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int MEMORY_LIMIT_EXCEEDED;
|
2020-08-07 19:36:04 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2016-10-24 04:06:27 +00:00
|
|
|
}
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2022-05-23 14:35:09 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
|
|
|
inline std::string_view toDescription(OvercommitResult result)
|
|
|
|
{
|
|
|
|
switch (result)
|
|
|
|
{
|
|
|
|
case OvercommitResult::NONE:
|
2023-01-29 01:52:37 +00:00
|
|
|
return "";
|
2022-05-23 14:35:09 +00:00
|
|
|
case OvercommitResult::DISABLED:
|
2023-01-29 01:52:37 +00:00
|
|
|
return "Memory overcommit isn't used. Waiting time or overcommit denominator are set to zero.";
|
2022-05-23 14:35:09 +00:00
|
|
|
case OvercommitResult::MEMORY_FREED:
|
|
|
|
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "OvercommitResult::MEMORY_FREED shouldn't be asked for description");
|
|
|
|
case OvercommitResult::SELECTED:
|
2023-01-29 01:52:37 +00:00
|
|
|
return "Query was selected to stop by OvercommitTracker.";
|
2022-05-23 14:35:09 +00:00
|
|
|
case OvercommitResult::TIMEOUTED:
|
2023-01-29 01:52:37 +00:00
|
|
|
return "Waiting timeout for memory to be freed is reached.";
|
2022-05-23 14:35:09 +00:00
|
|
|
case OvercommitResult::NOT_ENOUGH_FREED:
|
2023-01-29 01:52:37 +00:00
|
|
|
return "Memory overcommit has freed not enough memory.";
|
2022-05-23 14:35:09 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-01-16 14:10:31 +00:00
|
|
|
bool shouldTrackAllocation(DB::Float64 probability, void * ptr)
|
|
|
|
{
|
2023-07-20 17:19:33 +00:00
|
|
|
return intHash64(uintptr_t(ptr)) < std::numeric_limits<uint64_t>::max() * probability;
|
2023-01-16 14:10:31 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
2023-01-16 15:48:44 +00:00
|
|
|
void AllocationTrace::onAllocImpl(void * ptr, size_t size) const
|
2023-01-16 14:10:31 +00:00
|
|
|
{
|
|
|
|
if (sample_probability < 1 && !shouldTrackAllocation(sample_probability, ptr))
|
|
|
|
return;
|
|
|
|
|
|
|
|
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
|
|
|
DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = Int64(size), .ptr = ptr});
|
|
|
|
}
|
|
|
|
|
2023-01-16 15:48:44 +00:00
|
|
|
void AllocationTrace::onFreeImpl(void * ptr, size_t size) const
|
2023-01-16 14:10:31 +00:00
|
|
|
{
|
|
|
|
if (sample_probability < 1 && !shouldTrackAllocation(sample_probability, ptr))
|
|
|
|
return;
|
|
|
|
|
|
|
|
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
|
|
|
DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -Int64(size), .ptr = ptr});
|
2022-05-23 14:35:09 +00:00
|
|
|
}
|
|
|
|
|
2020-09-09 16:59:38 +00:00
|
|
|
namespace ProfileEvents
|
|
|
|
{
|
|
|
|
extern const Event QueryMemoryLimitExceeded;
|
2022-10-22 11:39:41 +00:00
|
|
|
extern const Event MemoryAllocatorPurge;
|
|
|
|
extern const Event MemoryAllocatorPurgeTimeMicroseconds;
|
2020-09-09 16:59:38 +00:00
|
|
|
}
|
2016-01-11 21:46:36 +00:00
|
|
|
|
2022-05-27 16:30:29 +00:00
|
|
|
using namespace std::chrono_literals;
|
|
|
|
|
2018-09-24 20:57:10 +00:00
|
|
|
static constexpr size_t log_peak_memory_usage_every = 1ULL << 30;
|
|
|
|
|
2020-04-19 21:43:06 +00:00
|
|
|
MemoryTracker total_memory_tracker(nullptr, VariableContext::Global);
|
2023-04-14 14:00:32 +00:00
|
|
|
MemoryTracker background_memory_tracker(&total_memory_tracker, VariableContext::User, false);
|
2020-04-19 21:43:06 +00:00
|
|
|
|
2022-09-20 21:56:10 +00:00
|
|
|
std::atomic<Int64> MemoryTracker::free_memory_in_allocator_arenas;
|
2020-04-19 21:43:06 +00:00
|
|
|
|
2022-07-08 14:46:11 +00:00
|
|
|
MemoryTracker::MemoryTracker(VariableContext level_) : parent(&total_memory_tracker), level(level_) {}
|
|
|
|
MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : parent(parent_), level(level_) {}
|
2023-04-14 14:00:32 +00:00
|
|
|
MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_, bool log_peak_memory_usage_in_destructor_)
|
|
|
|
: parent(parent_)
|
|
|
|
, log_peak_memory_usage_in_destructor(log_peak_memory_usage_in_destructor_)
|
|
|
|
, level(level_)
|
|
|
|
{}
|
2018-09-24 20:57:10 +00:00
|
|
|
|
2014-05-03 22:57:43 +00:00
|
|
|
MemoryTracker::~MemoryTracker()
|
|
|
|
{
|
2022-06-05 09:02:58 +00:00
|
|
|
if ((level == VariableContext::Process || level == VariableContext::User) && peak && log_peak_memory_usage_in_destructor)
|
2017-10-27 17:24:33 +00:00
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
logPeakMemoryUsage();
|
|
|
|
}
|
2023-09-25 20:19:09 +00:00
|
|
|
catch (...) // NOLINT(bugprone-empty-catch)
|
2017-10-27 17:24:33 +00:00
|
|
|
{
|
|
|
|
/// Exception in Logger, intentionally swallow.
|
|
|
|
}
|
|
|
|
}
|
2014-05-03 22:57:43 +00:00
|
|
|
}
|
|
|
|
|
2022-06-05 09:02:58 +00:00
|
|
|
void MemoryTracker::logPeakMemoryUsage()
|
2015-12-30 15:39:11 +00:00
|
|
|
{
|
2022-06-05 09:02:58 +00:00
|
|
|
log_peak_memory_usage_in_destructor = false;
|
2020-05-28 12:30:42 +00:00
|
|
|
const auto * description = description_ptr.load(std::memory_order_relaxed);
|
2023-08-15 13:37:09 +00:00
|
|
|
auto peak_bytes = peak.load(std::memory_order::relaxed);
|
|
|
|
if (peak_bytes < 128 * 1024)
|
|
|
|
return;
|
2020-10-21 00:31:12 +00:00
|
|
|
LOG_DEBUG(&Poco::Logger::get("MemoryTracker"),
|
2023-08-15 13:37:09 +00:00
|
|
|
"Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak_bytes));
|
2015-12-30 15:39:11 +00:00
|
|
|
}
|
|
|
|
|
2020-04-19 21:43:06 +00:00
|
|
|
void MemoryTracker::logMemoryUsage(Int64 current) const
|
2018-09-24 20:57:10 +00:00
|
|
|
{
|
2020-05-28 12:30:42 +00:00
|
|
|
const auto * description = description_ptr.load(std::memory_order_relaxed);
|
2020-10-21 00:31:12 +00:00
|
|
|
LOG_DEBUG(&Poco::Logger::get("MemoryTracker"),
|
|
|
|
"Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current));
|
2018-09-24 20:57:10 +00:00
|
|
|
}
|
|
|
|
|
2023-03-16 20:28:07 +00:00
|
|
|
void MemoryTracker::injectFault() const
|
|
|
|
{
|
|
|
|
if (!memoryTrackerCanThrow(level, true))
|
|
|
|
{
|
|
|
|
LOG_WARNING(&Poco::Logger::get("MemoryTracker"),
|
|
|
|
"Cannot inject fault at specific point. Uncaught exceptions: {}, stack trace:\n{}",
|
|
|
|
std::uncaught_exceptions(), StackTrace().toString());
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
|
|
|
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
|
|
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded);
|
|
|
|
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 : "");
|
|
|
|
}
|
2018-09-24 20:57:10 +00:00
|
|
|
|
Fix deadlock due to debug tracking of memory allocations
Since memory tracker should be blocked globally (by default it is blocked only
for VariableContext::User).
<details>
<summary>stacktrace</summary>
* thread 11, name = 'Collector', stop reason = trace
frame 1: 0x00007ffff7e37ea2 libc.so.6`pthread_mutex_lock + 274
frame 6: 0x000000001ee10298 clickhouse`toStringCached(pointers=0x00007fffedfef068, offset=0, size=45) + 88 at StackTrace.cpp:425
frame 7: 0x000000001ee1128e clickhouse`StackTrace::toString(this=0x00007fffedfef058) const + 46 at StackTrace.cpp:444
frame 8: 0x000000001ede58ff clickhouse`(anonymous namespace)::debugLogBigAllocationWithoutCheck(size=25187648) + 479 at MemoryTracker.cpp:98
frame 9: 0x000000001ede5481 clickhouse`MemoryTracker::allocImpl(this=0x0000000030729e68, size=25187648, throw_if_memory_exceeded=false, query_tracker=0x00007ffff3c22440) + 3937 at MemoryTracker.cpp:352
frame 10: 0x000000001ede477f clickhouse`MemoryTracker::allocImpl(this=0x00007ffff3c1e530, size=25187648, throw_if_memory_exceeded=false, query_tracker=0x00007ffff3c22440) + 607 at MemoryTracker.cpp:198
frame 11: 0x000000001ede477f clickhouse`MemoryTracker::allocImpl(this=0x00007ffff3c22440, size=25187648, throw_if_memory_exceeded=false, query_tracker=0x0000000000000000) + 607 at MemoryTracker.cpp:198
frame 12: 0x000000001ede477f clickhouse`MemoryTracker::allocImpl(this=0x00007fffedff3bf0, size=25187648, throw_if_memory_exceeded=false, query_tracker=0x0000000000000000) + 607 at MemoryTracker.cpp:198
frame 13: 0x000000001ed6c4c9 clickhouse`CurrentMemoryTracker::allocImpl(size=25165824, throw_if_memory_exceeded=false) + 265 at CurrentMemoryTracker.cpp:58
frame 14: 0x000000001ed6c6a1 clickhouse`CurrentMemoryTracker::allocNoThrow(size=25165824) + 33 at CurrentMemoryTracker.cpp:91
frame 15: 0x000000001ed63079 clickhouse`operator new(unsigned long) [inlined] void Memory::trackMemory<>(size=25165824) + 89 at memory.h:177
frame 16: 0x000000001ed63034 clickhouse`operator new(size=25165824) + 20 at new_delete.cpp:74
...
frame 27: 0x000000001ee15895 clickhouse`DB::(anonymous namespace)::collectSymbols(info=0x00007ffff7062000, (null)=<unavailable>, data_ptr=0x00007fffec2b7000) + 53 at SymbolIndex.cpp:483
frame 28: 0x000000002e1a5d57 clickhouse`::dl_iterate_phdr(callback=(clickhouse`DB::(anonymous namespace)::collectSymbols(dl_phdr_info*, unsigned long, void*) at SymbolIndex.cpp:480), data=0x00007fffec2b7000)(dl_phdr_info *, size_t, void *), void *) + 183 at phdr_cache.cpp:76
frame 29: 0x000000001ee15549 clickhouse`DB::SymbolIndex::update(this=0x00007fffec2b7000) + 41 at SymbolIndex.cpp:515
frame 30: 0x000000001ee1be35 clickhouse`DB::SymbolIndex::SymbolIndex(this=<unavailable>) + 53 at SymbolIndex.h:23
frame 31: 0x000000001ee17a6c clickhouse`DB::SymbolIndex::instanceImpl() + 108 at SymbolIndex.cpp:555
frame 32: 0x000000001ee17b4a clickhouse`DB::SymbolIndex::instance() + 10 at SymbolIndex.cpp:561
frame 33: 0x000000001ee1070f clickhouse`void toStringEveryLineImpl<toStringCached(std::__1::array<void*, 45ul> const&, unsigned long, unsigned long)::$_0>(fatal=false, stack_trace=0x00007fffedff1ef8, callback=0x00007fffedff1ee0)::$_0&&) + 207 at StackTrace.cpp:349
frame 34: 0x000000001ee1040b clickhouse`toStringCached(pointers=0x00007fffedff2928, offset=0, size=43) + 459 at StackTrace.cpp:436
frame 35: 0x000000001ee1128e clickhouse`StackTrace::toString(this=0x00007fffedff2918) const + 46 at StackTrace.cpp:444
frame 36: 0x000000001ede58ff clickhouse`(anonymous namespace)::debugLogBigAllocationWithoutCheck(size=25165824) + 479 at MemoryTracker.cpp:98
frame 37: 0x000000001ede5481 clickhouse`MemoryTracker::allocImpl(this=0x00007fffedff3bf0, size=25165824, throw_if_memory_exceeded=false, query_tracker=0x0000000000000000) + 3937 at MemoryTracker.cpp:352
frame 38: 0x000000001ed6c4c9 clickhouse`CurrentMemoryTracker::allocImpl(size=25165824, throw_if_memory_exceeded=false) + 265 at CurrentMemoryTracker.cpp:58
frame 39: 0x000000001ed6c6a1 clickhouse`CurrentMemoryTracker::allocNoThrow(size=25165824) + 33 at CurrentMemoryTracker.cpp:91
frame 40: 0x000000001ed63079 clickhouse`operator new(unsigned long) [inlined] void Memory::trackMemory<>(size=25165824) + 89 at memory.h:177
frame 41: 0x000000001ed63034 clickhouse`operator new(size=25165824) + 20 at new_delete.cpp:74
...
frame 55: 0x0000000025badace clickhouse`DB::WriteBufferFromS3::nextImpl(this=0x00007ffff3c58f00) + 366 at WriteBufferFromS3.cpp:110
</details>
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-04-06 12:43:52 +00:00
|
|
|
void MemoryTracker::debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused]])
|
|
|
|
{
|
|
|
|
/// Big allocations through allocNoThrow (without checking memory limits) may easily lead to OOM (and it's hard to debug).
|
|
|
|
/// Let's find them.
|
|
|
|
#ifdef ABORT_ON_LOGICAL_ERROR
|
|
|
|
if (size < 0)
|
|
|
|
return;
|
|
|
|
|
|
|
|
constexpr Int64 threshold = 16 * 1024 * 1024; /// The choice is arbitrary (maybe we should decrease it)
|
|
|
|
if (size < threshold)
|
|
|
|
return;
|
|
|
|
|
|
|
|
MemoryTrackerBlockerInThread blocker(VariableContext::Global);
|
|
|
|
LOG_TEST(&Poco::Logger::get("MemoryTracker"), "Too big allocation ({} bytes) without checking memory limits, "
|
|
|
|
"it may lead to OOM. Stack trace: {}", size, StackTrace().toString());
|
|
|
|
#else
|
|
|
|
return; /// Avoid trash logging in release builds
|
|
|
|
#endif
|
|
|
|
}
|
2018-09-24 20:57:10 +00:00
|
|
|
|
2023-01-18 13:28:57 +00:00
|
|
|
AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker, double _sample_probability)
|
2014-05-03 22:57:43 +00:00
|
|
|
{
|
2020-08-07 19:36:04 +00:00
|
|
|
if (size < 0)
|
|
|
|
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Negative size ({}) is passed to MemoryTracker. It is a bug.", size);
|
|
|
|
|
2023-01-18 13:28:57 +00:00
|
|
|
if (_sample_probability < 0)
|
|
|
|
_sample_probability = sample_probability;
|
|
|
|
|
2023-08-01 17:14:16 +00:00
|
|
|
if (!isSizeOkForSampling(size))
|
|
|
|
_sample_probability = 0;
|
|
|
|
|
2022-01-10 19:39:10 +00:00
|
|
|
if (MemoryTrackerBlockerInThread::isBlocked(level))
|
2020-12-05 08:23:12 +00:00
|
|
|
{
|
2022-09-20 21:56:10 +00:00
|
|
|
if (level == VariableContext::Global)
|
|
|
|
{
|
|
|
|
/// For global memory tracker always update memory usage.
|
|
|
|
amount.fetch_add(size, std::memory_order_relaxed);
|
|
|
|
|
|
|
|
auto metric_loaded = metric.load(std::memory_order_relaxed);
|
|
|
|
if (metric_loaded != CurrentMetrics::end())
|
|
|
|
CurrentMetrics::add(metric_loaded, size);
|
|
|
|
}
|
|
|
|
|
2022-01-10 19:39:10 +00:00
|
|
|
/// Since the MemoryTrackerBlockerInThread should respect the level, we should go to the next parent.
|
2020-12-05 08:23:12 +00:00
|
|
|
if (auto * loaded_next = parent.load(std::memory_order_relaxed))
|
2023-01-16 14:10:31 +00:00
|
|
|
{
|
|
|
|
MemoryTracker * tracker = level == VariableContext::Process ? this : query_tracker;
|
2023-01-18 13:28:57 +00:00
|
|
|
return loaded_next->allocImpl(size, throw_if_memory_exceeded, tracker, _sample_probability);
|
2023-01-16 14:10:31 +00:00
|
|
|
}
|
|
|
|
|
2023-01-18 13:28:57 +00:00
|
|
|
return AllocationTrace(_sample_probability);
|
2020-12-05 08:23:12 +00:00
|
|
|
}
|
2018-02-01 17:55:08 +00:00
|
|
|
|
2017-09-17 00:13:26 +00:00
|
|
|
/** Using memory_order_relaxed means that if allocations are done simultaneously,
|
2020-03-03 00:24:44 +00:00
|
|
|
* we allow exception about memory limit exceeded to be thrown only on next allocation.
|
2017-09-17 00:13:26 +00:00
|
|
|
* So, we allow over-allocations.
|
|
|
|
*/
|
2023-08-02 13:11:52 +00:00
|
|
|
Int64 will_be = size ? size + amount.fetch_add(size, std::memory_order_relaxed) : amount.load(std::memory_order_relaxed);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-01-17 12:40:14 +00:00
|
|
|
auto metric_loaded = metric.load(std::memory_order_relaxed);
|
2023-08-02 13:11:52 +00:00
|
|
|
if (metric_loaded != CurrentMetrics::end() && size)
|
2021-01-17 12:40:14 +00:00
|
|
|
CurrentMetrics::add(metric_loaded, size);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-01-21 13:53:30 +00:00
|
|
|
Int64 current_hard_limit = hard_limit.load(std::memory_order_relaxed);
|
2020-01-22 15:20:19 +00:00
|
|
|
Int64 current_profiler_limit = profiler_limit.load(std::memory_order_relaxed);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-08-17 20:04:10 +00:00
|
|
|
bool memory_limit_exceeded_ignored = false;
|
2021-05-25 20:54:48 +00:00
|
|
|
|
2021-10-07 18:13:56 +00:00
|
|
|
bool allocation_traced = false;
|
2021-05-25 20:54:48 +00:00
|
|
|
if (unlikely(current_profiler_limit && will_be > current_profiler_limit))
|
|
|
|
{
|
2022-01-10 19:39:10 +00:00
|
|
|
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
2022-11-24 19:54:39 +00:00
|
|
|
DB::TraceSender::send(DB::TraceType::Memory, StackTrace(), {.size = size});
|
2021-05-25 20:54:48 +00:00
|
|
|
setOrRaiseProfilerLimit((will_be + profiler_step - 1) / profiler_step * profiler_step);
|
2021-10-07 18:13:56 +00:00
|
|
|
allocation_traced = true;
|
2014-05-03 22:57:43 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-08-17 20:04:10 +00:00
|
|
|
std::bernoulli_distribution fault(fault_probability);
|
2022-09-10 02:07:51 +00:00
|
|
|
if (unlikely(fault_probability > 0.0 && fault(thread_local_rng)))
|
2021-05-25 20:54:48 +00:00
|
|
|
{
|
2022-08-17 20:04:10 +00:00
|
|
|
if (memoryTrackerCanThrow(level, true) && throw_if_memory_exceeded)
|
2021-10-22 12:56:09 +00:00
|
|
|
{
|
2022-08-15 21:34:02 +00:00
|
|
|
/// Revert
|
|
|
|
amount.fetch_sub(size, std::memory_order_relaxed);
|
|
|
|
|
2021-10-22 12:56:09 +00:00
|
|
|
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
2022-01-18 20:33:29 +00:00
|
|
|
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
2022-08-17 20:04:10 +00:00
|
|
|
|
2021-10-22 12:56:09 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded);
|
|
|
|
const auto * description = description_ptr.load(std::memory_order_relaxed);
|
|
|
|
throw DB::Exception(
|
|
|
|
DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
|
2022-08-17 20:04:10 +00:00
|
|
|
"Memory tracker{}{}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}",
|
2021-10-22 12:56:09 +00:00
|
|
|
description ? " " : "",
|
|
|
|
description ? description : "",
|
2022-08-14 06:50:12 +00:00
|
|
|
formatReadableSizeWithBinarySuffix(will_be),
|
2021-10-22 12:56:09 +00:00
|
|
|
size,
|
2022-08-17 20:04:10 +00:00
|
|
|
formatReadableSizeWithBinarySuffix(current_hard_limit));
|
2021-10-22 12:56:09 +00:00
|
|
|
}
|
|
|
|
else
|
2023-04-04 18:10:22 +00:00
|
|
|
{
|
2022-08-17 20:04:10 +00:00
|
|
|
memory_limit_exceeded_ignored = true;
|
2023-04-04 18:10:22 +00:00
|
|
|
debugLogBigAllocationWithoutCheck(size);
|
|
|
|
}
|
2021-05-25 20:54:48 +00:00
|
|
|
}
|
|
|
|
|
2022-09-20 21:56:10 +00:00
|
|
|
Int64 limit_to_check = current_hard_limit;
|
|
|
|
|
|
|
|
#if USE_JEMALLOC
|
2022-11-21 16:10:47 +00:00
|
|
|
if (level == VariableContext::Global && allow_use_jemalloc_memory.load(std::memory_order_relaxed))
|
2022-09-20 21:56:10 +00:00
|
|
|
{
|
|
|
|
/// Jemalloc arenas may keep some extra memory.
|
|
|
|
/// This memory was substucted from RSS to decrease memory drift.
|
|
|
|
/// In case memory is close to limit, try to pugre the arenas.
|
|
|
|
/// This is needed to avoid OOM, because some allocations are directly done with mmap.
|
|
|
|
Int64 current_free_memory_in_allocator_arenas = free_memory_in_allocator_arenas.load(std::memory_order_relaxed);
|
|
|
|
|
|
|
|
if (current_free_memory_in_allocator_arenas > 0 && current_hard_limit && current_free_memory_in_allocator_arenas + will_be > current_hard_limit)
|
2021-10-22 12:56:09 +00:00
|
|
|
{
|
2022-09-20 21:56:10 +00:00
|
|
|
if (free_memory_in_allocator_arenas.exchange(-current_free_memory_in_allocator_arenas) > 0)
|
|
|
|
{
|
2022-10-22 11:39:41 +00:00
|
|
|
Stopwatch watch;
|
2022-09-20 21:56:10 +00:00
|
|
|
mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0);
|
2022-10-22 11:39:41 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge);
|
|
|
|
ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, watch.elapsedMicroseconds());
|
2022-09-20 21:56:10 +00:00
|
|
|
}
|
2021-10-22 12:56:09 +00:00
|
|
|
}
|
2022-09-20 21:56:10 +00:00
|
|
|
|
|
|
|
limit_to_check += abs(current_free_memory_in_allocator_arenas);
|
2021-05-25 20:54:48 +00:00
|
|
|
}
|
2022-09-20 21:56:10 +00:00
|
|
|
#endif
|
2021-05-25 20:54:48 +00:00
|
|
|
|
2022-09-20 21:56:10 +00:00
|
|
|
if (unlikely(current_hard_limit && will_be > limit_to_check))
|
2021-05-26 20:31:35 +00:00
|
|
|
{
|
2022-08-17 20:04:10 +00:00
|
|
|
if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded)
|
|
|
|
{
|
|
|
|
OvercommitResult overcommit_result = OvercommitResult::NONE;
|
|
|
|
if (auto * overcommit_tracker_ptr = overcommit_tracker.load(std::memory_order_relaxed); overcommit_tracker_ptr != nullptr && query_tracker != nullptr)
|
|
|
|
overcommit_result = overcommit_tracker_ptr->needToStopQuery(query_tracker, size);
|
|
|
|
|
|
|
|
if (overcommit_result != OvercommitResult::MEMORY_FREED)
|
|
|
|
{
|
|
|
|
/// Revert
|
|
|
|
amount.fetch_sub(size, std::memory_order_relaxed);
|
|
|
|
|
|
|
|
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
|
|
|
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
|
|
|
ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded);
|
|
|
|
const auto * description = description_ptr.load(std::memory_order_relaxed);
|
|
|
|
throw DB::Exception(
|
2023-01-23 21:13:58 +00:00
|
|
|
DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
|
|
|
|
"Memory limit{}{} exceeded: "
|
2023-01-29 01:52:37 +00:00
|
|
|
"would use {} (attempt to allocate chunk of {} bytes), maximum: {}."
|
|
|
|
"{}{}",
|
2023-01-23 21:13:58 +00:00
|
|
|
description ? " " : "",
|
|
|
|
description ? description : "",
|
|
|
|
formatReadableSizeWithBinarySuffix(will_be),
|
|
|
|
size,
|
|
|
|
formatReadableSizeWithBinarySuffix(current_hard_limit),
|
2023-01-29 01:52:37 +00:00
|
|
|
overcommit_result == OvercommitResult::NONE ? "" : " OvercommitTracker decision: ",
|
2023-01-23 21:13:58 +00:00
|
|
|
toDescription(overcommit_result));
|
2022-08-17 20:04:10 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
// If OvercommitTracker::needToStopQuery returned false, it guarantees that enough memory is freed.
|
|
|
|
// This memory is already counted in variable `amount` in the moment of `will_be` initialization.
|
|
|
|
// Now we just need to update value stored in `will_be`, because it should have changed.
|
|
|
|
will_be = amount.load(std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
2023-04-04 18:10:22 +00:00
|
|
|
{
|
2022-08-17 20:04:10 +00:00
|
|
|
memory_limit_exceeded_ignored = true;
|
2023-04-04 18:10:22 +00:00
|
|
|
debugLogBigAllocationWithoutCheck(size);
|
|
|
|
}
|
2021-05-26 20:31:35 +00:00
|
|
|
}
|
2022-08-17 20:04:10 +00:00
|
|
|
|
|
|
|
bool peak_updated = false;
|
|
|
|
/// In case of MEMORY_LIMIT_EXCEEDED was ignored, will_be may include
|
|
|
|
/// memory of other allocations, that may fail but not reverted yet, and so
|
|
|
|
/// updating peak will be inaccurate.
|
|
|
|
if (!memory_limit_exceeded_ignored)
|
2021-05-26 20:31:35 +00:00
|
|
|
{
|
2022-08-17 20:04:10 +00:00
|
|
|
if (throw_if_memory_exceeded)
|
|
|
|
{
|
|
|
|
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
|
|
|
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
|
|
|
bool log_memory_usage = true;
|
|
|
|
peak_updated = updatePeak(will_be, log_memory_usage);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
bool log_memory_usage = false;
|
|
|
|
peak_updated = updatePeak(will_be, log_memory_usage);
|
2023-04-04 18:10:22 +00:00
|
|
|
debugLogBigAllocationWithoutCheck(size);
|
2022-08-17 20:04:10 +00:00
|
|
|
}
|
2021-10-07 18:13:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (peak_updated && allocation_traced)
|
|
|
|
{
|
2022-01-10 19:39:10 +00:00
|
|
|
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
2022-11-24 19:54:39 +00:00
|
|
|
DB::TraceSender::send(DB::TraceType::MemoryPeak, StackTrace(), {.size = will_be});
|
2021-05-26 20:31:35 +00:00
|
|
|
}
|
2020-04-19 21:43:06 +00:00
|
|
|
|
2020-04-22 00:29:38 +00:00
|
|
|
if (auto * loaded_next = parent.load(std::memory_order_relaxed))
|
2023-01-16 14:10:31 +00:00
|
|
|
{
|
|
|
|
MemoryTracker * tracker = level == VariableContext::Process ? this : query_tracker;
|
2023-01-18 13:28:57 +00:00
|
|
|
return loaded_next->allocImpl(size, throw_if_memory_exceeded, tracker, _sample_probability);
|
2023-01-16 14:10:31 +00:00
|
|
|
}
|
|
|
|
|
2023-01-18 13:28:57 +00:00
|
|
|
return AllocationTrace(_sample_probability);
|
2020-04-19 21:43:06 +00:00
|
|
|
}
|
|
|
|
|
2022-07-07 12:24:36 +00:00
|
|
|
void MemoryTracker::adjustWithUntrackedMemory(Int64 untracked_memory)
|
2021-05-25 09:50:55 +00:00
|
|
|
{
|
2022-07-07 12:24:36 +00:00
|
|
|
if (untracked_memory > 0)
|
2023-01-16 14:10:31 +00:00
|
|
|
std::ignore = allocImpl(untracked_memory, /*throw_if_memory_exceeded*/ false);
|
2022-07-07 12:24:36 +00:00
|
|
|
else
|
2023-01-16 14:10:31 +00:00
|
|
|
std::ignore = free(-untracked_memory);
|
2021-05-25 09:50:55 +00:00
|
|
|
}
|
2020-04-19 21:43:06 +00:00
|
|
|
|
2021-10-07 18:13:56 +00:00
|
|
|
bool MemoryTracker::updatePeak(Int64 will_be, bool log_memory_usage)
|
2020-04-19 21:43:06 +00:00
|
|
|
{
|
2018-09-24 20:57:10 +00:00
|
|
|
auto peak_old = peak.load(std::memory_order_relaxed);
|
|
|
|
if (will_be > peak_old) /// Races doesn't matter. Could rewrite with CAS, but not worth.
|
|
|
|
{
|
2016-07-31 03:53:16 +00:00
|
|
|
peak.store(will_be, std::memory_order_relaxed);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-05-25 20:54:48 +00:00
|
|
|
if (log_memory_usage && (level == VariableContext::Process || level == VariableContext::Global)
|
2020-04-19 21:43:06 +00:00
|
|
|
&& will_be / log_peak_memory_usage_every > peak_old / log_peak_memory_usage_every)
|
2018-09-24 20:57:10 +00:00
|
|
|
logMemoryUsage(will_be);
|
2021-10-07 18:13:56 +00:00
|
|
|
|
|
|
|
return true;
|
2018-09-24 20:57:10 +00:00
|
|
|
}
|
2021-10-07 18:13:56 +00:00
|
|
|
return false;
|
2015-12-30 15:39:11 +00:00
|
|
|
}
|
|
|
|
|
2023-01-18 13:28:57 +00:00
|
|
|
AllocationTrace MemoryTracker::free(Int64 size, double _sample_probability)
|
2015-12-30 15:39:11 +00:00
|
|
|
{
|
2023-01-18 13:28:57 +00:00
|
|
|
if (_sample_probability < 0)
|
|
|
|
_sample_probability = sample_probability;
|
|
|
|
|
2023-08-01 17:14:16 +00:00
|
|
|
if (!isSizeOkForSampling(size))
|
|
|
|
_sample_probability = 0;
|
|
|
|
|
2022-01-10 19:39:10 +00:00
|
|
|
if (MemoryTrackerBlockerInThread::isBlocked(level))
|
2021-01-05 23:42:34 +00:00
|
|
|
{
|
2022-09-20 21:56:10 +00:00
|
|
|
if (level == VariableContext::Global)
|
|
|
|
{
|
|
|
|
/// For global memory tracker always update memory usage.
|
|
|
|
amount.fetch_sub(size, std::memory_order_relaxed);
|
|
|
|
auto metric_loaded = metric.load(std::memory_order_relaxed);
|
|
|
|
if (metric_loaded != CurrentMetrics::end())
|
|
|
|
CurrentMetrics::sub(metric_loaded, size);
|
|
|
|
}
|
|
|
|
|
2022-01-10 19:39:10 +00:00
|
|
|
/// Since the MemoryTrackerBlockerInThread should respect the level, we should go to the next parent.
|
2021-01-05 23:42:34 +00:00
|
|
|
if (auto * loaded_next = parent.load(std::memory_order_relaxed))
|
2023-01-18 13:28:57 +00:00
|
|
|
return loaded_next->free(size, _sample_probability);
|
2018-02-01 17:55:08 +00:00
|
|
|
|
2023-01-18 13:28:57 +00:00
|
|
|
return AllocationTrace(_sample_probability);
|
2020-04-30 13:25:17 +00:00
|
|
|
}
|
|
|
|
|
2020-10-18 10:51:59 +00:00
|
|
|
Int64 accounted_size = size;
|
2022-09-20 21:56:10 +00:00
|
|
|
if (level == VariableContext::Thread || level == VariableContext::Global)
|
2017-09-17 00:13:26 +00:00
|
|
|
{
|
2018-06-09 15:29:08 +00:00
|
|
|
/// Could become negative if memory allocated in this thread is freed in another one
|
2020-10-18 10:51:59 +00:00
|
|
|
amount.fetch_sub(accounted_size, std::memory_order_relaxed);
|
2018-06-09 15:29:08 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2020-10-18 10:51:59 +00:00
|
|
|
Int64 new_amount = amount.fetch_sub(accounted_size, std::memory_order_relaxed) - accounted_size;
|
2018-06-09 15:29:08 +00:00
|
|
|
|
|
|
|
/** Sometimes, query could free some data, that was allocated outside of query context.
|
|
|
|
* Example: cache eviction.
|
|
|
|
* To avoid negative memory usage, we "saturate" amount.
|
|
|
|
* Memory usage will be calculated with some error.
|
|
|
|
* NOTE: The code is not atomic. Not worth to fix.
|
|
|
|
*/
|
|
|
|
if (unlikely(new_amount < 0))
|
|
|
|
{
|
|
|
|
amount.fetch_sub(new_amount);
|
2020-10-18 10:51:59 +00:00
|
|
|
accounted_size += new_amount;
|
2018-06-09 15:29:08 +00:00
|
|
|
}
|
2017-09-17 00:13:26 +00:00
|
|
|
}
|
2022-06-20 17:35:24 +00:00
|
|
|
if (auto * overcommit_tracker_ptr = overcommit_tracker.load(std::memory_order_relaxed))
|
|
|
|
overcommit_tracker_ptr->tryContinueQueryExecutionAfterFree(accounted_size);
|
2015-12-30 15:39:11 +00:00
|
|
|
|
2023-01-18 13:28:57 +00:00
|
|
|
/// free should never throw, we can update metric early.
|
2021-01-17 12:40:14 +00:00
|
|
|
auto metric_loaded = metric.load(std::memory_order_relaxed);
|
|
|
|
if (metric_loaded != CurrentMetrics::end())
|
|
|
|
CurrentMetrics::sub(metric_loaded, accounted_size);
|
2023-01-16 14:10:31 +00:00
|
|
|
|
2023-01-18 13:28:57 +00:00
|
|
|
if (auto * loaded_next = parent.load(std::memory_order_relaxed))
|
|
|
|
return loaded_next->free(size, _sample_probability);
|
|
|
|
|
|
|
|
return AllocationTrace(_sample_probability);
|
2015-12-30 15:39:11 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-10-22 12:56:09 +00:00
|
|
|
OvercommitRatio MemoryTracker::getOvercommitRatio()
|
|
|
|
{
|
|
|
|
return { amount.load(std::memory_order_relaxed), soft_limit.load(std::memory_order_relaxed) };
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-10-26 12:32:17 +00:00
|
|
|
OvercommitRatio MemoryTracker::getOvercommitRatio(Int64 limit)
|
|
|
|
{
|
|
|
|
return { amount.load(std::memory_order_relaxed), limit };
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-05-27 16:30:29 +00:00
|
|
|
void MemoryTracker::setOvercommitWaitingTime(UInt64 wait_time)
|
|
|
|
{
|
|
|
|
max_wait_time.store(wait_time * 1us, std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-06-09 15:29:08 +00:00
|
|
|
void MemoryTracker::resetCounters()
|
|
|
|
{
|
|
|
|
amount.store(0, std::memory_order_relaxed);
|
|
|
|
peak.store(0, std::memory_order_relaxed);
|
2021-10-22 12:56:09 +00:00
|
|
|
soft_limit.store(0, std::memory_order_relaxed);
|
2020-01-21 13:53:30 +00:00
|
|
|
hard_limit.store(0, std::memory_order_relaxed);
|
2020-01-22 15:20:19 +00:00
|
|
|
profiler_limit.store(0, std::memory_order_relaxed);
|
2018-06-09 15:29:08 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2015-12-30 15:39:11 +00:00
|
|
|
void MemoryTracker::reset()
|
|
|
|
{
|
2021-01-17 12:40:14 +00:00
|
|
|
auto metric_loaded = metric.load(std::memory_order_relaxed);
|
|
|
|
if (metric_loaded != CurrentMetrics::end())
|
|
|
|
CurrentMetrics::sub(metric_loaded, amount.load(std::memory_order_relaxed));
|
2016-01-21 01:47:28 +00:00
|
|
|
|
2018-06-09 15:29:08 +00:00
|
|
|
resetCounters();
|
2017-02-01 03:53:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-09-20 21:56:10 +00:00
|
|
|
void MemoryTracker::setRSS(Int64 rss_, Int64 free_memory_in_allocator_arenas_)
|
2020-04-19 21:43:06 +00:00
|
|
|
{
|
2022-10-22 16:39:36 +00:00
|
|
|
Int64 new_amount = rss_;
|
2022-09-20 21:56:10 +00:00
|
|
|
total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed);
|
|
|
|
free_memory_in_allocator_arenas.store(free_memory_in_allocator_arenas_, std::memory_order_relaxed);
|
|
|
|
|
|
|
|
auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed);
|
|
|
|
if (metric_loaded != CurrentMetrics::end())
|
|
|
|
CurrentMetrics::set(metric_loaded, new_amount);
|
2022-08-14 06:50:12 +00:00
|
|
|
|
|
|
|
bool log_memory_usage = true;
|
2022-09-20 21:56:10 +00:00
|
|
|
total_memory_tracker.updatePeak(rss_, log_memory_usage);
|
2020-04-19 21:43:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-10-22 12:56:09 +00:00
|
|
|
void MemoryTracker::setSoftLimit(Int64 value)
|
|
|
|
{
|
|
|
|
soft_limit.store(value, std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-01-16 15:33:34 +00:00
|
|
|
void MemoryTracker::setHardLimit(Int64 value)
|
|
|
|
{
|
|
|
|
hard_limit.store(value, std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-01-21 13:53:30 +00:00
|
|
|
void MemoryTracker::setOrRaiseHardLimit(Int64 value)
|
|
|
|
{
|
|
|
|
/// This is just atomic set to maximum.
|
|
|
|
Int64 old_value = hard_limit.load(std::memory_order_relaxed);
|
2021-08-13 10:23:54 +00:00
|
|
|
while ((value == 0 || old_value < value) && !hard_limit.compare_exchange_weak(old_value, value))
|
2020-01-21 13:53:30 +00:00
|
|
|
;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-01-22 15:20:19 +00:00
|
|
|
void MemoryTracker::setOrRaiseProfilerLimit(Int64 value)
|
2017-02-01 03:53:10 +00:00
|
|
|
{
|
2020-01-22 15:20:19 +00:00
|
|
|
Int64 old_value = profiler_limit.load(std::memory_order_relaxed);
|
2021-08-13 10:23:54 +00:00
|
|
|
while ((value == 0 || old_value < value) && !profiler_limit.compare_exchange_weak(old_value, value))
|
2017-02-01 03:53:10 +00:00
|
|
|
;
|
2014-05-03 22:57:43 +00:00
|
|
|
}
|
2023-01-16 14:10:31 +00:00
|
|
|
|
2023-08-03 09:35:45 +00:00
|
|
|
double MemoryTracker::getSampleProbability(UInt64 size)
|
2023-01-16 14:10:31 +00:00
|
|
|
{
|
2023-01-18 13:28:57 +00:00
|
|
|
if (sample_probability >= 0)
|
2023-08-03 09:35:45 +00:00
|
|
|
{
|
|
|
|
if (!isSizeOkForSampling(size))
|
|
|
|
return 0;
|
2023-01-18 13:28:57 +00:00
|
|
|
return sample_probability;
|
2023-08-03 09:35:45 +00:00
|
|
|
}
|
2023-01-16 14:10:31 +00:00
|
|
|
|
|
|
|
if (auto * loaded_next = parent.load(std::memory_order_relaxed))
|
2023-08-03 09:35:45 +00:00
|
|
|
return loaded_next->getSampleProbability(size);
|
2023-01-16 14:10:31 +00:00
|
|
|
|
|
|
|
return 0;
|
|
|
|
}
|
2023-06-27 17:26:25 +00:00
|
|
|
|
2023-07-30 12:01:34 +00:00
|
|
|
bool MemoryTracker::isSizeOkForSampling(UInt64 size) const
|
|
|
|
{
|
|
|
|
/// We can avoid comparison min_allocation_size_bytes with zero, because we cannot have 0 bytes allocation/deallocation
|
|
|
|
return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes);
|
|
|
|
}
|
|
|
|
|
2023-11-02 16:14:15 +00:00
|
|
|
void MemoryTracker::setParent(MemoryTracker * elem)
|
|
|
|
{
|
|
|
|
/// Untracked memory shouldn't be accounted to a query or a user if it was allocated before the thread was attached
|
|
|
|
/// to a query thread group or a user group, because this memory will be (🤞) freed outside of these scopes.
|
|
|
|
if (level == VariableContext::Thread && DB::current_thread)
|
|
|
|
DB::current_thread->flushUntrackedMemory();
|
|
|
|
|
|
|
|
parent.store(elem, std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
2023-04-14 14:34:19 +00:00
|
|
|
bool canEnqueueBackgroundTask()
|
|
|
|
{
|
|
|
|
auto limit = background_memory_tracker.getSoftLimit();
|
|
|
|
auto amount = background_memory_tracker.get();
|
|
|
|
return limit == 0 || amount < limit;
|
|
|
|
}
|