Skip to content

Commit

Permalink
Merge pull request #20312 from ztlpn/flex-assignment-decrease-core-count
Browse files Browse the repository at this point in the history
Node-local core assignment: core count decrease
  • Loading branch information
ztlpn authored Jun 28, 2024
2 parents f3b079e + a2a27c6 commit 27c5cee
Show file tree
Hide file tree
Showing 28 changed files with 1,606 additions and 1,125 deletions.
44 changes: 32 additions & 12 deletions src/v/cluster/cluster_utils.cc
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
#include "cluster/simple_batch_builder.h"
#include "cluster/types.h"
#include "config/configuration.h"
#include "raft/consensus_utils.h"
#include "raft/errc.h"
#include "rpc/backoff_policy.h"
#include "rpc/types.h"
Expand Down Expand Up @@ -400,18 +401,7 @@ std::optional<ss::sstring> check_result_configuration(
new_configuration.id());
}
auto& current_configuration = it->second.broker;
/**
* do no allow to decrease node core count
*/
if (
current_configuration.properties().cores
> new_configuration.properties().cores) {
return fmt::format(
"core count must not decrease on any broker, currently configured "
"core count: {}, requested core count: {}",
current_configuration.properties().cores,
new_configuration.properties().cores);
}

/**
* When cluster member configuration changes Redpanda by default doesn't
* allow the change if a new cluster configuration would have two
Expand Down Expand Up @@ -477,4 +467,34 @@ std::optional<ss::sstring> check_result_configuration(
}
return {};
}

ss::future<> copy_persistent_state(
const model::ntp& ntp,
raft::group_id group,
storage::kvstore& source_kvs,
ss::shard_id target_shard,
ss::sharded<storage::api>& storage) {
return ss::when_all_succeed(
storage::disk_log_impl::copy_kvstore_state(
ntp, source_kvs, target_shard, storage),
raft::details::copy_persistent_state(
group, source_kvs, target_shard, storage),
storage::offset_translator::copy_persistent_state(
group, source_kvs, target_shard, storage),
raft::copy_persistent_stm_state(
ntp, source_kvs, target_shard, storage))
.discard_result();
}

ss::future<> remove_persistent_state(
const model::ntp& ntp, raft::group_id group, storage::kvstore& source_kvs) {
return ss::when_all_succeed(
storage::disk_log_impl::remove_kvstore_state(ntp, source_kvs),
raft::details::remove_persistent_state(group, source_kvs),
storage::offset_translator::remove_persistent_state(
group, source_kvs),
raft::remove_persistent_stm_state(ntp, source_kvs))
.discard_result();
}

} // namespace cluster
13 changes: 13 additions & 0 deletions src/v/cluster/cluster_utils.h
Original file line number Diff line number Diff line change
Expand Up @@ -354,4 +354,17 @@ std::optional<ss::sstring> check_result_configuration(
const members_table::cache_t& current_brokers,
const model::broker& to_update);

/// Copies all bits of partition kvstore state from source kvstore to kvstore on
/// target shard.
ss::future<> copy_persistent_state(
const model::ntp&,
raft::group_id,
storage::kvstore& source_kvs,
ss::shard_id target_shard,
ss::sharded<storage::api>&);

/// Removes all bits of partition kvstore state in source kvstore.
ss::future<> remove_persistent_state(
const model::ntp&, raft::group_id, storage::kvstore& source_kvs);

} // namespace cluster
Loading

0 comments on commit 27c5cee

Please sign in to comment.