Skip to content

Commit 10405b5

Browse files
committed
stateloader: simplify SynthesizeHardState
Epic: none Release note: none
1 parent ba7131f commit 10405b5

File tree

3 files changed

+23
-28
lines changed

3 files changed

+23
-28
lines changed

pkg/kv/kvserver/logstore/stateloader.go

Lines changed: 5 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -167,17 +167,13 @@ func (sl StateLoader) SetHardState(
167167
// SynthesizeHardState synthesizes an on-disk HardState from the given input,
168168
// taking care that a HardState compatible with the existing data is written.
169169
func (sl StateLoader) SynthesizeHardState(
170-
ctx context.Context,
171-
writer storage.Writer,
172-
oldHS raftpb.HardState,
173-
truncState kvserverpb.RaftTruncatedState,
174-
raftAppliedIndex kvpb.RaftIndex,
170+
ctx context.Context, writer storage.Writer, oldHS raftpb.HardState, applied EntryID,
175171
) error {
176172
newHS := raftpb.HardState{
177-
Term: uint64(truncState.Term),
178-
// Note that when applying a Raft snapshot, the applied index is
179-
// equal to the Commit index represented by the snapshot.
180-
Commit: uint64(raftAppliedIndex),
173+
Term: uint64(applied.Term),
174+
// NB: when applying a Raft snapshot, the applied index is equal to the
175+
// Commit index represented by the snapshot.
176+
Commit: uint64(applied.Index),
181177
}
182178

183179
if oldHS.Commit > newHS.Commit {

pkg/kv/kvserver/stateloader/initial_test.go

Lines changed: 13 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -12,7 +12,7 @@ import (
1212
"testing"
1313

1414
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
15-
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
15+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore"
1616
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/print"
1717
"github.com/cockroachdb/cockroach/pkg/raft/raftpb"
1818
"github.com/cockroachdb/cockroach/pkg/roachpb"
@@ -64,20 +64,20 @@ func TestSynthesizeHardState(t *testing.T) {
6464
tHS := raftpb.HardState{Term: 2, Vote: 3, Commit: 4, Lead: 5, LeadEpoch: 6}
6565

6666
testCases := []struct {
67-
TruncTerm kvpb.RaftTerm
68-
RaftAppliedIndex kvpb.RaftIndex
69-
OldHS *raftpb.HardState
70-
NewHS raftpb.HardState
71-
Err string
67+
AppliedTerm kvpb.RaftTerm
68+
AppliedIndex kvpb.RaftIndex
69+
OldHS *raftpb.HardState
70+
NewHS raftpb.HardState
71+
Err string
7272
}{
73-
{OldHS: nil, TruncTerm: 42, RaftAppliedIndex: 24, NewHS: raftpb.HardState{Term: 42, Vote: 0, Commit: 24}},
73+
{OldHS: nil, AppliedTerm: 42, AppliedIndex: 24, NewHS: raftpb.HardState{Term: 42, Vote: 0, Commit: 24}},
7474
// Can't wind back the committed index of the new HardState.
75-
{OldHS: &tHS, RaftAppliedIndex: kvpb.RaftIndex(tHS.Commit - 1), Err: "can't decrease HardState.Commit"},
76-
{OldHS: &tHS, RaftAppliedIndex: kvpb.RaftIndex(tHS.Commit), NewHS: tHS},
77-
{OldHS: &tHS, RaftAppliedIndex: kvpb.RaftIndex(tHS.Commit + 1), NewHS: raftpb.HardState{Term: tHS.Term, Vote: 3, Commit: tHS.Commit + 1, Lead: 5, LeadEpoch: 6}},
75+
{OldHS: &tHS, AppliedIndex: kvpb.RaftIndex(tHS.Commit - 1), Err: "can't decrease HardState.Commit"},
76+
{OldHS: &tHS, AppliedIndex: kvpb.RaftIndex(tHS.Commit), NewHS: tHS},
77+
{OldHS: &tHS, AppliedIndex: kvpb.RaftIndex(tHS.Commit + 1), NewHS: raftpb.HardState{Term: tHS.Term, Vote: 3, Commit: tHS.Commit + 1, Lead: 5, LeadEpoch: 6}},
7878
// Higher Term is picked up, but Vote, Lead, and LeadEpoch aren't carried
7979
// over when the term changes.
80-
{OldHS: &tHS, RaftAppliedIndex: kvpb.RaftIndex(tHS.Commit), TruncTerm: 11, NewHS: raftpb.HardState{Term: 11, Vote: 0, Commit: tHS.Commit, Lead: 0}},
80+
{OldHS: &tHS, AppliedIndex: kvpb.RaftIndex(tHS.Commit), AppliedTerm: 11, NewHS: raftpb.HardState{Term: 11, Vote: 0, Commit: tHS.Commit, Lead: 0}},
8181
}
8282

8383
for i, test := range testCases {
@@ -97,9 +97,8 @@ func TestSynthesizeHardState(t *testing.T) {
9797
t.Fatal(err)
9898
}
9999

100-
err = rsl.SynthesizeHardState(
101-
context.Background(), batch, oldHS, kvserverpb.RaftTruncatedState{Term: test.TruncTerm}, test.RaftAppliedIndex,
102-
)
100+
err = rsl.SynthesizeHardState(context.Background(), batch, oldHS,
101+
logstore.EntryID{Index: test.AppliedIndex, Term: test.AppliedTerm})
103102
if !testutils.IsError(err, test.Err) {
104103
t.Fatalf("%d: expected %q got %v", i, test.Err, err)
105104
} else if err != nil {

pkg/kv/kvserver/stateloader/stateloader.go

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -453,13 +453,13 @@ func (rsl StateLoader) SynthesizeRaftState(
453453
if err != nil {
454454
return err
455455
}
456-
truncState, err := rsl.LoadRaftTruncatedState(ctx, readWriter)
457-
if err != nil {
458-
return err
459-
}
460456
as, err := rsl.LoadRangeAppliedState(ctx, readWriter)
461457
if err != nil {
462458
return err
463459
}
464-
return rsl.SynthesizeHardState(ctx, readWriter, hs, truncState, as.RaftAppliedIndex)
460+
applied := logstore.EntryID{
461+
Index: as.RaftAppliedIndex,
462+
Term: as.RaftAppliedIndexTerm,
463+
}
464+
return rsl.SynthesizeHardState(ctx, readWriter, hs, applied)
465465
}

0 commit comments

Comments
 (0)