mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Some fixups
This commit is contained in:
parent
e1730c070e
commit
fe86741059
@ -624,21 +624,17 @@ try
|
||||
buildLoggers(config(), logger());
|
||||
main_config_reloader->start();
|
||||
|
||||
std::optional<CgroupsMemoryUsageObserver> observer;
|
||||
std::optional<CgroupsMemoryUsageObserver> cgroups_memory_usage_observer;
|
||||
try
|
||||
{
|
||||
auto wait_time = config().getUInt64("keeper_server.cgroups_memory_observer_wait_time", 15);
|
||||
if (wait_time != 0)
|
||||
{
|
||||
observer.emplace(std::chrono::seconds(wait_time));
|
||||
/// Keeper use setting max_memory_usage_soft_limit to control memory limit. When memory usage
|
||||
/// hits the limit, Keeper will refuse all the request util memory usage is lower.
|
||||
/// We don't call setLimits() here because of this.
|
||||
observer->setOnMemoryAmountAvailableChanged([&]()
|
||||
{
|
||||
main_config_reloader->reload();
|
||||
});
|
||||
observer->startThread();
|
||||
cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time));
|
||||
/// Not calling cgroups_memory_usage_observer->setLimits() here (as for the normal ClickHouse server) because Keeper controls
|
||||
/// its memory usage by other means (via setting 'max_memory_usage_soft_limit').
|
||||
cgroups_memory_usage_observer->setOnMemoryAmountAvailableChangedFn([&]() { main_config_reloader->reload(); });
|
||||
cgroups_memory_usage_observer->startThread();
|
||||
}
|
||||
}
|
||||
catch (Exception &)
|
||||
|
@ -1702,10 +1702,7 @@ try
|
||||
|
||||
if (cgroups_memory_usage_observer)
|
||||
{
|
||||
cgroups_memory_usage_observer->setOnMemoryAmountAvailableChanged([&]()
|
||||
{
|
||||
main_config_reloader->reload();
|
||||
});
|
||||
cgroups_memory_usage_observer->setOnMemoryAmountAvailableChangedFn([&]() { main_config_reloader->reload(); });
|
||||
cgroups_memory_usage_observer->startThread();
|
||||
}
|
||||
|
||||
|
@ -84,7 +84,7 @@ void CgroupsMemoryUsageObserver::setMemoryUsageLimits(uint64_t hard_limit_, uint
|
||||
mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0);
|
||||
#endif
|
||||
/// Reset current usage in memory tracker. Expect zero for free_memory_in_allocator_arenas as we just purged them.
|
||||
uint64_t memory_usage = readMemoryUsage();
|
||||
uint64_t memory_usage = memory_usage_file.readMemoryUsage();
|
||||
MemoryTracker::setRSS(memory_usage, 0);
|
||||
|
||||
LOG_INFO(log, "Purged jemalloc arenas. Current memory usage is {}", ReadableSize(memory_usage));
|
||||
@ -98,17 +98,12 @@ void CgroupsMemoryUsageObserver::setMemoryUsageLimits(uint64_t hard_limit_, uint
|
||||
LOG_INFO(log, "Set new limits, soft limit: {}, hard limit: {}", ReadableSize(soft_limit_), ReadableSize(hard_limit_));
|
||||
}
|
||||
|
||||
void CgroupsMemoryUsageObserver::setOnMemoryAmountAvailableChanged(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_)
|
||||
void CgroupsMemoryUsageObserver::setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_)
|
||||
{
|
||||
std::lock_guard<std::mutex> memory_amount_change_lock(memory_amount_change_mutex);
|
||||
std::lock_guard<std::mutex> memory_amount_available_changed_lock(memory_amount_available_changed_mutex);
|
||||
on_memory_amount_available_changed = on_memory_amount_available_changed_;
|
||||
}
|
||||
|
||||
uint64_t CgroupsMemoryUsageObserver::readMemoryUsage() const
|
||||
{
|
||||
return memory_usage_file.readMemoryUsage();
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
@ -299,7 +294,7 @@ void CgroupsMemoryUsageObserver::runThread()
|
||||
{
|
||||
LOG_INFO(log, "Memory amount available to the process changed from {} to {}", ReadableSize(last_available_memory_amount), ReadableSize(available_memory_amount));
|
||||
last_available_memory_amount = available_memory_amount;
|
||||
std::lock_guard<std::mutex> memory_amount_change_lock(memory_amount_change_mutex);
|
||||
std::lock_guard<std::mutex> memory_amount_available_changed_lock(memory_amount_available_changed_mutex);
|
||||
on_memory_amount_available_changed();
|
||||
}
|
||||
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <Common/ThreadPool.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#include <mutex>
|
||||
|
||||
@ -14,11 +13,16 @@ namespace DB
|
||||
/// You can specify soft or hard memory limits:
|
||||
/// - When the soft memory limit is hit, drop jemalloc cache.
|
||||
/// - When the hard memory limit is hit, update MemoryTracking metric to throw memory exceptions faster.
|
||||
/// The goal of this is to avoid that the process hits the maximum allowed memory limit at which there is a good
|
||||
/// chance that the Limux OOM killer terminates it. All of this is done is because internal memory tracking in
|
||||
/// ClickHouse can unfortunately under-estimate the actually used memory.
|
||||
/// 2. Periodically reads the the maximum memory available to the process (which can change due to cgroups settings).
|
||||
/// You can specify a callback to react on changes. The callback typically performs circular logic: It reloads the
|
||||
/// configuration (e.g. server configuration file), which will check the memory amount again and re-calculate
|
||||
/// soft/hard limits (see 1.) and update the value of setting `max_server_memory_usage` for clickhouse-server or
|
||||
/// `max_memory_usage_soft_limit` for clickhouse-keeper.
|
||||
/// You can specify a callback to react on changes. The callback typically reloads the configuration, i.e. Server
|
||||
/// or Keeper configuration file. This reloads settings 'max_server_memory_usage' (Server) and 'max_memory_usage_soft_limit'
|
||||
/// (Keeper) from which various other internal limits are calculated, including the soft and hard limits for (1.).
|
||||
/// The goal of this is to provide elasticity when the container is scaled-up/scaled-down. The mechanism (polling
|
||||
/// cgroups) is quite implicit, unfortuantely there is currently no better way to communicate memory threshold changes
|
||||
/// to the database.
|
||||
#if defined(OS_LINUX)
|
||||
class CgroupsMemoryUsageObserver
|
||||
{
|
||||
@ -36,12 +40,10 @@ public:
|
||||
~CgroupsMemoryUsageObserver();
|
||||
|
||||
void setMemoryUsageLimits(uint64_t hard_limit_, uint64_t soft_limit_);
|
||||
void setOnMemoryAmountAvailableChanged(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed);
|
||||
void setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_);
|
||||
|
||||
void startThread();
|
||||
|
||||
uint64_t readMemoryUsage() const;
|
||||
|
||||
private:
|
||||
LoggerPtr log;
|
||||
|
||||
@ -52,8 +54,9 @@ private:
|
||||
size_t soft_limit TSA_GUARDED_BY(limit_mutex) = 0;
|
||||
OnMemoryLimitFn on_hard_limit TSA_GUARDED_BY(limit_mutex);
|
||||
OnMemoryLimitFn on_soft_limit TSA_GUARDED_BY(limit_mutex);
|
||||
std::mutex memory_amount_change_mutex;
|
||||
OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed TSA_GUARDED_BY(memory_amount_change_mutex);
|
||||
|
||||
std::mutex memory_amount_available_changed_mutex;
|
||||
OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed TSA_GUARDED_BY(memory_amount_available_changed_mutex);
|
||||
|
||||
uint64_t last_memory_usage = 0; /// how much memory does the process use
|
||||
uint64_t last_available_memory_amount; /// how much memory can the process use
|
||||
@ -93,9 +96,8 @@ public:
|
||||
explicit CgroupsMemoryUsageObserver(std::chrono::seconds) {}
|
||||
|
||||
void setMemoryUsageLimits(uint64_t, uint64_t) {}
|
||||
void setOnMemoryAmountAvailableChanged(OnMemoryAmountAvailableChangedFn) {}
|
||||
void setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn) {}
|
||||
void startThread() {}
|
||||
size_t readMemoryUsage() { return 0; }
|
||||
};
|
||||
#endif
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user