2018-05-31 15:54:08 +00:00
|
|
|
#include "MemoryTracker.h"
|
2020-01-16 12:37:29 +00:00
|
|
|
|
|
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
#include "Common/TraceCollector.h"
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/Exception.h>
|
|
|
|
#include <Common/formatReadable.h>
|
2020-01-16 12:37:29 +00:00
|
|
|
#include <common/logger_useful.h>
|
2020-09-09 16:59:38 +00:00
|
|
|
#include <Common/ProfileEvents.h>
|
2020-01-16 12:37:29 +00:00
|
|
|
|
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>
|
2014-05-03 22:57:43 +00:00
|
|
|
|
2021-01-12 14:34:50 +00:00
|
|
|
#ifdef MEMORY_TRACKER_DEBUG_CHECKS
|
|
|
|
thread_local bool _memory_tracker_always_throw_logical_error_on_allocation = false;
|
|
|
|
#endif
|
|
|
|
|
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-01-22 19:07:47 +00:00
|
|
|
/// - to avoid std::terminate(), when stack unwinding is currently in progress
|
|
|
|
/// in this thread.
|
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
|
|
|
{
|
2020-12-05 08:23:12 +00:00
|
|
|
return !MemoryTracker::LockExceptionInThread::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
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int MEMORY_LIMIT_EXCEEDED;
|
2020-08-07 19:36:04 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2020-09-09 16:59:38 +00:00
|
|
|
namespace ProfileEvents
|
|
|
|
{
|
|
|
|
extern const Event QueryMemoryLimitExceeded;
|
|
|
|
}
|
2016-01-11 21:46:36 +00:00
|
|
|
|
2018-09-24 20:57:10 +00:00
|
|
|
static constexpr size_t log_peak_memory_usage_every = 1ULL << 30;
|
|
|
|
|
2020-12-05 08:23:12 +00:00
|
|
|
// BlockerInThread
|
2020-12-01 07:34:25 +00:00
|
|
|
thread_local uint64_t MemoryTracker::BlockerInThread::counter = 0;
|
2020-12-05 08:23:12 +00:00
|
|
|
thread_local VariableContext MemoryTracker::BlockerInThread::level = VariableContext::Global;
|
|
|
|
MemoryTracker::BlockerInThread::BlockerInThread(VariableContext level_)
|
|
|
|
: previous_level(level)
|
|
|
|
{
|
|
|
|
++counter;
|
|
|
|
level = level_;
|
|
|
|
}
|
|
|
|
MemoryTracker::BlockerInThread::~BlockerInThread()
|
|
|
|
{
|
|
|
|
--counter;
|
|
|
|
level = previous_level;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// LockExceptionInThread
|
2020-12-01 07:34:25 +00:00
|
|
|
thread_local uint64_t MemoryTracker::LockExceptionInThread::counter = 0;
|
2020-12-05 08:23:12 +00:00
|
|
|
thread_local VariableContext MemoryTracker::LockExceptionInThread::level = VariableContext::Global;
|
|
|
|
thread_local bool MemoryTracker::LockExceptionInThread::block_fault_injections = false;
|
|
|
|
MemoryTracker::LockExceptionInThread::LockExceptionInThread(VariableContext level_, bool block_fault_injections_)
|
|
|
|
: previous_level(level)
|
|
|
|
, previous_block_fault_injections(block_fault_injections)
|
|
|
|
{
|
|
|
|
++counter;
|
|
|
|
level = level_;
|
|
|
|
block_fault_injections = block_fault_injections_;
|
|
|
|
}
|
|
|
|
MemoryTracker::LockExceptionInThread::~LockExceptionInThread()
|
|
|
|
{
|
|
|
|
--counter;
|
|
|
|
level = previous_level;
|
|
|
|
block_fault_injections = previous_block_fault_injections;
|
|
|
|
}
|
|
|
|
|
2020-10-21 00:31:12 +00:00
|
|
|
|
2020-04-19 21:43:06 +00:00
|
|
|
MemoryTracker total_memory_tracker(nullptr, VariableContext::Global);
|
|
|
|
|
|
|
|
|
|
|
|
MemoryTracker::MemoryTracker(VariableContext level_) : parent(&total_memory_tracker), level(level_) {}
|
|
|
|
MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : parent(parent_), level(level_) {}
|
|
|
|
|
2018-09-24 20:57:10 +00:00
|
|
|
|
2014-05-03 22:57:43 +00:00
|
|
|
MemoryTracker::~MemoryTracker()
|
|
|
|
{
|
2020-04-19 21:43:06 +00:00
|
|
|
if ((level == VariableContext::Process || level == VariableContext::User) && peak)
|
2017-10-27 17:24:33 +00:00
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
logPeakMemoryUsage();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
/// Exception in Logger, intentionally swallow.
|
|
|
|
}
|
|
|
|
}
|
2014-05-03 22:57:43 +00:00
|
|
|
}
|
|
|
|
|
2015-12-30 15:39:11 +00:00
|
|
|
|
|
|
|
void MemoryTracker::logPeakMemoryUsage() const
|
|
|
|
{
|
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"),
|
|
|
|
"Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak));
|
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
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-05-08 15:54:51 +00:00
|
|
|
void MemoryTracker::alloc(Int64 size)
|
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);
|
|
|
|
|
2020-12-05 08:23:12 +00:00
|
|
|
if (BlockerInThread::isBlocked(level))
|
|
|
|
{
|
|
|
|
/// Since the BlockerInThread should respect the level, we should go to the next parent.
|
|
|
|
if (auto * loaded_next = parent.load(std::memory_order_relaxed))
|
|
|
|
loaded_next->alloc(size);
|
2018-02-01 17:55:08 +00:00
|
|
|
return;
|
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.
|
|
|
|
*/
|
|
|
|
Int64 will_be = size + amount.fetch_add(size, 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);
|
|
|
|
if (metric_loaded != CurrentMetrics::end())
|
|
|
|
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
|
|
|
|
2020-10-21 00:31:12 +00:00
|
|
|
/// Cap the limit to the total_memory_tracker, since it may include some drift
|
|
|
|
/// for user-level memory tracker.
|
2020-07-06 22:43:39 +00:00
|
|
|
///
|
2020-08-08 00:47:03 +00:00
|
|
|
/// And since total_memory_tracker is reset to the process resident
|
2020-07-06 22:43:39 +00:00
|
|
|
/// memory peridically (in AsynchronousMetrics::update()), any limit can be
|
|
|
|
/// capped to it, to avoid possible drift.
|
2020-10-21 00:31:12 +00:00
|
|
|
if (unlikely(current_hard_limit
|
|
|
|
&& will_be > current_hard_limit
|
|
|
|
&& level == VariableContext::User))
|
2020-07-06 22:43:39 +00:00
|
|
|
{
|
2020-07-07 08:21:00 +00:00
|
|
|
Int64 total_amount = total_memory_tracker.get();
|
|
|
|
if (amount > total_amount)
|
|
|
|
{
|
|
|
|
set(total_amount);
|
|
|
|
will_be = size + total_amount;
|
|
|
|
}
|
2020-07-06 22:43:39 +00:00
|
|
|
}
|
|
|
|
|
2021-01-12 14:34:50 +00:00
|
|
|
#ifdef MEMORY_TRACKER_DEBUG_CHECKS
|
|
|
|
if (unlikely(_memory_tracker_always_throw_logical_error_on_allocation))
|
|
|
|
{
|
|
|
|
_memory_tracker_always_throw_logical_error_on_allocation = false;
|
|
|
|
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Memory tracker: allocations not allowed.");
|
|
|
|
}
|
|
|
|
#endif
|
|
|
|
|
2020-04-30 13:25:17 +00:00
|
|
|
std::bernoulli_distribution fault(fault_probability);
|
2020-12-05 08:23:12 +00:00
|
|
|
if (unlikely(fault_probability && fault(thread_local_rng)) && memoryTrackerCanThrow(level, true))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-07-12 17:06:02 +00:00
|
|
|
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
2021-01-15 19:39:10 +00:00
|
|
|
BlockerInThread untrack_lock(VariableContext::Global);
|
2019-07-12 17:06:02 +00:00
|
|
|
|
2020-09-09 16:59:38 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded);
|
2020-11-10 18:22:26 +00:00
|
|
|
const auto * description = description_ptr.load(std::memory_order_relaxed);
|
2020-10-21 00:31:12 +00:00
|
|
|
amount.fetch_sub(size, std::memory_order_relaxed);
|
2020-11-10 18:22:26 +00:00
|
|
|
throw DB::Exception(DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
|
2020-11-11 13:27:54 +00:00
|
|
|
"Memory tracker{}{}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}",
|
|
|
|
description ? " " : "", description ? description : "",
|
|
|
|
formatReadableSizeWithBinarySuffix(will_be),
|
2020-11-10 18:22:26 +00:00
|
|
|
size, formatReadableSizeWithBinarySuffix(current_hard_limit));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-01-22 15:20:19 +00:00
|
|
|
if (unlikely(current_profiler_limit && will_be > current_profiler_limit))
|
2020-01-21 13:53:30 +00:00
|
|
|
{
|
2021-01-15 19:39:10 +00:00
|
|
|
BlockerInThread untrack_lock(VariableContext::Global);
|
2020-03-03 00:24:44 +00:00
|
|
|
DB::TraceCollector::collect(DB::TraceType::Memory, StackTrace(), size);
|
|
|
|
setOrRaiseProfilerLimit((will_be + profiler_step - 1) / profiler_step * profiler_step);
|
2020-01-21 13:53:30 +00:00
|
|
|
}
|
|
|
|
|
2020-04-30 13:25:17 +00:00
|
|
|
std::bernoulli_distribution sample(sample_probability);
|
|
|
|
if (unlikely(sample_probability && sample(thread_local_rng)))
|
|
|
|
{
|
2021-01-15 19:39:10 +00:00
|
|
|
BlockerInThread untrack_lock(VariableContext::Global);
|
2020-04-30 13:25:17 +00:00
|
|
|
DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), size);
|
|
|
|
}
|
|
|
|
|
2020-12-05 08:23:12 +00:00
|
|
|
if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow(level, false))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-07-12 17:06:02 +00:00
|
|
|
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
2021-01-15 19:39:10 +00:00
|
|
|
BlockerInThread untrack_lock(VariableContext::Global);
|
2019-07-12 17:06:02 +00:00
|
|
|
|
2020-09-09 16:59:38 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded);
|
2020-11-10 18:22:26 +00:00
|
|
|
const auto * description = description_ptr.load(std::memory_order_relaxed);
|
2020-10-21 00:31:12 +00:00
|
|
|
amount.fetch_sub(size, std::memory_order_relaxed);
|
2020-11-10 18:22:26 +00:00
|
|
|
throw DB::Exception(DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
|
2020-11-11 13:27:54 +00:00
|
|
|
"Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}",
|
|
|
|
description ? " " : "", description ? description : "",
|
|
|
|
formatReadableSizeWithBinarySuffix(will_be),
|
2020-11-10 18:22:26 +00:00
|
|
|
size, formatReadableSizeWithBinarySuffix(current_hard_limit));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-04-19 21:43:06 +00:00
|
|
|
updatePeak(will_be);
|
|
|
|
|
2020-04-22 00:29:38 +00:00
|
|
|
if (auto * loaded_next = parent.load(std::memory_order_relaxed))
|
2020-04-19 21:43:06 +00:00
|
|
|
loaded_next->alloc(size);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void MemoryTracker::updatePeak(Int64 will_be)
|
|
|
|
{
|
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.
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
peak.store(will_be, std::memory_order_relaxed);
|
|
|
|
|
2020-04-19 21:43:06 +00:00
|
|
|
if ((level == VariableContext::Process || level == VariableContext::Global)
|
|
|
|
&& 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);
|
|
|
|
}
|
2015-12-30 15:39:11 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void MemoryTracker::free(Int64 size)
|
|
|
|
{
|
2020-12-05 08:23:12 +00:00
|
|
|
if (BlockerInThread::isBlocked(level))
|
2021-01-05 23:42:34 +00:00
|
|
|
{
|
|
|
|
/// Since the BlockerInThread should respect the level, we should go to the next parent.
|
|
|
|
if (auto * loaded_next = parent.load(std::memory_order_relaxed))
|
|
|
|
loaded_next->free(size);
|
2018-02-01 17:55:08 +00:00
|
|
|
return;
|
2021-01-05 23:42:34 +00:00
|
|
|
}
|
2018-02-01 17:55:08 +00:00
|
|
|
|
2020-04-30 13:25:17 +00:00
|
|
|
std::bernoulli_distribution sample(sample_probability);
|
|
|
|
if (unlikely(sample_probability && sample(thread_local_rng)))
|
|
|
|
{
|
2021-01-15 19:39:10 +00:00
|
|
|
BlockerInThread untrack_lock(VariableContext::Global);
|
2020-04-30 13:25:17 +00:00
|
|
|
DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), -size);
|
|
|
|
}
|
|
|
|
|
2020-10-18 10:51:59 +00:00
|
|
|
Int64 accounted_size = size;
|
2018-06-09 15:29:08 +00:00
|
|
|
if (level == VariableContext::Thread)
|
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
|
|
|
}
|
2015-12-30 15:39:11 +00:00
|
|
|
|
2020-04-22 00:29:38 +00:00
|
|
|
if (auto * loaded_next = parent.load(std::memory_order_relaxed))
|
2017-09-09 04:06:54 +00:00
|
|
|
loaded_next->free(size);
|
2018-10-08 05:30:03 +00:00
|
|
|
|
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);
|
2015-12-30 15:39:11 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
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);
|
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
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-04-19 21:43:06 +00:00
|
|
|
void MemoryTracker::set(Int64 to)
|
|
|
|
{
|
|
|
|
amount.store(to, std::memory_order_relaxed);
|
|
|
|
updatePeak(to);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
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);
|
|
|
|
while (old_value < value && !hard_limit.compare_exchange_weak(old_value, value))
|
|
|
|
;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
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);
|
|
|
|
while (old_value < value && !profiler_limit.compare_exchange_weak(old_value, value))
|
2017-04-01 07:20:54 +00:00
|
|
|
;
|
2014-05-03 22:57:43 +00:00
|
|
|
}
|