Skip to content

Commit

Permalink
Merge pull request redpanda-data#13607 from ztlpn/coordinated-recover…
Browse files Browse the repository at this point in the history
…y-debug-json

admin: add raft follower_recovery_state to debug partition json
  • Loading branch information
ztlpn authored Sep 22, 2023
2 parents a854bc7 + f01b65c commit 359c09e
Show file tree
Hide file tree
Showing 5 changed files with 64 additions and 2 deletions.
10 changes: 10 additions & 0 deletions src/v/cluster/cluster_utils.cc
Original file line number Diff line number Diff line change
Expand Up @@ -350,6 +350,16 @@ partition_raft_state get_partition_raft_state(consensus_ptr ptr) {
raft_state.followers = std::move(followers);
}
raft_state.stms = get_partition_stm_state(ptr);

const auto& frs = ptr->get_follower_recovery_state();
if (frs) {
raft_state.recovery_state
= partition_raft_state::follower_recovery_state{
.is_active = frs->is_active(),
.pending_offset_count = frs->pending_offset_count(),
};
}

return raft_state;
}

Expand Down
27 changes: 25 additions & 2 deletions src/v/cluster/types.h
Original file line number Diff line number Diff line change
Expand Up @@ -3557,7 +3557,7 @@ struct partition_stm_state
struct partition_raft_state
: serde::envelope<
partition_raft_state,
serde::version<1>,
serde::version<2>,
serde::compat_version<0>> {
using rpc_adl_exempt = std::true_type;

Expand Down Expand Up @@ -3619,10 +3619,32 @@ struct partition_raft_state
suppress_heartbeats,
is_recovering);
}

friend bool operator==(const follower_state&, const follower_state&)
= default;
};

struct follower_recovery_state
: serde::envelope<
follower_recovery_state,
serde::version<0>,
serde::compat_version<0>> {
bool is_active = false;
int64_t pending_offset_count = 0;

auto serde_fields() {
return std::tie(is_active, pending_offset_count);
}

friend bool operator==(
const follower_recovery_state&, const follower_recovery_state&)
= default;
};

// Set only on leaders.
std::optional<std::vector<follower_state>> followers;
// Set only on recovering followers.
std::optional<follower_recovery_state> recovery_state;

auto serde_fields() {
return std::tie(
Expand All @@ -3644,7 +3666,8 @@ struct partition_raft_state
is_leader,
is_elected_leader,
followers,
stms);
stms,
recovery_state);
}

friend bool
Expand Down
5 changes: 5 additions & 0 deletions src/v/raft/consensus.h
Original file line number Diff line number Diff line change
Expand Up @@ -499,6 +499,11 @@ class consensus {

void reset_last_sent_protocol_meta(const vnode&);

const std::optional<follower_recovery_state>&
get_follower_recovery_state() const {
return _follower_recovery_state;
}

private:
friend replicate_entries_stm;
friend vote_stm;
Expand Down
18 changes: 18 additions & 0 deletions src/v/redpanda/admin/api-doc/debug.json
Original file line number Diff line number Diff line change
Expand Up @@ -913,6 +913,20 @@
}
}
},
"follower_recovery_state": {
"id": "follower_recovery_state",
"description": "Follower-side Raft recovery state",
"properties": {
"is_active": {
"type": "boolean",
"description": "True if recovery is currently allowed by the scheduler"
},
"pending_offset_count": {
"type": "long",
"description": "Difference between leader and our last offsets"
}
}
},
"raft_replica_state": {
"id": "raft_replica_state",
"description": "Raft level state for a single replica of a partition",
Expand Down Expand Up @@ -998,6 +1012,10 @@
"type": "stm_state"
},
"description": "All snapshottable stms attached to this replica"
},
"follower_recovery_state": {
"type": "follower_recovery_state",
"description": "Raft recovery state if this replica is a follower in recovery"
}
}
},
Expand Down
6 changes: 6 additions & 0 deletions src/v/redpanda/admin_server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -4107,6 +4107,12 @@ void fill_raft_state(
state.max_collectible_offset = stm.last_applied_offset;
raft_state.stms.push(std::move(state));
}
if (src.recovery_state) {
ss::httpd::debug_json::follower_recovery_state frs;
frs.is_active = src.recovery_state->is_active;
frs.pending_offset_count = src.recovery_state->pending_offset_count;
raft_state.follower_recovery_state = std::move(frs);
}
replica.raft_state = std::move(raft_state);
}
ss::future<result<std::vector<cluster::partition_state>>>
Expand Down

0 comments on commit 359c09e

Please sign in to comment.