Skip to content

Commit

Permalink
resource_mgmt/smp: seperate implementation
Browse files Browse the repository at this point in the history
Signed-off-by: Tyler Rockwood <rockwood@redpanda.com>
  • Loading branch information
rockwotj committed Jan 16, 2024
1 parent cb310c5 commit 99a3ece
Show file tree
Hide file tree
Showing 3 changed files with 73 additions and 50 deletions.
1 change: 1 addition & 0 deletions src/v/resource_mgmt/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ v_cc_library(
memory_sampling.cc
cpu_profiler.cc
logger.cc
smp_groups.cc
DEPS
Seastar::seastar
v::ssx
Expand Down
67 changes: 67 additions & 0 deletions src/v/resource_mgmt/smp_groups.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* Copyright 2024 Redpanda Data, Inc.
*
* Use of this software is governed by the Business Source License
* included in the file licenses/BSL.md
*
* As of the Change Date specified in that file, in accordance with
* the Business Source License, use of this software will be governed
* by the Apache License, Version 2.0
*/

#include "resource_mgmt/smp_groups.h"

#include <seastar/core/coroutine.hh>

ss::future<> smp_groups::create_groups(config cfg) {
_raft = co_await create_service_group(
cfg.raft_group_max_non_local_requests);
_kafka = co_await create_service_group(
cfg.kafka_group_max_non_local_requests);
_cluster = co_await create_service_group(
cfg.cluster_group_max_non_local_requests);
_proxy = co_await create_service_group(
cfg.proxy_group_max_non_local_requests);
_transform = co_await create_service_group(
cfg.transform_group_max_non_local_requests);
}

ss::future<> smp_groups::destroy_groups() {
co_await destroy_smp_service_group(*_kafka);
co_await destroy_smp_service_group(*_raft);
co_await destroy_smp_service_group(*_cluster);
co_await destroy_smp_service_group(*_proxy);
co_await destroy_smp_service_group(*_transform);
}

uint32_t
smp_groups::default_raft_non_local_requests(uint32_t max_partitions_per_core) {
/**
* raft max non local requests
* - up to 7000 groups per core
* - up to 256 concurrent append entries per group
* - additional requests like (vote, snapshot, timeout now)
*
* All the values have to be multiplied by core count minus one since
* part of the requests will be core local
*
* 7000*256 * (number of cores-1) + 10 * 7000 * (number of cores-1)
* ^ ^
* append entries requests additional requests
*/

static constexpr uint32_t max_append_requests_per_follower = 256;
static constexpr uint32_t additional_requests_per_follower = 10;

return max_partitions_per_core
* (max_append_requests_per_follower + additional_requests_per_follower)
* (ss::smp::count - 1);
}

ss::future<std::unique_ptr<ss::smp_service_group>>
smp_groups::create_service_group(unsigned max_non_local_requests) {
ss::smp_service_group_config smp_sg_config{
.max_nonlocal_requests = max_non_local_requests};
auto sg = co_await create_smp_service_group(smp_sg_config);
co_return std::make_unique<ss::smp_service_group>(sg);
}
55 changes: 5 additions & 50 deletions src/v/resource_mgmt/smp_groups.h
Original file line number Diff line number Diff line change
Expand Up @@ -13,8 +13,7 @@

#include "base/seastarx.h"

#include <seastar/core/coroutine.hh>
#include <seastar/core/reactor.hh>
#include <seastar/core/smp.hh>

// manage SMP scheduling groups. These scheduling groups are global, so one
// instance of this class can be created at the top level and passed down into
Expand All @@ -37,66 +36,22 @@ class smp_groups {
};

smp_groups() = default;
ss::future<> create_groups(config cfg) {
_raft = co_await create_service_group(
cfg.raft_group_max_non_local_requests);
_kafka = co_await create_service_group(
cfg.kafka_group_max_non_local_requests);
_cluster = co_await create_service_group(
cfg.cluster_group_max_non_local_requests);
_proxy = co_await create_service_group(
cfg.proxy_group_max_non_local_requests);
_transform = co_await create_service_group(
cfg.transform_group_max_non_local_requests);
}
ss::future<> create_groups(config cfg);

ss::smp_service_group raft_smp_sg() { return *_raft; }
ss::smp_service_group kafka_smp_sg() { return *_kafka; }
ss::smp_service_group cluster_smp_sg() { return *_cluster; }
ss::smp_service_group proxy_smp_sg() { return *_proxy; }
ss::smp_service_group transform_smp_sg() { return *_transform; }

ss::future<> destroy_groups() {
co_await destroy_smp_service_group(*_kafka);
co_await destroy_smp_service_group(*_raft);
co_await destroy_smp_service_group(*_cluster);
co_await destroy_smp_service_group(*_proxy);
co_await destroy_smp_service_group(*_transform);
}
ss::future<> destroy_groups();

static uint32_t
default_raft_non_local_requests(uint32_t max_partitions_per_core) {
/**
* raft max non local requests
* - up to 7000 groups per core
* - up to 256 concurrent append entries per group
* - additional requests like (vote, snapshot, timeout now)
*
* All the values have to be multiplied by core count minus one since
* part of the requests will be core local
*
* 7000*256 * (number of cores-1) + 10 * 7000 * (number of cores-1)
* ^ ^
* append entries requests additional requests
*/

static constexpr uint32_t max_append_requests_per_follower = 256;
static constexpr uint32_t additional_requests_per_follower = 10;

return max_partitions_per_core
* (max_append_requests_per_follower + additional_requests_per_follower)
* (ss::smp::count - 1);
}
default_raft_non_local_requests(uint32_t max_partitions_per_core);

private:
ss::future<std::unique_ptr<ss::smp_service_group>>
create_service_group(unsigned max_non_local_requests) {
ss::smp_service_group_config smp_sg_config{
.max_nonlocal_requests = max_non_local_requests};
auto sg = co_await create_smp_service_group(smp_sg_config);

co_return std::make_unique<ss::smp_service_group>(sg);
}
create_service_group(unsigned max_non_local_requests);

std::unique_ptr<ss::smp_service_group> _raft;
std::unique_ptr<ss::smp_service_group> _kafka;
Expand Down

0 comments on commit 99a3ece

Please sign in to comment.