diff --git a/pkg/cli/BUILD.bazel b/pkg/cli/BUILD.bazel index 8ee3fbb49330..9933c44d5799 100644 --- a/pkg/cli/BUILD.bazel +++ b/pkg/cli/BUILD.bazel @@ -128,6 +128,7 @@ go_library( "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/gc", + "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/kvstorage", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/kv/kvserver/loqrecovery", diff --git a/pkg/cli/debug_check_store.go b/pkg/cli/debug_check_store.go index 5936dc051c71..a04415a9ee51 100644 --- a/pkg/cli/debug_check_store.go +++ b/pkg/cli/debug_check_store.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cli/clierrorplus" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" @@ -80,11 +81,11 @@ func runDebugCheckStoreCmd(cmd *cobra.Command, args []string) error { } type replicaCheckInfo struct { - truncatedIndex uint64 - appliedIndex uint64 - firstIndex uint64 - lastIndex uint64 - committedIndex uint64 + truncatedIndex roachpb.RaftIndex + appliedIndex roachpb.RaftIndex + firstIndex roachpb.RaftIndex + lastIndex roachpb.RaftIndex + committedIndex roachpb.RaftIndex } type checkInput struct { @@ -251,7 +252,7 @@ func checkStoreRaftState( if err := kv.Value.GetProto(&hs); err != nil { return err } - getReplicaInfo(rangeID).committedIndex = hs.Commit + getReplicaInfo(rangeID).committedIndex = roachpb.RaftIndex(hs.Commit) case bytes.Equal(suffix, keys.LocalRaftTruncatedStateSuffix): var trunc roachpb.RaftTruncatedState if err := kv.Value.GetProto(&trunc); err != nil { @@ -259,13 +260,14 @@ func checkStoreRaftState( } getReplicaInfo(rangeID).truncatedIndex = trunc.Index case bytes.Equal(suffix, keys.LocalRangeAppliedStateSuffix): - var state enginepb.RangeAppliedState + var state kvserverpb.RangeAppliedState if err := kv.Value.GetProto(&state); err != nil { return err } getReplicaInfo(rangeID).appliedIndex = state.RaftAppliedIndex case bytes.Equal(suffix, keys.LocalRaftLogSuffix): - _, index, err := encoding.DecodeUint64Ascending(detail) + _, uIndex, err := encoding.DecodeUint64Ascending(detail) + index := roachpb.RaftIndex(uIndex) if err != nil { return err } diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index ee2b4564fae0..d7b1e4a7b629 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -356,21 +356,21 @@ func RaftLogPrefix(rangeID roachpb.RangeID) roachpb.Key { } // RaftLogKey returns a system-local key for a Raft log entry. -func RaftLogKey(rangeID roachpb.RangeID, logIndex uint64) roachpb.Key { +func RaftLogKey(rangeID roachpb.RangeID, logIndex roachpb.RaftIndex) roachpb.Key { return MakeRangeIDPrefixBuf(rangeID).RaftLogKey(logIndex) } // RaftLogKeyFromPrefix returns a system-local key for a Raft log entry, using // the provided Raft log prefix. -func RaftLogKeyFromPrefix(raftLogPrefix []byte, logIndex uint64) roachpb.Key { - return encoding.EncodeUint64Ascending(raftLogPrefix, logIndex) +func RaftLogKeyFromPrefix(raftLogPrefix []byte, logIndex roachpb.RaftIndex) roachpb.Key { + return encoding.EncodeUint64Ascending(raftLogPrefix, uint64(logIndex)) } // DecodeRaftLogKeyFromSuffix parses the suffix of a system-local key for a Raft // log entry and returns the entry's log index. -func DecodeRaftLogKeyFromSuffix(raftLogSuffix []byte) (uint64, error) { +func DecodeRaftLogKeyFromSuffix(raftLogSuffix []byte) (roachpb.RaftIndex, error) { _, logIndex, err := encoding.DecodeUint64Ascending(raftLogSuffix) - return logIndex, err + return roachpb.RaftIndex(logIndex), err } // RaftReplicaIDKey returns a system-local key for a RaftReplicaID. @@ -1092,7 +1092,7 @@ func (b RangeIDPrefixBuf) RaftLogPrefix() roachpb.Key { } // RaftLogKey returns a system-local key for a Raft log entry. -func (b RangeIDPrefixBuf) RaftLogKey(logIndex uint64) roachpb.Key { +func (b RangeIDPrefixBuf) RaftLogKey(logIndex roachpb.RaftIndex) roachpb.Key { return RaftLogKeyFromPrefix(b.RaftLogPrefix(), logIndex) } diff --git a/pkg/keys/printer.go b/pkg/keys/printer.go index 6fa74e7b6b3f..48e8be4b9e43 100644 --- a/pkg/keys/printer.go +++ b/pkg/keys/printer.go @@ -305,7 +305,7 @@ func raftLogKeyParse(rangeID roachpb.RangeID, input string) (string, roachpb.Key if err != nil { panic(err) } - return "", RaftLogKey(rangeID, index) + return "", RaftLogKey(rangeID, roachpb.RaftIndex(index)) } func raftLogKeyPrint(buf *redact.StringBuilder, key roachpb.Key) { diff --git a/pkg/keys/printer_test.go b/pkg/keys/printer_test.go index 4b7ecd0c258f..9ec7e11297bf 100644 --- a/pkg/keys/printer_test.go +++ b/pkg/keys/printer_test.go @@ -250,7 +250,7 @@ func TestPrettyPrint(t *testing.T) { {keys.RaftHardStateKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/u/RaftHardState", revertSupportUnknown}, {keys.RangeTombstoneKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/u/RangeTombstone", revertSupportUnknown}, - {keys.RaftLogKey(roachpb.RangeID(1000001), uint64(200001)), "/Local/RangeID/1000001/u/RaftLog/logIndex:200001", revertSupportUnknown}, + {keys.RaftLogKey(roachpb.RangeID(1000001), roachpb.RaftIndex(200001)), "/Local/RangeID/1000001/u/RaftLog/logIndex:200001", revertSupportUnknown}, {keys.RangeLastReplicaGCTimestampKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/u/RangeLastReplicaGCTimestamp", revertSupportUnknown}, {keys.MakeRangeKeyPrefix(roachpb.RKey(tenSysCodec.TablePrefix(42))), `/Local/Range/Table/42`, revertSupportUnknown}, diff --git a/pkg/kv/kvpb/api.proto b/pkg/kv/kvpb/api.proto index 51c2588d8794..d1b5fd7a852f 100644 --- a/pkg/kv/kvpb/api.proto +++ b/pkg/kv/kvpb/api.proto @@ -1462,7 +1462,7 @@ message TruncateLogRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; // Log entries < this index are to be discarded. - uint64 index = 2; + uint64 index = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftIndex"]; // RangeID is used to double check that the correct range is being truncated. // The header specifies a span, start and end keys, but not the range id @@ -1482,7 +1482,7 @@ message TruncateLogRequest { // discarded from the raft log to be performed once, at the leaseholder. // // Populated starting at cluster version LooselyCoupledRaftLogTruncation. - uint64 expected_first_index = 4; + uint64 expected_first_index = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftIndex"]; } // TruncateLogResponse is the response to a TruncateLog() operation. @@ -2099,7 +2099,7 @@ message SubsumeResponse { // provide the sender of the Subsume request with an upper bound on the lease // applied index of the CPut that left an intent on the local copy of the // right-hand range descriptor. - uint64 lease_applied_index = 4; + int64 lease_applied_index = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.LeaseSequence"]; // FreezeStart is a timestamp that is guaranteed to be greater than the // timestamps at which any requests were serviced by the responding replica diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index ce1aa608b1f5..599df496b5b7 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -125,7 +125,6 @@ go_library( "//pkg/kv/kvserver/batcheval", "//pkg/kv/kvserver/batcheval/result", "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", "//pkg/kv/kvserver/closedts/sidetransport", "//pkg/kv/kvserver/closedts/tracker", "//pkg/kv/kvserver/concurrency", @@ -364,7 +363,6 @@ go_test( "//pkg/kv/kvserver/batcheval", "//pkg/kv/kvserver/batcheval/result", "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", "//pkg/kv/kvserver/closedts/tracker", "//pkg/kv/kvserver/concurrency", "//pkg/kv/kvserver/concurrency/isolation", diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go index d517d9770f7c..b893d9a1dc2e 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go @@ -1957,7 +1957,7 @@ func (a *Allocator) ValidLeaseTargets( leaseRepl interface { StoreID() roachpb.StoreID RaftStatus() *raft.Status - GetFirstIndex() uint64 + GetFirstIndex() roachpb.RaftIndex Desc() *roachpb.RangeDescriptor }, opts allocator.TransferLeaseOptions, @@ -2132,7 +2132,7 @@ func (a *Allocator) leaseholderShouldMoveDueToPreferences( leaseRepl interface { StoreID() roachpb.StoreID RaftStatus() *raft.Status - GetFirstIndex() uint64 + GetFirstIndex() roachpb.RaftIndex }, allExistingReplicas []roachpb.ReplicaDescriptor, ) bool { @@ -2217,7 +2217,7 @@ func (a *Allocator) TransferLeaseTarget( StoreID() roachpb.StoreID GetRangeID() roachpb.RangeID RaftStatus() *raft.Status - GetFirstIndex() uint64 + GetFirstIndex() roachpb.RaftIndex Desc() *roachpb.RangeDescriptor }, usageInfo allocator.RangeUsageInfo, @@ -2485,7 +2485,7 @@ func (a *Allocator) ShouldTransferLease( leaseRepl interface { StoreID() roachpb.StoreID RaftStatus() *raft.Status - GetFirstIndex() uint64 + GetFirstIndex() roachpb.RaftIndex Desc() *roachpb.RangeDescriptor }, usageInfo allocator.RangeUsageInfo, @@ -2852,7 +2852,10 @@ func FilterBehindReplicas( // Other replicas may be filtered out if this function is called with the // `raftStatus` of a non-raft leader replica. func excludeReplicasInNeedOfSnapshots( - ctx context.Context, st *raft.Status, firstIndex uint64, replicas []roachpb.ReplicaDescriptor, + ctx context.Context, + st *raft.Status, + firstIndex roachpb.RaftIndex, + replicas []roachpb.ReplicaDescriptor, ) []roachpb.ReplicaDescriptor { filled := 0 for _, repl := range replicas { diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go index fc8bac15aa1e..4ca9bbbe757a 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go @@ -1893,7 +1893,7 @@ func (r *mockRepl) RaftStatus() *raft.Status { return raftStatus } -func (r *mockRepl) GetFirstIndex() uint64 { +func (r *mockRepl) GetFirstIndex() roachpb.RaftIndex { return 0 } diff --git a/pkg/kv/kvserver/api.proto b/pkg/kv/kvserver/api.proto index 41f5f285b9ce..2a2b5b98d209 100644 --- a/pkg/kv/kvserver/api.proto +++ b/pkg/kv/kvserver/api.proto @@ -55,7 +55,7 @@ message WaitForApplicationRequest { StoreRequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; int64 range_id = 2 [(gogoproto.customname) = "RangeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; - uint64 lease_index = 3; + int64 lease_index = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.LeaseSequence"]; } message WaitForApplicationResponse { diff --git a/pkg/kv/kvserver/app_batch.go b/pkg/kv/kvserver/app_batch.go index 836c0765e0c0..1686b3dc32fb 100644 --- a/pkg/kv/kvserver/app_batch.go +++ b/pkg/kv/kvserver/app_batch.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -179,7 +180,7 @@ func (b *appBatch) runPostAddTriggers( env.st, env.eng, env.sideloaded, - cmd.Term, + roachpb.RaftTerm(cmd.Term), cmd.Index(), *res.AddSSTable, env.bulkLimiter, diff --git a/pkg/kv/kvserver/apply/BUILD.bazel b/pkg/kv/kvserver/apply/BUILD.bazel index 3ff451e07b8a..c147355d617c 100644 --- a/pkg/kv/kvserver/apply/BUILD.bazel +++ b/pkg/kv/kvserver/apply/BUILD.bazel @@ -11,6 +11,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply", visibility = ["//visibility:public"], deps = [ + "//pkg/roachpb", "@com_github_cockroachdb_errors//:errors", "@io_etcd_go_raft_v3//raftpb", ], @@ -26,6 +27,7 @@ go_test( args = ["-test.timeout=55s"], deps = [ ":apply", + "//pkg/roachpb", "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", "@io_etcd_go_raft_v3//raftpb", diff --git a/pkg/kv/kvserver/apply/cmd.go b/pkg/kv/kvserver/apply/cmd.go index 0c75f9ccd8c6..4822e02ae752 100644 --- a/pkg/kv/kvserver/apply/cmd.go +++ b/pkg/kv/kvserver/apply/cmd.go @@ -10,14 +10,18 @@ package apply -import "context" +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/roachpb" +) // Command is a command that has been successfully replicated through raft // by being durably committed to the raft log of a quorum of peers in a raft // group. type Command interface { // Index is the log index of the corresponding raft entry. - Index() uint64 + Index() roachpb.RaftIndex // IsTrivial returns whether the command can apply in a batch with other // "trivial" commands. This is the case if the log entry represented by the // Command is a simple write, as is the case for all user-issued mutations. diff --git a/pkg/kv/kvserver/apply/task_test.go b/pkg/kv/kvserver/apply/task_test.go index 7927bdca5492..dd919a4b5a04 100644 --- a/pkg/kv/kvserver/apply/task_test.go +++ b/pkg/kv/kvserver/apply/task_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" "go.etcd.io/raft/v3/raftpb" @@ -33,7 +34,7 @@ func setLogging(on bool) func() { } type cmd struct { - index uint64 + index roachpb.RaftIndex nonTrivial bool nonLocal bool shouldReject bool @@ -52,10 +53,10 @@ type appliedCmd struct { *checkedCmd } -func (c *cmd) Index() uint64 { return c.index } -func (c *cmd) IsTrivial() bool { return !c.nonTrivial } -func (c *cmd) IsLocal() bool { return !c.nonLocal } -func (c *cmd) Ctx() context.Context { return context.Background() } +func (c *cmd) Index() roachpb.RaftIndex { return c.index } +func (c *cmd) IsTrivial() bool { return !c.nonTrivial } +func (c *cmd) IsLocal() bool { return !c.nonLocal } +func (c *cmd) Ctx() context.Context { return context.Background() } func (c *cmd) AckErrAndFinish(_ context.Context, err error) error { c.acked = true c.finished = true @@ -121,9 +122,9 @@ var _ apply.CheckedCommandList = &checkedCmdSlice{} var _ apply.AppliedCommandList = &appliedCmdSlice{} type testStateMachine struct { - batches [][]uint64 - applied []uint64 - appliedSideEffects []uint64 + batches [][]roachpb.RaftIndex + applied []roachpb.RaftIndex + appliedSideEffects []roachpb.RaftIndex batchOpen bool } @@ -166,7 +167,7 @@ func (sm *testStateMachine) ApplySideEffects( type testBatch struct { sm *testStateMachine ephemeral bool - staged []uint64 + staged []roachpb.RaftIndex } func (b *testBatch) Stage(_ context.Context, cmdI apply.Command) (apply.CheckedCommand, error) { @@ -191,27 +192,27 @@ func (b *testBatch) Close() { } type testDecoder struct { - nonTrivial map[uint64]bool - nonLocal map[uint64]bool - shouldReject map[uint64]bool - shouldThrowErrRemoved map[uint64]bool + nonTrivial map[roachpb.RaftIndex]bool + nonLocal map[roachpb.RaftIndex]bool + shouldReject map[roachpb.RaftIndex]bool + shouldThrowErrRemoved map[roachpb.RaftIndex]bool cmds []*cmd } func newTestDecoder() *testDecoder { return &testDecoder{ - nonTrivial: make(map[uint64]bool), - nonLocal: make(map[uint64]bool), - shouldReject: make(map[uint64]bool), - shouldThrowErrRemoved: make(map[uint64]bool), + nonTrivial: make(map[roachpb.RaftIndex]bool), + nonLocal: make(map[roachpb.RaftIndex]bool), + shouldReject: make(map[roachpb.RaftIndex]bool), + shouldThrowErrRemoved: make(map[roachpb.RaftIndex]bool), } } func (d *testDecoder) DecodeAndBind(_ context.Context, ents []raftpb.Entry) (bool, error) { d.cmds = make([]*cmd, len(ents)) for i, ent := range ents { - idx := ent.Index + idx := roachpb.RaftIndex(ent.Index) cmd := &cmd{ index: idx, nonTrivial: d.nonTrivial[idx], @@ -258,9 +259,9 @@ func TestApplyCommittedEntries(t *testing.T) { // Assert that all commands were applied in the correct batches. exp := testStateMachine{ - batches: [][]uint64{{1, 2}, {3}, {4}, {5}, {6}}, - applied: []uint64{1, 2, 3, 4, 5, 6}, - appliedSideEffects: []uint64{1, 2, 3, 4, 5, 6}, + batches: [][]roachpb.RaftIndex{{1, 2}, {3}, {4}, {5}, {6}}, + applied: []roachpb.RaftIndex{1, 2, 3, 4, 5, 6}, + appliedSideEffects: []roachpb.RaftIndex{1, 2, 3, 4, 5, 6}, } require.Equal(t, exp, *sm) @@ -288,9 +289,9 @@ func TestApplyCommittedEntriesWithBatchSize(t *testing.T) { // Assert that all commands were applied in the correct batches. exp := testStateMachine{ - batches: [][]uint64{{1, 2}, {3}, {4}, {5, 6}, {7}}, - applied: []uint64{1, 2, 3, 4, 5, 6, 7}, - appliedSideEffects: []uint64{1, 2, 3, 4, 5, 6, 7}, + batches: [][]roachpb.RaftIndex{{1, 2}, {3}, {4}, {5, 6}, {7}}, + applied: []roachpb.RaftIndex{1, 2, 3, 4, 5, 6, 7}, + appliedSideEffects: []roachpb.RaftIndex{1, 2, 3, 4, 5, 6, 7}, } require.Equal(t, exp, *sm) @@ -358,9 +359,9 @@ func TestApplyCommittedEntriesWithErr(t *testing.T) { // Assert that only commands up to the replica removal were applied. exp := testStateMachine{ - batches: [][]uint64{{1, 2}, {3}}, - applied: []uint64{1, 2, 3}, - appliedSideEffects: []uint64{1, 2, 3}, + batches: [][]roachpb.RaftIndex{{1, 2}, {3}}, + applied: []roachpb.RaftIndex{1, 2, 3}, + appliedSideEffects: []roachpb.RaftIndex{1, 2, 3}, } require.Equal(t, exp, *sm) diff --git a/pkg/kv/kvserver/asim/storerebalancer/candidate_replica.go b/pkg/kv/kvserver/asim/storerebalancer/candidate_replica.go index 8845bfeaf479..81e0014764dc 100644 --- a/pkg/kv/kvserver/asim/storerebalancer/candidate_replica.go +++ b/pkg/kv/kvserver/asim/storerebalancer/candidate_replica.go @@ -70,7 +70,7 @@ func (sr *simulatorReplica) RaftStatus() *raft.Status { // GetFirstIndex returns the index of the first entry in the replica's Raft // log. -func (sr *simulatorReplica) GetFirstIndex() uint64 { +func (sr *simulatorReplica) GetFirstIndex() roachpb.RaftIndex { // TODO(kvoli): We always return 2 here as RaftStatus is unimplemented. // When it is implmeneted, this may become variable. return 2 diff --git a/pkg/kv/kvserver/batcheval/eval_context.go b/pkg/kv/kvserver/batcheval/eval_context.go index 4522de2d38ad..1641bb16e378 100644 --- a/pkg/kv/kvserver/batcheval/eval_context.go +++ b/pkg/kv/kvserver/batcheval/eval_context.go @@ -62,9 +62,9 @@ type EvalContext interface { GetNodeLocality() roachpb.Locality IsFirstRange() bool - GetFirstIndex() uint64 - GetTerm(uint64) (uint64, error) - GetLeaseAppliedIndex() uint64 + GetFirstIndex() roachpb.RaftIndex + GetTerm(index roachpb.RaftIndex) (roachpb.RaftTerm, error) + GetLeaseAppliedIndex() roachpb.LeaseSequence Desc() *roachpb.RangeDescriptor ContainsKey(key roachpb.Key) bool @@ -172,7 +172,8 @@ type MockEvalCtx struct { CPU float64 AbortSpan *abortspan.AbortSpan GCThreshold hlc.Timestamp - Term, FirstIndex uint64 + Term roachpb.RaftTerm + FirstIndex roachpb.RaftIndex CanCreateTxnRecordFn func() (bool, kvpb.TransactionAbortedReason) MinTxnCommitTSFn func() hlc.Timestamp Lease roachpb.Lease @@ -228,13 +229,13 @@ func (m *mockEvalCtxImpl) GetRangeID() roachpb.RangeID { func (m *mockEvalCtxImpl) IsFirstRange() bool { panic("unimplemented") } -func (m *mockEvalCtxImpl) GetFirstIndex() uint64 { +func (m *mockEvalCtxImpl) GetFirstIndex() roachpb.RaftIndex { return m.FirstIndex } -func (m *mockEvalCtxImpl) GetTerm(uint64) (uint64, error) { +func (m *mockEvalCtxImpl) GetTerm(roachpb.RaftIndex) (roachpb.RaftTerm, error) { return m.Term, nil } -func (m *mockEvalCtxImpl) GetLeaseAppliedIndex() uint64 { +func (m *mockEvalCtxImpl) GetLeaseAppliedIndex() roachpb.LeaseSequence { panic("unimplemented") } func (m *mockEvalCtxImpl) Desc() *roachpb.RangeDescriptor { diff --git a/pkg/kv/kvserver/below_raft_protos_test.go b/pkg/kv/kvserver/below_raft_protos_test.go index 585f7db59a69..d5fa623e53cd 100644 --- a/pkg/kv/kvserver/below_raft_protos_test.go +++ b/pkg/kv/kvserver/below_raft_protos_test.go @@ -17,6 +17,7 @@ import ( "regexp" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" @@ -50,7 +51,7 @@ func TestBelowRaftProtosDontChange(t *testing.T) { return m }, func(r *rand.Rand) protoutil.Message { - return enginepb.NewPopulatedRangeAppliedState(r, false) + return kvserverpb.NewPopulatedRangeAppliedState(r, false) }, func(r *rand.Rand) protoutil.Message { type expectedHardState struct { diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 51c547a413d7..56bc99c5f549 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -664,7 +664,7 @@ func mergeCheckingTimestampCaches( } // Applied to leaseholder after the partition heals. - var truncIndex uint64 + var truncIndex roachpb.RaftIndex restoredLeaseholderFuncs := noopRaftHandlerFuncs() restoredLeaseholderFuncs.dropReq = func(req *kvserverpb.RaftMessageRequest) bool { // Make sure that even going forward no MsgApp for what we just @@ -675,7 +675,7 @@ func mergeCheckingTimestampCaches( // // NB: the Index on the message is the log index that _precedes_ any of the // entries in the MsgApp, so filter where msg.Index < index, not <= index. - return req.Message.Type == raftpb.MsgApp && req.Message.Index < truncIndex + return req.Message.Type == raftpb.MsgApp && roachpb.RaftIndex(req.Message.Index) < truncIndex } // Because we enter a split leader-leaseholder state, none of the @@ -730,13 +730,13 @@ func mergeCheckingTimestampCaches( // largest log index on the leader, or it will panic. So we choose // the minimum of these two and just pick the smallest "last index" // in the range, which does the trick. - min := func(a, b uint64) uint64 { + min := func(a, b roachpb.RaftIndex) roachpb.RaftIndex { if a < b { return a } return b } - minLastIndex := uint64(math.MaxUint64) + minLastIndex := roachpb.RaftIndex(math.MaxUint64) for _, r := range lhsRepls { lastIndex := r.GetLastIndex() minLastIndex = min(minLastIndex, lastIndex) @@ -4026,7 +4026,7 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) { } // Truncate the logs of the LHS. - index := func() uint64 { + index := func() roachpb.RaftIndex { repl := store0.LookupReplica(roachpb.RKey(keyA)) index := repl.GetLastIndex() truncArgs := &kvpb.TruncateLogRequest{ @@ -4058,7 +4058,7 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) { // // NB: the Index on the message is the log index that _precedes_ any of the // entries in the MsgApp, so filter where msg.Index < index, not <= index. - return req.Message.Type == raftpb.MsgApp && req.Message.Index < index + return req.Message.Type == raftpb.MsgApp && roachpb.RaftIndex(req.Message.Index) < index }, // Don't drop heartbeats or responses. dropHB: func(*kvserverpb.RaftHeartbeat) bool { return false }, diff --git a/pkg/kv/kvserver/client_raft_log_queue_test.go b/pkg/kv/kvserver/client_raft_log_queue_test.go index afdac79d90ac..5852c0d57276 100644 --- a/pkg/kv/kvserver/client_raft_log_queue_test.go +++ b/pkg/kv/kvserver/client_raft_log_queue_test.go @@ -87,7 +87,7 @@ func TestRaftLogQueue(t *testing.T) { } } - var afterTruncationIndex uint64 + var afterTruncationIndex roachpb.RaftIndex testutils.SucceedsSoon(t, func() error { // Force a truncation check. for i := range tc.Servers { diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index 81826f1659cb..7f6d0d885a5e 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -826,7 +826,9 @@ func TestSnapshotAfterTruncation(t *testing.T) { } } -func waitForTruncationForTesting(t *testing.T, r *kvserver.Replica, newFirstIndex uint64) { +func waitForTruncationForTesting( + t *testing.T, r *kvserver.Replica, newFirstIndex roachpb.RaftIndex, +) { testutils.SucceedsSoon(t, func() error { // Flush the engine to advance durability, which triggers truncation. require.NoError(t, r.Store().TODOEngine().Flush()) @@ -1032,7 +1034,7 @@ func TestSnapshotAfterTruncationWithUncommittedTail(t *testing.T) { // to make the test pass reliably. // NB: the Index on the message is the log index that _precedes_ any of the // entries in the MsgApp, so filter where msg.Index < index, not <= index. - return req.Message.Type == raftpb.MsgApp && req.Message.Index < index + return req.Message.Type == raftpb.MsgApp && roachpb.RaftIndex(req.Message.Index) < index }, dropHB: func(*kvserverpb.RaftHeartbeat) bool { return false }, dropResp: func(*kvserverpb.RaftMessageResponse) bool { return false }, @@ -3149,8 +3151,8 @@ func TestReplicaGCRace(t *testing.T) { return errors.Errorf("%+v has not yet advanced", progress) } for i := range hbReq.Heartbeats { - hbReq.Heartbeats[i].Term = status.Term - hbReq.Heartbeats[i].Commit = progress.Match + hbReq.Heartbeats[i].Term = roachpb.RaftTerm(status.Term) + hbReq.Heartbeats[i].Commit = roachpb.RaftIndex(progress.Match) } return nil }) @@ -4617,7 +4619,7 @@ func TestStoreRangeWaitForApplication(t *testing.T) { _, err := targets[2].client.WaitForApplication(ctx, &kvserver.WaitForApplicationRequest{ StoreRequestHeader: targets[2].header, RangeID: desc.RangeID, - LeaseIndex: math.MaxUint64, + LeaseIndex: math.MaxInt64, }) errChs[2] <- err }() @@ -4646,7 +4648,7 @@ func TestStoreRangeWaitForApplication(t *testing.T) { _, err := targets[0].client.WaitForApplication(ctx, &kvserver.WaitForApplicationRequest{ StoreRequestHeader: targets[0].header, RangeID: desc.RangeID, - LeaseIndex: math.MaxUint64, + LeaseIndex: math.MaxInt64, }) if exp := "context deadline exceeded"; !testutils.IsError(err, exp) { t.Fatalf("expected %q error, but got %v", exp, err) @@ -5680,7 +5682,7 @@ func TestElectionAfterRestart(t *testing.T) { testutils.SucceedsSoon(t, func() error { for rangeID := range rangeIDs { var err error - var lastIndex uint64 + var lastIndex roachpb.RaftIndex for _, srv := range tc.Servers { _ = srv.Stores().VisitStores(func(s *kvserver.Store) error { s.VisitReplicas(func(replica *kvserver.Replica) (more bool) { diff --git a/pkg/kv/kvserver/client_test.go b/pkg/kv/kvserver/client_test.go index f4191948ef64..7c0c9512b958 100644 --- a/pkg/kv/kvserver/client_test.go +++ b/pkg/kv/kvserver/client_test.go @@ -102,7 +102,7 @@ func incrementArgs(key roachpb.Key, inc int64) *kvpb.IncrementRequest { } } -func truncateLogArgs(index uint64, rangeID roachpb.RangeID) *kvpb.TruncateLogRequest { +func truncateLogArgs(index roachpb.RaftIndex, rangeID roachpb.RangeID) *kvpb.TruncateLogRequest { return &kvpb.TruncateLogRequest{ Index: index, RangeID: rangeID, diff --git a/pkg/kv/kvserver/closedts/ctpb/service.go b/pkg/kv/kvserver/closedts/ctpb/service.go index d975d50946d5..6f4f8ff73703 100644 --- a/pkg/kv/kvserver/closedts/ctpb/service.go +++ b/pkg/kv/kvserver/closedts/ctpb/service.go @@ -26,13 +26,6 @@ type SeqNum int64 // SafeValue implements the redact.SafeValue interface. func (SeqNum) SafeValue() {} -// LAI is an int64 denoting a lease applied index with its own type to avoid -// mix-ups in positional arguments. -type LAI int64 - -// SafeValue implements the redact.SafeValue interface. -func (LAI) SafeValue() {} - func (m *Update) String() string { sb := &strings.Builder{} fmt.Fprintf(sb, "Seq num: %d, sending node: n%d, snapshot: %t, size: %d bytes", diff --git a/pkg/kv/kvserver/closedts/ctpb/service.proto b/pkg/kv/kvserver/closedts/ctpb/service.proto index 0337bbdf1ec2..506079912063 100644 --- a/pkg/kv/kvserver/closedts/ctpb/service.proto +++ b/pkg/kv/kvserver/closedts/ctpb/service.proto @@ -84,7 +84,7 @@ message Update { // being included in the added_or_updated field of a future snapshot. message RangeUpdate { uint64 range_id = 1 [(gogoproto.customname) = "RangeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; - int64 lai = 2 [(gogoproto.customname) = "LAI", (gogoproto.casttype) = "LAI"]; + int64 lai = 2 [(gogoproto.customname) = "LAI", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.LeaseSequence"]; roachpb.RangeClosedTimestampPolicy policy = 3; } repeated RangeUpdate added_or_updated = 6 [(gogoproto.nullable) = false]; diff --git a/pkg/kv/kvserver/closedts/sidetransport/receiver.go b/pkg/kv/kvserver/closedts/sidetransport/receiver.go index 50f5c968af19..c08fbe1cc50d 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/receiver.go +++ b/pkg/kv/kvserver/closedts/sidetransport/receiver.go @@ -98,7 +98,7 @@ func (s *Receiver) PushUpdates(stream ctpb.SideTransport_PushUpdatesServer) erro // for closed timestamp info about this range. func (s *Receiver) GetClosedTimestamp( ctx context.Context, rangeID roachpb.RangeID, leaseholderNode roachpb.NodeID, -) (hlc.Timestamp, ctpb.LAI) { +) (hlc.Timestamp, roachpb.LeaseSequence) { s.mu.RLock() conn, ok := s.mu.conns[leaseholderNode] s.mu.RUnlock() @@ -190,7 +190,7 @@ type Stores interface { // ForwardSideTransportClosedTimestampForRange forwards the side-transport // closed timestamp for the local replica(s) of the given range. ForwardSideTransportClosedTimestampForRange( - ctx context.Context, rangeID roachpb.RangeID, closedTS hlc.Timestamp, lai ctpb.LAI) + ctx context.Context, rangeID roachpb.RangeID, closedTS hlc.Timestamp, lai roachpb.LeaseSequence) } func newIncomingStream(s *Receiver, stores Stores) *incomingStream { @@ -208,7 +208,7 @@ func newIncomingStream(s *Receiver, stores Stores) *incomingStream { // does not have state for the range. func (r *incomingStream) GetClosedTimestamp( ctx context.Context, rangeID roachpb.RangeID, -) (hlc.Timestamp, ctpb.LAI) { +) (hlc.Timestamp, roachpb.LeaseSequence) { r.mu.RLock() defer r.mu.RUnlock() info, ok := r.mu.tracked[rangeID] diff --git a/pkg/kv/kvserver/closedts/sidetransport/receiver_test.go b/pkg/kv/kvserver/closedts/sidetransport/receiver_test.go index b33b3baca0d6..82870edfa474 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/receiver_test.go +++ b/pkg/kv/kvserver/closedts/sidetransport/receiver_test.go @@ -32,13 +32,13 @@ type mockStores struct { type rangeUpdate struct { rid roachpb.RangeID closedTS hlc.Timestamp - lai ctpb.LAI + lai roachpb.LeaseSequence } var _ Stores = &mockStores{} func (m *mockStores) ForwardSideTransportClosedTimestampForRange( - ctx context.Context, rangeID roachpb.RangeID, closedTS hlc.Timestamp, lai ctpb.LAI, + ctx context.Context, rangeID roachpb.RangeID, closedTS hlc.Timestamp, lai roachpb.LeaseSequence, ) { upd := rangeUpdate{ rid: rangeID, @@ -64,12 +64,12 @@ var ts12 = hlc.Timestamp{WallTime: 12} var ts20 = hlc.Timestamp{WallTime: 20, Synthetic: true} var ts21 = hlc.Timestamp{WallTime: 21, Synthetic: true} var ts22 = hlc.Timestamp{WallTime: 22, Synthetic: true} -var laiZero = ctpb.LAI(0) +var laiZero = roachpb.LeaseSequence(0) -const lai100 = ctpb.LAI(100) -const lai101 = ctpb.LAI(101) -const lai102 = ctpb.LAI(102) -const lai103 = ctpb.LAI(102) +const lai100 = roachpb.LeaseSequence(100) +const lai101 = roachpb.LeaseSequence(101) +const lai102 = roachpb.LeaseSequence(102) +const lai103 = roachpb.LeaseSequence(103) func TestIncomingStreamProcessUpdateBasic(t *testing.T) { defer leaktest.AfterTest(t)() diff --git a/pkg/kv/kvserver/closedts/sidetransport/sender.go b/pkg/kv/kvserver/closedts/sidetransport/sender.go index 3644bb5f7b8e..310391ccd784 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/sender.go +++ b/pkg/kv/kvserver/closedts/sidetransport/sender.go @@ -114,7 +114,7 @@ type connTestingKnobs struct { // trackedRange contains the information that the side-transport last published // about a particular range. type trackedRange struct { - lai ctpb.LAI + lai roachpb.LeaseSequence policy roachpb.RangeClosedTimestampPolicy } @@ -167,7 +167,7 @@ type BumpSideTransportClosedResult struct { // Fields only set when ok. // The range's current LAI, to be associated with the closed timestamp. - LAI ctpb.LAI + LAI roachpb.LeaseSequence // The range's current policy. Policy roachpb.RangeClosedTimestampPolicy } diff --git a/pkg/kv/kvserver/closedts/sidetransport/sender_test.go b/pkg/kv/kvserver/closedts/sidetransport/sender_test.go index 612809789fdd..1352753ae685 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/sender_test.go +++ b/pkg/kv/kvserver/closedts/sidetransport/sender_test.go @@ -46,7 +46,7 @@ type mockReplica struct { canBump bool cantBumpReason CantCloseReason - lai ctpb.LAI + lai roachpb.LeaseSequence policy roachpb.RangeClosedTimestampPolicy } diff --git a/pkg/kv/kvserver/debug_print.go b/pkg/kv/kvserver/debug_print.go index d528d972c446..69d455a6b0f7 100644 --- a/pkg/kv/kvserver/debug_print.go +++ b/pkg/kv/kvserver/debug_print.go @@ -416,7 +416,7 @@ func tryRangeIDKey(kv storage.MVCCKeyValue) (string, error) { msg = &roachpb.Lease{} case bytes.Equal(suffix, keys.LocalRangeAppliedStateSuffix): - msg = &enginepb.RangeAppliedState{} + msg = &kvserverpb.RangeAppliedState{} case bytes.Equal(suffix, keys.LocalRangeStatsLegacySuffix): msg = &enginepb.MVCCStats{} diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index 234e5cf72d72..c5ec7de1c8db 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -264,7 +264,7 @@ func (r *Replica) RaftReportUnreachable(id roachpb.ReplicaID) error { } // LastAssignedLeaseIndexRLocked returns the last assigned lease index. -func (r *Replica) LastAssignedLeaseIndex() uint64 { +func (r *Replica) LastAssignedLeaseIndex() roachpb.LeaseSequence { r.mu.RLock() defer r.mu.RUnlock() return r.mu.proposalBuf.LastAssignedLeaseIndexRLocked() @@ -272,7 +272,7 @@ func (r *Replica) LastAssignedLeaseIndex() uint64 { // LastAssignedLeaseIndexRLocked is like LastAssignedLeaseIndex, but requires // b.mu to be held in read mode. -func (b *propBuf) LastAssignedLeaseIndexRLocked() uint64 { +func (b *propBuf) LastAssignedLeaseIndexRLocked() roachpb.LeaseSequence { return b.assignedLAI } @@ -283,9 +283,9 @@ func (b *propBuf) LastAssignedLeaseIndexRLocked() uint64 { func (r *Replica) InitQuotaPool(quota uint64) error { r.mu.Lock() defer r.mu.Unlock() - var appliedIndex uint64 + var appliedIndex roachpb.RaftIndex err := r.withRaftGroupLocked(false, func(r *raft.RawNode) (unquiesceAndWakeLeader bool, err error) { - appliedIndex = r.BasicStatus().Applied + appliedIndex = roachpb.RaftIndex(r.BasicStatus().Applied) return false, nil }) if err != nil { @@ -357,7 +357,7 @@ func (r *Replica) GetRaftLogSize() (int64, bool) { // GetCachedLastTerm returns the cached last term value. May return // invalidLastTerm if the cache is not set. -func (r *Replica) GetCachedLastTerm() uint64 { +func (r *Replica) GetCachedLastTerm() roachpb.RaftTerm { r.mu.RLock() defer r.mu.RUnlock() return r.mu.lastTermNotDurable diff --git a/pkg/kv/kvserver/kvserverbase/forced_error.go b/pkg/kv/kvserver/kvserverbase/forced_error.go index 4a4adbbccb0d..ef7ec6933fed 100644 --- a/pkg/kv/kvserver/kvserverbase/forced_error.go +++ b/pkg/kv/kvserver/kvserverbase/forced_error.go @@ -45,7 +45,7 @@ var NoopOnProbeCommandErr = kvpb.NewErrorf("no-op on ProbeRequest") // ForcedErrResult is the output from CheckForcedErr. type ForcedErrResult struct { - LeaseIndex uint64 + LeaseIndex roachpb.LeaseSequence Rejection ProposalRejectionType ForcedError *kvpb.Error } diff --git a/pkg/kv/kvserver/kvserverpb/BUILD.bazel b/pkg/kv/kvserver/kvserverpb/BUILD.bazel index 0a1ca39a65ba..a3bfdc6d4b1a 100644 --- a/pkg/kv/kvserver/kvserverpb/BUILD.bazel +++ b/pkg/kv/kvserver/kvserverpb/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "log.go", "proposer_kv.go", "raft.go", + "state.go", ], embed = [":kvserverpb_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb", diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.go b/pkg/kv/kvserver/kvserverpb/proposer_kv.go index d2af609b2649..025755ff1730 100644 --- a/pkg/kv/kvserver/kvserverpb/proposer_kv.go +++ b/pkg/kv/kvserver/kvserverpb/proposer_kv.go @@ -18,7 +18,7 @@ import ( ) var maxRaftCommandFooterSize = (&RaftCommandFooter{ - MaxLeaseIndex: math.MaxUint64, + MaxLeaseIndex: math.MaxInt64, ClosedTimestamp: hlc.Timestamp{ WallTime: math.MaxInt64, Logical: math.MaxInt32, diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto index f2969b396052..4622b8476d80 100644 --- a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto +++ b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto @@ -151,7 +151,7 @@ message ReplicatedEvalResult { // RaftExpectedFirstIndex is populated starting at cluster version // LooselyCoupledRaftLogTruncation. When this is not populated, the replica // should not delay enacting the truncation. - uint64 raft_expected_first_index = 25; + uint64 raft_expected_first_index = 25 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftIndex"]; // MVCCHistoryMutation describes mutations of MVCC history that may violate // the closed timestamp, timestamp cache, and guarantees that rely on these @@ -316,7 +316,7 @@ message RaftCommand { // max_lease_index is a primitive type, so it does not get encoded when zero. // This alone ensures that the field is not encoded twice in the combined // RaftCommand+MaxLeaseFooter proto. - uint64 max_lease_index = 4; + int64 max_lease_index = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.LeaseSequence"]; // The closed timestamp carried by this command. Once a follower is told to // apply this command, it knows that there will be no further writes at @@ -384,7 +384,7 @@ message RaftCommand { // appended to the marshaled byte buffer. This minimizes the memory allocation // and marshaling work performed under lock. message RaftCommandFooter { - uint64 max_lease_index = 4; + uint64 max_lease_index = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.LeaseSequence"]; // NOTE: unlike in RaftCommand, there's no reason to make this field nullable // and so we make it non-nullable in order to save allocations. This means // that the field on a decoded RaftCommand will also never be nil, but we diff --git a/pkg/kv/kvserver/kvserverpb/raft.proto b/pkg/kv/kvserver/kvserverpb/raft.proto index 322904c35b5c..c80080a340ab 100644 --- a/pkg/kv/kvserver/kvserverpb/raft.proto +++ b/pkg/kv/kvserver/kvserverpb/raft.proto @@ -33,8 +33,8 @@ message RaftHeartbeat { (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.ReplicaID"]; uint32 to_replica_id = 3 [(gogoproto.customname) = "ToReplicaID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.ReplicaID"]; - uint64 term = 4; - uint64 commit = 5; + uint64 term = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftTerm"]; + uint64 commit = 5 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftIndex"]; bool quiesce = 6; reserved 7; repeated kv.kvserver.liveness.livenesspb.Liveness lagging_followers_on_quiesce = 8 [(gogoproto.nullable) = false]; @@ -292,10 +292,10 @@ message DelegateSendSnapshotRequest { // The Raft term of the coordinator (in most cases the leaseholder) replica. // The term is used during snapshot receiving to reject messages from an older term. - uint64 term = 7; + uint64 term = 7 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftTerm"]; // The first index of the Raft log on the coordinator replica. - uint64 first_index = 8; + uint64 first_index = 8 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftIndex"]; // The sending queue's name. SnapshotRequest.QueueName sender_queue_name = 9; diff --git a/pkg/kv/kvserver/kvserverpb/state.go b/pkg/kv/kvserver/kvserverpb/state.go new file mode 100644 index 000000000000..a2d19435c225 --- /dev/null +++ b/pkg/kv/kvserver/kvserverpb/state.go @@ -0,0 +1,23 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// 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, included in the file +// licenses/APL.txt. + +package kvserverpb + +import "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + +// ToStats converts the receiver to an MVCCStats. +func (ms *MVCCPersistentStats) ToStats() enginepb.MVCCStats { + return enginepb.MVCCStats(*ms) +} + +// ToStatsPtr converts the receiver to a *MVCCStats. +func (ms *MVCCPersistentStats) ToStatsPtr() *enginepb.MVCCStats { + return (*enginepb.MVCCStats)(ms) +} diff --git a/pkg/kv/kvserver/kvserverpb/state.proto b/pkg/kv/kvserver/kvserverpb/state.proto index 2de02efa86d3..2273c677168a 100644 --- a/pkg/kv/kvserver/kvserverpb/state.proto +++ b/pkg/kv/kvserver/kvserverpb/state.proto @@ -35,9 +35,9 @@ message ReplicaState { option (gogoproto.equal) = true; // The highest (and last) index applied to the state machine. - uint64 raft_applied_index = 1; + uint64 raft_applied_index = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftIndex"]; // The highest (and last) lease index applied to the state machine. - uint64 lease_applied_index = 2; + int64 lease_applied_index = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.LeaseSequence"]; // The Range descriptor. // The pointer may change, but the referenced RangeDescriptor struct itself // must be treated as immutable; it is leaked out of the lock. @@ -136,7 +136,7 @@ message ReplicaState { // The term corresponding to RaftAppliedIndex. This is derived from // RangeAppliedState.RaftAppliedIndexTerm. - uint64 raft_applied_index_term = 14; + uint64 raft_applied_index_term = 14 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftTerm"]; // GCHint contains GC hint information for the replica. If hint // timestamp is set that means all data in the range is expected to be @@ -154,7 +154,7 @@ message RangeInfo { ReplicaState state = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; // The highest (and last) index in the Raft log. - uint64 last_index = 2; + uint64 last_index = 2 [(gogoproto.casttype) ="github.com/cockroachdb/cockroach/pkg/roachpb.RaftIndex"]; uint64 num_pending = 3; reserved 4; // previously last verification timestamp for verify queue. uint64 num_dropped = 5; @@ -203,9 +203,77 @@ message RangeSideTransportInfo { util.hlc.Timestamp replica_closed = 1 [(gogoproto.nullable) = false]; int64 replica_lai = 2 [(gogoproto.customname) = "ReplicaLAI", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb.LAI"]; + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.LeaseSequence"]; util.hlc.Timestamp central_closed = 3 [(gogoproto.nullable) = false]; int64 central_lai = 4 [(gogoproto.customname) = "CentralLAI", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb.LAI"]; + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.LeaseSequence"]; } + + +// RangeAppliedState combines the raft and lease applied indices with +// mvcc stats. These are all persisted on each transition of the Raft +// state machine (i.e. on each Raft application), so they are stored +// in the same RocksDB key for efficiency. +message RangeAppliedState { + option (gogoproto.equal) = true; + option (gogoproto.populate) = true; + + // raft_applied_index is the highest (and last) index applied to the Raft + // state machine. + uint64 raft_applied_index = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftIndex"]; + // lease_applied_index is the highest (and last) lease index applied to the + // Raft state machine. + // NB: The LeaseSequence is a int64, but this field is a uint64 for + // compatibility reasons. + uint64 lease_applied_index = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.LeaseSequence"]; + // range_stats is the set of mvcc stats that accounts for the current value + // of the Raft state machine. + MVCCPersistentStats range_stats = 3 [(gogoproto.nullable) = false]; + + // raft_closed_timestamp is the largest timestamp that is known to have been + // closed through Raft commands as of this lease applied index. This means + // that the current leaseholder (if any) and any future leaseholder will not + // evaluate writes at or below this timestamp, and also that any in-flight + // commands that can still apply are writing at higher timestamps. + // Non-leaseholder replicas are free to serve "follower reads" at or below + // this timestamp. + util.hlc.Timestamp raft_closed_timestamp = 4 [(gogoproto.nullable) = false]; + + // raft_applied_index_term is the term corresponding to raft_applied_index. + // The serialized proto will not contain this field until code starts + // setting it to a value > 0 (in v22.1). This is desirable since we don't + // want a mixed version cluster (v21.2 and v22.1) to have divergent replica + // state simply because we have introduced this field. + uint64 raft_applied_index_term = 5 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftTerm"]; +} +// MVCCPersistentStats is convertible to MVCCStats, but uses signed variable +// width encodings for most fields that make it efficient to store positive +// values but inefficient to store negative values. This makes the encodings +// incompatible. +message MVCCPersistentStats { + option (gogoproto.equal) = true; + option (gogoproto.populate) = true; + + int64 contains_estimates = 14; // must never go negative absent a bug + sfixed64 last_update_nanos = 1; + sfixed64 intent_age = 2; + sfixed64 gc_bytes_age = 3 [(gogoproto.customname) = "GCBytesAge"]; + int64 live_bytes = 4; + int64 live_count = 5; + int64 key_bytes = 6; + int64 key_count = 7; + int64 val_bytes = 8; + int64 val_count = 9; + int64 intent_bytes = 10; + int64 intent_count = 11; + int64 separated_intent_count = 16; + int64 range_key_count = 17; + int64 range_key_bytes = 18; + int64 range_val_count = 19; + int64 range_val_bytes = 20; + int64 sys_bytes = 12; + int64 sys_count = 13; + int64 abort_span_bytes = 15; +} + diff --git a/pkg/kv/kvserver/kvstorage/replica_state.go b/pkg/kv/kvserver/kvstorage/replica_state.go index ee397a5dcb5e..48c0b22fa26e 100644 --- a/pkg/kv/kvserver/kvstorage/replica_state.go +++ b/pkg/kv/kvserver/kvstorage/replica_state.go @@ -29,7 +29,7 @@ import ( // TODO(pavelkalinnikov): integrate with kvstorage.Replica. type LoadedReplicaState struct { ReplicaID roachpb.ReplicaID - LastIndex uint64 + LastIndex roachpb.RaftIndex ReplState kvserverpb.ReplicaState hardState raftpb.HardState diff --git a/pkg/kv/kvserver/logstore/logstore.go b/pkg/kv/kvserver/logstore/logstore.go index a621f71fb284..bbdf0ae32938 100644 --- a/pkg/kv/kvserver/logstore/logstore.go +++ b/pkg/kv/kvserver/logstore/logstore.go @@ -68,8 +68,8 @@ func MakeMsgStorageAppend(m raftpb.Message) MsgStorageAppend { // RaftState stores information about the last entry and the size of the log. type RaftState struct { - LastIndex uint64 - LastTerm uint64 + LastIndex roachpb.RaftIndex + LastTerm roachpb.RaftTerm ByteSize int64 } @@ -161,7 +161,7 @@ func (s *LogStore) storeEntriesAndCommitBatch( prevLastIndex := state.LastIndex overwriting := false if len(m.Entries) > 0 { - firstPurge := m.Entries[0].Index // first new entry written + firstPurge := roachpb.RaftIndex(m.Entries[0].Index) // first new entry written overwriting = firstPurge <= prevLastIndex stats.Begin = timeutil.Now() // All of the entries are appended to distinct keys, returning a new @@ -283,8 +283,8 @@ func (s *LogStore) storeEntriesAndCommitBatch( // entries that we didn't overwrite). Remove any such leftover on-disk // payloads (we can do that now because we've committed the deletion // just above). - firstPurge := m.Entries[0].Index // first new entry written - purgeTerm := m.Entries[0].Term - 1 + firstPurge := roachpb.RaftIndex(m.Entries[0].Index) // first new entry written + purgeTerm := roachpb.RaftTerm(m.Entries[0].Term - 1) lastPurge := prevLastIndex // old end of the log, include in deletion purgedSize, err := maybePurgeSideloaded(ctx, s.Sideload, firstPurge, lastPurge, purgeTerm) if err != nil { @@ -374,14 +374,14 @@ func logAppend( defer valPool.Put(value) for i := range entries { ent := &entries[i] - key := keys.RaftLogKeyFromPrefix(raftLogPrefix, ent.Index) + key := keys.RaftLogKeyFromPrefix(raftLogPrefix, roachpb.RaftIndex(ent.Index)) if err := value.SetProto(ent); err != nil { return RaftState{}, err } value.InitChecksum(key) var err error - if ent.Index > prev.LastIndex { + if roachpb.RaftIndex(ent.Index) > prev.LastIndex { err = storage.MVCCBlindPut(ctx, rw, &diff, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, *value, nil /* txn */) } else { err = storage.MVCCPut(ctx, rw, &diff, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, *value, nil /* txn */) @@ -391,7 +391,7 @@ func logAppend( } } - newLastIndex := entries[len(entries)-1].Index + newLastIndex := roachpb.RaftIndex(entries[len(entries)-1].Index) // Delete any previously appended log entries which never committed. if prev.LastIndex > 0 { for i := newLastIndex + 1; i <= prev.LastIndex; i++ { @@ -406,7 +406,7 @@ func logAppend( } return RaftState{ LastIndex: newLastIndex, - LastTerm: entries[len(entries)-1].Term, + LastTerm: roachpb.RaftTerm(entries[len(entries)-1].Term), ByteSize: prev.ByteSize + diff.SysBytes, }, nil } @@ -419,11 +419,11 @@ func LoadTerm( eng storage.Engine, rangeID roachpb.RangeID, eCache *raftentry.Cache, - index uint64, -) (uint64, error) { + index roachpb.RaftIndex, +) (roachpb.RaftTerm, error) { entry, found := eCache.Get(rangeID, index) if found { - return entry.Term, nil + return roachpb.RaftTerm(entry.Term), nil } reader := eng.NewReadOnly(storage.StandardDurability) @@ -442,7 +442,7 @@ func LoadTerm( if found { // Found an entry. Double-check that it has a correct index. - if got, want := entry.Index, index; got != want { + if got, want := roachpb.RaftIndex(entry.Index), index; got != want { return 0, errors.Errorf("there is a gap at index %d, found entry #%d", want, got) } // Cache the entry except if it is sideloaded. We don't load/inline the @@ -456,7 +456,7 @@ func LoadTerm( if !typ.IsSideloaded() { eCache.Add(rangeID, []raftpb.Entry{entry}, false /* truncate */) } - return entry.Term, nil + return roachpb.RaftTerm(entry.Term), nil } // Otherwise, the entry at the given index is not found. This can happen if // the index is ahead of lastIndex, or it has been compacted away. @@ -495,7 +495,8 @@ func LoadEntries( rangeID roachpb.RangeID, eCache *raftentry.Cache, sideloaded SideloadStorage, - lo, hi, maxBytes uint64, + lo, hi roachpb.RaftIndex, + maxBytes uint64, ) ([]raftpb.Entry, error) { if lo > hi { return nil, errors.Errorf("lo:%d is greater than hi:%d", lo, hi) @@ -511,7 +512,7 @@ func LoadEntries( // Return results if the correct number of results came back or if // we ran into the max bytes limit. - if uint64(len(ents)) == hi-lo || exceededMaxBytes { + if roachpb.RaftIndex(len(ents)) == hi-lo || exceededMaxBytes { return ents, nil } @@ -521,7 +522,7 @@ func LoadEntries( scanFunc := func(ent raftpb.Entry) error { // Exit early if we have any gaps or it has been compacted. - if ent.Index != expectedIndex { + if roachpb.RaftIndex(ent.Index) != expectedIndex { return iterutil.StopIteration() } expectedIndex++ @@ -564,7 +565,7 @@ func LoadEntries( eCache.Add(rangeID, ents, false /* truncate */) // Did the correct number of results come back? If so, we're all good. - if uint64(len(ents)) == hi-lo { + if roachpb.RaftIndex(len(ents)) == hi-lo { return ents, nil } diff --git a/pkg/kv/kvserver/logstore/sideload.go b/pkg/kv/kvserver/logstore/sideload.go index 2c21a24fae44..db3d5ff83fb8 100644 --- a/pkg/kv/kvserver/logstore/sideload.go +++ b/pkg/kv/kvserver/logstore/sideload.go @@ -32,30 +32,30 @@ type SideloadStorage interface { Dir() string // Writes the given contents to the file specified by the given index and // term. Overwrites the file if it already exists. - Put(_ context.Context, index, term uint64, contents []byte) error + Put(_ context.Context, index roachpb.RaftIndex, term roachpb.RaftTerm, contents []byte) error // Load the file at the given index and term. Return errSideloadedFileNotFound when no // such file is present. - Get(_ context.Context, index, term uint64) ([]byte, error) + Get(_ context.Context, index roachpb.RaftIndex, term roachpb.RaftTerm) ([]byte, error) // Purge removes the file at the given index and term. It may also // remove any leftover files at the same index and earlier terms, but // is not required to do so. When no file at the given index and term // exists, returns errSideloadedFileNotFound. // // Returns the total size of the purged payloads. - Purge(_ context.Context, index, term uint64) (int64, error) + Purge(_ context.Context, index roachpb.RaftIndex, term roachpb.RaftTerm) (int64, error) // Clear files that may have been written by this SideloadStorage. Clear(context.Context) error // TruncateTo removes all files belonging to an index strictly smaller than // the given one. Returns the number of bytes freed, the number of bytes in // files that remain, or an error. - TruncateTo(_ context.Context, index uint64) (freed, retained int64, _ error) + TruncateTo(_ context.Context, index roachpb.RaftIndex) (freed, retained int64, _ error) // BytesIfTruncatedFromTo returns the number of bytes that would be freed, // if one were to truncate [from, to). Additionally, it returns the the // number of bytes that would be retained >= to. - BytesIfTruncatedFromTo(_ context.Context, from, to uint64) (freed, retained int64, _ error) + BytesIfTruncatedFromTo(_ context.Context, from roachpb.RaftIndex, to roachpb.RaftIndex) (freed, retained int64, _ error) // Returns an absolute path to the file that Get() would return the contents // of. Does not check whether the file actually exists. - Filename(_ context.Context, index, term uint64) (string, error) + Filename(_ context.Context, index roachpb.RaftIndex, term roachpb.RaftTerm) (string, error) } // MaybeSideloadEntries optimizes handling for AddSST requests. AddSST are @@ -133,7 +133,7 @@ func MaybeSideloadEntries( } log.Eventf(ctx, "writing payload at index=%d term=%d", outputEnt.Index, outputEnt.Term) - if err := sideloaded.Put(ctx, outputEnt.Index, outputEnt.Term, dataToSideload); err != nil { // TODO could verify checksum here + if err := sideloaded.Put(ctx, roachpb.RaftIndex(outputEnt.Index), roachpb.RaftTerm(outputEnt.Term), dataToSideload); err != nil { // TODO could verify checksum here return nil, 0, 0, 0, err } sideloadedEntriesSize += int64(len(dataToSideload)) @@ -173,7 +173,7 @@ func MaybeInlineSideloadedRaftCommand( // are very likely to have appended it recently, in which case // we can save work. cachedSingleton, _, _, _ := entryCache.Scan( - nil, rangeID, ent.Index, ent.Index+1, 1<<20, + nil, rangeID, roachpb.RaftIndex(ent.Index), roachpb.RaftIndex(ent.Index+1), 1<<20, ) if len(cachedSingleton) > 0 { @@ -204,7 +204,7 @@ func MaybeInlineSideloadedRaftCommand( return &ent, nil } - sideloadedData, err := sideloaded.Get(ctx, ent.Index, ent.Term) + sideloadedData, err := sideloaded.Get(ctx, roachpb.RaftIndex(ent.Index), roachpb.RaftTerm(ent.Term)) if err != nil { return nil, errors.Wrap(err, "loading sideloaded data") } @@ -251,7 +251,10 @@ func AssertSideloadedRaftCommandInlined(ctx context.Context, ent *raftpb.Entry) // and returns the total number of bytes removed. Nonexistent entries are // silently skipped over. func maybePurgeSideloaded( - ctx context.Context, ss SideloadStorage, firstIndex, lastIndex uint64, term uint64, + ctx context.Context, + ss SideloadStorage, + firstIndex, lastIndex roachpb.RaftIndex, + term roachpb.RaftTerm, ) (int64, error) { var totalSize int64 for i := firstIndex; i <= lastIndex; i++ { diff --git a/pkg/kv/kvserver/logstore/sideload_disk.go b/pkg/kv/kvserver/logstore/sideload_disk.go index e2bfc6765581..d716327314d2 100644 --- a/pkg/kv/kvserver/logstore/sideload_disk.go +++ b/pkg/kv/kvserver/logstore/sideload_disk.go @@ -87,7 +87,9 @@ func (ss *DiskSideloadStorage) Dir() string { } // Put implements SideloadStorage. -func (ss *DiskSideloadStorage) Put(ctx context.Context, index, term uint64, contents []byte) error { +func (ss *DiskSideloadStorage) Put( + ctx context.Context, index roachpb.RaftIndex, term roachpb.RaftTerm, contents []byte, +) error { filename := ss.filename(ctx, index, term) // There's a chance the whole path is missing (for example after Clear()), // in which case handle that transparently. @@ -109,7 +111,9 @@ func (ss *DiskSideloadStorage) Put(ctx context.Context, index, term uint64, cont } // Get implements SideloadStorage. -func (ss *DiskSideloadStorage) Get(ctx context.Context, index, term uint64) ([]byte, error) { +func (ss *DiskSideloadStorage) Get( + ctx context.Context, index roachpb.RaftIndex, term roachpb.RaftTerm, +) ([]byte, error) { filename := ss.filename(ctx, index, term) b, err := fs.ReadFile(ss.eng, filename) if oserror.IsNotExist(err) { @@ -119,16 +123,22 @@ func (ss *DiskSideloadStorage) Get(ctx context.Context, index, term uint64) ([]b } // Filename implements SideloadStorage. -func (ss *DiskSideloadStorage) Filename(ctx context.Context, index, term uint64) (string, error) { +func (ss *DiskSideloadStorage) Filename( + ctx context.Context, index roachpb.RaftIndex, term roachpb.RaftTerm, +) (string, error) { return ss.filename(ctx, index, term), nil } -func (ss *DiskSideloadStorage) filename(ctx context.Context, index, term uint64) string { +func (ss *DiskSideloadStorage) filename( + ctx context.Context, index roachpb.RaftIndex, term roachpb.RaftTerm, +) string { return filepath.Join(ss.dir, fmt.Sprintf("i%d.t%d", index, term)) } // Purge implements SideloadStorage. -func (ss *DiskSideloadStorage) Purge(ctx context.Context, index, term uint64) (int64, error) { +func (ss *DiskSideloadStorage) Purge( + ctx context.Context, index roachpb.RaftIndex, term roachpb.RaftTerm, +) (int64, error) { return ss.purgeFile(ctx, ss.filename(ctx, index, term)) } @@ -166,17 +176,17 @@ func (ss *DiskSideloadStorage) Clear(_ context.Context) error { // TruncateTo implements SideloadStorage. func (ss *DiskSideloadStorage) TruncateTo( - ctx context.Context, firstIndex uint64, + ctx context.Context, firstIndex roachpb.RaftIndex, ) (bytesFreed, bytesRetained int64, _ error) { return ss.possiblyTruncateTo(ctx, 0, firstIndex, true /* doTruncate */) } // Helper for truncation or byte calculation for [from, to). func (ss *DiskSideloadStorage) possiblyTruncateTo( - ctx context.Context, from uint64, to uint64, doTruncate bool, + ctx context.Context, from roachpb.RaftIndex, to roachpb.RaftIndex, doTruncate bool, ) (bytesFreed, bytesRetained int64, _ error) { deletedAll := true - if err := ss.forEach(ctx, func(index uint64, filename string) error { + if err := ss.forEach(ctx, func(index roachpb.RaftIndex, filename string) error { if index >= to { size, err := ss.fileSize(filename) if err != nil { @@ -220,13 +230,13 @@ func (ss *DiskSideloadStorage) possiblyTruncateTo( // BytesIfTruncatedFromTo implements SideloadStorage. func (ss *DiskSideloadStorage) BytesIfTruncatedFromTo( - ctx context.Context, from uint64, to uint64, + ctx context.Context, from roachpb.RaftIndex, to roachpb.RaftIndex, ) (freed, retained int64, _ error) { return ss.possiblyTruncateTo(ctx, from, to, false /* doTruncate */) } func (ss *DiskSideloadStorage) forEach( - ctx context.Context, visit func(index uint64, filename string) error, + ctx context.Context, visit func(index roachpb.RaftIndex, filename string) error, ) error { matches, err := ss.eng.List(ss.dir) if oserror.IsNotExist(err) { @@ -253,7 +263,7 @@ func (ss *DiskSideloadStorage) forEach( log.Infof(ctx, "unexpected file %s in sideloaded directory %s", match, ss.dir) continue } - if err := visit(logIdx, match); err != nil { + if err := visit(roachpb.RaftIndex(logIdx), match); err != nil { return errors.Wrapf(err, "matching pattern %q on dir %s", match, ss.dir) } } @@ -264,7 +274,7 @@ func (ss *DiskSideloadStorage) forEach( func (ss *DiskSideloadStorage) String() string { var buf strings.Builder var count int - if err := ss.forEach(context.Background(), func(_ uint64, filename string) error { + if err := ss.forEach(context.Background(), func(_ roachpb.RaftIndex, filename string) error { count++ _, _ = fmt.Fprintln(&buf, filename) return nil diff --git a/pkg/kv/kvserver/logstore/sideload_test.go b/pkg/kv/kvserver/logstore/sideload_test.go index 800ff9b9ce7b..bb7f04154c76 100644 --- a/pkg/kv/kvserver/logstore/sideload_test.go +++ b/pkg/kv/kvserver/logstore/sideload_test.go @@ -114,11 +114,11 @@ func testSideloadingSideloadedStorage(t *testing.T, eng storage.Engine) { highTerm ) - file := func(i uint64) []byte { // take uint64 for convenience - return []byte("content-" + strconv.Itoa(int(i))) + file := func(index roachpb.RaftIndex, term roachpb.RaftTerm) []byte { // take uint64 for convenience + return []byte("content-" + strconv.Itoa(int(index)*int(term))) } - if err := ss.Put(ctx, 1, highTerm, file(1)); err != nil { + if err := ss.Put(ctx, 1, highTerm, file(1, 1)); err != nil { t.Fatal(err) } @@ -126,19 +126,19 @@ func testSideloadingSideloadedStorage(t *testing.T, eng storage.Engine) { if c, err := ss.Get(ctx, 1, highTerm); err != nil { t.Fatal(err) - } else if exp := file(1); !bytes.Equal(c, exp) { + } else if exp := file(1, 1); !bytes.Equal(c, exp) { t.Fatalf("got %q, wanted %q", c, exp) } // Overwrites the occupied slot. - if err := ss.Put(ctx, 1, highTerm, file(12345)); err != nil { + if err := ss.Put(ctx, 1, highTerm, file(12345, 1)); err != nil { t.Fatal(err) } // ... consequently the old entry is gone. if c, err := ss.Get(ctx, 1, highTerm); err != nil { t.Fatal(err) - } else if exp := file(12345); !bytes.Equal(c, exp) { + } else if exp := file(12345, 1); !bytes.Equal(c, exp) { t.Fatalf("got %q, wanted %q", c, exp) } @@ -192,10 +192,10 @@ func testSideloadingSideloadedStorage(t *testing.T, eng storage.Engine) { // Write some payloads at various indexes. Note that this tests Put // on a recently Clear()ed storage. Randomize order for fun. - payloads := []uint64{3, 5, 7, 9, 10} + payloads := []roachpb.RaftIndex{3, 5, 7, 9, 10} for n := range rand.Perm(len(payloads)) { i := payloads[n] - if err := ss.Put(ctx, i, highTerm, file(i*highTerm)); err != nil { + if err := ss.Put(ctx, i, highTerm, file(i, highTerm)); err != nil { t.Fatalf("%d: %+v", i, err) } } @@ -203,19 +203,19 @@ func testSideloadingSideloadedStorage(t *testing.T, eng storage.Engine) { assertCreated(true) // Write some more payloads, overlapping, at the past term. - pastPayloads := append([]uint64{81}, payloads...) + pastPayloads := append([]roachpb.RaftIndex{81}, payloads...) for _, i := range pastPayloads { - if err := ss.Put(ctx, i, lowTerm, file(i*lowTerm)); err != nil { + if err := ss.Put(ctx, i, lowTerm, file(i, lowTerm)); err != nil { t.Fatal(err) } } // Just a sanity check that for the overlapping terms, we see both entries. - for _, term := range []uint64{lowTerm, highTerm} { + for _, term := range []roachpb.RaftTerm{lowTerm, highTerm} { index := payloads[0] // exists at both lowTerm and highTerm if c, err := ss.Get(ctx, index, term); err != nil { t.Fatal(err) - } else if exp := file(term * index); !bytes.Equal(c, exp) { + } else if exp := file(index, term); !bytes.Equal(c, exp) { t.Fatalf("got %q, wanted %q", c, exp) } } @@ -238,7 +238,7 @@ func testSideloadingSideloadedStorage(t *testing.T, eng storage.Engine) { require.Equal(t, retainedByTruncateTo, retained) // Index payloads[n] and above are still there (truncation is exclusive) // at both terms. - for _, term := range []uint64{lowTerm, highTerm} { + for _, term := range []roachpb.RaftTerm{lowTerm, highTerm} { for _, i := range payloads[n:] { if _, err := ss.Get(ctx, i, term); err != nil { t.Fatalf("%d.%d: %+v", n, i, err) @@ -286,10 +286,10 @@ func testSideloadingSideloadedStorage(t *testing.T, eng storage.Engine) { // Repopulate with some random indexes to test deletion when there are a // non-zero number of filepath.Glob matches. - payloads := []uint64{3, 5, 7, 9, 10} + payloads := []roachpb.RaftIndex{3, 5, 7, 9, 10} for n := range rand.Perm(len(payloads)) { i := payloads[n] - require.NoError(t, ss.Put(ctx, i, highTerm, file(i*highTerm))) + require.NoError(t, ss.Put(ctx, i, highTerm, file(i, highTerm))) } assertCreated(true) freed, retained, err := ss.BytesIfTruncatedFromTo(ctx, 0, math.MaxUint64) @@ -323,7 +323,7 @@ func testSideloadingSideloadedStorage(t *testing.T, eng storage.Engine) { // Repopulate with a few entries at indexes=1,2,4 and term 10 to test `maybePurgeSideloaded` // with. - for index := uint64(1); index < 5; index++ { + for index := roachpb.RaftIndex(1); index < 5; index++ { if index == 3 { continue } @@ -332,7 +332,7 @@ func testSideloadingSideloadedStorage(t *testing.T, eng storage.Engine) { } // Term too high and too low, respectively. Shouldn't delete anything. - for _, term := range []uint64{9, 11} { + for _, term := range []roachpb.RaftTerm{9, 11} { if size, err := maybePurgeSideloaded(ctx, ss, 1, 10, term); err != nil || size != 0 { t.Fatalf("expected noop for term %d, got (%d, %v)", term, size, err) } diff --git a/pkg/kv/kvserver/logstore/stateloader.go b/pkg/kv/kvserver/logstore/stateloader.go index e36ca0639208..470310d1a6cf 100644 --- a/pkg/kv/kvserver/logstore/stateloader.go +++ b/pkg/kv/kvserver/logstore/stateloader.go @@ -50,13 +50,15 @@ func NewStateLoader(rangeID roachpb.RangeID) StateLoader { } // LoadLastIndex loads the last index. -func (sl StateLoader) LoadLastIndex(ctx context.Context, reader storage.Reader) (uint64, error) { +func (sl StateLoader) LoadLastIndex( + ctx context.Context, reader storage.Reader, +) (roachpb.RaftIndex, error) { prefix := sl.RaftLogPrefix() // NB: raft log has no intents. iter := reader.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{LowerBound: prefix}) defer iter.Close() - var lastIndex uint64 + var lastIndex roachpb.RaftIndex iter.SeekLT(storage.MakeMVCCMetadataKey(keys.RaftLogKeyFromPrefix(prefix, math.MaxUint64))) if ok, _ := iter.Valid(); ok { key := iter.UnsafeKey().Key @@ -154,13 +156,13 @@ func (sl StateLoader) SynthesizeHardState( readWriter storage.ReadWriter, oldHS raftpb.HardState, truncState roachpb.RaftTruncatedState, - raftAppliedIndex uint64, + raftAppliedIndex roachpb.RaftIndex, ) error { newHS := raftpb.HardState{ - Term: truncState.Term, + Term: uint64(truncState.Term), // Note that when applying a Raft snapshot, the applied index is // equal to the Commit index represented by the snapshot. - Commit: raftAppliedIndex, + Commit: uint64(raftAppliedIndex), } if oldHS.Commit > newHS.Commit { diff --git a/pkg/kv/kvserver/loqrecovery/collect.go b/pkg/kv/kvserver/loqrecovery/collect.go index 487c5d31e90b..848ee4e57a54 100644 --- a/pkg/kv/kvserver/loqrecovery/collect.go +++ b/pkg/kv/kvserver/loqrecovery/collect.go @@ -194,7 +194,7 @@ func visitStoreReplicas( NodeID: nodeID, Desc: desc, RaftAppliedIndex: rstate.RaftAppliedIndex, - RaftCommittedIndex: hstate.Commit, + RaftCommittedIndex: roachpb.RaftIndex(hstate.Commit), RaftLogDescriptorChanges: rangeUpdates, LocalAssumesLeaseholder: localIsLeaseholder, }) @@ -208,7 +208,7 @@ func visitStoreReplicas( // lo (inclusive) and hi (exclusive) and searches for changes to range // descriptors, as identified by presence of a commit trigger. func GetDescriptorChangesFromRaftLog( - rangeID roachpb.RangeID, lo, hi uint64, reader storage.Reader, + rangeID roachpb.RangeID, lo, hi roachpb.RaftIndex, reader storage.Reader, ) ([]loqrecoverypb.DescriptorChangeInfo, error) { var changes []loqrecoverypb.DescriptorChangeInfo if err := raftlog.Visit(reader, rangeID, lo, hi, func(ent raftpb.Entry) error { diff --git a/pkg/kv/kvserver/loqrecovery/loqrecoverypb/recovery.proto b/pkg/kv/kvserver/loqrecovery/loqrecoverypb/recovery.proto index 9e78dd63e261..c9142188d00c 100644 --- a/pkg/kv/kvserver/loqrecovery/loqrecoverypb/recovery.proto +++ b/pkg/kv/kvserver/loqrecovery/loqrecoverypb/recovery.proto @@ -51,8 +51,8 @@ message ReplicaInfo { int32 store_id = 2 [(gogoproto.customname) = "StoreID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID"]; roachpb.RangeDescriptor desc = 3 [(gogoproto.nullable) = false]; - uint64 raft_applied_index = 4; - uint64 raft_committed_index = 5; + uint64 raft_applied_index = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftIndex"] ; + uint64 raft_committed_index = 5 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RaftIndex"]; repeated DescriptorChangeInfo raft_log_descriptor_changes = 6 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "raft_log_descriptor_changes,omitempty"]; bool local_assumes_leaseholder = 7 [ diff --git a/pkg/kv/kvserver/loqrecovery/recovery_env_test.go b/pkg/kv/kvserver/loqrecovery/recovery_env_test.go index 556acf11e9f7..b3d007bb0e15 100644 --- a/pkg/kv/kvserver/loqrecovery/recovery_env_test.go +++ b/pkg/kv/kvserver/loqrecovery/recovery_env_test.go @@ -115,8 +115,8 @@ type testReplicaInfo struct { Generation roachpb.RangeGeneration `yaml:"Generation,omitempty"` // Raft state. - RangeAppliedIndex uint64 `yaml:"RangeAppliedIndex"` - RaftCommittedIndex uint64 `yaml:"RaftCommittedIndex"` + RangeAppliedIndex roachpb.RaftIndex `yaml:"RangeAppliedIndex"` + RaftCommittedIndex roachpb.RaftIndex `yaml:"RaftCommittedIndex"` DescriptorUpdates []testReplicaDescriptorChange `yaml:"DescriptorUpdates,flow,omitempty"` // TODO(oleg): Add ability to have descriptor intents in the store for testing purposes @@ -345,7 +345,7 @@ func (e *quorumRecoveryEnv) handleReplicationData(t *testing.T, d datadriven.Tes t.Fatalf("failed to serialize metadata entry for raft log") } if err := eng.PutUnversioned(keys.RaftLogKey(replica.RangeID, - uint64(i)+hardState.Commit+1), value); err != nil { + roachpb.RaftIndex(uint64(i)+hardState.Commit+1)), value); err != nil { t.Fatalf("failed to insert raft log entry into store: %s", err) } } @@ -422,11 +422,11 @@ func buildReplicaDescriptorFromTestData( hardState := raftpb.HardState{ Term: 0, Vote: 0, - Commit: replica.RaftCommittedIndex, + Commit: uint64(replica.RaftCommittedIndex), } var raftLog []enginepb.MVCCMetadata for i, u := range replica.DescriptorUpdates { - entry := raftLogFromPendingDescriptorUpdate(t, replica, u, desc, uint64(i)) + entry := raftLogFromPendingDescriptorUpdate(t, replica, u, desc, roachpb.RaftIndex(i)) raftLog = append(raftLog, enginepb.MVCCMetadata{RawBytes: entry.RawBytes}) } return replicaID, key, desc, replicaState, hardState, raftLog @@ -437,7 +437,7 @@ func raftLogFromPendingDescriptorUpdate( replica testReplicaInfo, update testReplicaDescriptorChange, desc roachpb.RangeDescriptor, - entryIndex uint64, + entryIndex roachpb.RaftIndex, ) roachpb.Value { // We mimic EndTxn messages with commit triggers here. We don't construct // full batches with descriptor updates as we only need data that would be @@ -488,7 +488,7 @@ func raftLogFromPendingDescriptorUpdate( raftlog.EntryEncodingStandardWithoutAC, kvserverbase.CmdIDKey(fmt.Sprintf("%08d", entryIndex)), out) ent := raftpb.Entry{ Term: 1, - Index: replica.RaftCommittedIndex + entryIndex, + Index: uint64(replica.RaftCommittedIndex + entryIndex), Type: raftpb.EntryNormal, Data: data, } diff --git a/pkg/kv/kvserver/raft_log_queue.go b/pkg/kv/kvserver/raft_log_queue.go index e27095d3e991..3e917666f4d6 100644 --- a/pkg/kv/kvserver/raft_log_queue.go +++ b/pkg/kv/kvserver/raft_log_queue.go @@ -374,8 +374,8 @@ type truncateDecisionInput struct { RaftStatus raft.Status LogSize, MaxLogSize int64 LogSizeTrusted bool // false when LogSize might be off - FirstIndex, LastIndex uint64 - PendingSnapshotIndex uint64 + FirstIndex, LastIndex roachpb.RaftIndex + PendingSnapshotIndex roachpb.RaftIndex } func (input truncateDecisionInput) LogTooLarge() bool { @@ -388,13 +388,13 @@ func (input truncateDecisionInput) LogTooLarge() bool { // cluster data. type truncateDecision struct { Input truncateDecisionInput - CommitIndex uint64 + CommitIndex roachpb.RaftIndex - NewFirstIndex uint64 // first index of the resulting log after truncation + NewFirstIndex roachpb.RaftIndex // first index of the resulting log after truncation ChosenVia string } -func (td *truncateDecision) raftSnapshotsForIndex(index uint64) int { +func (td *truncateDecision) raftSnapshotsForIndex(index roachpb.RaftIndex) int { var n int for _, p := range td.Input.RaftStatus.Progress { if p.State != tracker.StateReplicate { @@ -412,7 +412,7 @@ func (td *truncateDecision) raftSnapshotsForIndex(index uint64) int { // need a truncation to catch up. A follower in that state will have a // Match equaling committed-1, but a Next of committed+1 (indicating that // an append at 'committed' is already ongoing). - if p.Match < index && p.Next <= index { + if roachpb.RaftIndex(p.Match) < index && roachpb.RaftIndex(p.Next) <= index { n++ } } @@ -476,7 +476,7 @@ func (td *truncateDecision) ShouldTruncate() bool { // if it would be truncating at a point past it. If a change is made, the // ChosenVia is updated with the one given. This protection is not guaranteed if // the protected index is outside of the existing [FirstIndex,LastIndex] bounds. -func (td *truncateDecision) ProtectIndex(index uint64, chosenVia string) { +func (td *truncateDecision) ProtectIndex(index roachpb.RaftIndex, chosenVia string) { if td.NewFirstIndex > index { td.NewFirstIndex = index td.ChosenVia = chosenVia @@ -502,7 +502,7 @@ func (td *truncateDecision) ProtectIndex(index uint64, chosenVia string) { // snapshots. See #8629. func computeTruncateDecision(input truncateDecisionInput) truncateDecision { decision := truncateDecision{Input: input} - decision.CommitIndex = input.RaftStatus.Commit + decision.CommitIndex = roachpb.RaftIndex(input.RaftStatus.Commit) // The last index is most aggressive possible truncation that we could do. // Everything else in this method makes the truncation less aggressive. @@ -545,7 +545,7 @@ func computeTruncateDecision(input truncateDecisionInput) truncateDecision { if progress.State == tracker.StateProbe { decision.ProtectIndex(input.FirstIndex, truncatableIndexChosenViaProbingFollower) } else { - decision.ProtectIndex(progress.Match, truncatableIndexChosenViaFollowers) + decision.ProtectIndex(roachpb.RaftIndex(progress.Match), truncatableIndexChosenViaFollowers) } continue } @@ -553,7 +553,7 @@ func computeTruncateDecision(input truncateDecisionInput) truncateDecision { // Second, if the follower has not been recently active, we don't // truncate it off as long as the raft log is not too large. if !input.LogTooLarge() { - decision.ProtectIndex(progress.Match, truncatableIndexChosenViaFollowers) + decision.ProtectIndex(roachpb.RaftIndex(progress.Match), truncatableIndexChosenViaFollowers) } // Otherwise, we let it truncate to the committed index. @@ -613,7 +613,7 @@ func computeTruncateDecision(input truncateDecisionInput) truncateDecision { // https://github.com/nvanbenschoten/optional could help us emulate an // `option` type if we care enough. logEmpty := input.FirstIndex > input.LastIndex - noCommittedEntries := input.FirstIndex > input.RaftStatus.Commit + noCommittedEntries := input.FirstIndex > roachpb.RaftIndex(input.RaftStatus.Commit) logIndexValid := logEmpty || (decision.NewFirstIndex >= input.FirstIndex) && (decision.NewFirstIndex <= input.LastIndex) diff --git a/pkg/kv/kvserver/raft_log_queue_test.go b/pkg/kv/kvserver/raft_log_queue_test.go index a80d5dd1e948..635eccc97e36 100644 --- a/pkg/kv/kvserver/raft_log_queue_test.go +++ b/pkg/kv/kvserver/raft_log_queue_test.go @@ -42,7 +42,7 @@ func TestShouldTruncate(t *testing.T) { defer log.Scope(t).Close(t) testCases := []struct { - truncatableIndexes uint64 + truncatableIndexes roachpb.RaftIndex raftLogSize int64 expected bool }{ @@ -77,12 +77,12 @@ func TestComputeTruncateDecision(t *testing.T) { // truncate: false", because these tests don't simulate enough data to be over // the truncation threshold. testCases := []struct { - commit uint64 + commit roachpb.RaftIndex progress []uint64 raftLogSize int64 - firstIndex uint64 - lastIndex uint64 - pendingSnapshot uint64 + firstIndex roachpb.RaftIndex + lastIndex roachpb.RaftIndex + pendingSnapshot roachpb.RaftIndex exp string }{ { @@ -183,7 +183,7 @@ func TestComputeTruncateDecision(t *testing.T) { Next: v + 1, } } - status.Commit = c.commit + status.Commit = uint64(c.commit) input := truncateDecisionInput{ RaftStatus: status, LogSize: c.raftLogSize, @@ -248,8 +248,8 @@ func TestComputeTruncateDecisionProgressStatusProbe(t *testing.T) { status := raft.Status{ Progress: make(map[uint64]tracker.Progress), } - progress := []uint64{100, 200, 300, 400, 500} - lastIndex := uint64(500) + progress := []roachpb.RaftIndex{100, 200, 300, 400, 500} + lastIndex := roachpb.RaftIndex(500) status.Commit = 300 for i, v := range progress { @@ -262,12 +262,12 @@ func TestComputeTruncateDecisionProgressStatusProbe(t *testing.T) { RecentActive: active, State: tracker.StateProbe, Match: 0, - Next: v, + Next: uint64(v), } } else { // everyone else pr = tracker.Progress{ - Match: v, - Next: v + 1, + Match: uint64(v), + Next: uint64(v + 1), RecentActive: true, State: tracker.StateReplicate, } @@ -472,7 +472,7 @@ func TestNewTruncateDecision(t *testing.T) { t.Fatal(err) } - getIndexes := func() (uint64, int, uint64, error) { + getIndexes := func() (roachpb.RaftIndex, int, roachpb.RaftIndex, error) { d, err := newTruncateDecision(ctx, r) if err != nil { return 0, 0, 0, err @@ -518,7 +518,7 @@ func TestNewTruncateDecision(t *testing.T) { // There can be a delay from when the truncation command is issued and the // indexes updating. - var cFirst, cOldest uint64 + var cFirst, cOldest roachpb.RaftIndex var numTruncatable int testutils.SucceedsSoon(t, func() error { var err error @@ -666,7 +666,7 @@ func TestSnapshotLogTruncationConstraints(t *testing.T) { // Helper that grabs the min constraint index (which can trigger GC as a // byproduct) and asserts. - assertMin := func(exp uint64, now time.Time) { + assertMin := func(exp roachpb.RaftIndex, now time.Time) { t.Helper() const anyRecipientStore roachpb.StoreID = 0 if _, maxIndex := r.getSnapshotLogTruncationConstraintsRLocked(anyRecipientStore, false /* initialOnly */); maxIndex != exp { @@ -718,7 +718,7 @@ func TestTruncateLog(t *testing.T) { looselyCoupledTruncationEnabled.Override(ctx, &st.SV, looselyCoupled) // Populate the log with 10 entries. Save the LastIndex after each write. - var indexes []uint64 + var indexes []roachpb.RaftIndex for i := 0; i < 10; i++ { args := incrementArgs([]byte("a"), int64(i)) @@ -913,7 +913,7 @@ func TestTruncateLogRecompute(t *testing.T) { } func waitForTruncationForTesting( - t *testing.T, r *Replica, newFirstIndex uint64, looselyCoupled bool, + t *testing.T, r *Replica, newFirstIndex roachpb.RaftIndex, looselyCoupled bool, ) { testutils.SucceedsSoon(t, func() error { if looselyCoupled { diff --git a/pkg/kv/kvserver/raft_log_truncator.go b/pkg/kv/kvserver/raft_log_truncator.go index 17f2ffb790c5..68c1d874ec4d 100644 --- a/pkg/kv/kvserver/raft_log_truncator.go +++ b/pkg/kv/kvserver/raft_log_truncator.go @@ -92,7 +92,9 @@ func (p *pendingLogTruncations) computePostTruncLogSize(raftLogSize int64) int64 // computePostTruncFirstIndex computes the first log index that is not // truncated, under the pretense that the pending truncations have been // enacted. -func (p *pendingLogTruncations) computePostTruncFirstIndex(firstIndex uint64) uint64 { +func (p *pendingLogTruncations) computePostTruncFirstIndex( + firstIndex roachpb.RaftIndex, +) roachpb.RaftIndex { p.mu.Lock() defer p.mu.Unlock() p.iterateLocked(func(_ int, trunc pendingTruncation) { @@ -167,14 +169,14 @@ type pendingTruncation struct { // are making an effort to have consecutive TruncateLogRequests provide us // stats for index intervals that are adjacent and non-overlapping, but // that behavior is best-effort. - expectedFirstIndex uint64 + expectedFirstIndex roachpb.RaftIndex // logDeltaBytes includes the bytes from sideloaded files. Like // ReplicatedEvalResult.RaftLogDelta, this is <= 0. logDeltaBytes int64 isDeltaTrusted bool } -func (pt *pendingTruncation) firstIndexAfterTrunc() uint64 { +func (pt *pendingTruncation) firstIndexAfterTrunc() roachpb.RaftIndex { // Reminder: RaftTruncatedState.Index is inclusive. return pt.Index + 1 } @@ -249,7 +251,7 @@ type replicaForTruncator interface { getTruncatedState() roachpb.RaftTruncatedState // Updates the replica state after the truncation is enacted. setTruncatedStateAndSideEffects( - _ context.Context, _ *roachpb.RaftTruncatedState, expectedFirstIndexPreTruncation uint64, + _ context.Context, _ *roachpb.RaftTruncatedState, expectedFirstIndexPreTruncation roachpb.RaftIndex, ) (expectedFirstIndexWasAccurate bool) // Updates the stats related to the raft log size after the truncation is // enacted. @@ -260,7 +262,7 @@ type replicaForTruncator interface { // Returns the sideloaded bytes that would be freed if we were to truncate // [from, to). sideloadedBytesIfTruncatedFromTo( - _ context.Context, from, to uint64) (freed int64, _ error) + _ context.Context, from, to roachpb.RaftIndex) (freed int64, _ error) getStateLoader() stateloader.StateLoader // NB: Setting the persistent raft state is via the Engine exposed by // storeForTruncator. @@ -273,7 +275,7 @@ func (t *raftLogTruncator) addPendingTruncation( ctx context.Context, r replicaForTruncator, trunc roachpb.RaftTruncatedState, - raftExpectedFirstIndex uint64, + raftExpectedFirstIndex roachpb.RaftIndex, raftLogDelta int64, ) { pendingTrunc := pendingTruncation{ diff --git a/pkg/kv/kvserver/raft_log_truncator_test.go b/pkg/kv/kvserver/raft_log_truncator_test.go index 7481bfeebbea..ce507885b9c9 100644 --- a/pkg/kv/kvserver/raft_log_truncator_test.go +++ b/pkg/kv/kvserver/raft_log_truncator_test.go @@ -149,7 +149,7 @@ func (r *replicaTruncatorTest) setTruncationDeltaAndTrusted(deltaBytes int64, is } func (r *replicaTruncatorTest) sideloadedBytesIfTruncatedFromTo( - _ context.Context, from, to uint64, + _ context.Context, from, to roachpb.RaftIndex, ) (freed int64, _ error) { fmt.Fprintf(r.buf, "r%d.sideloadedBytesIfTruncatedFromTo(%d, %d)\n", r.rangeID, from, to) return r.sideloadedFreed, r.sideloadedErr @@ -161,7 +161,9 @@ func (r *replicaTruncatorTest) getStateLoader() stateloader.StateLoader { } func (r *replicaTruncatorTest) setTruncatedStateAndSideEffects( - _ context.Context, truncState *roachpb.RaftTruncatedState, expectedFirstIndexPreTruncation uint64, + _ context.Context, + truncState *roachpb.RaftTruncatedState, + expectedFirstIndexPreTruncation roachpb.RaftIndex, ) (expectedFirstIndexWasAccurate bool) { expectedFirstIndexWasAccurate = r.truncState.Index+1 == expectedFirstIndexPreTruncation r.truncState = *truncState @@ -172,7 +174,7 @@ func (r *replicaTruncatorTest) setTruncatedStateAndSideEffects( } func (r *replicaTruncatorTest) writeRaftStateToEngine( - t *testing.T, eng storage.Engine, truncIndex uint64, lastLogEntry uint64, + t *testing.T, eng storage.Engine, truncIndex roachpb.RaftIndex, lastLogEntry roachpb.RaftIndex, ) { require.NoError(t, r.stateLoader.SetRaftTruncatedState(context.Background(), eng, &roachpb.RaftTruncatedState{Index: truncIndex})) @@ -182,7 +184,7 @@ func (r *replicaTruncatorTest) writeRaftStateToEngine( } func (r *replicaTruncatorTest) writeRaftAppliedIndex( - t *testing.T, eng storage.Engine, raftAppliedIndex uint64, flush bool, + t *testing.T, eng storage.Engine, raftAppliedIndex roachpb.RaftIndex, flush bool, ) { require.NoError(t, r.stateLoader.SetRangeAppliedState(context.Background(), eng, raftAppliedIndex, 0, 0, &enginepb.MVCCStats{}, hlc.Timestamp{}, nil)) @@ -299,8 +301,8 @@ func TestRaftLogTruncator(t *testing.T) { var lastLogEntry uint64 d.ScanArgs(t, "last-log-entry", &lastLogEntry) r := makeReplicaTT(rangeID, &buf) - r.truncState.Index = truncIndex - r.writeRaftStateToEngine(t, eng, truncIndex, lastLogEntry) + r.truncState.Index = roachpb.RaftIndex(truncIndex) + r.writeRaftStateToEngine(t, eng, roachpb.RaftIndex(truncIndex), roachpb.RaftIndex(lastLogEntry)) store.replicas[rangeID] = r return flushAndReset() @@ -326,7 +328,7 @@ func TestRaftLogTruncator(t *testing.T) { } r.sideloadedFreed = int64(sideloadedBytes) truncator.addPendingTruncation(context.Background(), r, - roachpb.RaftTruncatedState{Index: truncIndex}, firstIndex, int64(deltaBytes)) + roachpb.RaftTruncatedState{Index: roachpb.RaftIndex(truncIndex)}, roachpb.RaftIndex(firstIndex), int64(deltaBytes)) printTruncatorState(t, &buf, truncator) r.sideloadedErr = nil return flushAndReset() @@ -348,7 +350,7 @@ func TestRaftLogTruncator(t *testing.T) { if d.HasArg("no-flush") { d.ScanArgs(t, "no-flush", &noFlush) } - store.replicas[rangeID].writeRaftAppliedIndex(t, eng, raftAppliedIndex, !noFlush) + store.replicas[rangeID].writeRaftAppliedIndex(t, eng, roachpb.RaftIndex(raftAppliedIndex), !noFlush) return flushAndReset() case "add-replica-to-truncator": diff --git a/pkg/kv/kvserver/raft_truncator_replica.go b/pkg/kv/kvserver/raft_truncator_replica.go index 2547a533f277..56ef981b9cda 100644 --- a/pkg/kv/kvserver/raft_truncator_replica.go +++ b/pkg/kv/kvserver/raft_truncator_replica.go @@ -34,7 +34,9 @@ func (r *raftTruncatorReplica) getTruncatedState() roachpb.RaftTruncatedState { } func (r *raftTruncatorReplica) setTruncatedStateAndSideEffects( - ctx context.Context, trunc *roachpb.RaftTruncatedState, expectedFirstIndexPreTruncation uint64, + ctx context.Context, + trunc *roachpb.RaftTruncatedState, + expectedFirstIndexPreTruncation roachpb.RaftIndex, ) (expectedFirstIndexWasAccurate bool) { _, expectedFirstIndexAccurate := (*Replica)(r).handleTruncatedStateResult( ctx, trunc, expectedFirstIndexPreTruncation) @@ -64,7 +66,7 @@ func (r *raftTruncatorReplica) getPendingTruncs() *pendingLogTruncations { } func (r *raftTruncatorReplica) sideloadedBytesIfTruncatedFromTo( - ctx context.Context, from, to uint64, + ctx context.Context, from, to roachpb.RaftIndex, ) (freed int64, err error) { freed, _, err = r.raftMu.sideloaded.BytesIfTruncatedFromTo(ctx, from, to) return freed, err diff --git a/pkg/kv/kvserver/raftentry/cache.go b/pkg/kv/kvserver/raftentry/cache.go index b3df6ada2af5..a8d9cfa32848 100644 --- a/pkg/kv/kvserver/raftentry/cache.go +++ b/pkg/kv/kvserver/raftentry/cache.go @@ -99,11 +99,11 @@ const partitionSize = int32(unsafe.Sizeof(partition{})) // implement the below interface. type rangeCache interface { add(ent []raftpb.Entry) (bytesAdded, entriesAdded int32) - truncateFrom(lo uint64) (bytesRemoved, entriesRemoved int32) - clearTo(hi uint64) (bytesRemoved, entriesRemoved int32) - get(index uint64) (raftpb.Entry, bool) - scan(ents []raftpb.Entry, lo, hi, maxBytes uint64) ( - _ []raftpb.Entry, bytes uint64, nextIdx uint64, exceededMaxBytes bool) + truncateFrom(lo roachpb.RaftIndex) (bytesRemoved, entriesRemoved int32) + clearTo(hi roachpb.RaftIndex) (bytesRemoved, entriesRemoved int32) + get(index roachpb.RaftIndex) (raftpb.Entry, bool) + scan(ents []raftpb.Entry, lo, hi roachpb.RaftIndex, maxBytes uint64) ( + _ []raftpb.Entry, bytes uint64, nextIdx roachpb.RaftIndex, exceededMaxBytes bool) } // ringBuf implements rangeCache. @@ -188,7 +188,7 @@ func (c *Cache) Add(id roachpb.RangeID, ents []raftpb.Entry, truncate bool) { // Note that ents[0].Index may not even be in the cache // at this point. `truncateFrom` will still remove any entries // it may have at indexes >= truncIdx, as instructed. - truncIdx := ents[0].Index + truncIdx := roachpb.RaftIndex(ents[0].Index) bytesRemoved, entriesRemoved = p.truncateFrom(truncIdx) } if add { @@ -198,7 +198,7 @@ func (c *Cache) Add(id roachpb.RangeID, ents []raftpb.Entry, truncate bool) { } // Clear removes all entries on the given range with index less than hi. -func (c *Cache) Clear(id roachpb.RangeID, hi uint64) { +func (c *Cache) Clear(id roachpb.RangeID, hi roachpb.RaftIndex) { c.mu.Lock() p := c.getPartLocked(id, false /* create */, false /* recordUse */) if p == nil { @@ -214,7 +214,7 @@ func (c *Cache) Clear(id roachpb.RangeID, hi uint64) { // Get returns the entry for the specified index and true for the second return // value. If the index is not present in the cache, false is returned. -func (c *Cache) Get(id roachpb.RangeID, idx uint64) (e raftpb.Entry, ok bool) { +func (c *Cache) Get(id roachpb.RangeID, idx roachpb.RaftIndex) (e raftpb.Entry, ok bool) { c.metrics.Accesses.Inc(1) c.mu.Lock() p := c.getPartLocked(id, false /* create */, true /* recordUse */) @@ -238,8 +238,8 @@ func (c *Cache) Get(id roachpb.RangeID, idx uint64) (e raftpb.Entry, ok bool) { // cache miss occurs. The returned size reflects the size of the returned // entries. func (c *Cache) Scan( - ents []raftpb.Entry, id roachpb.RangeID, lo, hi, maxBytes uint64, -) (_ []raftpb.Entry, bytes uint64, nextIdx uint64, exceededMaxBytes bool) { + ents []raftpb.Entry, id roachpb.RangeID, lo, hi roachpb.RaftIndex, maxBytes uint64, +) (_ []raftpb.Entry, bytes uint64, nextIdx roachpb.RaftIndex, exceededMaxBytes bool) { c.metrics.Accesses.Inc(1) c.mu.Lock() p := c.getPartLocked(id, false /* create */, true /* recordUse */) diff --git a/pkg/kv/kvserver/raftentry/cache_test.go b/pkg/kv/kvserver/raftentry/cache_test.go index 529395b2ab2d..435f7aad4f93 100644 --- a/pkg/kv/kvserver/raftentry/cache_test.go +++ b/pkg/kv/kvserver/raftentry/cache_test.go @@ -26,18 +26,18 @@ import ( const noLimit = math.MaxUint64 -func newEntry(index, size uint64) raftpb.Entry { - r := rand.New(rand.NewSource(int64(index * size))) +func newEntry(index roachpb.RaftIndex, size int) raftpb.Entry { + r := rand.New(rand.NewSource(int64(index) * int64(size))) data := make([]byte, size) if _, err := r.Read(data); err != nil { panic(err) } ent := raftpb.Entry{ - Index: index, + Index: uint64(index), Data: data, } for { - entSize := uint64(ent.Size()) + entSize := ent.Size() if entSize == size { return ent } @@ -45,22 +45,22 @@ func newEntry(index, size uint64) raftpb.Entry { panic("size undershot") } delta := entSize - size - if uint64(len(ent.Data)) < delta { + if len(ent.Data) < delta { panic("can't shorten ent.Data to target size") } ent.Data = ent.Data[delta:] } } -func newEntries(lo, hi, size uint64) []raftpb.Entry { - ents := []raftpb.Entry{} +func newEntries(lo, hi roachpb.RaftIndex, size int) []raftpb.Entry { + var ents []raftpb.Entry for i := lo; i < hi; i++ { ents = append(ents, newEntry(i, size)) } return ents } -func addEntries(c *Cache, rangeID roachpb.RangeID, lo, hi uint64) []raftpb.Entry { +func addEntries(c *Cache, rangeID roachpb.RangeID, lo, hi roachpb.RaftIndex) []raftpb.Entry { ents := newEntries(lo, hi, 9) c.Add(rangeID, ents, false) return ents @@ -70,9 +70,9 @@ func verifyGet( t *testing.T, c *Cache, rangeID roachpb.RangeID, - lo, hi uint64, + lo, hi roachpb.RaftIndex, expEnts []raftpb.Entry, - expNextIndex uint64, + expNextIndex roachpb.RaftIndex, allowEviction bool, ) { t.Helper() @@ -87,7 +87,7 @@ func verifyGet( t.Fatalf("expected next index %d; got %d", expNextIndex, nextIndex) } for _, e := range ents { - found, ok := c.Get(rangeID, e.Index) + found, ok := c.Get(rangeID, roachpb.RaftIndex(e.Index)) if !ok { if allowEviction { break @@ -100,7 +100,7 @@ func verifyGet( } } -func requireEqual(t *testing.T, c *Cache, rangeID roachpb.RangeID, idxs ...uint64) { +func requireEqual(t *testing.T, c *Cache, rangeID roachpb.RangeID, idxs ...roachpb.RaftIndex) { t.Helper() p := c.getPartLocked(rangeID, false /* create */, false /* recordUse */) if p == nil { @@ -111,7 +111,7 @@ func requireEqual(t *testing.T, c *Cache, rangeID roachpb.RangeID, idxs ...uint6 } b := &p.ringBuf it := first(b) - var act []uint64 + var act []roachpb.RaftIndex ok := it.valid(b) for ok { act = append(act, it.index(b)) @@ -126,7 +126,7 @@ func TestEntryCache(t *testing.T) { rangeID := roachpb.RangeID(2) otherRangeID := rangeID + 1 // Note 9 bytes per entry with data size of 1 - verify := func(rangeID roachpb.RangeID, lo, hi uint64, ents []raftpb.Entry, expNextIndex uint64) { + verify := func(rangeID roachpb.RangeID, lo, hi roachpb.RaftIndex, ents []raftpb.Entry, expNextIndex roachpb.RaftIndex) { t.Helper() verifyGet(t, c, rangeID, lo, hi, ents, expNextIndex, false) } @@ -220,21 +220,21 @@ func TestIgnoredAdd(t *testing.T) { // Cache has entries 4, 5, 6. Offer an oversize entry at index 7 (which is // notably after 6) and request truncation. This should be a no-op. - c.Add(rangeID, []raftpb.Entry{newEntry(7, uint64(c.maxBytes+1))}, true /* truncate */) + c.Add(rangeID, []raftpb.Entry{newEntry(7, int(c.maxBytes+1))}, true /* truncate */) requireEqual(t, c, rangeID, 4, 5, 6) verifyGet(t, c, rangeID, 4, 7, ents, 7, false) // Cache has entries 4, 5, 6. Offer an oversize entry at index 6 and request // truncation. This should remove index 6 (as requested due to the truncation) // without replacing it with the input entry. - c.Add(rangeID, []raftpb.Entry{newEntry(6, uint64(c.maxBytes+1))}, true /* truncate */) + c.Add(rangeID, []raftpb.Entry{newEntry(6, int(c.maxBytes+1))}, true /* truncate */) requireEqual(t, c, rangeID, 4, 5) verifyGet(t, c, rangeID, 4, 7, ents[:len(ents)-1], 6, false) // Cache has entries 4, 5. Offer an oversize entry at index 3 (which is // notably before 4) and request truncation. This should clear all entries // >= 3, i.e. everything. - c.Add(rangeID, []raftpb.Entry{newEntry(3, uint64(c.maxBytes+1))}, true /* truncate */) + c.Add(rangeID, []raftpb.Entry{newEntry(3, int(c.maxBytes+1))}, true /* truncate */) // And it did. requireEqual(t, c, rangeID) verifyGet(t, c, rangeID, 0, 0, nil, 0, false) @@ -421,8 +421,8 @@ func TestConcurrentEvictions(t *testing.T) { ents := rangeData[r] offset := rand.Intn(len(ents)) length := rand.Intn(len(ents) - offset) - lo := ents[offset].Index - hi := lo + uint64(length) + lo := roachpb.RaftIndex(ents[offset].Index) + hi := lo + roachpb.RaftIndex(length) wg.Add(1) go func() { time.Sleep(time.Duration(rand.Intn(int(time.Microsecond)))) @@ -439,12 +439,12 @@ func TestConcurrentEvictions(t *testing.T) { if offset >= 0 && offset < len(ents) { lo := ents[offset].Index hi := lo + uint64(length) - toAdd = newEntries(lo, hi, 9) + toAdd = newEntries(roachpb.RaftIndex(lo), roachpb.RaftIndex(hi), 9) ents = append(ents[:offset], toAdd...) } else { lo := uint64(offset + 2) hi := lo + uint64(length) - toAdd = newEntries(lo, hi, 9) + toAdd = newEntries(roachpb.RaftIndex(lo), roachpb.RaftIndex(hi), 9) ents = toAdd } rangeData[r] = ents @@ -476,7 +476,7 @@ func TestConcurrentEvictions(t *testing.T) { if len(data) == 0 { continue } - c.Clear(r, data[len(data)-1].Index+1) + c.Clear(r, roachpb.RaftIndex(data[len(data)-1].Index+1)) } verifyMetrics(t, c, 0, int64(len(c.parts))*int64(partitionSize)) } @@ -576,7 +576,7 @@ func TestConcurrentUpdates(t *testing.T) { clear func() }{ {"drop", func() { c.Drop(r1) }}, - {"clear", func() { c.Clear(r1, ents[len(ents)-1].Index+1) }}, + {"clear", func() { c.Clear(r1, roachpb.RaftIndex(ents[len(ents)-1].Index+1)) }}, } { t.Run(clearMethod.name, func(t *testing.T) { // NB: N is chosen based on the race detector's limit of 8128 goroutines. @@ -651,7 +651,7 @@ func TestConcurrentAddGetAndEviction(t *testing.T) { ents := []raftpb.Entry{newEntry(1, 500)} doAddAndGetToRange := func(rangeID roachpb.RangeID) { doAction(func() { c.Add(rangeID, ents, true) }) - doAction(func() { c.Get(rangeID, ents[0].Index) }) + doAction(func() { c.Get(rangeID, roachpb.RaftIndex(ents[0].Index)) }) } doAddAndGetToRange(1) doAddAndGetToRange(2) @@ -662,7 +662,7 @@ func BenchmarkEntryCache(b *testing.B) { rangeID := roachpb.RangeID(1) ents := make([]raftpb.Entry, 1000) for i := range ents { - ents[i] = newEntry(uint64(i+1), 9) + ents[i] = newEntry(roachpb.RaftIndex(i+1), 9) } b.ResetTimer() for i := 0; i < b.N; i++ { @@ -675,8 +675,8 @@ func BenchmarkEntryCache(b *testing.B) { } b.StartTimer() c.Add(rangeID, ents, true) - _, _, _, _ = c.Scan(nil, rangeID, 0, uint64(len(ents)-10), noLimit) - c.Clear(rangeID, uint64(len(ents)-10)) + _, _, _, _ = c.Scan(nil, rangeID, 0, roachpb.RaftIndex(len(ents)-10), noLimit) + c.Clear(rangeID, roachpb.RaftIndex(len(ents)-10)) } } @@ -684,7 +684,7 @@ func BenchmarkEntryCacheClearTo(b *testing.B) { rangeID := roachpb.RangeID(1) ents := make([]raftpb.Entry, 1000) for i := range ents { - ents[i] = newEntry(uint64(i+1), 9) + ents[i] = newEntry(roachpb.RaftIndex(i+1), 9) } b.ResetTimer() for i := 0; i < b.N; i++ { @@ -692,6 +692,6 @@ func BenchmarkEntryCacheClearTo(b *testing.B) { c := NewCache(uint64(10 * len(ents) * len(ents[0].Data))) c.Add(rangeID, ents, true) b.StartTimer() - c.Clear(rangeID, uint64(len(ents)-10)) + c.Clear(rangeID, roachpb.RaftIndex(len(ents)-10)) } } diff --git a/pkg/kv/kvserver/raftentry/ring_buffer.go b/pkg/kv/kvserver/raftentry/ring_buffer.go index 96f94d172d6e..fe9a9ba6b7a4 100644 --- a/pkg/kv/kvserver/raftentry/ring_buffer.go +++ b/pkg/kv/kvserver/raftentry/ring_buffer.go @@ -13,6 +13,7 @@ package raftentry import ( "math/bits" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/errors" "go.etcd.io/raft/v3/raftpb" @@ -34,20 +35,20 @@ const ( // given that ents may overlap with existing entries or may be rejected from // the buffer. ents must not be empty. func (b *ringBuf) add(ents []raftpb.Entry) (addedBytes, addedEntries int32) { - if it := last(b); it.valid(b) && ents[0].Index > it.index(b)+1 { + if it := last(b); it.valid(b) && roachpb.RaftIndex(ents[0].Index) > it.index(b)+1 { // If ents is non-contiguous and later than the currently cached range then // remove the current entries and add ents in their place. removedBytes, removedEntries := b.clearTo(it.index(b) + 1) addedBytes, addedEntries = -1*removedBytes, -1*removedEntries } - before, after, ok := computeExtension(b, ents[0].Index, ents[len(ents)-1].Index) + before, after, ok := computeExtension(b, roachpb.RaftIndex(ents[0].Index), roachpb.RaftIndex(ents[len(ents)-1].Index)) if !ok { return } extend(b, before, after) it := first(b) if before == 0 && after != b.len { // skip unchanged prefix - it, _ = iterateFrom(b, ents[0].Index) // safe by construction + it, _ = iterateFrom(b, roachpb.RaftIndex(ents[0].Index)) // safe by construction } firstNewAfter := len(ents) - after for i, e := range ents { @@ -65,7 +66,7 @@ func (b *ringBuf) add(ents []raftpb.Entry) (addedBytes, addedEntries int32) { // truncateFrom clears all entries from the ringBuf with index equal to or // greater than lo. The method returns the aggregate size and count of entries // removed. Note that lo itself may or may not be in the cache. -func (b *ringBuf) truncateFrom(lo uint64) (removedBytes, removedEntries int32) { +func (b *ringBuf) truncateFrom(lo roachpb.RaftIndex) (removedBytes, removedEntries int32) { if b.len == 0 { return } @@ -103,7 +104,7 @@ func (b *ringBuf) truncateFrom(lo uint64) (removedBytes, removedEntries int32) { // clearTo clears all entries from the ringBuf with index less than hi. The // method returns the aggregate size and count of entries removed. -func (b *ringBuf) clearTo(hi uint64) (removedBytes, removedEntries int32) { +func (b *ringBuf) clearTo(hi roachpb.RaftIndex) (removedBytes, removedEntries int32) { if b.len == 0 || hi < first(b).index(b) { return } @@ -128,7 +129,7 @@ func (b *ringBuf) clearTo(hi uint64) (removedBytes, removedEntries int32) { return } -func (b *ringBuf) get(index uint64) (e raftpb.Entry, ok bool) { +func (b *ringBuf) get(index roachpb.RaftIndex) (e raftpb.Entry, ok bool) { it, ok := iterateFrom(b, index) if !ok { return e, ok @@ -137,8 +138,8 @@ func (b *ringBuf) get(index uint64) (e raftpb.Entry, ok bool) { } func (b *ringBuf) scan( - ents []raftpb.Entry, lo, hi, maxBytes uint64, -) (_ []raftpb.Entry, bytes uint64, nextIdx uint64, exceededMaxBytes bool) { + ents []raftpb.Entry, lo roachpb.RaftIndex, hi roachpb.RaftIndex, maxBytes uint64, +) (_ []raftpb.Entry, bytes uint64, nextIdx roachpb.RaftIndex, exceededMaxBytes bool) { var it iterator nextIdx = lo it, ok := iterateFrom(b, lo) @@ -203,7 +204,7 @@ func extend(b *ringBuf, before, after int) { // after are counts, not indices, of number of entries which precede and follow // the currently cached range. If [lo, hi] is not overlapping or directly // adjacent to the current cache bounds, ok will be false. -func computeExtension(b *ringBuf, lo, hi uint64) (before, after int, ok bool) { +func computeExtension(b *ringBuf, lo, hi roachpb.RaftIndex) (before, after int, ok bool) { if b.len == 0 { return 0, int(hi) - int(lo) + 1, true } @@ -223,7 +224,7 @@ func computeExtension(b *ringBuf, lo, hi uint64) (before, after int, ok bool) { // iterator indexes into a ringBuf. A value of -1 is not valid. type iterator int -func iterateFrom(b *ringBuf, index uint64) (_ iterator, ok bool) { +func iterateFrom(b *ringBuf, index roachpb.RaftIndex) (_ iterator, ok bool) { if b.len == 0 { return -1, false } @@ -257,8 +258,8 @@ func (it iterator) valid(b *ringBuf) bool { } // index returns the index of the entry at iterator's curent position. -func (it iterator) index(b *ringBuf) uint64 { - return b.buf[it].Index +func (it iterator) index(b *ringBuf) roachpb.RaftIndex { + return roachpb.RaftIndex(b.buf[it].Index) } // entry returns the entry at iterator's curent position. diff --git a/pkg/kv/kvserver/raftentry/ring_buffer_test.go b/pkg/kv/kvserver/raftentry/ring_buffer_test.go index 461019542f60..3cdecaddab98 100644 --- a/pkg/kv/kvserver/raftentry/ring_buffer_test.go +++ b/pkg/kv/kvserver/raftentry/ring_buffer_test.go @@ -13,6 +13,7 @@ package raftentry import ( "testing" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/stretchr/testify/require" ) @@ -80,12 +81,13 @@ func TestRingBuffer_Add(t *testing.T) { func TestRingBuffer_Scan(t *testing.T) { for _, tc := range []struct { - desc string - lo, hi, mb uint64 + desc string + lo, hi roachpb.RaftIndex + mb uint64 idxs []uint64 // full range is 10,11,12,13,14, each of size 10 scanBytes uint64 - nextIdx uint64 + nextIdx roachpb.RaftIndex exceededMaxBytes bool }{ { @@ -190,9 +192,9 @@ func TestRingBuffer_ClearTo(t *testing.T) { eq(t, b) } -func eq(t *testing.T, b *ringBuf, idxs ...uint64) { +func eq(t *testing.T, b *ringBuf, idxs ...roachpb.RaftIndex) { t.Helper() - var sl []uint64 + var sl []roachpb.RaftIndex it := first(b) for it.valid(b) { idx := it.index(b) @@ -200,7 +202,7 @@ func eq(t *testing.T, b *ringBuf, idxs ...uint64) { it, _ = it.next(b) ent, ok := b.get(idx) require.True(t, ok) - require.Equal(t, idx, ent.Index) + require.Equal(t, idx, roachpb.RaftIndex(ent.Index)) } require.Equal(t, idxs, sl) if len(sl) == 0 { diff --git a/pkg/kv/kvserver/raftlog/command.go b/pkg/kv/kvserver/raftlog/command.go index 95cf45c22c57..710dc57d51a0 100644 --- a/pkg/kv/kvserver/raftlog/command.go +++ b/pkg/kv/kvserver/raftlog/command.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/errors" "go.etcd.io/raft/v3/raftpb" ) @@ -59,8 +60,8 @@ func (c *ReplicatedCmd) Decode(e *raftpb.Entry) error { // Index implements apply.Command. It returns the index of the log entry that // this Command applies. -func (c *ReplicatedCmd) Index() uint64 { - return c.Entry.Index +func (c *ReplicatedCmd) Index() roachpb.RaftIndex { + return roachpb.RaftIndex(c.Entry.Index) } // IsTrivial implements apply.Command. Trivial commands may be batched in an diff --git a/pkg/kv/kvserver/raftlog/iter_bench_test.go b/pkg/kv/kvserver/raftlog/iter_bench_test.go index adc0850b9c37..d136d85dc7ce 100644 --- a/pkg/kv/kvserver/raftlog/iter_bench_test.go +++ b/pkg/kv/kvserver/raftlog/iter_bench_test.go @@ -178,7 +178,7 @@ func BenchmarkVisit(b *testing.B) { defer eng.Close() ent, metaB := mkBenchEnt(b) - require.NoError(b, eng.PutUnversioned(keys.RaftLogKey(rangeID, ent.Index), metaB)) + require.NoError(b, eng.PutUnversioned(keys.RaftLogKey(rangeID, roachpb.RaftIndex(ent.Index)), metaB)) b.ReportAllocs() b.ResetTimer() diff --git a/pkg/kv/kvserver/raftlog/iter_test.go b/pkg/kv/kvserver/raftlog/iter_test.go index 6d372e654885..4e0391583099 100644 --- a/pkg/kv/kvserver/raftlog/iter_test.go +++ b/pkg/kv/kvserver/raftlog/iter_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -31,7 +32,7 @@ func ents(inds ...uint64) []raftpb.Entry { sl := make([]raftpb.Entry, 0, len(inds)) for _, ind := range inds { cmd := kvserverpb.RaftCommand{ - MaxLeaseIndex: ind, // just to have something nontrivial in here + MaxLeaseIndex: roachpb.LeaseSequence(ind), // just to have something nontrivial in here } b, err := protoutil.Marshal(&cmd) if err != nil { @@ -99,7 +100,7 @@ func ents(inds ...uint64) []raftpb.Entry { type modelIter struct { idx int ents []raftpb.Entry - hi uint64 + hi roachpb.RaftIndex } func (it *modelIter) load() (raftpb.Entry, error) { @@ -113,12 +114,12 @@ func (it *modelIter) check() error { return err } -func (it *modelIter) SeekGE(lo uint64) (bool, error) { +func (it *modelIter) SeekGE(lo roachpb.RaftIndex) (bool, error) { for { - if it.idx >= len(it.ents) || (it.hi > 0 && it.ents[it.idx].Index >= it.hi) { + if it.idx >= len(it.ents) || (it.hi > 0 && roachpb.RaftIndex(it.ents[it.idx].Index) >= it.hi) { return false, nil } - if ind := it.ents[it.idx].Index; ind >= lo && (it.hi == 0 || ind < it.hi) { + if ind := roachpb.RaftIndex(it.ents[it.idx].Index); ind >= lo && (it.hi == 0 || ind < it.hi) { if err := it.check(); err != nil { return false, err } @@ -133,7 +134,7 @@ func (it *modelIter) Next() (bool, error) { if it.idx >= len(it.ents) { return false, nil } - if it.hi > 0 && it.ents[it.idx].Index >= it.hi { + if it.hi > 0 && roachpb.RaftIndex(it.ents[it.idx].Index) >= it.hi { return false, nil } err := it.check() @@ -152,7 +153,7 @@ func TestIteratorEmptyLog(t *testing.T) { defer log.Scope(t).Close(t) eng := storage.NewDefaultInMemForTesting() - for _, hi := range []uint64{0, 1} { + for _, hi := range []roachpb.RaftIndex{0, 1} { it := NewIterator(rangeID, eng, IterOptions{Hi: hi}) ok, err := it.SeekGE(0) it.Close() @@ -186,7 +187,7 @@ func TestIterator(t *testing.T) { ents: ents(math.MaxUint64-1, math.MaxUint64), }, } { - indToName := func(ind uint64) string { + indToName := func(ind roachpb.RaftIndex) string { if ind > math.MaxUint64/2 { if ind == math.MaxUint64 { return "max" @@ -197,7 +198,7 @@ func TestIterator(t *testing.T) { } var inds []string for _, ent := range tc.ents { - inds = append(inds, indToName(ent.Index)) + inds = append(inds, indToName(roachpb.RaftIndex(ent.Index))) } if len(inds) == 0 { inds = []string{"empty"} @@ -214,17 +215,17 @@ func TestIterator(t *testing.T) { require.NoError(t, err) metaB, err := e.ToRawBytes() require.NoError(t, err) - require.NoError(t, eng.PutUnversioned(keys.RaftLogKey(rangeID, ent.Index), metaB)) + require.NoError(t, eng.PutUnversioned(keys.RaftLogKey(rangeID, roachpb.RaftIndex(ent.Index)), metaB)) } // Rather than handcrafting some invocations, just run all possible ones // and verify against our expectations. There's no need to hard-code them // since they're so simple to express. - var fi uint64 // firstIndex - var li uint64 // lastIndex + var fi roachpb.RaftIndex // firstIndex + var li roachpb.RaftIndex // lastIndex if n := len(tc.ents); n > 0 { - fi = tc.ents[0].Index - li = tc.ents[n-1].Index + fi = roachpb.RaftIndex(tc.ents[0].Index) + li = roachpb.RaftIndex(tc.ents[n-1].Index) } else { // Make sure we do some bogus scans on the empty log as well. fi = 1 @@ -274,12 +275,12 @@ func TestIterator(t *testing.T) { } type iter interface { - SeekGE(idx uint64) (bool, error) + SeekGE(idx roachpb.RaftIndex) (bool, error) Next() (bool, error) Entry() raftpb.Entry } -func consumeIter(it iter, lo uint64) ([]uint64, error) { +func consumeIter(it iter, lo roachpb.RaftIndex) ([]uint64, error) { var sl []uint64 ok, err := it.SeekGE(lo) diff --git a/pkg/kv/kvserver/raftlog/iterator.go b/pkg/kv/kvserver/raftlog/iterator.go index 25126821efab..5ac24deb8e75 100644 --- a/pkg/kv/kvserver/raftlog/iterator.go +++ b/pkg/kv/kvserver/raftlog/iterator.go @@ -60,7 +60,7 @@ type Iterator struct { type IterOptions struct { // Hi ensures the Iterator never seeks to any entry with index >= Hi. This is // useful when the caller is interested in a slice [Lo, Hi) of the raft log. - Hi uint64 + Hi roachpb.RaftIndex } // NewIterator initializes an Iterator that reads the raft log for the given @@ -107,7 +107,7 @@ func (it *Iterator) load() (bool, error) { // SeekGE positions the Iterator at the first raft log with index greater than // or equal to idx. Returns (true, nil) on success, (false, nil) if no such // entry exists. -func (it *Iterator) SeekGE(idx uint64) (bool, error) { +func (it *Iterator) SeekGE(idx roachpb.RaftIndex) (bool, error) { it.iter.SeekGE(storage.MakeMVCCMetadataKey(it.prefixBuf.RaftLogKey(idx))) return it.load() } @@ -134,7 +134,9 @@ func (it *Iterator) Entry() raftpb.Entry { // // The closure may return iterutil.StopIteration(), which will stop iteration // without returning an error. -func Visit(eng Reader, rangeID roachpb.RangeID, lo, hi uint64, fn func(raftpb.Entry) error) error { +func Visit( + eng Reader, rangeID roachpb.RangeID, lo, hi roachpb.RaftIndex, fn func(raftpb.Entry) error, +) error { it := NewIterator(rangeID, eng, IterOptions{Hi: hi}) defer it.Close() ok, err := it.SeekGE(lo) diff --git a/pkg/kv/kvserver/raftutil/util.go b/pkg/kv/kvserver/raftutil/util.go index 54d26a984401..402e48d50a2c 100644 --- a/pkg/kv/kvserver/raftutil/util.go +++ b/pkg/kv/kvserver/raftutil/util.go @@ -128,7 +128,7 @@ func (s ReplicaNeedsSnapshotStatus) String() string { // indicates that our local replica is not the raft leader, we pessimistically // assume that replicaID may need a snapshot. func ReplicaMayNeedSnapshot( - st *raft.Status, firstIndex uint64, replicaID roachpb.ReplicaID, + st *raft.Status, firstIndex roachpb.RaftIndex, replicaID roachpb.ReplicaID, ) ReplicaNeedsSnapshotStatus { if st == nil { // Testing only. @@ -158,7 +158,7 @@ func ReplicaMayNeedSnapshot( default: panic("unknown tracker.StateType") } - if progress.Match+1 < firstIndex { + if roachpb.RaftIndex(progress.Match+1) < firstIndex { // Even if the follower is in StateReplicate, it could have been cut off // from the log by a recent log truncation that hasn't been recognized yet // by raft. Confirm that this is not the case. diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 51b5a66a695b..1df783662dc7 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -405,7 +405,8 @@ type Replica struct { // thus invalid) even when lastIndexNotDurable is known, in which case the // term will have to be retrieved from the Raft log entry. Use the // invalidLastTerm constant for this case. - lastIndexNotDurable, lastTermNotDurable uint64 + lastIndexNotDurable roachpb.RaftIndex + lastTermNotDurable roachpb.RaftTerm // A map of raft log index of pending snapshots to deadlines. // Used to prohibit raft log truncations that would leave a gap between // the snapshot and the new first index. The map entry has a zero @@ -705,7 +706,7 @@ type Replica struct { // The base index is the index up to (including) which quota was already // released. That is, the first element in quotaReleaseQueue below is // released as the base index moves up by one, etc. - proposalQuotaBaseIndex uint64 + proposalQuotaBaseIndex roachpb.RaftIndex // Once the leader observes a proposal come 'out of Raft', we add the size // of the associated command to a queue of quotas we have yet to release diff --git a/pkg/kv/kvserver/replica_app_batch.go b/pkg/kv/kvserver/replica_app_batch.go index 77923343b014..9e2c1e3ed724 100644 --- a/pkg/kv/kvserver/replica_app_batch.go +++ b/pkg/kv/kvserver/replica_app_batch.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -65,7 +64,7 @@ type replicaAppBatch struct { followerStoreWriteBytes kvadmission.FollowerStoreWriteBytes // Reused by addAppliedStateKeyToBatch to avoid heap allocations. - asAlloc enginepb.RangeAppliedState + asAlloc kvserverpb.RangeAppliedState } // Stage implements the apply.Batch interface. The method handles the first @@ -510,11 +509,11 @@ func (b *replicaAppBatch) stageTrivialReplicatedEvalResult( ctx context.Context, cmd *replicatedCmd, ) { b.state.RaftAppliedIndex = cmd.Index() - b.state.RaftAppliedIndexTerm = cmd.Term + b.state.RaftAppliedIndexTerm = roachpb.RaftTerm(cmd.Term) - // NB: since the command is "trivial" we know the LeaseIndex field is set to + // NB: since the command is "trivial" we know the LeaseSequence field is set to // something meaningful if it's nonzero (e.g. cmd is not a lease request). For - // a rejected command, cmd.LeaseIndex was zeroed out earlier. + // a rejected command, cmd.LeaseSequence was zeroed out earlier. if leaseAppliedIndex := cmd.LeaseIndex; leaseAppliedIndex != 0 { b.state.LeaseAppliedIndex = leaseAppliedIndex } diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index 3ab270040d2a..0a7c9b69ded0 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -331,7 +331,9 @@ func (r *Replica) handleLeaseResult( } func (r *Replica) handleTruncatedStateResult( - ctx context.Context, t *roachpb.RaftTruncatedState, expectedFirstIndexPreTruncation uint64, + ctx context.Context, + t *roachpb.RaftTruncatedState, + expectedFirstIndexPreTruncation roachpb.RaftIndex, ) (raftLogDelta int64, expectedFirstIndexWasAccurate bool) { r.mu.Lock() expectedFirstIndexWasAccurate = diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index 2615b0cf6cde..f3fcb9739985 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -439,7 +438,7 @@ type closedTimestampSetterInfo struct { // lease represents the lease under which the command is being applied. lease *roachpb.Lease // leaseIdx is the LAI of the command. - leaseIdx ctpb.LAI + leaseIdx roachpb.LeaseSequence // leaseReq is set if the request that generated this command was a // RequestLeaseRequest. This is only ever set on the leaseholder replica since // only the leaseholder has information about the request corresponding to a @@ -457,7 +456,7 @@ type closedTimestampSetterInfo struct { // timestamp. func (s *closedTimestampSetterInfo) record(cmd *replicatedCmd, lease *roachpb.Lease) { *s = closedTimestampSetterInfo{} - s.leaseIdx = ctpb.LAI(cmd.LeaseIndex) + s.leaseIdx = cmd.LeaseIndex s.lease = lease if !cmd.IsLocal() { return diff --git a/pkg/kv/kvserver/replica_application_state_machine_test.go b/pkg/kv/kvserver/replica_application_state_machine_test.go index bffcb6311131..da21aefb6f3e 100644 --- a/pkg/kv/kvserver/replica_application_state_machine_test.go +++ b/pkg/kv/kvserver/replica_application_state_machine_test.go @@ -94,7 +94,7 @@ func TestReplicaStateMachineChangeReplicas(t *testing.T) { // Stage a command with the ChangeReplicas trigger. ent := &raftlog.Entry{ Entry: raftpb.Entry{ - Index: r.mu.state.RaftAppliedIndex + 1, + Index: uint64(r.mu.state.RaftAppliedIndex + 1), Type: raftpb.EntryConfChange, }, ID: makeIDKey(), @@ -198,7 +198,7 @@ func TestReplicaStateMachineRaftLogTruncationStronglyCoupled(t *testing.T) { // byte size of 1. ent := &raftlog.Entry{ Entry: raftpb.Entry{ - Index: raftAppliedIndex + 1, + Index: uint64(raftAppliedIndex + 1), Type: raftpb.EntryNormal, }, ID: makeIDKey(), @@ -285,7 +285,7 @@ func TestReplicaStateMachineRaftLogTruncationLooselyCoupled(t *testing.T) { require.NoError(t, pErr.GoError()) } - raftLogSize, truncatedIndex := func() (_rls int64, truncIdx uint64) { + raftLogSize, truncatedIndex := func() (_rls int64, truncIdx roachpb.RaftIndex) { // Lock the replica. We do this early to avoid interference from any other // moving parts on the Replica, whatever they may be. For example, we don't // want a skewed lease applied index because commands are applying concurrently @@ -317,7 +317,7 @@ func TestReplicaStateMachineRaftLogTruncationLooselyCoupled(t *testing.T) { // byte size of 1. ent := &raftlog.Entry{ Entry: raftpb.Entry{ - Index: raftAppliedIndex + 1, + Index: uint64(raftAppliedIndex + 1), Type: raftpb.EntryNormal, }, ID: makeIDKey(), @@ -448,7 +448,7 @@ func TestReplicaStateMachineEphemeralAppBatchRejection(t *testing.T) { req, repr := descWriteRepr(s) ent := &raftlog.Entry{ Entry: raftpb.Entry{ - Index: raftAppliedIndex + 1, + Index: uint64(raftAppliedIndex + 1), Type: raftpb.EntryNormal, }, ID: makeIDKey(), diff --git a/pkg/kv/kvserver/replica_closedts.go b/pkg/kv/kvserver/replica_closedts.go index f9548e94ab7f..2ecf5281aac8 100644 --- a/pkg/kv/kvserver/replica_closedts.go +++ b/pkg/kv/kvserver/replica_closedts.go @@ -14,7 +14,6 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -54,7 +53,7 @@ func (r *Replica) BumpSideTransportClosed( return res } - lai := ctpb.LAI(r.mu.state.LeaseAppliedIndex) + lai := r.mu.state.LeaseAppliedIndex policy := r.closedTimestampPolicyRLocked() target := targetByPolicy[policy] st := r.leaseStatusForRequestRLocked(ctx, now, hlc.Timestamp{} /* reqTS */) @@ -130,7 +129,7 @@ func (r *Replica) closedTimestampTargetRLocked() hlc.Timestamp { // ForwardSideTransportClosedTimestamp forwards the side-transport closed // timestamp. It is called by the closed timestamp side-transport receiver. func (r *Replica) ForwardSideTransportClosedTimestamp( - ctx context.Context, closed hlc.Timestamp, lai ctpb.LAI, + ctx context.Context, closed hlc.Timestamp, lai roachpb.LeaseSequence, ) { // We pass knownApplied = false because we don't know whether this lease // applied index has been applied locally yet. @@ -177,7 +176,7 @@ type sidetransportAccess struct { type sidetransportReceiver interface { GetClosedTimestamp( ctx context.Context, rangeID roachpb.RangeID, leaseholderNode roachpb.NodeID, - ) (hlc.Timestamp, ctpb.LAI) + ) (hlc.Timestamp, roachpb.LeaseSequence) HTML() string } @@ -186,7 +185,7 @@ type sidetransportReceiver interface { // lease applied index has been locally applied. type closedTimestamp struct { ts hlc.Timestamp - lai ctpb.LAI + lai roachpb.LeaseSequence } // regression returns whether the combination of the two closed timestamps @@ -226,7 +225,7 @@ func (st *sidetransportAccess) init(receiver sidetransportReceiver, rangeID roac // // The method returns the current applied closed timestamp. func (st *sidetransportAccess) forward( - ctx context.Context, closed hlc.Timestamp, lai ctpb.LAI, knownApplied bool, + ctx context.Context, closed hlc.Timestamp, lai roachpb.LeaseSequence, knownApplied bool, ) closedTimestamp { st.mu.Lock() defer st.mu.Unlock() @@ -280,7 +279,10 @@ func (st *sidetransportAccess) assertNoRegression(ctx context.Context, cur, up c // previous caller passed in. This means that get can be called without holding // the replica.mu. func (st *sidetransportAccess) get( - ctx context.Context, leaseholder roachpb.NodeID, appliedLAI ctpb.LAI, sufficient hlc.Timestamp, + ctx context.Context, + leaseholder roachpb.NodeID, + appliedLAI roachpb.LeaseSequence, + sufficient hlc.Timestamp, ) hlc.Timestamp { st.mu.RLock() cur, next := st.mu.cur, st.mu.next diff --git a/pkg/kv/kvserver/replica_closedts_internal_test.go b/pkg/kv/kvserver/replica_closedts_internal_test.go index ddf7ac7a722b..ab2df7630a03 100644 --- a/pkg/kv/kvserver/replica_closedts_internal_test.go +++ b/pkg/kv/kvserver/replica_closedts_internal_test.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -51,7 +50,7 @@ func TestSideTransportClosed(t *testing.T) { curSet bool nextSet bool recSet bool - applied ctpb.LAI + applied roachpb.LeaseSequence sufficient hlc.Timestamp expClosed hlc.Timestamp @@ -443,10 +442,10 @@ func TestSideTransportClosedMonotonic(t *testing.T) { for i := 0; i < observers; i++ { g.Go(func() error { var lastTS hlc.Timestamp - var lastLAI ctpb.LAI + var lastLAI roachpb.LeaseSequence for atomic.LoadInt32(&done) == 0 { // Determine which lease applied index to use. - var lai ctpb.LAI + var lai roachpb.LeaseSequence switch rand.Intn(3) { case 0: lai = lastLAI @@ -500,7 +499,7 @@ var _ sidetransportReceiver = &mockReceiver{} // GetClosedTimestamp is part of the sidetransportReceiver interface. func (r *mockReceiver) GetClosedTimestamp( ctx context.Context, rangeID roachpb.RangeID, leaseholderNode roachpb.NodeID, -) (hlc.Timestamp, ctpb.LAI) { +) (hlc.Timestamp, roachpb.LeaseSequence) { r.Lock() defer r.Unlock() return r.ts, r.lai @@ -522,10 +521,10 @@ func TestReplicaClosedTimestamp(t *testing.T) { for _, test := range []struct { name string - applied ctpb.LAI + applied roachpb.LeaseSequence raftClosed hlc.Timestamp sidetransportClosed hlc.Timestamp - sidetransportLAI ctpb.LAI + sidetransportLAI roachpb.LeaseSequence expClosed hlc.Timestamp }{ { @@ -569,7 +568,7 @@ func TestReplicaClosedTimestamp(t *testing.T) { tc.repl.mu.Lock() defer tc.repl.mu.Unlock() tc.repl.mu.state.RaftClosedTimestamp = test.raftClosed - tc.repl.mu.state.LeaseAppliedIndex = uint64(test.applied) + tc.repl.mu.state.LeaseAppliedIndex = test.applied // NB: don't release the mutex to make this test a bit more resilient to // problems that could arise should something propose a command to this // replica whose LeaseAppliedIndex we've mutated. diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 12f7082c0650..6caec54dfa2c 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -817,7 +817,7 @@ func waitForApplication( dialer *nodedialer.Dialer, rangeID roachpb.RangeID, replicas []roachpb.ReplicaDescriptor, - leaseIndex uint64, + leaseIndex roachpb.LeaseSequence, ) error { g := ctxgroup.WithContext(ctx) for _, repl := range replicas { @@ -2815,7 +2815,7 @@ func (r *Replica) sendSnapshotUsingDelegate( SenderQueueName: senderQueueName, SenderQueuePriority: senderQueuePriority, Type: snapType, - Term: status.Term, + Term: roachpb.RaftTerm(status.Term), DelegatedSender: sender, FirstIndex: appliedIndex, DescriptorGeneration: r.Desc().Generation, @@ -2936,7 +2936,7 @@ func (r *Replica) validateSnapshotDelegationRequest( // haven't woken up yet. return errors.Errorf("raft status not initialized") } - replTerm := status.Term + replTerm := roachpb.RaftTerm(status.Term) r.mu.RUnlock() // Delegate has a different term than the coordinator. This typically means @@ -3088,8 +3088,8 @@ func (r *Replica) followerSendSnapshot( // kvBatchSnapshotStrategy.Send and results in no log entries being sent at // all. Note that Metadata.Index is really the applied index of the replica. snap.State.TruncatedState = &roachpb.RaftTruncatedState{ - Index: snap.RaftSnap.Metadata.Index, - Term: snap.RaftSnap.Metadata.Term, + Index: roachpb.RaftIndex(snap.RaftSnap.Metadata.Index), + Term: roachpb.RaftTerm(snap.RaftSnap.Metadata.Term), } // See comment on DeprecatedUsingAppliedStateKey for why we need to set this @@ -3108,7 +3108,7 @@ func (r *Replica) followerSendSnapshot( Type: raftpb.MsgSnap, From: uint64(req.CoordinatorReplica.ReplicaID), To: uint64(req.RecipientReplica.ReplicaID), - Term: req.Term, + Term: uint64(req.Term), Snapshot: &snap.RaftSnap, }, }, diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go index 40f86e4e9372..464ae1fbc93e 100644 --- a/pkg/kv/kvserver/replica_eval_context_span.go +++ b/pkg/kv/kvserver/replica_eval_context_span.go @@ -85,17 +85,17 @@ func (rec *SpanSetReplicaEvalContext) GetNodeLocality() roachpb.Locality { } // GetFirstIndex returns the first index. -func (rec *SpanSetReplicaEvalContext) GetFirstIndex() uint64 { +func (rec *SpanSetReplicaEvalContext) GetFirstIndex() roachpb.RaftIndex { return rec.i.GetFirstIndex() } // GetTerm returns the term for the given index in the Raft log. -func (rec *SpanSetReplicaEvalContext) GetTerm(i uint64) (uint64, error) { +func (rec *SpanSetReplicaEvalContext) GetTerm(i roachpb.RaftIndex) (roachpb.RaftTerm, error) { return rec.i.GetTerm(i) } // GetLeaseAppliedIndex returns the lease index of the last applied command. -func (rec *SpanSetReplicaEvalContext) GetLeaseAppliedIndex() uint64 { +func (rec *SpanSetReplicaEvalContext) GetLeaseAppliedIndex() roachpb.LeaseSequence { return rec.i.GetLeaseAppliedIndex() } diff --git a/pkg/kv/kvserver/replica_follower_read.go b/pkg/kv/kvserver/replica_follower_read.go index 9120a2422aee..99432ff77247 100644 --- a/pkg/kv/kvserver/replica_follower_read.go +++ b/pkg/kv/kvserver/replica_follower_read.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvbase" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -138,7 +137,7 @@ func (r *Replica) canServeFollowerReadRLocked(ctx context.Context, ba *kvpb.Batc func (r *Replica) getCurrentClosedTimestampLocked( ctx context.Context, sufficient hlc.Timestamp, ) hlc.Timestamp { - appliedLAI := ctpb.LAI(r.mu.state.LeaseAppliedIndex) + appliedLAI := r.mu.state.LeaseAppliedIndex leaseholder := r.mu.state.Lease.Replica.NodeID raftClosed := r.mu.state.RaftClosedTimestamp sideTransportClosed := r.sideTransportClosedTimestamp.get(ctx, leaseholder, appliedLAI, sufficient) diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index a8cd8e100a60..2d82f05ec7e5 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -509,7 +509,8 @@ func addSSTablePreApply( st *cluster.Settings, eng storage.Engine, sideloaded logstore.SideloadStorage, - term, index uint64, + term roachpb.RaftTerm, + index roachpb.RaftIndex, sst kvserverpb.ReplicatedEvalResult_AddSSTable, limiter *rate.Limiter, ) bool { diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 7b6c3dea6169..cfff9cac217d 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -75,7 +75,7 @@ type propBuf struct { // assignedLAI represents the highest LAI that was assigned to a proposal. // This is set at the same time as assignedClosedTimestamp. - assignedLAI uint64 + assignedLAI roachpb.LeaseSequence // assignedClosedTimestamp is the largest "closed timestamp" - i.e. the // largest timestamp that was communicated to other replicas as closed, // representing a promise that this leaseholder will not evaluate writes with @@ -97,7 +97,7 @@ type propBuf struct { testing struct { // leaseIndexFilter can be used by tests to override the max lease index // assigned to a proposal by returning a non-zero lease index. - leaseIndexFilter func(*ProposalData) (indexOverride uint64) + leaseIndexFilter func(*ProposalData) roachpb.LeaseSequence // insertFilter allows tests to inject errors at Insert() time. insertFilter func(*ProposalData) error // submitProposalFilter can be used by tests to observe and optionally @@ -139,8 +139,8 @@ type proposer interface { // The following require the proposer to hold (at least) a shared lock. getReplicaID() roachpb.ReplicaID destroyed() destroyStatus - firstIndex() uint64 - leaseAppliedIndex() uint64 + firstIndex() roachpb.RaftIndex + leaseAppliedIndex() roachpb.LeaseSequence enqueueUpdateCheck() closedTimestampTarget() hlc.Timestamp leaderStatus(ctx context.Context, raftGroup proposerRaft) rangeLeaderInfo @@ -744,7 +744,7 @@ func (b *propBuf) leaderStatusRLocked(ctx context.Context, raftGroup proposerRaf // in the local replica's raft entry cache). func (b *propBuf) allocateLAIAndClosedTimestampLocked( ctx context.Context, p *ProposalData, closedTSTarget hlc.Timestamp, -) (uint64, hlc.Timestamp, error) { +) (roachpb.LeaseSequence, hlc.Timestamp, error) { // Assign a LeaseAppliedIndex (see checkForcedErr). These provide replay // protection. @@ -762,7 +762,7 @@ func (b *propBuf) allocateLAIAndClosedTimestampLocked( // are only ever proposed by leaseholders, and they use the LAI to prevent // replays (though they could in principle also be handled like lease // requests). - var lai uint64 + var lai roachpb.LeaseSequence if !p.Request.IsSingleRequestLeaseRequest() { b.assignedLAI++ lai = b.assignedLAI @@ -860,7 +860,7 @@ func (b *propBuf) allocateLAIAndClosedTimestampLocked( // marshallLAIAndClosedTimestampToProposalLocked modifies p.encodedCommand, // adding the LAI and closed timestamp. func (b *propBuf) marshallLAIAndClosedTimestampToProposalLocked( - ctx context.Context, p *ProposalData, lai uint64, closedTimestamp hlc.Timestamp, + ctx context.Context, p *ProposalData, lai roachpb.LeaseSequence, closedTimestamp hlc.Timestamp, ) error { buf := &b.scratchFooter buf.MaxLeaseIndex = lai @@ -888,7 +888,7 @@ func (b *propBuf) marshallLAIAndClosedTimestampToProposalLocked( return err } -func (b *propBuf) forwardAssignedLAILocked(v uint64) { +func (b *propBuf) forwardAssignedLAILocked(v roachpb.LeaseSequence) { if b.assignedLAI < v { b.assignedLAI = v } @@ -939,7 +939,7 @@ func (b *propBuf) FlushLockedWithoutProposing(ctx context.Context) { // Similarly, appliedLAI is the highest LAI of an applied command; the propBuf // will propose commands with higher LAIs. func (b *propBuf) OnLeaseChangeLocked( - leaseOwned bool, appliedClosedTS hlc.Timestamp, appliedLAI uint64, + leaseOwned bool, appliedClosedTS hlc.Timestamp, appliedLAI roachpb.LeaseSequence, ) { if leaseOwned { b.forwardClosedTimestampLocked(appliedClosedTS) @@ -1151,11 +1151,11 @@ func (rp *replicaProposer) destroyed() destroyStatus { return rp.mu.destroyStatus } -func (rp *replicaProposer) firstIndex() uint64 { +func (rp *replicaProposer) firstIndex() roachpb.RaftIndex { return (*Replica)(rp).raftFirstIndexRLocked() } -func (rp *replicaProposer) leaseAppliedIndex() uint64 { +func (rp *replicaProposer) leaseAppliedIndex() roachpb.LeaseSequence { return rp.mu.state.LeaseAppliedIndex } diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index 77d542d1b061..c88b3a2fb4df 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -46,8 +46,8 @@ type testProposer struct { syncutil.RWMutex clock *hlc.Clock ds destroyStatus - fi uint64 - lai uint64 + fi roachpb.RaftIndex + lai roachpb.LeaseSequence enqueued int registered int @@ -148,11 +148,11 @@ func (t *testProposer) destroyed() destroyStatus { return t.ds } -func (t *testProposer) firstIndex() uint64 { +func (t *testProposer) firstIndex() roachpb.RaftIndex { return t.fi } -func (t *testProposer) leaseAppliedIndex() uint64 { +func (t *testProposer) leaseAppliedIndex() roachpb.LeaseSequence { return t.lai } @@ -357,12 +357,12 @@ func TestProposalBuffer(t *testing.T) { require.Equal(t, num, p.registered) // We've flushed num requests, out of which one is a lease request (so that // one did not increment the MLAI). - require.Equal(t, uint64(num)-1, b.assignedLAI) + require.Equal(t, roachpb.LeaseSequence(num-1), b.assignedLAI) require.Equal(t, 2*propBufArrayMinSize, b.arr.len()) require.Equal(t, 1, b.evalTracker.Count()) proposals := r.consumeProposals() require.Len(t, proposals, propBufArrayMinSize) - var lai uint64 + var lai roachpb.LeaseSequence for i, p := range proposals { if i != leaseReqIdx { lai++ @@ -649,7 +649,7 @@ func TestProposalBufferRejectUnsafeLeaseTransfer(t *testing.T) { ctx := context.Background() proposer := uint64(1) - proposerFirstIndex := uint64(5) + proposerFirstIndex := roachpb.RaftIndex(5) target := uint64(2) // Each subtest will try to propose a lease transfer in a different Raft @@ -659,7 +659,7 @@ func TestProposalBufferRejectUnsafeLeaseTransfer(t *testing.T) { proposerState raft.StateType // math.MaxUint64 if the target is not in the raft group. targetState rafttracker.StateType - targetMatch uint64 + targetMatch roachpb.RaftIndex expRejection bool expRejectionReason raftutil.ReplicaNeedsSnapshotStatus @@ -747,11 +747,11 @@ func TestProposalBufferRejectUnsafeLeaseTransfer(t *testing.T) { if tc.proposerState == raft.StateLeader { raftStatus.Lead = proposer raftStatus.Progress = map[uint64]rafttracker.Progress{ - proposer: {State: rafttracker.StateReplicate, Match: proposerFirstIndex}, + proposer: {State: rafttracker.StateReplicate, Match: uint64(proposerFirstIndex)}, } if tc.targetState != math.MaxUint64 { raftStatus.Progress[target] = rafttracker.Progress{ - State: tc.targetState, Match: tc.targetMatch, + State: tc.targetState, Match: uint64(tc.targetMatch), } } } diff --git a/pkg/kv/kvserver/replica_proposal_quota.go b/pkg/kv/kvserver/replica_proposal_quota.go index 3c0ad9db1f77..42788d5d01ed 100644 --- a/pkg/kv/kvserver/replica_proposal_quota.go +++ b/pkg/kv/kvserver/replica_proposal_quota.go @@ -93,7 +93,7 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( // After the proposal quota is enabled all entries applied by this replica // will be appended to the quotaReleaseQueue. The proposalQuotaBaseIndex // and the quotaReleaseQueue together track status.Applied exactly. - r.mu.proposalQuotaBaseIndex = status.Applied + r.mu.proposalQuotaBaseIndex = roachpb.RaftIndex(status.Applied) if r.mu.proposalQuota != nil { log.Fatal(ctx, "proposalQuota was not nil before becoming the leader") } @@ -138,12 +138,12 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( now := timeutil.Now() // commitIndex is used to determine whether a newly added replica has fully // caught up. - commitIndex := status.Commit + commitIndex := roachpb.RaftIndex(status.Commit) // Initialize minIndex to the currently applied index. The below progress // checks will only decrease the minIndex. Given that the quotaReleaseQueue // cannot correspond to values beyond the applied index there's no reason // to consider progress beyond it as meaningful. - minIndex := status.Applied + minIndex := roachpb.RaftIndex(status.Applied) r.mu.internalRaftGroup.WithProgress(func(id uint64, _ raft.ProgressType, progress tracker.Progress) { rep, ok := r.mu.state.Desc.GetReplicaDescriptorByID(roachpb.ReplicaID(id)) @@ -203,7 +203,7 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( // Only consider followers who are in advance of the quota base // index. This prevents a follower from coming back online and // preventing throughput to the range until it has caught up. - if progress.Match < r.mu.proposalQuotaBaseIndex { + if roachpb.RaftIndex(progress.Match) < r.mu.proposalQuotaBaseIndex { return } if _, paused := r.mu.pausedFollowers[roachpb.ReplicaID(id)]; paused { @@ -214,13 +214,13 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( // See #79215. return } - if progress.Match > 0 && progress.Match < minIndex { - minIndex = progress.Match + if progress.Match > 0 && roachpb.RaftIndex(progress.Match) < minIndex { + minIndex = roachpb.RaftIndex(progress.Match) } // If this is the most recently added replica, and it has caught up, clear // our state that was tracking it. This is unrelated to managing proposal // quota, but this is a convenient place to do so. - if rep.ReplicaID == r.mu.lastReplicaAdded && progress.Match >= commitIndex { + if rep.ReplicaID == r.mu.lastReplicaAdded && roachpb.RaftIndex(progress.Match) >= commitIndex { r.mu.lastReplicaAdded = 0 r.mu.lastReplicaAddedTime = time.Time{} } @@ -245,8 +245,8 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( // correspond to applied entries. It should not be possible for the base // index and the not yet released applied entries to not equal the applied // index. - releasableIndex := r.mu.proposalQuotaBaseIndex + uint64(len(r.mu.quotaReleaseQueue)) - if releasableIndex != status.Applied { + releasableIndex := r.mu.proposalQuotaBaseIndex + roachpb.RaftIndex(len(r.mu.quotaReleaseQueue)) + if releasableIndex != roachpb.RaftIndex(status.Applied) { log.Fatalf(ctx, "proposalQuotaBaseIndex (%d) + quotaReleaseQueueLen (%d) = %d"+ " must equal the applied index (%d)", r.mu.proposalQuotaBaseIndex, len(r.mu.quotaReleaseQueue), releasableIndex, diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 898892163df1..3d66f092f56e 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -66,7 +66,7 @@ var ( // have grown abnormally large. RaftLogTruncationThreshold will typically not // trigger it, unless the average log entry is <= 160 bytes. The key size is // ~16 bytes, so Pebble point deletion batches will be bounded at ~1.6MB. - raftLogTruncationClearRangeThreshold = uint64(util.ConstantWithMetamorphicTestRange( + raftLogTruncationClearRangeThreshold = roachpb.RaftIndex(util.ConstantWithMetamorphicTestRange( "raft-log-truncation-clearrange-threshold", 100000 /* default */, 1 /* min */, 1e6 /* max */)) ) @@ -356,7 +356,7 @@ func (r *Replica) propose( // Failure to propose will propagate to the client. An invariant of this // package is that proposals which are finished carry a raft command with a // MaxLeaseIndex equal to the proposal command's max lease index. - defer func(prev uint64) { + defer func(prev roachpb.LeaseSequence) { if pErr != nil { p.command.MaxLeaseIndex = prev } @@ -1464,8 +1464,8 @@ func (r *Replica) maybeCoalesceHeartbeat( RangeID: r.RangeID, ToReplicaID: toReplica.ReplicaID, FromReplicaID: fromReplica.ReplicaID, - Term: msg.Term, - Commit: msg.Commit, + Term: roachpb.RaftTerm(msg.Term), + Commit: roachpb.RaftIndex(msg.Commit), Quiesce: quiesce, LaggingFollowersOnQuiesce: lagging, LaggingFollowersOnQuiesceAccurate: quiesce, @@ -1783,7 +1783,7 @@ func (r *Replica) reportSnapshotStatus(ctx context.Context, to roachpb.ReplicaID } type snapTruncationInfo struct { - index uint64 + index roachpb.RaftIndex recipientStore roachpb.StoreID initial bool } @@ -1805,7 +1805,7 @@ type snapTruncationInfo struct { // queue to a new replica; some callers only care about these snapshots. func (r *Replica) addSnapshotLogTruncationConstraint( ctx context.Context, snapUUID uuid.UUID, initial bool, recipientStore roachpb.StoreID, -) (uint64, func()) { +) (roachpb.RaftIndex, func()) { r.mu.Lock() defer r.mu.Unlock() appliedIndex := r.mu.state.RaftAppliedIndex @@ -1854,7 +1854,7 @@ func (r *Replica) addSnapshotLogTruncationConstraint( // to new replicas are considered. func (r *Replica) getSnapshotLogTruncationConstraintsRLocked( recipientStore roachpb.StoreID, initialOnly bool, -) (_ []snapTruncationInfo, minSnapIndex uint64) { +) (_ []snapTruncationInfo, minSnapIndex roachpb.RaftIndex) { var sl []snapTruncationInfo for _, item := range r.mu.snapshotLogTruncationConstraints { if initialOnly && !item.initial { diff --git a/pkg/kv/kvserver/replica_raft_overload.go b/pkg/kv/kvserver/replica_raft_overload.go index 61366df54452..7a3baed07ae2 100644 --- a/pkg/kv/kvserver/replica_raft_overload.go +++ b/pkg/kv/kvserver/replica_raft_overload.go @@ -72,7 +72,7 @@ type computeExpendableOverloadedFollowersInput struct { // that the original store can now contribute to quorum. However, that store // is likely behind on the log, and we should consider it as non-live until // it has caught up. - minLiveMatchIndex uint64 + minLiveMatchIndex roachpb.RaftIndex } type nonLiveReason byte @@ -129,7 +129,7 @@ func computeExpendableOverloadedFollowers( if pr.IsPaused() { nonLive[roachpb.ReplicaID(id)] = nonLiveReasonPaused } - if pr.Match < d.minLiveMatchIndex { + if roachpb.RaftIndex(pr.Match) < d.minLiveMatchIndex { nonLive[roachpb.ReplicaID(id)] = nonLiveReasonBehind } } diff --git a/pkg/kv/kvserver/replica_raft_overload_test.go b/pkg/kv/kvserver/replica_raft_overload_test.go index 695ba01a0fd1..c4cd42832998 100644 --- a/pkg/kv/kvserver/replica_raft_overload_test.go +++ b/pkg/kv/kvserver/replica_raft_overload_test.go @@ -53,7 +53,7 @@ func TestReplicaRaftOverload_computeExpendableOverloadedFollowers(t *testing.T) snapshotMap := map[roachpb.ReplicaID]struct{}{} downMap := map[roachpb.ReplicaID]struct{}{} match := map[roachpb.ReplicaID]uint64{} - minLiveMatchIndex := uint64(0) // accept all live followers by default + minLiveMatchIndex := roachpb.RaftIndex(0) // accept all live followers by default for _, arg := range d.CmdArgs { for i := range arg.Vals { sl := strings.SplitN(arg.Vals[i], "@", 2) @@ -67,7 +67,7 @@ func TestReplicaRaftOverload_computeExpendableOverloadedFollowers(t *testing.T) arg.Scan(t, i, &id) switch arg.Key { case "min-live-match-index": - minLiveMatchIndex = id + minLiveMatchIndex = roachpb.RaftIndex(id) case "self": self = roachpb.ReplicaID(id) case "voters", "learners": diff --git a/pkg/kv/kvserver/replica_raft_quiesce.go b/pkg/kv/kvserver/replica_raft_quiesce.go index 0dd07956fcdb..ff3d95dd1bdd 100644 --- a/pkg/kv/kvserver/replica_raft_quiesce.go +++ b/pkg/kv/kvserver/replica_raft_quiesce.go @@ -194,7 +194,7 @@ type quiescer interface { isRaftLeaderRLocked() bool raftSparseStatusRLocked() *raftSparseStatus raftBasicStatusRLocked() raft.BasicStatus - raftLastIndexRLocked() uint64 + raftLastIndexRLocked() roachpb.RaftIndex hasRaftReadyRLocked() bool hasPendingProposalsRLocked() bool hasPendingProposalQuotaRLocked() bool @@ -380,7 +380,7 @@ func shouldReplicaQuiesce( return nil, nil, false } lastIndex := q.raftLastIndexRLocked() - if status.Commit != lastIndex { + if roachpb.RaftIndex(status.Commit) != lastIndex { if log.V(4) { log.Infof(ctx, "not quiescing: commit (%d) != lastIndex (%d)", status.Commit, lastIndex) diff --git a/pkg/kv/kvserver/replica_raft_truncation_test.go b/pkg/kv/kvserver/replica_raft_truncation_test.go index 16fdcd45e920..4e8fe90c437f 100644 --- a/pkg/kv/kvserver/replica_raft_truncation_test.go +++ b/pkg/kv/kvserver/replica_raft_truncation_test.go @@ -50,8 +50,11 @@ func TestHandleTruncatedStateBelowRaft(t *testing.T) { datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "prev": - d.ScanArgs(t, "index", &prevTruncatedState.Index) - d.ScanArgs(t, "term", &prevTruncatedState.Term) + var v uint64 + d.ScanArgs(t, "index", &v) + prevTruncatedState.Index = roachpb.RaftIndex(v) + d.ScanArgs(t, "term", &v) + prevTruncatedState.Term = roachpb.RaftTerm(v) return "" case "put": @@ -60,8 +63,8 @@ func TestHandleTruncatedStateBelowRaft(t *testing.T) { d.ScanArgs(t, "term", &term) truncState := &roachpb.RaftTruncatedState{ - Index: index, - Term: term, + Index: roachpb.RaftIndex(index), + Term: roachpb.RaftTerm(term), } require.NoError(t, loader.SetRaftTruncatedState(ctx, eng, truncState)) @@ -74,15 +77,15 @@ func TestHandleTruncatedStateBelowRaft(t *testing.T) { d.ScanArgs(t, "term", &term) suggestedTruncatedState := &roachpb.RaftTruncatedState{ - Index: index, - Term: term, + Index: roachpb.RaftIndex(index), + Term: roachpb.RaftTerm(term), } currentTruncatedState, err := loader.LoadRaftTruncatedState(ctx, eng) require.NoError(t, err) // Write log entries at start, middle, end, and above the truncated interval. if suggestedTruncatedState.Index > currentTruncatedState.Index { - indexes := []uint64{ + indexes := []roachpb.RaftIndex{ currentTruncatedState.Index + 1, // start (suggestedTruncatedState.Index + currentTruncatedState.Index + 1) / 2, // middle suggestedTruncatedState.Index, // end @@ -90,7 +93,7 @@ func TestHandleTruncatedStateBelowRaft(t *testing.T) { } for _, idx := range indexes { meta := enginepb.MVCCMetadata{RawBytes: make([]byte, 8)} - binary.BigEndian.PutUint64(meta.RawBytes, idx) + binary.BigEndian.PutUint64(meta.RawBytes, uint64(idx)) value, err := protoutil.Marshal(&meta) require.NoError(t, err) require.NoError(t, eng.PutUnversioned(prefixBuf.RaftLogKey(idx), value)) diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index a8dc4298bda9..fc92dc8fdf6e 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -79,7 +79,13 @@ func (r *replicaRaftStorage) InitialState() (raftpb.HardState, raftpb.ConfState, // access to r.mu.stateLoader. // // Entries can return log entries that are not yet stable in durable storage. -func (r *replicaRaftStorage) Entries(lo, hi, maxBytes uint64) ([]raftpb.Entry, error) { +func (r *replicaRaftStorage) Entries(lo, hi uint64, maxBytes uint64) ([]raftpb.Entry, error) { + return r.TypedEntries(roachpb.RaftIndex(lo), roachpb.RaftIndex(hi), maxBytes) +} + +func (r *replicaRaftStorage) TypedEntries( + lo, hi roachpb.RaftIndex, maxBytes uint64, +) ([]raftpb.Entry, error) { ctx := r.AnnotateCtx(context.TODO()) if r.raftMu.sideloaded == nil { return nil, errors.New("sideloaded storage is uninitialized") @@ -89,8 +95,10 @@ func (r *replicaRaftStorage) Entries(lo, hi, maxBytes uint64) ([]raftpb.Entry, e } // raftEntriesLocked requires that r.mu is held for writing. -func (r *Replica) raftEntriesLocked(lo, hi, maxBytes uint64) ([]raftpb.Entry, error) { - return (*replicaRaftStorage)(r).Entries(lo, hi, maxBytes) +func (r *Replica) raftEntriesLocked( + lo, hi roachpb.RaftIndex, maxBytes uint64, +) ([]raftpb.Entry, error) { + return (*replicaRaftStorage)(r).TypedEntries(lo, hi, maxBytes) } // invalidLastTerm is an out-of-band value for r.mu.lastTermNotDurable that @@ -99,9 +107,14 @@ func (r *Replica) raftEntriesLocked(lo, hi, maxBytes uint64) ([]raftpb.Entry, er const invalidLastTerm = 0 // Term implements the raft.Storage interface. -// Term requires that r.mu is held for writing because it requires exclusive -// access to r.mu.stateLoader. func (r *replicaRaftStorage) Term(i uint64) (uint64, error) { + term, err := r.TypedTerm(roachpb.RaftIndex(i)) + return uint64(term), err +} + +// TypedTerm requires that r.mu is held for writing because it requires exclusive +// access to r.mu.stateLoader. +func (r *replicaRaftStorage) TypedTerm(i roachpb.RaftIndex) (roachpb.RaftTerm, error) { // TODO(nvanbenschoten): should we set r.mu.lastTermNotDurable when // r.mu.lastIndexNotDurable == i && r.mu.lastTermNotDurable == invalidLastTerm? if r.mu.lastIndexNotDurable == i && r.mu.lastTermNotDurable != invalidLastTerm { @@ -113,38 +126,43 @@ func (r *replicaRaftStorage) Term(i uint64) (uint64, error) { } // raftTermLocked requires that r.mu is locked for writing. -func (r *Replica) raftTermLocked(i uint64) (uint64, error) { - return (*replicaRaftStorage)(r).Term(i) +func (r *Replica) raftTermLocked(i roachpb.RaftIndex) (roachpb.RaftTerm, error) { + return (*replicaRaftStorage)(r).TypedTerm(i) } // GetTerm returns the term of the given index in the raft log. It requires that // r.mu is not held. -func (r *Replica) GetTerm(i uint64) (uint64, error) { +func (r *Replica) GetTerm(i roachpb.RaftIndex) (roachpb.RaftTerm, error) { r.mu.Lock() defer r.mu.Unlock() return r.raftTermLocked(i) } // raftLastIndexRLocked requires that r.mu is held for reading. -func (r *Replica) raftLastIndexRLocked() uint64 { +func (r *Replica) raftLastIndexRLocked() roachpb.RaftIndex { return r.mu.lastIndexNotDurable } // LastIndex implements the raft.Storage interface. // LastIndex requires that r.mu is held for reading. func (r *replicaRaftStorage) LastIndex() (uint64, error) { + index, err := r.TypedLastIndex() + return uint64(index), err +} + +func (r *replicaRaftStorage) TypedLastIndex() (roachpb.RaftIndex, error) { return (*Replica)(r).raftLastIndexRLocked(), nil } // GetLastIndex returns the index of the last entry in the replica's Raft log. -func (r *Replica) GetLastIndex() uint64 { +func (r *Replica) GetLastIndex() roachpb.RaftIndex { r.mu.RLock() defer r.mu.RUnlock() return r.raftLastIndexRLocked() } // raftFirstIndexRLocked requires that r.mu is held for reading. -func (r *Replica) raftFirstIndexRLocked() uint64 { +func (r *Replica) raftFirstIndexRLocked() roachpb.RaftIndex { // TruncatedState is guaranteed to be non-nil. return r.mu.state.TruncatedState.Index + 1 } @@ -152,18 +170,23 @@ func (r *Replica) raftFirstIndexRLocked() uint64 { // FirstIndex implements the raft.Storage interface. // FirstIndex requires that r.mu is held for reading. func (r *replicaRaftStorage) FirstIndex() (uint64, error) { + index, err := r.TypedFirstIndex() + return uint64(index), err +} + +func (r *replicaRaftStorage) TypedFirstIndex() (roachpb.RaftIndex, error) { return (*Replica)(r).raftFirstIndexRLocked(), nil } // GetFirstIndex returns the index of the first entry in the replica's Raft log. -func (r *Replica) GetFirstIndex() uint64 { +func (r *Replica) GetFirstIndex() roachpb.RaftIndex { r.mu.RLock() defer r.mu.RUnlock() return r.raftFirstIndexRLocked() } // GetLeaseAppliedIndex returns the lease index of the last applied command. -func (r *Replica) GetLeaseAppliedIndex() uint64 { +func (r *Replica) GetLeaseAppliedIndex() roachpb.LeaseSequence { r.mu.RLock() defer r.mu.RUnlock() return r.mu.state.LeaseAppliedIndex @@ -189,8 +212,8 @@ func (r *replicaRaftStorage) Snapshot() (raftpb.Snapshot, error) { r.mu.AssertHeld() return raftpb.Snapshot{ Metadata: raftpb.SnapshotMetadata{ - Index: r.mu.state.RaftAppliedIndex, - Term: r.mu.state.RaftAppliedIndexTerm, + Index: uint64(r.mu.state.RaftAppliedIndex), + Term: uint64(r.mu.state.RaftAppliedIndexTerm), }, }, nil } @@ -288,7 +311,7 @@ type IncomingSnapshot struct { DataSize int64 snapType kvserverpb.SnapshotRequest_Type placeholder *ReplicaPlaceholder - raftAppliedIndex uint64 // logging only + raftAppliedIndex roachpb.RaftIndex // logging only } func (s IncomingSnapshot) String() string { @@ -336,8 +359,8 @@ func snapshot( RaftSnap: raftpb.Snapshot{ Data: snapUUID.GetBytes(), Metadata: raftpb.SnapshotMetadata{ - Index: state.RaftAppliedIndex, - Term: state.RaftAppliedIndexTerm, + Index: uint64(state.RaftAppliedIndex), + Term: uint64(state.RaftAppliedIndexTerm), // Synthesize our raftpb.ConfState from desc. ConfState: desc.Replicas().ConfState(), }, @@ -569,11 +592,11 @@ func (r *Replica) applySnapshot( log.Fatalf(ctx, "unable to load replica state: %s", err) } - if state.RaftAppliedIndex != nonemptySnap.Metadata.Index { + if uint64(state.RaftAppliedIndex) != nonemptySnap.Metadata.Index { log.Fatalf(ctx, "snapshot RaftAppliedIndex %d doesn't match its metadata index %d", state.RaftAppliedIndex, nonemptySnap.Metadata.Index) } - if state.RaftAppliedIndexTerm != nonemptySnap.Metadata.Term { + if uint64(state.RaftAppliedIndexTerm) != nonemptySnap.Metadata.Term { log.Fatalf(ctx, "snapshot RaftAppliedIndexTerm %d doesn't match its metadata term %d", state.RaftAppliedIndexTerm, nonemptySnap.Metadata.Term) } @@ -749,8 +772,8 @@ func writeUnreplicatedSST( if err := sl.SetRaftTruncatedState( ctx, &unreplicatedSST, &roachpb.RaftTruncatedState{ - Index: meta.Index, - Term: meta.Term, + Index: roachpb.RaftIndex(meta.Index), + Term: roachpb.RaftTerm(meta.Term), }, ); err != nil { return nil, false, errors.Wrapf(err, "unable to write TruncatedState to unreplicated SST writer") diff --git a/pkg/kv/kvserver/replica_rangefeed_test.go b/pkg/kv/kvserver/replica_rangefeed_test.go index 77281986930e..1ad0a7086a4d 100644 --- a/pkg/kv/kvserver/replica_rangefeed_test.go +++ b/pkg/kv/kvserver/replica_rangefeed_test.go @@ -859,7 +859,7 @@ func TestReplicaRangefeedErrors(t *testing.T) { // to make the test pass reliably. // NB: the Index on the message is the log index that _precedes_ any of the // entries in the MsgApp, so filter where msg.Index < index, not <= index. - return req.Message.Type == raftpb.MsgApp && req.Message.Index < index + return req.Message.Type == raftpb.MsgApp && roachpb.RaftIndex(req.Message.Index) < index }, dropHB: func(*kvserverpb.RaftHeartbeat) bool { return false }, dropResp: func(*kvserverpb.RaftMessageResponse) bool { return false }, diff --git a/pkg/kv/kvserver/replica_rankings.go b/pkg/kv/kvserver/replica_rankings.go index 147fd5062ac0..4c47bbbac2c3 100644 --- a/pkg/kv/kvserver/replica_rankings.go +++ b/pkg/kv/kvserver/replica_rankings.go @@ -43,7 +43,7 @@ type CandidateReplica interface { RaftStatus() *raft.Status // GetFirstIndex returns the index of the first entry in the replica's Raft // log. - GetFirstIndex() uint64 + GetFirstIndex() roachpb.RaftIndex // DescAndSpanConfig returns the authoritative range descriptor as well // as the span config for the replica. DescAndSpanConfig() (*roachpb.RangeDescriptor, roachpb.SpanConfig) diff --git a/pkg/kv/kvserver/replica_sideload_test.go b/pkg/kv/kvserver/replica_sideload_test.go index 6375b3953b28..c2293c7a0cc4 100644 --- a/pkg/kv/kvserver/replica_sideload_test.go +++ b/pkg/kv/kvserver/replica_sideload_test.go @@ -205,7 +205,7 @@ func TestRaftSSTableSideloading(t *testing.T) { } ent, err := logstore.MaybeInlineSideloadedRaftCommand(ctx, tc.repl.RangeID, ents[idx], tc.repl.raftMu.sideloaded, tc.store.raftEntryCache) require.NoError(t, err) - sst, err := tc.repl.raftMu.sideloaded.Get(ctx, ent.Index, ent.Term) + sst, err := tc.repl.raftMu.sideloaded.Get(ctx, roachpb.RaftIndex(ent.Index), roachpb.RaftTerm(ent.Term)) require.NoError(t, err) require.Equal(t, origSSTData, sst) break @@ -229,7 +229,7 @@ func TestRaftSSTableSideloadingTruncation(t *testing.T) { const count = 10 - var indexes []uint64 + var indexes []roachpb.RaftIndex addLastIndex := func() { lastIndex := tc.repl.GetLastIndex() indexes = append(indexes, lastIndex) diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index d655970614b1..6a4059e6d4c8 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -1512,7 +1512,7 @@ func internalMergeArgs(key []byte, value roachpb.Value) kvpb.MergeRequest { } } -func truncateLogArgs(index uint64, rangeID roachpb.RangeID) kvpb.TruncateLogRequest { +func truncateLogArgs(index roachpb.RaftIndex, rangeID roachpb.RangeID) kvpb.TruncateLogRequest { return kvpb.TruncateLogRequest{ Index: index, RangeID: rangeID, @@ -6552,7 +6552,7 @@ func TestAppliedIndex(t *testing.T) { sc.TestingKnobs.DisableCanAckBeforeApplication = true tc.StartWithStoreConfig(ctx, t, stopper, sc) - var appliedIndex uint64 + var appliedIndex roachpb.RaftIndex var sum int64 for i := int64(1); i <= 10; i++ { args := incrementArgs([]byte("a"), i) @@ -7207,10 +7207,10 @@ func TestEntries(t *testing.T) { repl := tc.repl rangeID := repl.RangeID - var indexes []uint64 + var indexes []roachpb.RaftIndex - populateLogs := func(from, to int) []uint64 { - var newIndexes []uint64 + populateLogs := func(from, to int) []roachpb.RaftIndex { + var newIndexes []roachpb.RaftIndex for i := from; i < to; i++ { args := incrementArgs([]byte("a"), int64(i)) if _, pErr := tc.SendWrapped(args); pErr != nil { @@ -7239,8 +7239,8 @@ func TestEntries(t *testing.T) { indexes = append(indexes, populateLogs(0, 10)...) for i, tc := range []struct { - lo uint64 - hi uint64 + lo roachpb.RaftIndex + hi roachpb.RaftIndex maxBytes uint64 expResultCount int expCacheCount int @@ -7330,7 +7330,7 @@ func TestEntries(t *testing.T) { if len(ents) != tc.expResultCount { t.Errorf("%d: expected %d entries, got %d", i, tc.expResultCount, len(ents)) } else if tc.expResultCount > 0 { - expHitLimit := ents[len(ents)-1].Index < tc.hi-1 + expHitLimit := roachpb.RaftIndex(ents[len(ents)-1].Index) < tc.hi-1 if hitLimit != expHitLimit { t.Errorf("%d: unexpected hit limit: %t", i, hitLimit) } @@ -7365,7 +7365,7 @@ func TestTerm(t *testing.T) { rangeID := repl.RangeID // Populate the log with 10 entries. Save the LastIndex after each write. - var indexes []uint64 + var indexes []roachpb.RaftIndex for i := 0; i < 10; i++ { args := incrementArgs([]byte("a"), int64(i)) @@ -7687,11 +7687,11 @@ func TestReplicaRetryRaftProposal(t *testing.T) { type magicKey struct{} - var c int32 // updated atomically - var wrongLeaseIndex uint64 // populated below + var c int32 // updated atomically + var wrongLeaseIndex roachpb.LeaseSequence // populated below tc.repl.mu.Lock() - tc.repl.mu.proposalBuf.testing.leaseIndexFilter = func(p *ProposalData) (indexOverride uint64) { + tc.repl.mu.proposalBuf.testing.leaseIndexFilter = func(p *ProposalData) (indexOverride roachpb.LeaseSequence) { if v := p.ctx.Value(magicKey{}); v != nil { if curAttempt := atomic.AddInt32(&c, 1); curAttempt == 1 { return wrongLeaseIndex @@ -8138,7 +8138,7 @@ func TestReplicaRefreshMultiple(t *testing.T) { t.Fatalf("test requires LeaseAppliedIndex >= 2 at this point, have %d", ai) } assigned := false - repl.mu.proposalBuf.testing.leaseIndexFilter = func(p *ProposalData) (indexOverride uint64) { + repl.mu.proposalBuf.testing.leaseIndexFilter = func(p *ProposalData) roachpb.LeaseSequence { if p == proposal && !assigned { assigned = true t.Logf("assigned wrong LAI %d", ai-1) @@ -8223,7 +8223,7 @@ func TestReplicaReproposalWithNewLeaseIndexError(t *testing.T) { var curFlushAttempt, curInsertAttempt int32 // updated atomically tc.repl.mu.Lock() - tc.repl.mu.proposalBuf.testing.leaseIndexFilter = func(p *ProposalData) (indexOverride uint64) { + tc.repl.mu.proposalBuf.testing.leaseIndexFilter = func(p *ProposalData) roachpb.LeaseSequence { if v := p.ctx.Value(magicKey{}); v != nil { flushAttempts := atomic.AddInt32(&curFlushAttempt, 1) switch flushAttempts { @@ -8233,7 +8233,7 @@ func TestReplicaReproposalWithNewLeaseIndexError(t *testing.T) { // write. Two requests can't have the same lease applied index, // so this will cause it to be rejected beneath raft with an // illegal lease index error. - wrongLeaseIndex := uint64(1) + wrongLeaseIndex := roachpb.LeaseSequence(1) return wrongLeaseIndex default: // Unexpected. Asserted against below. @@ -8327,7 +8327,7 @@ func TestFailureToProcessCommandClearsLocalResult(t *testing.T) { r := tc.repl r.mu.Lock() - r.mu.proposalBuf.testing.leaseIndexFilter = func(p *ProposalData) (indexOverride uint64) { + r.mu.proposalBuf.testing.leaseIndexFilter = func(p *ProposalData) roachpb.LeaseSequence { // We're going to recognize the first time the commnand for the EndTxn is // proposed and we're going to hackily force a low MaxLeaseIndex, so that // the processing gets rejected further on. @@ -9902,7 +9902,7 @@ type testQuiescer struct { numProposals int pendingQuota bool status *raftSparseStatus - lastIndex uint64 + lastIndex roachpb.RaftIndex raftReady bool leaseStatus kvserverpb.LeaseStatus storeID roachpb.StoreID @@ -9930,7 +9930,7 @@ func (q *testQuiescer) raftBasicStatusRLocked() raft.BasicStatus { return q.status.BasicStatus } -func (q *testQuiescer) raftLastIndexRLocked() uint64 { +func (q *testQuiescer) raftLastIndexRLocked() roachpb.RaftIndex { return q.lastIndex } diff --git a/pkg/kv/kvserver/stateloader/initial_test.go b/pkg/kv/kvserver/stateloader/initial_test.go index b940e98dcf71..18286b3fa441 100644 --- a/pkg/kv/kvserver/stateloader/initial_test.go +++ b/pkg/kv/kvserver/stateloader/initial_test.go @@ -33,19 +33,20 @@ func TestSynthesizeHardState(t *testing.T) { tHS := raftpb.HardState{Term: 2, Vote: 3, Commit: 4} testCases := []struct { - TruncTerm, RaftAppliedIndex uint64 - OldHS *raftpb.HardState - NewHS raftpb.HardState - Err string + TruncTerm roachpb.RaftTerm + RaftAppliedIndex roachpb.RaftIndex + OldHS *raftpb.HardState + NewHS raftpb.HardState + Err string }{ {OldHS: nil, TruncTerm: 42, RaftAppliedIndex: 24, NewHS: raftpb.HardState{Term: 42, Vote: 0, Commit: 24}}, // Can't wind back the committed index of the new HardState. - {OldHS: &tHS, RaftAppliedIndex: tHS.Commit - 1, Err: "can't decrease HardState.Commit"}, - {OldHS: &tHS, RaftAppliedIndex: tHS.Commit, NewHS: tHS}, - {OldHS: &tHS, RaftAppliedIndex: tHS.Commit + 1, NewHS: raftpb.HardState{Term: tHS.Term, Vote: 3, Commit: tHS.Commit + 1}}, + {OldHS: &tHS, RaftAppliedIndex: roachpb.RaftIndex(tHS.Commit - 1), Err: "can't decrease HardState.Commit"}, + {OldHS: &tHS, RaftAppliedIndex: roachpb.RaftIndex(tHS.Commit), NewHS: tHS}, + {OldHS: &tHS, RaftAppliedIndex: roachpb.RaftIndex(tHS.Commit + 1), NewHS: raftpb.HardState{Term: tHS.Term, Vote: 3, Commit: tHS.Commit + 1}}, // Higher Term is picked up, but vote isn't carried over when the term // changes. - {OldHS: &tHS, RaftAppliedIndex: tHS.Commit, TruncTerm: 11, NewHS: raftpb.HardState{Term: 11, Vote: 0, Commit: tHS.Commit}}, + {OldHS: &tHS, RaftAppliedIndex: roachpb.RaftIndex(tHS.Commit), TruncTerm: 11, NewHS: raftpb.HardState{Term: 11, Vote: 0, Commit: tHS.Commit}}, } for i, test := range testCases { diff --git a/pkg/kv/kvserver/stateloader/stateloader.go b/pkg/kv/kvserver/stateloader/stateloader.go index 3ca9496cd36a..3979475f681e 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -172,8 +172,8 @@ func (rsl StateLoader) SetLease( // LoadRangeAppliedState loads the Range applied state. func (rsl StateLoader) LoadRangeAppliedState( ctx context.Context, reader storage.Reader, -) (*enginepb.RangeAppliedState, error) { - var as enginepb.RangeAppliedState +) (*kvserverpb.RangeAppliedState, error) { + var as kvserverpb.RangeAppliedState _, err := storage.MVCCGetProto(ctx, reader, rsl.RangeAppliedStateKey(), hlc.Timestamp{}, &as, storage.MVCCGetOptions{}) return &as, err @@ -200,19 +200,21 @@ func (rsl StateLoader) LoadMVCCStats( func (rsl StateLoader) SetRangeAppliedState( ctx context.Context, readWriter storage.ReadWriter, - appliedIndex, leaseAppliedIndex, appliedIndexTerm uint64, + appliedIndex roachpb.RaftIndex, + leaseAppliedIndex roachpb.LeaseSequence, + appliedIndexTerm roachpb.RaftTerm, newMS *enginepb.MVCCStats, raftClosedTimestamp hlc.Timestamp, - asAlloc *enginepb.RangeAppliedState, // optional + asAlloc *kvserverpb.RangeAppliedState, // optional ) error { if asAlloc == nil { - asAlloc = new(enginepb.RangeAppliedState) + asAlloc = new(kvserverpb.RangeAppliedState) } as := asAlloc - *as = enginepb.RangeAppliedState{ + *as = kvserverpb.RangeAppliedState{ RaftAppliedIndex: appliedIndex, LeaseAppliedIndex: leaseAppliedIndex, - RangeStats: newMS.ToPersistentStats(), + RangeStats: kvserverpb.MVCCPersistentStats(*newMS), RaftClosedTimestamp: raftClosedTimestamp, RaftAppliedIndexTerm: appliedIndexTerm, } diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 7adb84b074a1..37fa40930fb2 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -280,11 +280,15 @@ func testStoreConfig(clock *hlc.Clock, version roachpb.Version) StoreConfig { } func newRaftConfig( - strg raft.Storage, id uint64, appliedIndex uint64, storeCfg StoreConfig, logger raft.Logger, + strg raft.Storage, + id uint64, + appliedIndex roachpb.RaftIndex, + storeCfg StoreConfig, + logger raft.Logger, ) *raft.Config { return &raft.Config{ ID: id, - Applied: appliedIndex, + Applied: uint64(appliedIndex), AsyncStorageWrites: true, ElectionTick: storeCfg.RaftElectionTimeoutTicks, HeartbeatTick: storeCfg.RaftHeartbeatIntervalTicks, diff --git a/pkg/kv/kvserver/store_raft.go b/pkg/kv/kvserver/store_raft.go index dc99302c14ae..73f717259045 100644 --- a/pkg/kv/kvserver/store_raft.go +++ b/pkg/kv/kvserver/store_raft.go @@ -233,8 +233,8 @@ func (s *Store) uncoalesceBeats( Type: msgT, From: uint64(beat.FromReplicaID), To: uint64(beat.ToReplicaID), - Term: beat.Term, - Commit: beat.Commit, + Term: uint64(beat.Term), + Commit: uint64(beat.Commit), } beatReqs[i] = kvserverpb.RaftMessageRequest{ RangeID: beat.RangeID, diff --git a/pkg/kv/kvserver/stores.go b/pkg/kv/kvserver/stores.go index c4e7e7ae9d9c..cb2c0ba96b0f 100644 --- a/pkg/kv/kvserver/stores.go +++ b/pkg/kv/kvserver/stores.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -125,7 +124,7 @@ func (ls *Stores) RemoveStore(s *Store) { // ForwardSideTransportClosedTimestampForRange forwards the side-transport // closed timestamp for the local replicas of the given range. func (ls *Stores) ForwardSideTransportClosedTimestampForRange( - ctx context.Context, rangeID roachpb.RangeID, closedTS hlc.Timestamp, lai ctpb.LAI, + ctx context.Context, rangeID roachpb.RangeID, closedTS hlc.Timestamp, lai roachpb.LeaseSequence, ) { if err := ls.VisitStores(func(s *Store) error { r := s.GetReplicaIfExists(rangeID) diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 6408f0b7cce1..a5525dd32fe6 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -1740,12 +1740,6 @@ func (crt ChangeReplicasTrigger) Removed() []ReplicaDescriptor { return crt.InternalRemovedReplicas } -// LeaseSequence is a custom type for a lease sequence number. -type LeaseSequence int64 - -// SafeValue implements the redact.SafeValue interface. -func (s LeaseSequence) SafeValue() {} - var _ fmt.Stringer = &Lease{} func (l Lease) String() string { @@ -2481,3 +2475,25 @@ func (tid *TenantID) UnmarshalYAML(unmarshal func(interface{}) error) error { return unmarshal(tid) } } + +// RaftTerm represents the term of a raft message. This corresponds to Term in +// HardState.Term in the Raft library. That type is a uint64, so it is necessary +// to cast to/from that type when dealing with the Raft library, however +// internally RaftTerm is used for all fields in CRDB. +type RaftTerm uint64 + +// RaftIndex represents the term of a raft message. This corresponds to Index in +// HardState.Index in the Raft library. That type is a uint64, so it is +// necessary to cast to/from that type when dealing with the Raft library, +// however internally RaftIndex is used for all fields in CRDB. +type RaftIndex uint64 + +// LeaseSequence is a custom type for a lease sequence number. +// It starts at 0, but was initially created as an int, so it is staying that +// way to avoid any compatibility considerations. +type LeaseSequence int64 + +// SafeValue implements the redact.SafeValue interface. +func (s RaftTerm) SafeValue() {} +func (s RaftIndex) SafeValue() {} +func (s LeaseSequence) SafeValue() {} diff --git a/pkg/roachpb/internal_raft.proto b/pkg/roachpb/internal_raft.proto index aa13c04b49e6..1c2b7492c4a8 100644 --- a/pkg/roachpb/internal_raft.proto +++ b/pkg/roachpb/internal_raft.proto @@ -22,9 +22,9 @@ message RaftTruncatedState { option (gogoproto.populate) = true; // The highest index that has been removed from the log. - optional uint64 index = 1 [(gogoproto.nullable) = false]; + optional uint64 index = 1 [(gogoproto.nullable) = false, (gogoproto.casttype) = "RaftIndex"]; // The term corresponding to 'index'. - optional uint64 term = 2 [(gogoproto.nullable) = false]; + optional uint64 term = 2 [(gogoproto.nullable) = false, (gogoproto.casttype) = "RaftTerm"]; } // RangeTombstone contains information about a replica that has been deleted. diff --git a/pkg/storage/enginepb/mvcc3.go b/pkg/storage/enginepb/mvcc3.go index 69fe1e128193..7ddfcd8ef110 100644 --- a/pkg/storage/enginepb/mvcc3.go +++ b/pkg/storage/enginepb/mvcc3.go @@ -28,24 +28,9 @@ func (ms *MVCCStats) ToStatsDelta() MVCCStatsDelta { return MVCCStatsDelta(*ms) } -// ToStats converts the receiver to an MVCCStats. -func (ms *MVCCPersistentStats) ToStats() MVCCStats { - return MVCCStats(*ms) -} - -// ToStatsPtr converts the receiver to a *MVCCStats. -func (ms *MVCCPersistentStats) ToStatsPtr() *MVCCStats { - return (*MVCCStats)(ms) -} - // SafeValue implements the redact.SafeValue interface. func (ms *MVCCStats) SafeValue() {} -// ToPersistentStats converts the receiver to an MVCCPersistentStats. -func (ms *MVCCStats) ToPersistentStats() MVCCPersistentStats { - return MVCCPersistentStats(*ms) -} - // MustSetValue is like SetValue, except it resets the enum and panics if the // provided value is not a valid variant type. func (op *MVCCLogicalOp) MustSetValue(value interface{}) { diff --git a/pkg/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index f815cb46d862..e92c7344ae4a 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -246,70 +246,6 @@ message MVCCStatsDelta { // whole message is marked as safe for log redaction. } -// MVCCPersistentStats is convertible to MVCCStats, but uses signed variable -// width encodings for most fields that make it efficient to store positive -// values but inefficient to store negative values. This makes the encodings -// incompatible. -message MVCCPersistentStats { - option (gogoproto.equal) = true; - option (gogoproto.populate) = true; - - int64 contains_estimates = 14; // must never go negative absent a bug - sfixed64 last_update_nanos = 1; - sfixed64 intent_age = 2; - sfixed64 gc_bytes_age = 3 [(gogoproto.customname) = "GCBytesAge"]; - int64 live_bytes = 4; - int64 live_count = 5; - int64 key_bytes = 6; - int64 key_count = 7; - int64 val_bytes = 8; - int64 val_count = 9; - int64 intent_bytes = 10; - int64 intent_count = 11; - int64 separated_intent_count = 16; - int64 range_key_count = 17; - int64 range_key_bytes = 18; - int64 range_val_count = 19; - int64 range_val_bytes = 20; - int64 sys_bytes = 12; - int64 sys_count = 13; - int64 abort_span_bytes = 15; -} - -// RangeAppliedState combines the raft and lease applied indices with -// mvcc stats. These are all persisted on each transition of the Raft -// state machine (i.e. on each Raft application), so they are stored -// in the same RocksDB key for efficiency. -message RangeAppliedState { - option (gogoproto.equal) = true; - option (gogoproto.populate) = true; - - // raft_applied_index is the highest (and last) index applied to the Raft - // state machine. - uint64 raft_applied_index = 1; - // lease_applied_index is the highest (and last) lease index applied to the - // Raft state machine. - uint64 lease_applied_index = 2; - // range_stats is the set of mvcc stats that accounts for the current value - // of the Raft state machine. - MVCCPersistentStats range_stats = 3 [(gogoproto.nullable) = false]; - - // raft_closed_timestamp is the largest timestamp that is known to have been - // closed through Raft commands as of this lease applied index. This means - // that the current leaseholder (if any) and any future leaseholder will not - // evaluate writes at or below this timestamp, and also that any in-flight - // commands that can still apply are writing at higher timestamps. - // Non-leaseholder replicas are free to serve "follower reads" at or below - // this timestamp. - util.hlc.Timestamp raft_closed_timestamp = 4 [(gogoproto.nullable) = false]; - - // raft_applied_index_term is the term corresponding to raft_applied_index. - // The serialized proto will not contain this field until code starts - // setting it to a value > 0 (in v22.1). This is desirable since we don't - // want a mixed version cluster (v21.2 and v22.1) to have divergent replica - // state simply because we have introduced this field. - uint64 raft_applied_index_term = 5; -} // MVCCWriteValueOp corresponds to a value being written outside of a // transaction. Inline values (without timestamp) are not logged. diff --git a/pkg/testutils/lint/passes/redactcheck/redactcheck.go b/pkg/testutils/lint/passes/redactcheck/redactcheck.go index 9f57165bf266..e0d6ce951f32 100644 --- a/pkg/testutils/lint/passes/redactcheck/redactcheck.go +++ b/pkg/testutils/lint/passes/redactcheck/redactcheck.go @@ -80,7 +80,6 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { "Method": {}, }, "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb": { - "LAI": {}, "SeqNum": {}, }, "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation": { @@ -112,6 +111,8 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { "StoreID": {}, "TenantID": {}, "TransactionStatus": {}, + "RaftIndex": {}, + "RaftTerm": {}, }, "github.com/cockroachdb/cockroach/pkg/rpc": { "ConnectionClass": {},