Skip to content

Commit

Permalink
r/tests: added test validating state after recovering from snapshot
Browse files Browse the repository at this point in the history
Signed-off-by: Michał Maślanka <michal@redpanda.com>
  • Loading branch information
mmaslankaprv committed Oct 18, 2024
1 parent bf10ec4 commit ceea064
Show file tree
Hide file tree
Showing 3 changed files with 80 additions and 1 deletion.
25 changes: 24 additions & 1 deletion src/v/raft/tests/BUILD
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
load("//bazel:test.bzl", "redpanda_cc_btest", "redpanda_test_cc_library")
load("//bazel:test.bzl", "redpanda_cc_btest", "redpanda_cc_gtest", "redpanda_test_cc_library")

redpanda_test_cc_library(
name = "simple_raft_fixture",
Expand Down Expand Up @@ -181,3 +181,26 @@ redpanda_cc_btest(
"@seastar//:testing",
],
)

redpanda_cc_gtest(
name = "snapshot_recovery_test",
timeout = "short",
srcs = [
"snapshot_recovery_test.cc",
],
cpu = 1,
deps = [
":raft_fixiture_retry_policy",
":raft_fixture",
"//src/v/config",
"//src/v/model",
"//src/v/raft",
"//src/v/resource_mgmt:io_priority",
"//src/v/storage",
"//src/v/test_utils:gtest",
"//src/v/utils:unresolved_address",
"@fmt",
"@seastar",
"@seastar//:testing",
],
)
1 change: 1 addition & 0 deletions src/v/raft/tests/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ set(gsrcs
persisted_stm_test.cc
replication_monitor_tests.cc
mux_state_machine_test.cc
snapshot_recovery_test.cc
)

rp_test(
Expand Down
55 changes: 55 additions & 0 deletions src/v/raft/tests/snapshot_recovery_test.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
// Copyright 2023 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 "model/fundamental.h"
#include "model/metadata.h"
#include "model/record.h"
#include "model/record_batch_reader.h"
#include "raft/tests/raft_fixture.h"
#include "raft/tests/raft_fixture_retry_policy.h"
#include "test_utils/test.h"

using namespace raft;

TEST_F_CORO(raft_fixture, test_snapshot_recovery) {
auto& n0 = add_node(model::node_id(0), model::revision_id(0));
auto& n1 = add_node(model::node_id(2), model::revision_id(0));
auto& n2 = add_node(model::node_id(3), model::revision_id(0));

// seed one of the nodes with snapshot
auto base_dir = n1.base_directory();
auto ntp = n1.ntp();

snapshot_metadata md{
.last_included_index = model::offset(128),
.last_included_term = model::term_id(64),
.latest_configuration = raft::group_configuration(
all_vnodes(), model::revision_id(0)),
.cluster_time = clock_type::time_point::min(),
.log_start_delta = offset_translator_delta(10),
};
co_await ss::recursive_touch_directory(n1.work_directory());
storage::simple_snapshot_manager snapshot_manager(
std::filesystem::path(n1.work_directory()),
storage::simple_snapshot_manager::default_snapshot_filename,
ss::default_priority_class());

co_await raft::details::persist_snapshot(snapshot_manager, md, iobuf{});

co_await n0.init_and_start(all_vnodes());
co_await n1.init_and_start(all_vnodes());
co_await n2.init_and_start(all_vnodes());

auto leader_id = co_await wait_for_leader(30s);
auto& leader_node = node(leader_id);
ASSERT_GT_CORO(leader_node.raft()->term(), md.last_included_term);
ASSERT_EQ_CORO(
leader_node.raft()->start_offset(),
model::next_offset(md.last_included_index));
}

0 comments on commit ceea064

Please sign in to comment.