Impala
Impalaistheopensource,nativeanalyticdatabaseforApacheHadoop.
|
This class is thread-safe. More...
#include <mem-tracker.h>
Public Types | |
typedef boost::function< void()> | GcFunction |
Signature for function that can be called to free some memory after limit is reached. More... | |
Public Member Functions | |
MemTracker (int64_t byte_limit=-1, int64_t rm_reserved_limit=-1, const std::string &label=std::string(), MemTracker *parent=NULL, bool log_usage_if_zero=true) | |
MemTracker (RuntimeProfile *profile, int64_t byte_limit, int64_t rm_reserved_limit=-1, const std::string &label=std::string(), MemTracker *parent=NULL) | |
MemTracker (UIntGauge *consumption_metric, int64_t byte_limit=-1, int64_t rm_reserved_limit=-1, const std::string &label=std::string()) | |
~MemTracker () | |
void | UnregisterFromParent () |
Removes this tracker from parent_->child_trackers_. More... | |
int64_t | effective_limit () const |
Returns the minimum of limit and rm_reserved_limit. More... | |
void | Consume (int64_t bytes) |
Increases consumption of this tracker and its ancestors by 'bytes'. More... | |
void | ConsumeLocal (int64_t bytes, MemTracker *end_tracker) |
void | ReleaseLocal (int64_t bytes, MemTracker *end_tracker) |
bool | TryConsume (int64_t bytes) |
void | Release (int64_t bytes) |
Decreases consumption of this tracker and its ancestors by 'bytes'. More... | |
bool | AnyLimitExceeded () |
bool | LimitExceeded () |
int64_t | SpareCapacity () const |
int64_t | limit () const |
bool | has_limit () const |
const std::string & | label () const |
int64_t | lowest_limit () const |
int64_t | consumption () const |
Returns the memory consumed in bytes. More... | |
int64_t | peak_consumption () const |
MemTracker * | parent () const |
void | AddGcFunction (GcFunction f) |
void | RegisterMetrics (MetricGroup *metrics, const std::string &prefix) |
std::string | LogUsage (const std::string &prefix="") const |
Logs the usage of this tracker and all of its children (recursively). More... | |
void | EnableLogging (bool enable, bool log_stack) |
Static Public Member Functions | |
static boost::shared_ptr < MemTracker > | GetQueryMemTracker (const TUniqueId &id, int64_t byte_limit, int64_t rm_reserved_limit, MemTracker *parent, QueryResourceMgr *res_mgr) |
static MemTracker * | GetRequestPoolMemTracker (const std::string &pool_name, MemTracker *parent) |
Static Public Attributes | |
static const std::string | COUNTER_NAME = "PeakMemoryUsage" |
Private Types | |
typedef boost::unordered_map < TUniqueId, boost::weak_ptr < MemTracker > > | RequestTrackersMap |
typedef boost::unordered_map < std::string, MemTracker * > | PoolTrackersMap |
Private Member Functions | |
bool | CheckLimitExceeded () const |
bool | GcMemory (int64_t max_consumption) |
void | GcTcmalloc () |
void | SetQueryResourceMgr (QueryResourceMgr *context) |
Set the resource mgr to allow expansion of limits (if NULL, no expansion is possible) More... | |
void | Init () |
void | AddChildTracker (MemTracker *tracker) |
Adds tracker to child_trackers_. More... | |
void | LogUpdate (bool is_consume, int64_t bytes) const |
Logs the stack of the current consume/release. Used for debugging only. More... | |
bool | ExpandRmReservation (int64_t bytes) |
Static Private Member Functions | |
static std::string | LogUsage (const std::string &prefix, const std::list< MemTracker * > &trackers) |
Static Private Attributes | |
static const int64_t | GC_RELEASE_SIZE = 128 * 1024L * 1024L |
static AtomicInt< int64_t > | released_memory_since_gc_ |
static boost::mutex | static_mem_trackers_lock_ |
Protects request_to_mem_trackers_ and pool_to_mem_trackers_. More... | |
static RequestTrackersMap | request_to_mem_trackers_ |
static PoolTrackersMap | pool_to_mem_trackers_ |
This class is thread-safe.
A MemTracker tracks memory consumption; it contains an optional limit and can be arranged into a tree structure such that the consumption tracked by a MemTracker is also tracked by its ancestors. By default, memory consumption is tracked via calls to Consume()/Release(), either to the tracker itself or to one of its descendents. Alternatively, a consumption metric can specified, and then the metric's value is used as the consumption rather than the tally maintained by Consume() and Release(). A tcmalloc metric is used to track process memory consumption, since the process memory usage may be higher than the computed total memory (tcmalloc does not release deallocated memory immediately). GcFunctions can be attached to a MemTracker in order to free up memory if the limit is reached. If LimitExceeded() is called and the limit is exceeded, it will first call the GcFunctions to try to free memory and recheck the limit. For example, the process tracker has a GcFunction that releases any unused memory still held by tcmalloc, so this will be called before the process limit is reported as exceeded. GcFunctions are called in the order they are added, so expensive functions should be added last.
Definition at line 61 of file mem-tracker.h.
typedef boost::function<void ()> impala::MemTracker::GcFunction |
Signature for function that can be called to free some memory after limit is reached.
Definition at line 308 of file mem-tracker.h.
|
private |
All per-request pool MemTracker objects. It is assumed that request pools will live for the entire duration of the process lifetime.
Definition at line 393 of file mem-tracker.h.
|
private |
All per-request MemTracker objects that are in use. For memory management, this map contains only weak ptrs. MemTrackers that are handed out via GetQueryMemTracker() are shared ptrs. When all the shared ptrs are no longer referenced, the MemTracker d'tor will be called at which point the weak ptr will be removed from the map.
Definition at line 388 of file mem-tracker.h.
impala::MemTracker::MemTracker | ( | int64_t | byte_limit = -1 , |
int64_t | rm_reserved_limit = -1 , |
||
const std::string & | label = std::string() , |
||
MemTracker * | parent = NULL , |
||
bool | log_usage_if_zero = true |
||
) |
'byte_limit' < 0 means no limit 'label' is the label used in the usage string (LogUsage()) If 'log_usage_if_zero' is false, this tracker (and its children) will not be included in LogUsage() output if consumption is 0.
Referenced by GetQueryMemTracker(), and GetRequestPoolMemTracker().
impala::MemTracker::MemTracker | ( | RuntimeProfile * | profile, |
int64_t | byte_limit, | ||
int64_t | rm_reserved_limit = -1 , |
||
const std::string & | label = std::string() , |
||
MemTracker * | parent = NULL |
||
) |
C'tor for tracker for which consumption counter is created as part of a profile. The counter is created with name COUNTER_NAME.
Definition at line 67 of file mem-tracker.cc.
References AddChildTracker(), Init(), and parent_.
impala::MemTracker::MemTracker | ( | UIntGauge * | consumption_metric, |
int64_t | byte_limit = -1 , |
||
int64_t | rm_reserved_limit = -1 , |
||
const std::string & | label = std::string() |
||
) |
C'tor for tracker that uses consumption_metric as the consumption value. Consume()/Release() can still be called. This is used for the process tracker.
impala::MemTracker::~MemTracker | ( | ) |
Definition at line 193 of file mem-tracker.cc.
References auto_unregister_, pool_name_, pool_to_mem_trackers_, query_id_, request_to_mem_trackers_, static_mem_trackers_lock_, and UnregisterFromParent().
|
private |
Adds tracker to child_trackers_.
Definition at line 122 of file mem-tracker.cc.
References child_tracker_it_, child_trackers_, child_trackers_lock_, and tracker.
Referenced by MemTracker().
|
inline |
Add a function 'f' to be called if the limit is reached. 'f' does not need to be thread-safe as long as it is added to only one MemTracker. Note that 'f' must be valid for the lifetime of this MemTracker.
Definition at line 313 of file mem-tracker.h.
References gc_functions_.
Referenced by impala::DiskIoMgr::Init(), and impala::TEST().
|
inline |
Returns true if a valid limit of this tracker or one of its ancestors is exceeded.
Definition at line 246 of file mem-tracker.h.
References limit_trackers_, and tracker.
Referenced by impala::TEST().
|
inlineprivate |
Definition at line 330 of file mem-tracker.h.
References consumption(), and limit_.
Referenced by LimitExceeded().
|
inline |
Increases consumption of this tracker and its ancestors by 'bytes'.
Definition at line 118 of file mem-tracker.h.
References all_trackers_, consumption_, consumption_metric_, enable_logging_, LogUpdate(), parent_, Release(), impala::RuntimeProfile::HighWaterMarkCounter::Set(), tracker, and UNLIKELY.
Referenced by impala::MemPool::AcquireData(), impala::MemPool::FindChunk(), impala::DiskIoMgr::GetFreeBuffer(), impala::HdfsSequenceTableWriter::HdfsSequenceTableWriter(), impala::HdfsTextTableWriter::Init(), impala::OldHashTable::OldHashTable(), Release(), impala::DiskIoMgr::BufferDescriptor::SetMemTracker(), and impala::TEST().
|
inline |
Increases/Decreases the consumption of this tracker and the ancestors up to (but not including) end_tracker. This is useful if we want to move tracking between trackers that share a common (i.e. end_tracker) ancestor. This happens when we want to update tracking on a particular mem tracker but the consumption against the limit recorded in one of its ancestors already happened.
Definition at line 145 of file mem-tracker.h.
References all_trackers_, consumption_metric_, enable_logging_, has_limit(), LogUpdate(), and UNLIKELY.
Referenced by impala::BufferedBlockMgr::ConsumeMemory(), and ReleaseLocal().
|
inline |
Returns the memory consumed in bytes.
Definition at line 298 of file mem-tracker.h.
References consumption_, and impala::RuntimeProfile::HighWaterMarkCounter::current_value().
Referenced by CheckLimitExceeded(), impala::HdfsScanNode::EnoughMemoryForScannerThread(), GcMemory(), LimitExceeded(), LogUpdate(), MemUsageHandler(), impala::PlanFragmentExecutor::Prepare(), impala::TEST(), impala::TEST_F(), and impala::AdmissionController::UpdateLocalMemUsage().
|
inline |
Returns the minimum of limit and rm_reserved_limit.
Definition at line 110 of file mem-tracker.h.
References limit_, and rm_reserved_limit_.
Referenced by TryConsume().
Definition at line 322 of file mem-tracker.h.
References enable_logging_, and log_stack_.
|
private |
Try to expand the limit (by asking the resource broker for more memory) by at least 'bytes'. Returns false if not possible, true if the request succeeded. May allocate more memory than was requested.
Definition at line 294 of file mem-tracker.cc.
References all_trackers_, consumption_, impala::QueryResourceMgr::CreateExpansionRequest(), impala::RuntimeProfile::HighWaterMarkCounter::current_value(), impala::ResourceBroker::Expand(), impala::Status::GetDetail(), impala::ExecEnv::GetInstance(), limit_, impala::Status::ok(), impala::PrettyPrinter::Print(), query_resource_mgr_, resource_acquisition_lock_, impala::ExecEnv::resource_broker(), rm_reserved_limit_, and tracker.
Referenced by TryConsume().
|
private |
If consumption is higher than max_consumption, attempts to free memory by calling any added GC functions. Returns true if max_consumption is still exceeded. Takes gc_lock. Updates metrics if initialized.
Definition at line 263 of file mem-tracker.cc.
References bytes_freed_by_last_gc_metric_, consumption(), consumption_, consumption_metric_, gc_functions_, gc_lock_, num_gcs_metric_, and impala::RuntimeProfile::HighWaterMarkCounter::Set().
Referenced by LimitExceeded(), and TryConsume().
|
private |
Called when the total release memory is larger than GC_RELEASE_SIZE. TcMalloc holds onto released memory and very slowly (if ever) releases it back to the OS. This is problematic since it is memory we are not constantly tracking which can cause us to go way over mem limits.
Definition at line 285 of file mem-tracker.cc.
References released_memory_since_gc_.
Referenced by Release().
|
static |
Returns a MemTracker object for query 'id'. Calling this with the same id will return the same MemTracker object. An example of how this is used is to pass it the same query id for all fragments of that query running on this machine. This way, we have per-query limits rather than per-fragment. The first time this is called for an id, a new MemTracker object is created with 'parent' as the parent tracker. byte_limit and parent must be the same for all GetMemTracker() calls with the same id.
Definition at line 156 of file mem-tracker.cc.
References MemTracker(), parent_, impala::MemInfo::physical_mem(), impala::PrettyPrinter::Print(), request_to_mem_trackers_, static_mem_trackers_lock_, tracker, and VLOG_QUERY.
Referenced by impala::RuntimeState::InitMemTrackers().
|
static |
Returns a MemTracker object for request pool 'pool_name'. Calling this with the same 'pool_name' will return the same MemTracker object. This is used to track the local memory usage of all requests executing in this pool. The first time this is called for a pool, a new MemTracker object is created with the parent tracker if it is not NULL. If the parent is NULL, no new tracker will be created and NULL is returned. There is no explicit per-pool byte_limit set at any particular impalad, so newly created trackers will always have a limit of -1.
Definition at line 134 of file mem-tracker.cc.
References auto_unregister_, MemTracker(), pool_name_, pool_to_mem_trackers_, impala::REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT, static_mem_trackers_lock_, and tracker.
Referenced by impala::RuntimeState::InitMemTrackers(), and impala::AdmissionController::UpdateLocalMemUsage().
|
inline |
Definition at line 282 of file mem-tracker.h.
References limit_.
Referenced by ConsumeLocal(), Init(), lowest_limit(), and impala::TEST().
|
private |
Walks the MemTracker hierarchy and populates all_trackers_ and limit_trackers_
Definition at line 109 of file mem-tracker.cc.
References all_trackers_, has_limit(), limit_, limit_trackers_, parent_, rm_reserved_limit_, and tracker.
Referenced by MemTracker().
|
inline |
Definition at line 283 of file mem-tracker.h.
References label_.
Referenced by impala::RuntimeState::SetMemLimitExceeded().
|
inline |
Definition at line 281 of file mem-tracker.h.
References limit_.
Referenced by lowest_limit(), MemUsageHandler(), and TryConsume().
|
inline |
If this tracker has a limit, checks the limit and attempts to free up some memory if the limit is exceeded by calling any added GC functions. Returns true if the limit is exceeded after calling the GC functions. Returns false if there is no limit.
Definition at line 257 of file mem-tracker.h.
References bytes_over_limit_metric_, CheckLimitExceeded(), consumption(), GcMemory(), limit_, and UNLIKELY.
Referenced by impala::DiskIoMgr::GetNextRequestRange(), impala::OldHashTable::GrowNodeArray(), impala::HashTable::GrowNodeArray(), impala::HashTableTest::GrowTableTest(), impala::RuntimeState::SetMemLimitExceeded(), impala::TEST(), and impala::TEST_F().
|
private |
Logs the stack of the current consume/release. Used for debugging only.
Definition at line 255 of file mem-tracker.cc.
References consumption(), impala::GetStackTrace(), limit_, and log_stack_.
Referenced by Consume(), ConsumeLocal(), Release(), and TryConsume().
std::string impala::MemTracker::LogUsage | ( | const std::string & | prefix = "" | ) | const |
Logs the usage of this tracker and all of its children (recursively).
Referenced by impala::ExecNode::Close(), MemUsageHandler(), and impala::RuntimeState::SetMemLimitExceeded().
|
staticprivate |
|
inline |
Returns the lowest limit for this tracker and its ancestors. Returns -1 if there is no limit.
Definition at line 287 of file mem-tracker.h.
References has_limit(), limit(), and limit_trackers_.
|
inline |
Definition at line 305 of file mem-tracker.h.
References parent_.
|
inline |
Note that if consumption_ is based on consumption_metric_, this will the max value we've recorded in consumption(), not necessarily the highest value consumption_metric_ has ever reached.
Definition at line 303 of file mem-tracker.h.
References consumption_, and impala::RuntimeProfile::Counter::value().
void impala::MemTracker::RegisterMetrics | ( | MetricGroup * | metrics, |
const std::string & | prefix | ||
) |
Register this MemTracker's metrics. Each key will be of the form "<prefix>.<metric name>".
Definition at line 203 of file mem-tracker.cc.
References impala::MetricGroup::AddCounter(), impala::MetricGroup::AddGauge(), bytes_freed_by_last_gc_metric_, bytes_over_limit_metric_, and num_gcs_metric_.
|
inline |
Decreases consumption of this tracker and its ancestors by 'bytes'.
If a UDF calls FunctionContext::TrackAllocation() but allocates less than the reported amount, the subsequent call to FunctionContext::Free() may cause the process mem tracker to go negative until it is synced back to the tcmalloc metric. Don't blow up in this case. (Note that this doesn't affect non-process trackers since we can enforce that the reported memory usage is internally consistent.)
TODO: Release brokered memory?
Definition at line 209 of file mem-tracker.h.
References all_trackers_, Consume(), consumption_, consumption_metric_, enable_logging_, GC_RELEASE_SIZE, GcTcmalloc(), LogUpdate(), parent_, released_memory_since_gc_, impala::RuntimeProfile::HighWaterMarkCounter::Set(), tracker, UNLIKELY, and impala::AtomicInt< T >::UpdateAndFetch().
Referenced by impala::MemPool::AcquireData(), impala::HdfsTextTableWriter::Close(), impala::OldHashTable::Close(), Consume(), impala::BufferedBlockMgr::DeleteBlock(), impala::ImpalaServer::QueryExecState::FetchRowsInternal(), impala::MemPool::FreeAll(), impala::GcFunctionHelper::GcFunc(), impala::DiskIoMgr::GcIoBuffers(), MaintenanceThread(), impala::DiskIoMgr::ReturnFreeBuffer(), and impala::TEST().
|
inline |
Definition at line 155 of file mem-tracker.h.
References ConsumeLocal().
Referenced by impala::BufferedBlockMgr::ReleaseMemory().
|
inlineprivate |
Set the resource mgr to allow expansion of limits (if NULL, no expansion is possible)
Definition at line 344 of file mem-tracker.h.
References query_resource_mgr_.
|
inline |
Returns the maximum consumption that can be made without exceeding the limit on this tracker or any of its parents. Returns int64_t::max() if there are no limits and a negative value if any limit is already exceeded.
Definition at line 270 of file mem-tracker.h.
References limit_trackers_, and tracker.
Referenced by impala::HdfsScanNode::EnoughMemoryForScannerThread(), impala::PartitionedHashJoinNode::PrepareNextPartition(), and impala::DiskIoMgr::ReadRange().
|
inline |
Increases consumption of this tracker and its ancestors by 'bytes' only if they can all consume 'bytes'. If this brings any of them over, none of them are updated. Returns true if the try succeeded.
Definition at line 163 of file mem-tracker.h.
References impala::RuntimeProfile::HighWaterMarkCounter::Add(), all_trackers_, consumption_, consumption_metric_, effective_limit(), enable_logging_, ExpandRmReservation(), GcMemory(), limit(), limit_, LogUpdate(), impala::RuntimeProfile::HighWaterMarkCounter::Set(), tracker, impala::RuntimeProfile::HighWaterMarkCounter::TryAdd(), and UNLIKELY.
Referenced by impala::ImpalaServer::QueryExecState::FetchRowsInternal(), impala::MemPool::FindChunk(), impala::BufferedBlockMgr::GetNewBlock(), impala::OldHashTable::ResizeBuckets(), and impala::TEST().
void impala::MemTracker::UnregisterFromParent | ( | ) |
Removes this tracker from parent_->child_trackers_.
Definition at line 127 of file mem-tracker.cc.
References child_tracker_it_, child_trackers_, child_trackers_lock_, and parent_.
Referenced by ~MemTracker().
|
private |
Definition at line 425 of file mem-tracker.h.
Referenced by Consume(), ConsumeLocal(), ExpandRmReservation(), Init(), Release(), and TryConsume().
|
private |
If true, calls UnregisterFromParent() in the dtor. This is only used for the query wide trackers to remove it from the process mem tracker. The process tracker never gets deleted so it is safe to reference it in the dtor. The query tracker has lifetime shared by multiple plan fragments so it's hard to do cleanup another way.
Definition at line 445 of file mem-tracker.h.
Referenced by GetRequestPoolMemTracker(), and ~MemTracker().
|
private |
The number of bytes freed by the last round of calling the GcFunctions (-1 before any GCs are performed).
Definition at line 469 of file mem-tracker.h.
Referenced by GcMemory(), and RegisterMetrics().
|
private |
The number of bytes over the limit we were the last time LimitExceeded() was called and the limit was exceeded pre-GC. -1 if there is no limit or the limit was never exceeded.
Definition at line 474 of file mem-tracker.h.
Referenced by LimitExceeded(), and RegisterMetrics().
|
private |
Iterator into parent_->child_trackers_ for this object. Stored to have O(1) remove.
Definition at line 435 of file mem-tracker.h.
Referenced by AddChildTracker(), and UnregisterFromParent().
|
private |
Definition at line 431 of file mem-tracker.h.
Referenced by AddChildTracker(), and UnregisterFromParent().
|
mutableprivate |
All the child trackers of this tracker. Used for error reporting only. i.e., Updating a parent tracker does not update the children.
Definition at line 430 of file mem-tracker.h.
Referenced by AddChildTracker(), and UnregisterFromParent().
|
private |
in bytes; not owned
Definition at line 415 of file mem-tracker.h.
Referenced by Consume(), consumption(), ExpandRmReservation(), GcMemory(), peak_consumption(), Release(), and TryConsume().
|
private |
If non-NULL, used to measure consumption (in bytes) rather than the values provided to Consume()/Release(). Only used for the process tracker, thus parent_ should be NULL if consumption_metric_ is set.
Definition at line 423 of file mem-tracker.h.
Referenced by Consume(), ConsumeLocal(), GcMemory(), Release(), and TryConsume().
|
static |
Definition at line 327 of file mem-tracker.h.
|
private |
If true, logs to INFO every consume/release called. Used for debugging.
Definition at line 448 of file mem-tracker.h.
Referenced by Consume(), ConsumeLocal(), EnableLogging(), Release(), and TryConsume().
|
private |
Functions to call after the limit is reached to free memory.
Definition at line 438 of file mem-tracker.h.
Referenced by AddGcFunction(), and GcMemory().
|
private |
Lock to protect GcMemory(). This prevents many GCs from occurring at once.
Definition at line 378 of file mem-tracker.h.
Referenced by GcMemory().
|
staticprivate |
Size, in bytes, that is considered a large value for Release() (or Consume() with a negative value). If tcmalloc is used, this can trigger it to GC. A higher value will make us call into tcmalloc less often (and therefore more efficient). A lower value will mean our memory overhead is lower. TODO: this is a stopgap.
Definition at line 371 of file mem-tracker.h.
Referenced by Release().
|
private |
Definition at line 411 of file mem-tracker.h.
Referenced by label().
|
private |
Hard limit on memory consumption, in bytes. May not be exceeded. If limit_ == -1, there is no consumption limit.
Definition at line 404 of file mem-tracker.h.
Referenced by CheckLimitExceeded(), effective_limit(), ExpandRmReservation(), has_limit(), Init(), limit(), LimitExceeded(), LogUpdate(), and TryConsume().
|
private |
Definition at line 426 of file mem-tracker.h.
Referenced by AnyLimitExceeded(), Init(), lowest_limit(), and SpareCapacity().
|
private |
holds consumption_ counter if not tied to a profile
Definition at line 418 of file mem-tracker.h.
|
private |
If true, log the stack as well.
Definition at line 450 of file mem-tracker.h.
Referenced by EnableLogging(), and LogUpdate().
|
private |
If false, this tracker (and its children) will not be included in LogUsage() output if consumption is 0.
Definition at line 454 of file mem-tracker.h.
|
private |
The number of times the GcFunctions were called.
Definition at line 465 of file mem-tracker.h.
Referenced by GcMemory(), and RegisterMetrics().
|
private |
Definition at line 412 of file mem-tracker.h.
Referenced by Consume(), GetQueryMemTracker(), Init(), MemTracker(), parent(), Release(), and UnregisterFromParent().
|
private |
Only valid for MemTrackers returned from GetRequestPoolMemTracker()
Definition at line 400 of file mem-tracker.h.
Referenced by GetRequestPoolMemTracker(), and ~MemTracker().
|
staticprivate |
Definition at line 394 of file mem-tracker.h.
Referenced by GetRequestPoolMemTracker(), and ~MemTracker().
|
private |
Only valid for MemTrackers returned from GetQueryMemTracker()
Definition at line 397 of file mem-tracker.h.
Referenced by ~MemTracker().
|
private |
If non-NULL, contains all the information required to expand resource reservations if required.
Definition at line 462 of file mem-tracker.h.
Referenced by ExpandRmReservation(), and SetQueryResourceMgr().
|
staticprivate |
Total amount of memory from calls to Release() since the last GC. If this is greater than GC_RELEASE_SIZE, this will trigger a tcmalloc gc.
Definition at line 375 of file mem-tracker.h.
Referenced by GcTcmalloc(), and Release().
|
staticprivate |
Definition at line 389 of file mem-tracker.h.
Referenced by GetQueryMemTracker(), and ~MemTracker().
|
private |
Lock is taken during ExpandRmReservation() to prevent concurrent acquisition of new resources.
Definition at line 458 of file mem-tracker.h.
Referenced by ExpandRmReservation().
|
private |
If > -1, when RM is enabled this is the limit after which this memtracker needs to acquire more memory from Llama. This limit is always less than or equal to the hard limit.
Definition at line 409 of file mem-tracker.h.
Referenced by effective_limit(), ExpandRmReservation(), and Init().
|
staticprivate |
Protects request_to_mem_trackers_ and pool_to_mem_trackers_.
Definition at line 381 of file mem-tracker.h.
Referenced by GetQueryMemTracker(), GetRequestPoolMemTracker(), and ~MemTracker().