2014-05-03 22:57:43 +00:00
|
|
|
#pragma once
|
|
|
|
|
2016-07-31 03:53:16 +00:00
|
|
|
#include <atomic>
|
2022-05-27 16:30:29 +00:00
|
|
|
#include <chrono>
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/types.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/CurrentMetrics.h>
|
2018-06-20 17:49:52 +00:00
|
|
|
#include <Common/VariableContext.h>
|
2022-07-07 11:54:41 +00:00
|
|
|
#include <Common/AllocationTrace.h>
|
2016-12-20 02:29:35 +00:00
|
|
|
|
2021-01-13 12:19:41 +00:00
|
|
|
#if !defined(NDEBUG)
|
2021-01-12 14:34:50 +00:00
|
|
|
#define MEMORY_TRACKER_DEBUG_CHECKS
|
|
|
|
#endif
|
|
|
|
|
2021-01-13 12:19:41 +00:00
|
|
|
/// DENY_ALLOCATIONS_IN_SCOPE macro makes MemoryTracker throw LOGICAL_ERROR on any allocation attempt
|
|
|
|
/// until the end of the scope. It's useful to ensure that no allocations happen in signal handlers and
|
|
|
|
/// outside of try/catch block of thread functions. ALLOW_ALLOCATIONS_IN_SCOPE cancels effect of
|
|
|
|
/// DENY_ALLOCATIONS_IN_SCOPE in the inner scope. In Release builds these macros do nothing.
|
2021-01-12 14:34:50 +00:00
|
|
|
#ifdef MEMORY_TRACKER_DEBUG_CHECKS
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/scope_guard.h>
|
2021-08-27 08:21:19 +00:00
|
|
|
extern thread_local bool memory_tracker_always_throw_logical_error_on_allocation;
|
2022-03-11 21:47:28 +00:00
|
|
|
|
|
|
|
/// NOLINTNEXTLINE
|
2021-01-12 14:34:50 +00:00
|
|
|
#define ALLOCATIONS_IN_SCOPE_IMPL_CONCAT(n, val) \
|
2021-08-27 08:21:19 +00:00
|
|
|
bool _allocations_flag_prev_val##n = memory_tracker_always_throw_logical_error_on_allocation; \
|
|
|
|
memory_tracker_always_throw_logical_error_on_allocation = val; \
|
|
|
|
SCOPE_EXIT({ memory_tracker_always_throw_logical_error_on_allocation = _allocations_flag_prev_val##n; })
|
2022-03-11 21:47:28 +00:00
|
|
|
|
|
|
|
/// NOLINTNEXTLINE
|
2021-01-12 14:34:50 +00:00
|
|
|
#define ALLOCATIONS_IN_SCOPE_IMPL(n, val) ALLOCATIONS_IN_SCOPE_IMPL_CONCAT(n, val)
|
2022-03-11 21:47:28 +00:00
|
|
|
|
|
|
|
/// NOLINTNEXTLINE
|
2021-01-12 14:34:50 +00:00
|
|
|
#define DENY_ALLOCATIONS_IN_SCOPE ALLOCATIONS_IN_SCOPE_IMPL(__LINE__, true)
|
2022-03-11 21:47:28 +00:00
|
|
|
|
|
|
|
/// NOLINTNEXTLINE
|
2021-01-12 14:34:50 +00:00
|
|
|
#define ALLOW_ALLOCATIONS_IN_SCOPE ALLOCATIONS_IN_SCOPE_IMPL(__LINE__, false)
|
|
|
|
#else
|
|
|
|
#define DENY_ALLOCATIONS_IN_SCOPE static_assert(true)
|
|
|
|
#define ALLOW_ALLOCATIONS_IN_SCOPE static_assert(true)
|
|
|
|
#endif
|
2016-12-20 02:29:35 +00:00
|
|
|
|
2021-10-22 12:56:09 +00:00
|
|
|
struct OvercommitRatio;
|
|
|
|
struct OvercommitTracker;
|
|
|
|
|
2016-12-23 20:23:46 +00:00
|
|
|
/** Tracks memory consumption.
|
|
|
|
* It throws an exception if amount of consumed memory become greater than certain limit.
|
|
|
|
* The same memory tracker could be simultaneously used in different threads.
|
2022-01-10 19:39:10 +00:00
|
|
|
*
|
|
|
|
* @see LockMemoryExceptionInThread
|
|
|
|
* @see MemoryTrackerBlockerInThread
|
2014-05-03 22:57:43 +00:00
|
|
|
*/
|
|
|
|
class MemoryTracker
|
|
|
|
{
|
2020-04-19 21:43:06 +00:00
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
std::atomic<Int64> amount {0};
|
|
|
|
std::atomic<Int64> peak {0};
|
2021-10-22 12:56:09 +00:00
|
|
|
std::atomic<Int64> soft_limit {0};
|
2020-01-21 13:53:30 +00:00
|
|
|
std::atomic<Int64> hard_limit {0};
|
2020-01-22 15:20:19 +00:00
|
|
|
std::atomic<Int64> profiler_limit {0};
|
2020-01-21 13:53:30 +00:00
|
|
|
|
2020-01-22 15:20:19 +00:00
|
|
|
Int64 profiler_step = 0;
|
2014-05-03 22:57:43 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// To test exception safety of calling code, memory tracker throws an exception on each memory allocation with specified probability.
|
|
|
|
double fault_probability = 0;
|
2015-12-23 07:39:28 +00:00
|
|
|
|
2020-04-30 13:25:17 +00:00
|
|
|
/// To randomly sample allocations and deallocations in trace_log.
|
|
|
|
double sample_probability = 0;
|
2022-07-07 11:54:41 +00:00
|
|
|
/// Sample probability for the whole chain.
|
|
|
|
std::atomic<double> total_sample_probability = 0;
|
2020-04-30 13:25:17 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy).
|
|
|
|
/// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker.
|
2018-02-01 17:55:08 +00:00
|
|
|
std::atomic<MemoryTracker *> parent {};
|
2015-12-30 15:39:11 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// You could specify custom metric to track memory usage.
|
2021-01-17 12:40:14 +00:00
|
|
|
std::atomic<CurrentMetrics::Metric> metric = CurrentMetrics::end();
|
2016-12-20 02:29:35 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// This description will be used as prefix into log messages (if isn't nullptr)
|
2020-05-28 12:30:42 +00:00
|
|
|
std::atomic<const char *> description_ptr = nullptr;
|
2015-12-30 15:39:11 +00:00
|
|
|
|
2022-05-27 16:30:29 +00:00
|
|
|
std::atomic<std::chrono::microseconds> max_wait_time;
|
|
|
|
|
2022-05-02 22:45:13 +00:00
|
|
|
std::atomic<OvercommitTracker *> overcommit_tracker = nullptr;
|
2021-10-22 12:56:09 +00:00
|
|
|
|
2021-10-07 18:13:56 +00:00
|
|
|
bool updatePeak(Int64 will_be, bool log_memory_usage);
|
2020-04-19 21:43:06 +00:00
|
|
|
void logMemoryUsage(Int64 current) const;
|
|
|
|
|
2021-10-07 07:51:03 +00:00
|
|
|
void setOrRaiseProfilerLimit(Int64 value);
|
|
|
|
|
2022-07-07 11:54:41 +00:00
|
|
|
void updateTotalSampleProbability(MemoryTracker * parent_elem);
|
|
|
|
|
2014-05-03 22:57:43 +00:00
|
|
|
public:
|
2021-09-17 16:47:54 +00:00
|
|
|
|
|
|
|
static constexpr auto USAGE_EVENT_NAME = "MemoryTrackerUsage";
|
|
|
|
|
2021-09-08 00:21:21 +00:00
|
|
|
explicit MemoryTracker(VariableContext level_ = VariableContext::Thread);
|
|
|
|
explicit MemoryTracker(MemoryTracker * parent_, VariableContext level_ = VariableContext::Thread);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
~MemoryTracker();
|
|
|
|
|
2018-06-09 15:29:08 +00:00
|
|
|
VariableContext level;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/** Call the following functions before calling of corresponding operations with memory allocators.
|
|
|
|
*/
|
2022-07-07 11:54:41 +00:00
|
|
|
[[nodiscard]] AllocationTrace alloc(Int64 size);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-07-07 11:54:41 +00:00
|
|
|
[[nodiscard]] AllocationTrace allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr);
|
2021-05-25 09:50:55 +00:00
|
|
|
|
2022-07-07 11:54:41 +00:00
|
|
|
[[nodiscard]] AllocationTrace allocNoThrow(Int64 size);
|
2021-05-25 09:50:55 +00:00
|
|
|
|
2022-07-07 11:54:41 +00:00
|
|
|
[[nodiscard]] AllocationTrace realloc(Int64 old_size, Int64 new_size);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/** This function should be called after memory deallocation.
|
|
|
|
*/
|
2022-07-07 11:54:41 +00:00
|
|
|
[[nodiscard]] AllocationTrace free(Int64 size);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
Int64 get() const
|
|
|
|
{
|
|
|
|
return amount.load(std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
|
|
|
Int64 getPeak() const
|
|
|
|
{
|
|
|
|
return peak.load(std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
2021-10-22 12:56:09 +00:00
|
|
|
void setSoftLimit(Int64 value);
|
2021-01-16 15:33:34 +00:00
|
|
|
void setHardLimit(Int64 value);
|
|
|
|
|
2022-02-18 09:04:36 +00:00
|
|
|
Int64 getHardLimit() const
|
|
|
|
{
|
|
|
|
return hard_limit.load(std::memory_order_relaxed);
|
|
|
|
}
|
2021-10-26 12:32:17 +00:00
|
|
|
Int64 getSoftLimit() const
|
|
|
|
{
|
|
|
|
return soft_limit.load(std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/** Set limit if it was not set.
|
|
|
|
* Otherwise, set limit to new value, if new value is greater than previous limit.
|
|
|
|
*/
|
2020-01-21 13:53:30 +00:00
|
|
|
void setOrRaiseHardLimit(Int64 value);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
void setFaultProbability(double value)
|
|
|
|
{
|
|
|
|
fault_probability = value;
|
|
|
|
}
|
|
|
|
|
2020-04-30 13:25:17 +00:00
|
|
|
void setSampleProbability(double value)
|
|
|
|
{
|
|
|
|
sample_probability = value;
|
2022-07-07 11:54:41 +00:00
|
|
|
updateTotalSampleProbability(parent.load());
|
|
|
|
}
|
|
|
|
|
|
|
|
double getSampleProbabilityTotal()
|
|
|
|
{
|
|
|
|
return total_sample_probability.load(std::memory_order_relaxed);
|
2020-04-30 13:25:17 +00:00
|
|
|
}
|
|
|
|
|
2020-01-22 15:20:19 +00:00
|
|
|
void setProfilerStep(Int64 value)
|
2020-01-21 13:53:30 +00:00
|
|
|
{
|
2020-01-22 15:20:19 +00:00
|
|
|
profiler_step = value;
|
2021-10-07 07:51:03 +00:00
|
|
|
setOrRaiseProfilerLimit(value);
|
2020-01-21 13:53:30 +00:00
|
|
|
}
|
|
|
|
|
2017-09-09 04:06:54 +00:00
|
|
|
/// next should be changed only once: from nullptr to some value.
|
2018-02-01 17:55:08 +00:00
|
|
|
/// NOTE: It is not true in MergeListElement
|
|
|
|
void setParent(MemoryTracker * elem)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-02-01 17:55:08 +00:00
|
|
|
parent.store(elem, std::memory_order_relaxed);
|
2022-07-07 11:54:41 +00:00
|
|
|
updateTotalSampleProbability(elem);
|
2018-02-01 17:55:08 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
MemoryTracker * getParent()
|
|
|
|
{
|
|
|
|
return parent.load(std::memory_order_relaxed);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// The memory consumption could be shown in realtime via CurrentMetrics counter
|
|
|
|
void setMetric(CurrentMetrics::Metric metric_)
|
|
|
|
{
|
2021-01-17 12:40:14 +00:00
|
|
|
metric.store(metric_, std::memory_order_relaxed);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2021-08-30 11:04:59 +00:00
|
|
|
CurrentMetrics::Metric getMetric()
|
|
|
|
{
|
|
|
|
return metric.load(std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
2020-05-28 12:30:42 +00:00
|
|
|
void setDescription(const char * description)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-05-28 12:30:42 +00:00
|
|
|
description_ptr.store(description, std::memory_order_relaxed);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2021-10-22 12:56:09 +00:00
|
|
|
OvercommitRatio getOvercommitRatio();
|
2021-10-26 12:32:17 +00:00
|
|
|
OvercommitRatio getOvercommitRatio(Int64 limit);
|
2021-10-22 12:56:09 +00:00
|
|
|
|
2022-05-27 16:30:29 +00:00
|
|
|
std::chrono::microseconds getOvercommitWaitingTime()
|
|
|
|
{
|
|
|
|
return max_wait_time.load(std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
|
|
|
void setOvercommitWaitingTime(UInt64 wait_time);
|
|
|
|
|
2021-10-22 12:56:09 +00:00
|
|
|
void setOvercommitTracker(OvercommitTracker * tracker) noexcept
|
|
|
|
{
|
2022-05-02 22:45:13 +00:00
|
|
|
overcommit_tracker.store(tracker, std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
|
|
|
void resetOvercommitTracker() noexcept
|
|
|
|
{
|
|
|
|
overcommit_tracker.store(nullptr, std::memory_order_relaxed);
|
2021-10-22 12:56:09 +00:00
|
|
|
}
|
|
|
|
|
2018-06-09 15:29:08 +00:00
|
|
|
/// Reset the accumulated data
|
|
|
|
void resetCounters();
|
|
|
|
|
2022-05-02 22:45:13 +00:00
|
|
|
/// Reset the accumulated data.
|
2017-04-01 07:20:54 +00:00
|
|
|
void reset();
|
|
|
|
|
2020-04-19 21:43:06 +00:00
|
|
|
/// Reset current counter to a new value.
|
|
|
|
void set(Int64 to);
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Prints info about peak memory consumption into log.
|
|
|
|
void logPeakMemoryUsage() const;
|
2018-02-01 17:55:08 +00:00
|
|
|
};
|
2014-05-03 22:57:43 +00:00
|
|
|
|
2020-04-19 21:43:06 +00:00
|
|
|
extern MemoryTracker total_memory_tracker;
|