Skip to content

Commit

Permalink
Merge pull request #15854 from vbotbuildovich/backport-pr-15844-v23.3…
Browse files Browse the repository at this point in the history
….x-695

[v23.3.x] Disable cloud metadata upload and cluster restore in recovery mode
  • Loading branch information
piyushredpanda committed Dec 22, 2023
2 parents 05a5a6d + 9973970 commit ce41701
Show file tree
Hide file tree
Showing 7 changed files with 25 additions and 9 deletions.
5 changes: 4 additions & 1 deletion src/v/cluster/cloud_metadata/cluster_recovery_backend.cc
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,10 @@ cluster_recovery_backend::cluster_recovery_backend(
, _producer_id_recovery(std::move(producer_id_recovery))
, _offsets_recovery(std::move(offsets_recovery))
, _recovery_table(recovery_table)
, _raft0(std::move(raft0)) {}
, _raft0(std::move(raft0)) {
vassert(_producer_id_recovery, "expected initialized producer_id_recovery");
vassert(_offsets_recovery, "expected initialized offsets_recovery");
}

void cluster_recovery_backend::start() {
_leader_cb_id = _raft_group_manager.register_leadership_notification(
Expand Down
4 changes: 3 additions & 1 deletion src/v/cluster/cloud_metadata/uploader.cc
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,9 @@ uploader::uploader(
, _bucket(bucket)
, _upload_interval_ms(
config::shard_local_cfg()
.cloud_storage_cluster_metadata_upload_interval_ms.bind()) {}
.cloud_storage_cluster_metadata_upload_interval_ms.bind()) {
vassert(_offsets_uploader, "expected initialized offsets_uploader");
}

ss::future<bool> uploader::term_has_changed(model::term_id term) {
if (!_raft0->is_leader() || _raft0->term() != term) {
Expand Down
3 changes: 3 additions & 0 deletions src/v/cluster/cluster_recovery_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,9 @@ cluster_recovery_manager::initialize_recovery(
if (!_remote.local_is_initialized()) {
co_return cluster::errc::invalid_request;
}
if (config::node().recovery_mode_enabled()) {
co_return cluster::errc::feature_disabled;
}
auto synced_term = co_await sync_leader(_sharded_as.local());
if (!synced_term.has_value()) {
co_return cluster::errc::not_leader_controller;
Expand Down
3 changes: 3 additions & 0 deletions src/v/cluster/controller.cc
Original file line number Diff line number Diff line change
Expand Up @@ -638,6 +638,9 @@ ss::future<> controller::start(
&partition_balancer_backend::start);
})
.then([this, offsets_uploader, producer_id_recovery, offsets_recovery] {
if (config::node().recovery_mode_enabled()) {
return;
}
auto bucket_opt = get_configured_bucket();
if (!bucket_opt.has_value()) {
return;
Expand Down
4 changes: 0 additions & 4 deletions src/v/cluster/controller.h
Original file line number Diff line number Diff line change
Expand Up @@ -144,10 +144,6 @@ class controller {
return _recovery_manager;
}

cloud_metadata::cluster_recovery_backend& get_cluster_recovery_backend() {
return *_recovery_backend;
}

ss::sharded<cluster_recovery_table>& get_cluster_recovery_table() {
return _recovery_table;
}
Expand Down
4 changes: 4 additions & 0 deletions src/v/redpanda/admin/server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -3568,6 +3568,10 @@ admin_server::initialize_cluster_recovery(
std::unique_ptr<ss::http::request> request,
std::unique_ptr<ss::http::reply> reply) {
reply->set_content_type("json");
if (config::node().recovery_mode_enabled()) {
throw ss::httpd::bad_request_exception(
"Cluster restore is not available, recovery mode enabled");
}
if (need_redirect_to_leader(model::controller_ntp, _metadata_cache)) {
throw co_await redirect_to_leader(*request, model::controller_ntp);
}
Expand Down
11 changes: 8 additions & 3 deletions tests/rptest/tests/recovery_mode_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
from requests.exceptions import HTTPError

from rptest.tests.redpanda_test import RedpandaTest
from rptest.services.redpanda import RESTART_LOG_ALLOW_LIST
from rptest.services.redpanda import SISettings
from rptest.services.cluster import cluster
from rptest.clients.rpk import RpkTool, RpkException
from rptest.services.admin import Admin
Expand Down Expand Up @@ -45,8 +45,13 @@ def assert_rpk_fails(cmd, error_msg):


class RecoveryModeTest(RedpandaTest):
def __init__(self, *args, **kwargs):
super().__init__(*args, num_brokers=4, **kwargs)
def __init__(self, test_ctx, *args, **kwargs):
self.si_settings = SISettings(test_ctx)
super().__init__(*args,
test_ctx,
num_brokers=4,
si_settings=self.si_settings,
**kwargs)

def setUp(self):
# start the nodes manually
Expand Down

0 comments on commit ce41701

Please sign in to comment.