Skip to content

Commit

Permalink
Merge pull request #10535 from VladLazar/10219-runner
Browse files Browse the repository at this point in the history
cloud_storage: tweak offsets used in post-recovery bootstrap
  • Loading branch information
piyushredpanda committed Jun 5, 2023
2 parents c4f3793 + ceeeb3d commit 3c2b7dd
Show file tree
Hide file tree
Showing 2 changed files with 7 additions and 5 deletions.
11 changes: 7 additions & 4 deletions src/v/raft/consensus_utils.cc
Original file line number Diff line number Diff line change
Expand Up @@ -514,11 +514,11 @@ ss::future<> create_offset_translator_state_for_pre_existing_partition(
raftlog.debug,
"{} Set highest_known_offset in kv-store to {}",
ntp_cfg.ntp(),
get_prev_offset(max_rp_offset));
max_rp_offset);
co_await api.kvs().put(
storage::kvstore::key_space::offset_translator,
raft::offset_translator::kvstore_highest_known_offset_key(group),
reflection::to_iobuf(get_prev_offset(max_rp_offset)));
reflection::to_iobuf(max_rp_offset));
}

ss::future<> create_raft_state_for_pre_existing_partition(
Expand All @@ -534,18 +534,21 @@ ss::future<> create_raft_state_for_pre_existing_partition(
raftlog.debug,
"{} Prepare raft state, set latest_known_offset {} to the kv-store",
ntp_cfg.ntp(),
get_prev_offset(max_rp_offset));
max_rp_offset);
auto key = raft::details::serialize_group_key(
group, raft::metadata_key::config_latest_known_offset);
co_await api.kvs().put(
storage::kvstore::key_space::consensus,
key,
reflection::to_iobuf(get_prev_offset(max_rp_offset)));
reflection::to_iobuf(max_rp_offset));

// Prepare Raft snapshot
raft::group_configuration group_config(
initial_nodes, ntp_cfg.get_revision());
raft::snapshot_metadata meta = {
// `last_included_index` should be the last offset included in
// this fake snapshot. That's why we set it to be the first offest
// before the start of the recovered log.
.last_included_index = get_prev_offset(min_rp_offset),
.last_included_term = last_included_term,
.version = raft::snapshot_metadata::current_version,
Expand Down
1 change: 0 additions & 1 deletion tests/rptest/tests/topic_recovery_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -1297,7 +1297,6 @@ def test_no_data(self, cloud_storage_type):
self.logger, self.rpk_producer_maker)
self.do_run(test_case)

@ok_to_fail # https://github.com/redpanda-data/redpanda/issues/10219
@cluster(num_nodes=3,
log_allow_list=MISSING_DATA_ERRORS + TRANSIENT_ERRORS)
@matrix(cloud_storage_type=get_cloud_storage_type())
Expand Down

0 comments on commit 3c2b7dd

Please sign in to comment.