Merge pull request #57699 from hanfei1991/hanfei/keeper_soft_limit_ratio

support keeper memory soft limit ratio
This commit is contained in:
alesapin 2023-12-11 21:43:24 +01:00 committed by GitHub
commit cbae29d26a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 50 additions and 5 deletions

View File

@ -14,6 +14,7 @@
#include <Common/assertProcessUserMatchesDataOwner.h>
#include <Common/makeSocketAddress.h>
#include <Server/waitServersToFinish.h>
#include <base/getMemoryAmount.h>
#include <base/scope_guard.h>
#include <base/safeExit.h>
#include <Poco/Net/NetException.h>
@ -289,6 +290,33 @@ try
if (!config().has("keeper_server"))
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Keeper configuration (<keeper_server> section) not found in config");
auto updateMemorySoftLimitInConfig = [&](Poco::Util::AbstractConfiguration & config)
{
UInt64 memory_soft_limit = 0;
if (config.has("keeper_server.max_memory_usage_soft_limit"))
{
memory_soft_limit = config.getUInt64("keeper_server.max_memory_usage_soft_limit");
}
/// if memory soft limit is not set, we will use default value
if (memory_soft_limit == 0)
{
Float64 ratio = 0.9;
if (config.has("keeper_server.max_memory_usage_soft_limit_ratio"))
ratio = config.getDouble("keeper_server.max_memory_usage_soft_limit_ratio");
size_t physical_server_memory = getMemoryAmount();
if (ratio > 0 && physical_server_memory > 0)
{
memory_soft_limit = static_cast<UInt64>(physical_server_memory * ratio);
config.setUInt64("keeper_server.max_memory_usage_soft_limit", memory_soft_limit);
}
}
LOG_INFO(log, "keeper_server.max_memory_usage_soft_limit is set to {}", formatReadableSizeWithBinarySuffix(memory_soft_limit));
};
updateMemorySoftLimitInConfig(config());
std::string path;
if (config().has("keeper_server.storage_path"))
@ -499,6 +527,8 @@ try
{
updateLevels(*config, logger());
updateMemorySoftLimitInConfig(*config);
if (config->has("keeper_server"))
global_context->updateKeeperConfiguration(*config);

View File

@ -43,7 +43,6 @@ struct Settings;
M(UInt64, max_requests_batch_bytes_size, 100*1024, "Max size in bytes of batch of requests that can be sent to RAFT", 0) \
M(UInt64, max_flush_batch_size, 1000, "Max size of batch of requests that can be flushed together", 0) \
M(UInt64, max_requests_quick_batch_size, 100, "Max size of batch of requests to try to get before proceeding with RAFT. Keeper will not wait for requests but take only requests that are already in queue" , 0) \
M(UInt64, max_memory_usage_soft_limit, 0, "Soft limit in bytes of keeper memory usage", 0) \
M(Bool, quorum_reads, false, "Execute read requests as writes through whole RAFT consesus with similar speed", 0) \
M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \
M(Bool, compress_logs, false, "Write compressed coordination logs in ZSTD format", 0) \

View File

@ -59,6 +59,8 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config,
}
}
updateKeeperMemorySoftLimit(config);
digest_enabled = config.getBool("keeper_server.digest_enabled", false);
ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false);
@ -375,4 +377,10 @@ void KeeperContext::initializeFeatureFlags(const Poco::Util::AbstractConfigurati
feature_flags.logFlags(&Poco::Logger::get("KeeperContext"));
}
void KeeperContext::updateKeeperMemorySoftLimit(const Poco::Util::AbstractConfiguration & config)
{
if (config.hasProperty("keeper_server.max_memory_usage_soft_limit"))
memory_soft_limit = config.getUInt64("keeper_server.max_memory_usage_soft_limit");
}
}

View File

@ -53,6 +53,9 @@ public:
constexpr KeeperDispatcher * getDispatcher() const { return dispatcher; }
UInt64 getKeeperMemorySoftLimit() const { return memory_soft_limit; }
void updateKeeperMemorySoftLimit(const Poco::Util::AbstractConfiguration & config);
/// set to true when we have preprocessed or committed all the logs
/// that were already present locally during startup
std::atomic<bool> local_logs_preprocessed = false;
@ -92,6 +95,8 @@ private:
KeeperFeatureFlags feature_flags;
KeeperDispatcher * dispatcher{nullptr};
std::atomic<UInt64> memory_soft_limit = 0;
};
using KeeperContextPtr = std::shared_ptr<KeeperContext>;

View File

@ -143,7 +143,7 @@ void KeeperDispatcher::requestThread()
if (shutdown_called)
break;
Int64 mem_soft_limit = configuration_and_settings->coordination_settings->max_memory_usage_soft_limit;
Int64 mem_soft_limit = keeper_context->getKeeperMemorySoftLimit();
if (configuration_and_settings->standalone_keeper && mem_soft_limit > 0 && total_memory_tracker.get() >= mem_soft_limit && checkIfRequestIncreaseMem(request.request))
{
LOG_TRACE(log, "Processing requests refused because of max_memory_usage_soft_limit {}, the total used memory is {}, request type is {}", mem_soft_limit, total_memory_tracker.get(), request.request->getOpNum());
@ -930,6 +930,8 @@ void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfigurati
throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push configuration update to queue");
snapshot_s3.updateS3Configuration(config, macros);
keeper_context->updateKeeperMemorySoftLimit(config);
}
void KeeperDispatcher::updateKeeperStatLatency(uint64_t process_time_ms)

View File

@ -15,6 +15,7 @@
<value>az-zoo1</value>
</availability_zone>
<server_id>1</server_id>
<max_memory_usage_soft_limit>200000000</max_memory_usage_soft_limit>
<coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms>
@ -23,7 +24,6 @@
<force_sync>false</force_sync>
<election_timeout_lower_bound_ms>2000</election_timeout_lower_bound_ms>
<election_timeout_upper_bound_ms>4000</election_timeout_upper_bound_ms>
<max_memory_usage_soft_limit>200000000</max_memory_usage_soft_limit>
<async_replication>1</async_replication>
</coordination_settings>

View File

@ -16,6 +16,7 @@
<value>az-zoo2</value>
<enable_auto_detection_on_cloud>1</enable_auto_detection_on_cloud>
</availability_zone>
<max_memory_usage_soft_limit>20000000</max_memory_usage_soft_limit>
<coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms>
@ -24,7 +25,6 @@
<force_sync>false</force_sync>
<election_timeout_lower_bound_ms>2000</election_timeout_lower_bound_ms>
<election_timeout_upper_bound_ms>4000</election_timeout_upper_bound_ms>
<max_memory_usage_soft_limit>20000000</max_memory_usage_soft_limit>
<async_replication>1</async_replication>
</coordination_settings>

View File

@ -13,6 +13,8 @@
<tcp_port>2181</tcp_port>
<server_id>3</server_id>
<max_memory_usage_soft_limit>20000000</max_memory_usage_soft_limit>
<coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms>
<session_timeout_ms>15000</session_timeout_ms>
@ -20,7 +22,6 @@
<force_sync>false</force_sync>
<election_timeout_lower_bound_ms>2000</election_timeout_lower_bound_ms>
<election_timeout_upper_bound_ms>4000</election_timeout_upper_bound_ms>
<max_memory_usage_soft_limit>20000000</max_memory_usage_soft_limit>
<async_replication>1</async_replication>
</coordination_settings>