Skip to content

Commit 341815e

Browse files
committed
kvserver: pass struct into SetRangeAppliedState
The signature of this method was error-prone and verbose. All the callers copy the RangeAppliedState from ReplicaState fields, or already have a fully populated RangeAppliedState. Epic: none Release note: none
1 parent e484d80 commit 341815e

File tree

4 files changed

+34
-51
lines changed

4 files changed

+34
-51
lines changed

pkg/kv/kvserver/kvserverpb/state.go

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,3 +16,18 @@ func (ms *MVCCPersistentStats) ToStats() enginepb.MVCCStats {
1616
func (ms *MVCCPersistentStats) ToStatsPtr() *enginepb.MVCCStats {
1717
return (*enginepb.MVCCStats)(ms)
1818
}
19+
20+
// ToRangeAppliedState converts the ReplicaState to a RangeAppliedState.
21+
func (m *ReplicaState) ToRangeAppliedState() RangeAppliedState {
22+
// NB: RangeAppliedState materializes in the state machine on every raft
23+
// commands batch application, and must be byte-to-byte consistent across
24+
// replicas. If you need to change the returned fields here, most likely this
25+
// needs to be accompanied by a below-raft migration.
26+
return RangeAppliedState{
27+
RaftAppliedIndex: m.RaftAppliedIndex,
28+
LeaseAppliedIndex: m.LeaseAppliedIndex,
29+
RangeStats: MVCCPersistentStats(*m.Stats),
30+
RaftClosedTimestamp: m.RaftClosedTimestamp,
31+
RaftAppliedIndexTerm: m.RaftAppliedIndexTerm,
32+
}
33+
}

pkg/kv/kvserver/kvstorage/stateloader.go

Lines changed: 9 additions & 38 deletions
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,6 @@ package kvstorage
88
import (
99
"context"
1010

11-
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
1211
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
1312
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore"
1413
"github.com/cockroachdb/cockroach/pkg/roachpb"
@@ -128,16 +127,10 @@ func (s StateLoader) Save(
128127
return enginepb.MVCCStats{}, err
129128
}
130129
}
131-
if err := s.SetRangeAppliedState(
132-
ctx,
133-
stateRW,
134-
state.RaftAppliedIndex,
135-
state.LeaseAppliedIndex,
136-
state.RaftAppliedIndexTerm,
137-
ms,
138-
state.RaftClosedTimestamp,
139-
nil,
140-
); err != nil {
130+
state.Stats = ms // no-op, just an acknowledgement that the stats were updated
131+
132+
as := state.ToRangeAppliedState()
133+
if err := s.SetRangeAppliedState(ctx, stateRW, &as); err != nil {
141134
return enginepb.MVCCStats{}, err
142135
}
143136
return *ms, nil
@@ -216,26 +209,8 @@ func (s StateLoader) LoadMVCCStats(
216209
// keys. We now deem those keys to be "legacy" because they have been replaced
217210
// by the range applied state key.
218211
func (s StateLoader) SetRangeAppliedState(
219-
ctx context.Context,
220-
stateRW StateRW,
221-
appliedIndex kvpb.RaftIndex,
222-
leaseAppliedIndex kvpb.LeaseAppliedIndex,
223-
appliedIndexTerm kvpb.RaftTerm,
224-
newMS *enginepb.MVCCStats,
225-
raftClosedTimestamp hlc.Timestamp,
226-
asAlloc *kvserverpb.RangeAppliedState, // optional
212+
ctx context.Context, stateRW StateRW, as *kvserverpb.RangeAppliedState,
227213
) error {
228-
if asAlloc == nil {
229-
asAlloc = new(kvserverpb.RangeAppliedState)
230-
}
231-
as := asAlloc
232-
*as = kvserverpb.RangeAppliedState{
233-
RaftAppliedIndex: appliedIndex,
234-
LeaseAppliedIndex: leaseAppliedIndex,
235-
RangeStats: kvserverpb.MVCCPersistentStats(*newMS),
236-
RaftClosedTimestamp: raftClosedTimestamp,
237-
RaftAppliedIndexTerm: appliedIndexTerm,
238-
}
239214
// The RangeAppliedStateKey is not included in stats. This is also reflected
240215
// in ComputeStats.
241216
ms := (*enginepb.MVCCStats)(nil)
@@ -253,10 +228,8 @@ func (s StateLoader) SetMVCCStats(
253228
if err != nil {
254229
return err
255230
}
256-
alloc := as // reuse
257-
return s.SetRangeAppliedState(
258-
ctx, stateRW, as.RaftAppliedIndex, as.LeaseAppliedIndex, as.RaftAppliedIndexTerm, newMS,
259-
as.RaftClosedTimestamp, alloc)
231+
as.RangeStats = kvserverpb.MVCCPersistentStats(*newMS)
232+
return s.SetRangeAppliedState(ctx, stateRW, as)
260233
}
261234

262235
// SetClosedTimestamp overwrites the closed timestamp.
@@ -267,10 +240,8 @@ func (s StateLoader) SetClosedTimestamp(
267240
if err != nil {
268241
return err
269242
}
270-
alloc := as // reuse
271-
return s.SetRangeAppliedState(
272-
ctx, stateRW, as.RaftAppliedIndex, as.LeaseAppliedIndex, as.RaftAppliedIndexTerm,
273-
as.RangeStats.ToStatsPtr(), closedTS, alloc)
243+
as.RaftClosedTimestamp = closedTS
244+
return s.SetRangeAppliedState(ctx, stateRW, as)
274245
}
275246

276247
// LoadGCThreshold loads the GC threshold.

pkg/kv/kvserver/raft_log_truncator_test.go

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -18,11 +18,9 @@ import (
1818
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage"
1919
"github.com/cockroachdb/cockroach/pkg/roachpb"
2020
"github.com/cockroachdb/cockroach/pkg/storage"
21-
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
2221
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
2322
"github.com/cockroachdb/cockroach/pkg/testutils/dd"
2423
"github.com/cockroachdb/cockroach/pkg/util/encoding"
25-
"github.com/cockroachdb/cockroach/pkg/util/hlc"
2624
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2725
"github.com/cockroachdb/cockroach/pkg/util/log"
2826
"github.com/cockroachdb/cockroach/pkg/util/stop"
@@ -181,8 +179,10 @@ func (r *replicaTruncatorTest) writeRaftStateToEngine(
181179
func (r *replicaTruncatorTest) writeRaftAppliedIndex(
182180
t *testing.T, eng storage.Engine, raftAppliedIndex kvpb.RaftIndex, flush bool,
183181
) {
184-
require.NoError(t, r.stateLoader.SetRangeAppliedState(context.Background(), eng,
185-
raftAppliedIndex, 0, 0, &enginepb.MVCCStats{}, hlc.Timestamp{}, nil))
182+
require.NoError(t, r.stateLoader.SetRangeAppliedState(
183+
context.Background(), eng,
184+
&kvserverpb.RangeAppliedState{RaftAppliedIndex: raftAppliedIndex},
185+
))
186186
// Flush to make it satisfy the contract of OnlyReadGuaranteedDurable in
187187
// Pebble.
188188
if flush {

pkg/kv/kvserver/replica_app_batch.go

Lines changed: 6 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -704,16 +704,13 @@ func (b *replicaAppBatch) ApplyToStateMachine(ctx context.Context) error {
704704
}
705705

706706
// addAppliedStateKeyToBatch adds the applied state key to the application
707-
// batch's RocksDB batch. This records the highest raft and lease index that
708-
// have been applied as of this batch. It also records the Range's mvcc stats.
707+
// batch's Pebble batch. This records the highest raft and lease index that have
708+
// been applied as of this batch. It also records the Range's MVCC stats.
709709
func (b *replicaAppBatch) addAppliedStateKeyToBatch(ctx context.Context) error {
710-
// Set the range applied state, which includes the last applied raft and
711-
// lease index along with the mvcc stats, all in one key.
712-
loader := &b.r.raftMu.stateLoader
713-
return loader.SetRangeAppliedState(
714-
ctx, b.batch, b.state.RaftAppliedIndex, b.state.LeaseAppliedIndex, b.state.RaftAppliedIndexTerm,
715-
b.state.Stats, b.state.RaftClosedTimestamp, &b.asAlloc,
716-
)
710+
// Set the range applied state, which includes the last applied raft and lease
711+
// index along with the MVCC stats, all in one key.
712+
b.asAlloc = b.state.ToRangeAppliedState()
713+
return b.r.raftMu.stateLoader.SetRangeAppliedState(ctx, b.batch, &b.asAlloc)
717714
}
718715

719716
func (b *replicaAppBatch) recordStatsOnCommit() {

0 commit comments

Comments
 (0)