diff --git a/pkg/cli/debug_check_store.go b/pkg/cli/debug_check_store.go index 5936dc051c71..05d1ac45896f 100644 --- a/pkg/cli/debug_check_store.go +++ b/pkg/cli/debug_check_store.go @@ -80,11 +80,11 @@ func runDebugCheckStoreCmd(cmd *cobra.Command, args []string) error { } type replicaCheckInfo struct { - truncatedIndex uint64 - appliedIndex uint64 - firstIndex uint64 - lastIndex uint64 - committedIndex uint64 + truncatedIndex enginepb.RaftIndex + appliedIndex enginepb.RaftIndex + firstIndex enginepb.RaftIndex + lastIndex enginepb.RaftIndex + committedIndex enginepb.RaftIndex } type checkInput struct { @@ -251,7 +251,7 @@ func checkStoreRaftState( if err := kv.Value.GetProto(&hs); err != nil { return err } - getReplicaInfo(rangeID).committedIndex = hs.Commit + getReplicaInfo(rangeID).committedIndex = enginepb.RaftIndex(hs.Commit) case bytes.Equal(suffix, keys.LocalRaftTruncatedStateSuffix): var trunc roachpb.RaftTruncatedState if err := kv.Value.GetProto(&trunc); err != nil { @@ -265,7 +265,8 @@ func checkStoreRaftState( } getReplicaInfo(rangeID).appliedIndex = state.RaftAppliedIndex case bytes.Equal(suffix, keys.LocalRaftLogSuffix): - _, index, err := encoding.DecodeUint64Ascending(detail) + _, uIndex, err := encoding.DecodeUint64Ascending(detail) + index := enginepb.RaftIndex(uIndex) if err != nil { return err } diff --git a/pkg/keys/BUILD.bazel b/pkg/keys/BUILD.bazel index 47513301d44c..7cf9a214ddbc 100644 --- a/pkg/keys/BUILD.bazel +++ b/pkg/keys/BUILD.bazel @@ -17,6 +17,7 @@ go_library( deps = [ "//pkg/kv/kvpb", "//pkg/roachpb", + "//pkg/storage/enginepb", "//pkg/util/encoding", "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", @@ -37,6 +38,7 @@ go_test( deps = [ "//pkg/kv/kvpb", "//pkg/roachpb", + "//pkg/storage/enginepb", "//pkg/testutils", "//pkg/util/bitarray", "//pkg/util/duration", diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index ee2b4564fae0..1d3852bfe1c3 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -356,21 +357,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 enginepb.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 enginepb.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) (enginepb.RaftIndex, error) { _, logIndex, err := encoding.DecodeUint64Ascending(raftLogSuffix) - return logIndex, err + return enginepb.RaftIndex(logIndex), err } // RaftReplicaIDKey returns a system-local key for a RaftReplicaID. @@ -1092,7 +1093,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 enginepb.RaftIndex) roachpb.Key { return RaftLogKeyFromPrefix(b.RaftLogPrefix(), logIndex) } diff --git a/pkg/keys/printer.go b/pkg/keys/printer.go index 6fa74e7b6b3f..aa9f672fabd4 100644 --- a/pkg/keys/printer.go +++ b/pkg/keys/printer.go @@ -18,6 +18,7 @@ import ( "unicode/utf8" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -305,7 +306,7 @@ func raftLogKeyParse(rangeID roachpb.RangeID, input string) (string, roachpb.Key if err != nil { panic(err) } - return "", RaftLogKey(rangeID, index) + return "", RaftLogKey(rangeID, enginepb.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..a4fe0b3ac753 100644 --- a/pkg/keys/printer_test.go +++ b/pkg/keys/printer_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/apd/v3" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/bitarray" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -250,7 +251,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), enginepb.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/kvclient/kvcoord/dist_sender_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_test.go index 39355c8edd66..c0f4efb6050a 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_test.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util" @@ -813,7 +814,7 @@ func TestBackoffOnNotLeaseHolderErrorDuringTransfer(t *testing.T) { t.Fatal(err) } } - var sequences []roachpb.LeaseSequence + var sequences []enginepb.LeaseSequence var testFn simpleSendFn = func(_ context.Context, args *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { reply := &kvpb.BatchResponse{} if len(sequences) > 0 { @@ -856,12 +857,12 @@ func TestBackoffOnNotLeaseHolderErrorDuringTransfer(t *testing.T) { Settings: cluster.MakeTestingClusterSettings(), } for i, c := range []struct { - leaseSequences []roachpb.LeaseSequence + leaseSequences []enginepb.LeaseSequence expected int64 }{ - {[]roachpb.LeaseSequence{2, 1, 2, 3}, 2}, - {[]roachpb.LeaseSequence{0}, 0}, - {[]roachpb.LeaseSequence{2, 1, 2, 3, 2}, 3}, + {[]enginepb.LeaseSequence{2, 1, 2, 3}, 2}, + {[]enginepb.LeaseSequence{0}, 0}, + {[]enginepb.LeaseSequence{2, 1, 2, 3, 2}, 3}, } { t.Run("", func(t *testing.T) { sequences = c.leaseSequences @@ -1189,7 +1190,7 @@ func TestDistSenderIgnoresNLHEBasedOnOldRangeGeneration(t *testing.T) { } if tc.nlheLeaseSequence != 0 { nlhe.Lease = &roachpb.Lease{ - Sequence: roachpb.LeaseSequence(tc.nlheLeaseSequence), + Sequence: enginepb.LeaseSequence(tc.nlheLeaseSequence), Replica: roachpb.ReplicaDescriptor{NodeID: 4, StoreID: 4, ReplicaID: 4}, } } else { @@ -1202,7 +1203,7 @@ func TestDistSenderIgnoresNLHEBasedOnOldRangeGeneration(t *testing.T) { cachedLease := roachpb.Lease{ Replica: desc.InternalReplicas[1], - Sequence: roachpb.LeaseSequence(tc.cachedLeaseSequence), + Sequence: enginepb.LeaseSequence(tc.cachedLeaseSequence), } // The cache starts with a lease on node 2, so the first request will be @@ -1415,7 +1416,7 @@ func TestDistSenderDownNodeEvictLeaseholder(t *testing.T) { // The client has cleared the lease in the cache after the failure of the // first RPC. assert.Equal(t, desc.Generation, ba.ClientRangeInfo.DescriptorGeneration) - assert.Equal(t, roachpb.LeaseSequence(0), ba.ClientRangeInfo.LeaseSequence) + assert.Equal(t, enginepb.LeaseSequence(0), ba.ClientRangeInfo.LeaseSequence) assert.Equal(t, roachpb.LEAD_FOR_GLOBAL_READS, ba.ClientRangeInfo.ClosedTimestampPolicy) contacted2 = true br := ba.CreateReply() diff --git a/pkg/kv/kvclient/rangecache/BUILD.bazel b/pkg/kv/kvclient/rangecache/BUILD.bazel index b84278b53d1f..3d10089af068 100644 --- a/pkg/kv/kvclient/rangecache/BUILD.bazel +++ b/pkg/kv/kvclient/rangecache/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings/cluster", + "//pkg/storage/enginepb", "//pkg/util", "//pkg/util/cache", "//pkg/util/contextutil", @@ -35,6 +36,7 @@ go_test( "//pkg/keys", "//pkg/roachpb", "//pkg/settings/cluster", + "//pkg/storage/enginepb", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/stop", diff --git a/pkg/kv/kvclient/rangecache/range_cache.go b/pkg/kv/kvclient/rangecache/range_cache.go index 26b04b388958..df33c34c83bc 100644 --- a/pkg/kv/kvclient/rangecache/range_cache.go +++ b/pkg/kv/kvclient/rangecache/range_cache.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/cache" "github.com/cockroachdb/cockroach/pkg/util/contextutil" @@ -376,7 +377,7 @@ func (et EvictionToken) Lease() *roachpb.Lease { // LeaseSeq returns the sequence of the cached lease. If no lease is cached, or // the cached lease is speculative, 0 is returned. -func (et EvictionToken) LeaseSeq() roachpb.LeaseSequence { +func (et EvictionToken) LeaseSeq() enginepb.LeaseSequence { if !et.Valid() { panic("invalid LeaseSeq() call on empty EvictionToken") } diff --git a/pkg/kv/kvclient/rangecache/range_cache_test.go b/pkg/kv/kvclient/rangecache/range_cache_test.go index 950434d37dc7..1ebae8a3d5d5 100644 --- a/pkg/kv/kvclient/rangecache/range_cache_test.go +++ b/pkg/kv/kvclient/rangecache/range_cache_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -1536,7 +1537,7 @@ func TestRangeCacheEvictAndReplace(t *testing.T) { require.Equal(t, desc2, *tok.Desc()) require.NotNil(t, tok.Leaseholder()) require.Equal(t, rep1, *tok.Leaseholder()) - require.Equal(t, roachpb.LeaseSequence(1), tok.LeaseSeq()) + require.Equal(t, enginepb.LeaseSequence(1), tok.LeaseSeq()) require.Equal(t, lag, tok.ClosedTimestampPolicy(lead)) // EvictAndReplace() with a new closed timestamp policy. @@ -1547,7 +1548,7 @@ func TestRangeCacheEvictAndReplace(t *testing.T) { require.Equal(t, desc2, *tok.Desc()) require.NotNil(t, tok.Leaseholder()) require.Equal(t, rep1, *tok.Leaseholder()) - require.Equal(t, roachpb.LeaseSequence(1), tok.LeaseSeq()) + require.Equal(t, enginepb.LeaseSequence(1), tok.LeaseSeq()) require.Equal(t, lead, tok.ClosedTimestampPolicy(lag)) // EvictAndReplace() with a speculative descriptor. Should update decriptor, @@ -1937,7 +1938,7 @@ func TestRangeCacheSyncTokenAndMaybeUpdateCache(t *testing.T) { require.True(t, updatedLeaseholder) require.Equal(t, &desc2, tok.Desc()) require.Equal(t, &rep2, tok.Leaseholder()) - require.Equal(t, roachpb.LeaseSequence(0), tok.Lease().Sequence) + require.Equal(t, enginepb.LeaseSequence(0), tok.Lease().Sequence) }, }, } @@ -2151,7 +2152,7 @@ func TestRangeCacheEntryMaybeUpdate(t *testing.T) { Replica: rep1, Sequence: 1, } - require.Equal(t, roachpb.LeaseSequence(2), e.Lease().Sequence) + require.Equal(t, enginepb.LeaseSequence(2), e.Lease().Sequence) updated, updatedLease, e = e.maybeUpdate(ctx, l, &desc3) require.True(t, updated) require.False(t, updatedLease) diff --git a/pkg/kv/kvpb/api.proto b/pkg/kv/kvpb/api.proto index 51c2588d8794..a6ec3fc9a230 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/storage/enginepb.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/storage/enginepb.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/storage/enginepb.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/kvpb/errors.proto b/pkg/kv/kvpb/errors.proto index b58f61c4db68..20a800e1cb58 100644 --- a/pkg/kv/kvpb/errors.proto +++ b/pkg/kv/kvpb/errors.proto @@ -325,7 +325,7 @@ message WriteIntentError { // operating under. Used on the server to avoid adding discovered locks // which were discovered under old leases to the lock table. optional int64 lease_sequence = 3 [(gogoproto.nullable) = false, - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.LeaseSequence"]; + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.LeaseSequence"]; enum Reason { // The reason for the WriteIntentError is unspecified. This will // always be the case for errors returned from MVCC. diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 297bad31df8e..2d20e709eabf 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/lock", diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel b/pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel index 04f19b9dc06f..c1179a06859c 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel +++ b/pkg/kv/kvserver/allocator/allocatorimpl/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/storage/enginepb", "//pkg/util/admission/admissionpb", "//pkg/util/log", "//pkg/util/metric", @@ -54,6 +55,7 @@ go_test( "//pkg/kv/kvserver/replicastats", "//pkg/roachpb", "//pkg/settings/cluster", + "//pkg/storage/enginepb", "//pkg/testutils", "//pkg/testutils/gossiputil", "//pkg/util", diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go index d517d9770f7c..8e2cfa6d164d 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -1957,7 +1958,7 @@ func (a *Allocator) ValidLeaseTargets( leaseRepl interface { StoreID() roachpb.StoreID RaftStatus() *raft.Status - GetFirstIndex() uint64 + GetFirstIndex() enginepb.RaftIndex Desc() *roachpb.RangeDescriptor }, opts allocator.TransferLeaseOptions, @@ -2132,7 +2133,7 @@ func (a *Allocator) leaseholderShouldMoveDueToPreferences( leaseRepl interface { StoreID() roachpb.StoreID RaftStatus() *raft.Status - GetFirstIndex() uint64 + GetFirstIndex() enginepb.RaftIndex }, allExistingReplicas []roachpb.ReplicaDescriptor, ) bool { @@ -2217,7 +2218,7 @@ func (a *Allocator) TransferLeaseTarget( StoreID() roachpb.StoreID GetRangeID() roachpb.RangeID RaftStatus() *raft.Status - GetFirstIndex() uint64 + GetFirstIndex() enginepb.RaftIndex Desc() *roachpb.RangeDescriptor }, usageInfo allocator.RangeUsageInfo, @@ -2485,7 +2486,7 @@ func (a *Allocator) ShouldTransferLease( leaseRepl interface { StoreID() roachpb.StoreID RaftStatus() *raft.Status - GetFirstIndex() uint64 + GetFirstIndex() enginepb.RaftIndex Desc() *roachpb.RangeDescriptor }, usageInfo allocator.RangeUsageInfo, @@ -2852,7 +2853,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 enginepb.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..7a689a732200 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/replicastats" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/gossiputil" "github.com/cockroachdb/cockroach/pkg/util" @@ -1893,7 +1894,7 @@ func (r *mockRepl) RaftStatus() *raft.Status { return raftStatus } -func (r *mockRepl) GetFirstIndex() uint64 { +func (r *mockRepl) GetFirstIndex() enginepb.RaftIndex { return 0 } diff --git a/pkg/kv/kvserver/api.proto b/pkg/kv/kvserver/api.proto index 41f5f285b9ce..ee6d01395217 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/storage/enginepb.LeaseSequence"]; } message WaitForApplicationResponse { diff --git a/pkg/kv/kvserver/app_batch.go b/pkg/kv/kvserver/app_batch.go index 836c0765e0c0..43966eeb4e2b 100644 --- a/pkg/kv/kvserver/app_batch.go +++ b/pkg/kv/kvserver/app_batch.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" "golang.org/x/time/rate" @@ -179,7 +180,7 @@ func (b *appBatch) runPostAddTriggers( env.st, env.eng, env.sideloaded, - cmd.Term, + enginepb.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..c32ff6585f97 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/storage/enginepb", "@com_github_cockroachdb_errors//:errors", "@io_etcd_go_raft_v3//raftpb", ], @@ -26,6 +27,7 @@ go_test( args = ["-test.timeout=55s"], deps = [ ":apply", + "//pkg/storage/enginepb", "@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..02efb5b3961a 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/storage/enginepb" +) // 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() enginepb.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..254507ee9c0d 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/storage/enginepb" "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 enginepb.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() enginepb.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 [][]enginepb.RaftIndex + applied []enginepb.RaftIndex + appliedSideEffects []enginepb.RaftIndex batchOpen bool } @@ -166,7 +167,7 @@ func (sm *testStateMachine) ApplySideEffects( type testBatch struct { sm *testStateMachine ephemeral bool - staged []uint64 + staged []enginepb.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[enginepb.RaftIndex]bool + nonLocal map[enginepb.RaftIndex]bool + shouldReject map[enginepb.RaftIndex]bool + shouldThrowErrRemoved map[enginepb.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[enginepb.RaftIndex]bool), + nonLocal: make(map[enginepb.RaftIndex]bool), + shouldReject: make(map[enginepb.RaftIndex]bool), + shouldThrowErrRemoved: make(map[enginepb.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 := enginepb.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: [][]enginepb.RaftIndex{{1, 2}, {3}, {4}, {5}, {6}}, + applied: []enginepb.RaftIndex{1, 2, 3, 4, 5, 6}, + appliedSideEffects: []enginepb.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: [][]enginepb.RaftIndex{{1, 2}, {3}, {4}, {5, 6}, {7}}, + applied: []enginepb.RaftIndex{1, 2, 3, 4, 5, 6, 7}, + appliedSideEffects: []enginepb.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: [][]enginepb.RaftIndex{{1, 2}, {3}}, + applied: []enginepb.RaftIndex{1, 2, 3}, + appliedSideEffects: []enginepb.RaftIndex{1, 2, 3}, } require.Equal(t, exp, *sm) diff --git a/pkg/kv/kvserver/asim/storerebalancer/BUILD.bazel b/pkg/kv/kvserver/asim/storerebalancer/BUILD.bazel index 76129418a80a..df32054940d2 100644 --- a/pkg/kv/kvserver/asim/storerebalancer/BUILD.bazel +++ b/pkg/kv/kvserver/asim/storerebalancer/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/kv/kvserver/asim/op", "//pkg/kv/kvserver/asim/state", "//pkg/roachpb", + "//pkg/storage/enginepb", "//pkg/util/hlc", "@com_github_cockroachdb_errors//:errors", "@io_etcd_go_raft_v3//:raft", diff --git a/pkg/kv/kvserver/asim/storerebalancer/candidate_replica.go b/pkg/kv/kvserver/asim/storerebalancer/candidate_replica.go index 8845bfeaf479..016967f50c75 100644 --- a/pkg/kv/kvserver/asim/storerebalancer/candidate_replica.go +++ b/pkg/kv/kvserver/asim/storerebalancer/candidate_replica.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" "go.etcd.io/raft/v3" @@ -70,7 +71,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() enginepb.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..ea8ee501ecd6 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() enginepb.RaftIndex + GetTerm(index enginepb.RaftIndex) (enginepb.RaftTerm, error) + GetLeaseAppliedIndex() enginepb.LeaseSequence Desc() *roachpb.RangeDescriptor ContainsKey(key roachpb.Key) bool @@ -118,7 +118,7 @@ type EvalContext interface { // RevokeLease stops the replica from using its current lease, if that lease // matches the provided lease sequence. All future calls to leaseStatus on // this node with the current lease will now return a PROSCRIBED status. - RevokeLease(context.Context, roachpb.LeaseSequence) + RevokeLease(context.Context, enginepb.LeaseSequence) // WatchForMerge arranges to block all requests until the in-progress merge // completes. Returns an error if no in-progress merge is detected. @@ -172,13 +172,14 @@ type MockEvalCtx struct { CPU float64 AbortSpan *abortspan.AbortSpan GCThreshold hlc.Timestamp - Term, FirstIndex uint64 + Term enginepb.RaftTerm + FirstIndex enginepb.RaftIndex CanCreateTxnRecordFn func() (bool, kvpb.TransactionAbortedReason) MinTxnCommitTSFn func() hlc.Timestamp Lease roachpb.Lease CurrentReadSummary rspb.ReadSummary ClosedTimestamp hlc.Timestamp - RevokedLeaseSeq roachpb.LeaseSequence + RevokedLeaseSeq enginepb.LeaseSequence MaxBytes int64 ApproxDiskBytes uint64 } @@ -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() enginepb.RaftIndex { return m.FirstIndex } -func (m *mockEvalCtxImpl) GetTerm(uint64) (uint64, error) { +func (m *mockEvalCtxImpl) GetTerm(enginepb.RaftIndex) (enginepb.RaftTerm, error) { return m.Term, nil } -func (m *mockEvalCtxImpl) GetLeaseAppliedIndex() uint64 { +func (m *mockEvalCtxImpl) GetLeaseAppliedIndex() enginepb.LeaseSequence { panic("unimplemented") } func (m *mockEvalCtxImpl) Desc() *roachpb.RangeDescriptor { @@ -293,7 +294,7 @@ func (m *mockEvalCtxImpl) GetCurrentClosedTimestamp(ctx context.Context) hlc.Tim return m.ClosedTimestamp } -func (m *mockEvalCtxImpl) RevokeLease(_ context.Context, seq roachpb.LeaseSequence) { +func (m *mockEvalCtxImpl) RevokeLease(_ context.Context, seq enginepb.LeaseSequence) { m.RevokedLeaseSeq = seq } func (m *mockEvalCtxImpl) WatchForMerge(ctx context.Context) error { diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 5a3611f313f9..2a96e5626c6d 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -663,7 +663,7 @@ func mergeCheckingTimestampCaches( } // Applied to leaseholder after the partition heals. - var truncIndex uint64 + var truncIndex enginepb.RaftIndex restoredLeaseholderFuncs := noopRaftHandlerFuncs() restoredLeaseholderFuncs.dropReq = func(req *kvserverpb.RaftMessageRequest) bool { // Make sure that even going forward no MsgApp for what we just @@ -674,7 +674,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 && enginepb.RaftIndex(req.Message.Index) < truncIndex } // Because we enter a split leader-leaseholder state, none of the @@ -729,13 +729,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 enginepb.RaftIndex) enginepb.RaftIndex { if a < b { return a } return b } - minLastIndex := uint64(math.MaxUint64) + minLastIndex := enginepb.RaftIndex(math.MaxUint64) for _, r := range lhsRepls { lastIndex := r.GetLastIndex() minLastIndex = min(minLastIndex, lastIndex) @@ -4025,7 +4025,7 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) { } // Truncate the logs of the LHS. - index := func() uint64 { + index := func() enginepb.RaftIndex { repl := store0.LookupReplica(roachpb.RKey(keyA)) index := repl.GetLastIndex() truncArgs := &kvpb.TruncateLogRequest{ @@ -4057,7 +4057,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 && enginepb.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..9ef19e74227b 100644 --- a/pkg/kv/kvserver/client_raft_log_queue_test.go +++ b/pkg/kv/kvserver/client_raft_log_queue_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -87,7 +88,7 @@ func TestRaftLogQueue(t *testing.T) { } } - var afterTruncationIndex uint64 + var afterTruncationIndex enginepb.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..0414661381b3 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 enginepb.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 && enginepb.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 = enginepb.RaftTerm(status.Term) + hbReq.Heartbeats[i].Commit = enginepb.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 enginepb.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..c879d109010e 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 enginepb.RaftIndex, rangeID roachpb.RangeID) *kvpb.TruncateLogRequest { return &kvpb.TruncateLogRequest{ Index: index, RangeID: rangeID, diff --git a/pkg/kv/kvserver/closedts/ctpb/BUILD.bazel b/pkg/kv/kvserver/closedts/ctpb/BUILD.bazel index c2884a50c24d..502b434dc678 100644 --- a/pkg/kv/kvserver/closedts/ctpb/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/ctpb/BUILD.bazel @@ -11,6 +11,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/roachpb", + "//pkg/storage/enginepb", "//pkg/util/timeutil", ], ) diff --git a/pkg/kv/kvserver/closedts/ctpb/service.go b/pkg/kv/kvserver/closedts/ctpb/service.go index d975d50946d5..d3743494b3b8 100644 --- a/pkg/kv/kvserver/closedts/ctpb/service.go +++ b/pkg/kv/kvserver/closedts/ctpb/service.go @@ -17,6 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) @@ -26,12 +27,8 @@ 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() {} +// Force import of LeaseSequence in this package. +var _ = enginepb.LeaseSequence(0) func (m *Update) String() string { sb := &strings.Builder{} diff --git a/pkg/kv/kvserver/closedts/ctpb/service.proto b/pkg/kv/kvserver/closedts/ctpb/service.proto index 0337bbdf1ec2..74554a31048a 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/storage/enginepb.LeaseSequence"]; roachpb.RangeClosedTimestampPolicy policy = 3; } repeated RangeUpdate added_or_updated = 6 [(gogoproto.nullable) = false]; diff --git a/pkg/kv/kvserver/closedts/sidetransport/BUILD.bazel b/pkg/kv/kvserver/closedts/sidetransport/BUILD.bazel index 8126a02008bd..d6a7940869b3 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/sidetransport/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/rpc", "//pkg/rpc/nodedialer", "//pkg/settings/cluster", + "//pkg/storage/enginepb", "//pkg/util/hlc", "//pkg/util/intsets", "//pkg/util/log", @@ -46,6 +47,7 @@ go_test( "//pkg/roachpb", "//pkg/rpc", "//pkg/settings/cluster", + "//pkg/storage/enginepb", "//pkg/testutils", "//pkg/util/hlc", "//pkg/util/leaktest", diff --git a/pkg/kv/kvserver/closedts/sidetransport/receiver.go b/pkg/kv/kvserver/closedts/sidetransport/receiver.go index 50f5c968af19..cb793ff987ba 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/receiver.go +++ b/pkg/kv/kvserver/closedts/sidetransport/receiver.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -98,7 +99,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, enginepb.LeaseSequence) { s.mu.RLock() conn, ok := s.mu.conns[leaseholderNode] s.mu.RUnlock() @@ -190,7 +191,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 enginepb.LeaseSequence) } func newIncomingStream(s *Receiver, stores Stores) *incomingStream { @@ -208,7 +209,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, enginepb.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..72efb906c688 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/receiver_test.go +++ b/pkg/kv/kvserver/closedts/sidetransport/receiver_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -32,13 +33,13 @@ type mockStores struct { type rangeUpdate struct { rid roachpb.RangeID closedTS hlc.Timestamp - lai ctpb.LAI + lai enginepb.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 enginepb.LeaseSequence, ) { upd := rangeUpdate{ rid: rangeID, @@ -64,12 +65,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 = enginepb.LeaseSequence(0) -const lai100 = ctpb.LAI(100) -const lai101 = ctpb.LAI(101) -const lai102 = ctpb.LAI(102) -const lai103 = ctpb.LAI(102) +const lai100 = enginepb.LeaseSequence(100) +const lai101 = enginepb.LeaseSequence(101) +const lai102 = enginepb.LeaseSequence(102) +const lai103 = enginepb.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..5443a776f705 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/sender.go +++ b/pkg/kv/kvserver/closedts/sidetransport/sender.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/intsets" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -114,7 +115,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 enginepb.LeaseSequence policy roachpb.RangeClosedTimestampPolicy } @@ -122,7 +123,7 @@ type trackedRange struct { // the sender and can send closed timestamp updates through the side transport. type leaseholder struct { Replica - leaseSeq roachpb.LeaseSequence + leaseSeq enginepb.LeaseSequence } // Replica represents a *Replica object, but with only the capabilities needed @@ -167,7 +168,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 enginepb.LeaseSequence // The range's current policy. Policy roachpb.RangeClosedTimestampPolicy } @@ -266,7 +267,7 @@ func (s *Sender) Run(ctx context.Context, nodeID roachpb.NodeID) { // on, until the replica is unregistered, the side-transport will try to advance // this replica's closed timestamp. func (s *Sender) RegisterLeaseholder( - ctx context.Context, r Replica, leaseSeq roachpb.LeaseSequence, + ctx context.Context, r Replica, leaseSeq enginepb.LeaseSequence, ) { s.leaseholdersMu.Lock() defer s.leaseholdersMu.Unlock() diff --git a/pkg/kv/kvserver/closedts/sidetransport/sender_test.go b/pkg/kv/kvserver/closedts/sidetransport/sender_test.go index 612809789fdd..05dab36ed35d 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/sender_test.go +++ b/pkg/kv/kvserver/closedts/sidetransport/sender_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -46,7 +47,7 @@ type mockReplica struct { canBump bool cantBumpReason CantCloseReason - lai ctpb.LAI + lai enginepb.LeaseSequence policy roachpb.RangeClosedTimestampPolicy } diff --git a/pkg/kv/kvserver/concurrency/concurrency_control.go b/pkg/kv/kvserver/concurrency/concurrency_control.go index f65627fc8d54..0d64e1e5752c 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_control.go +++ b/pkg/kv/kvserver/concurrency/concurrency_control.go @@ -229,7 +229,7 @@ type ContentionHandler interface { // lock table and observes the lock on key K, so it enters the lock's // wait-queue and waits for it to be resolved. HandleWriterIntentError( - context.Context, *Guard, roachpb.LeaseSequence, *kvpb.WriteIntentError, + context.Context, *Guard, enginepb.LeaseSequence, *kvpb.WriteIntentError, ) (*Guard, *Error) // HandleTransactionPushError consumes a TransactionPushError thrown by a @@ -291,7 +291,7 @@ type RangeStateListener interface { // OnRangeLeaseUpdated informs the concurrency manager that its range's // lease has been updated. The argument indicates whether this manager's // replica is the leaseholder going forward. - OnRangeLeaseUpdated(_ roachpb.LeaseSequence, isLeaseholder bool) + OnRangeLeaseUpdated(_ enginepb.LeaseSequence, isLeaseholder bool) // OnRangeSplit informs the concurrency manager that its range has split off // a new range to its RHS. @@ -644,7 +644,7 @@ type lockTable interface { // true) or whether it was ignored because the lockTable is currently // disabled (false). AddDiscoveredLock( - intent *roachpb.Intent, seq roachpb.LeaseSequence, consultFinalizedTxnCache bool, + intent *roachpb.Intent, seq enginepb.LeaseSequence, consultFinalizedTxnCache bool, guard lockTableGuard) (bool, error) // AcquireLock informs the lockTable that a new lock was acquired or an @@ -1001,7 +1001,7 @@ type requestQueuer interface { // Enable allows requests to be queued. The method is idempotent. // The lease sequence is used to avoid mixing incompatible requests // or other state from different leasing periods. - Enable(roachpb.LeaseSequence) + Enable(enginepb.LeaseSequence) // Clear empties the queue(s) and causes all waiting requests to // return. If disable is true, future requests must not be enqueued. diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index ec90fb971340..3bb7737edf1c 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -436,7 +436,7 @@ func (m *managerImpl) FinishReq(g *Guard) { // HandleWriterIntentError implements the ContentionHandler interface. func (m *managerImpl) HandleWriterIntentError( - ctx context.Context, g *Guard, seq roachpb.LeaseSequence, t *kvpb.WriteIntentError, + ctx context.Context, g *Guard, seq enginepb.LeaseSequence, t *kvpb.WriteIntentError, ) (*Guard, *Error) { if g.ltg == nil { log.Fatalf(ctx, "cannot handle WriteIntentError %v for request without "+ @@ -548,7 +548,7 @@ func (m *managerImpl) OnRangeDescUpdated(desc *roachpb.RangeDescriptor) { } // OnRangeLeaseUpdated implements the RangeStateListener interface. -func (m *managerImpl) OnRangeLeaseUpdated(seq roachpb.LeaseSequence, isLeaseholder bool) { +func (m *managerImpl) OnRangeLeaseUpdated(seq enginepb.LeaseSequence, isLeaseholder bool) { if isLeaseholder { m.lt.Enable(seq) m.twq.Enable(seq) diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index e866ab6646fb..16c26e9eef3a 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -327,7 +327,7 @@ func TestConcurrencyManagerBasic(t *testing.T) { opName := fmt.Sprintf("handle write intent error %s", reqName) mon.runAsync(opName, func(ctx context.Context) { - seq := roachpb.LeaseSequence(leaseSeq) + seq := enginepb.LeaseSequence(leaseSeq) wiErr := &kvpb.WriteIntentError{Intents: intents} guard, err := m.HandleWriterIntentError(ctx, prev, seq, wiErr) if err != nil { @@ -509,7 +509,7 @@ func TestConcurrencyManagerBasic(t *testing.T) { } else { log.Event(ctx, "released") } - m.OnRangeLeaseUpdated(roachpb.LeaseSequence(leaseSeq), isLeaseholder) + m.OnRangeLeaseUpdated(enginepb.LeaseSequence(leaseSeq), isLeaseholder) }) return c.waitAndCollect(t, mon) diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index 30a0058f275b..f0e61871f926 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -187,7 +187,7 @@ type lockTableImpl struct { // may no longer be accurate. enabled bool enabledMu syncutil.RWMutex - enabledSeq roachpb.LeaseSequence + enabledSeq enginepb.LeaseSequence // A sequence number is assigned to each request seen by the lockTable. This // is to preserve fairness despite the design choice of allowing @@ -2571,7 +2571,7 @@ func (t *lockTableImpl) Dequeue(guard lockTableGuard) { // finalizedTxnCache eagerly when adding discovered locks. func (t *lockTableImpl) AddDiscoveredLock( intent *roachpb.Intent, - seq roachpb.LeaseSequence, + seq enginepb.LeaseSequence, consultFinalizedTxnCache bool, guard lockTableGuard, ) (added bool, _ error) { @@ -2895,7 +2895,7 @@ func (t *lockTableImpl) TransactionIsFinalized(txn *roachpb.Transaction) { } // Enable implements the lockTable interface. -func (t *lockTableImpl) Enable(seq roachpb.LeaseSequence) { +func (t *lockTableImpl) Enable(seq enginepb.LeaseSequence) { // Avoid disrupting other requests if the lockTable is already enabled. // NOTE: This may be a premature optimization, but it can't hurt. t.enabledMu.RLock() diff --git a/pkg/kv/kvserver/concurrency/lock_table_test.go b/pkg/kv/kvserver/concurrency/lock_table_test.go index 1fdf427269e8..1b8fc6b49c22 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_test.go @@ -467,7 +467,7 @@ func TestLockTableBasic(t *testing.T) { if d.HasArg("consult-finalized-txn-cache") { d.ScanArgs(t, "consult-finalized-txn-cache", &consultFinalizedTxnCache) } - leaseSeq := roachpb.LeaseSequence(seq) + leaseSeq := enginepb.LeaseSequence(seq) if _, err := lt.AddDiscoveredLock( &intent, leaseSeq, consultFinalizedTxnCache, g); err != nil { return err.Error() @@ -594,7 +594,7 @@ func TestLockTableBasic(t *testing.T) { if d.HasArg("lease-seq") { d.ScanArgs(t, "lease-seq", &seq) } - lt.Enable(roachpb.LeaseSequence(seq)) + lt.Enable(enginepb.LeaseSequence(seq)) return "" case "clear": diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index 234e5cf72d72..690df15794fa 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() enginepb.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() enginepb.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 enginepb.RaftIndex err := r.withRaftGroupLocked(false, func(r *raft.RawNode) (unquiesceAndWakeLeader bool, err error) { - appliedIndex = r.BasicStatus().Applied + appliedIndex = enginepb.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() enginepb.RaftTerm { r.mu.RLock() defer r.mu.RUnlock() return r.mu.lastTermNotDurable diff --git a/pkg/kv/kvserver/kvserverbase/BUILD.bazel b/pkg/kv/kvserver/kvserverbase/BUILD.bazel index 1bb735f64459..c2f6a6c20bdf 100644 --- a/pkg/kv/kvserver/kvserverbase/BUILD.bazel +++ b/pkg/kv/kvserver/kvserverbase/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/storage/enginepb", "//pkg/util/errorutil", "//pkg/util/hlc", "//pkg/util/humanizeutil", diff --git a/pkg/kv/kvserver/kvserverbase/forced_error.go b/pkg/kv/kvserver/kvserverbase/forced_error.go index 4a4adbbccb0d..1f27c49082aa 100644 --- a/pkg/kv/kvserver/kvserverbase/forced_error.go +++ b/pkg/kv/kvserver/kvserverbase/forced_error.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "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/util/log" ) @@ -45,7 +46,7 @@ var NoopOnProbeCommandErr = kvpb.NewErrorf("no-op on ProbeRequest") // ForcedErrResult is the output from CheckForcedErr. type ForcedErrResult struct { - LeaseIndex uint64 + LeaseIndex enginepb.LeaseSequence Rejection ProposalRejectionType ForcedError *kvpb.Error } 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..10ac04e9d198 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/storage/enginepb.RaftIndex"]; // MVCCHistoryMutation describes mutations of MVCC history that may violate // the closed timestamp, timestamp cache, and guarantees that rely on these @@ -267,7 +267,7 @@ message RaftCommand { // - the command applies on replica 1 // - replica 2 serves anomalous reads which don't see the write // - the command applies on replica 2 - int64 proposer_lease_sequence = 6 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.LeaseSequence"]; + int64 proposer_lease_sequence = 6 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.LeaseSequence"]; // deprecated_proposer_lease served the same purpose as proposer_lease_seq. // As of VersionLeaseSequence, it is no longer in use. @@ -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/storage/enginepb.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/storage/enginepb.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..2966e3c88e4a 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/storage/enginepb.RaftTerm"]; + uint64 commit = 5 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.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/storage/enginepb.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/storage/enginepb.RaftIndex"]; // The sending queue's name. SnapshotRequest.QueueName sender_queue_name = 9; diff --git a/pkg/kv/kvserver/kvserverpb/state.proto b/pkg/kv/kvserver/kvserverpb/state.proto index 2de02efa86d3..14db81217c4c 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/storage/enginepb.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/storage/enginepb.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/storage/enginepb.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/storage/enginepb.RaftIndex"]; uint64 num_pending = 3; reserved 4; // previously last verification timestamp for verify queue. uint64 num_dropped = 5; @@ -203,9 +203,9 @@ 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/storage/enginepb.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/storage/enginepb.LeaseSequence"]; } diff --git a/pkg/kv/kvserver/kvstorage/BUILD.bazel b/pkg/kv/kvserver/kvstorage/BUILD.bazel index 31e6d409cad7..2fc2c04b95fb 100644 --- a/pkg/kv/kvserver/kvstorage/BUILD.bazel +++ b/pkg/kv/kvserver/kvstorage/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//pkg/kv/kvserver/stateloader", "//pkg/roachpb", "//pkg/storage", + "//pkg/storage/enginepb", "//pkg/util/hlc", "//pkg/util/iterutil", "//pkg/util/log", diff --git a/pkg/kv/kvserver/kvstorage/replica_state.go b/pkg/kv/kvserver/kvstorage/replica_state.go index ee397a5dcb5e..974f333f8483 100644 --- a/pkg/kv/kvserver/kvstorage/replica_state.go +++ b/pkg/kv/kvserver/kvstorage/replica_state.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "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/hlc" "github.com/cockroachdb/errors" "go.etcd.io/raft/v3/raftpb" @@ -29,7 +30,7 @@ import ( // TODO(pavelkalinnikov): integrate with kvstorage.Replica. type LoadedReplicaState struct { ReplicaID roachpb.ReplicaID - LastIndex uint64 + LastIndex enginepb.RaftIndex ReplState kvserverpb.ReplicaState hardState raftpb.HardState diff --git a/pkg/kv/kvserver/logstore/BUILD.bazel b/pkg/kv/kvserver/logstore/BUILD.bazel index 1a7ec0fba2c1..9f9e24679e64 100644 --- a/pkg/kv/kvserver/logstore/BUILD.bazel +++ b/pkg/kv/kvserver/logstore/BUILD.bazel @@ -59,6 +59,7 @@ go_test( "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/storage", + "//pkg/storage/enginepb", "//pkg/testutils", "//pkg/util/humanizeutil", "//pkg/util/leaktest", diff --git a/pkg/kv/kvserver/logstore/logstore.go b/pkg/kv/kvserver/logstore/logstore.go index a621f71fb284..b62975122b59 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 enginepb.RaftIndex + LastTerm enginepb.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 := enginepb.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 := enginepb.RaftIndex(m.Entries[0].Index) // first new entry written + purgeTerm := enginepb.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, enginepb.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 enginepb.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 := enginepb.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: enginepb.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 enginepb.RaftIndex, +) (enginepb.RaftTerm, error) { entry, found := eCache.Get(rangeID, index) if found { - return entry.Term, nil + return enginepb.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 := enginepb.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 enginepb.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 enginepb.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 enginepb.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 enginepb.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 enginepb.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..b23acee0dfc1 100644 --- a/pkg/kv/kvserver/logstore/sideload.go +++ b/pkg/kv/kvserver/logstore/sideload.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftentry" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" @@ -32,30 +33,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 enginepb.RaftIndex, term enginepb.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 enginepb.RaftIndex, term enginepb.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 enginepb.RaftIndex, term enginepb.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 enginepb.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 enginepb.RaftIndex, to enginepb.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 enginepb.RaftIndex, term enginepb.RaftTerm) (string, error) } // MaybeSideloadEntries optimizes handling for AddSST requests. AddSST are @@ -133,7 +134,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, enginepb.RaftIndex(outputEnt.Index), enginepb.RaftTerm(outputEnt.Term), dataToSideload); err != nil { // TODO could verify checksum here return nil, 0, 0, 0, err } sideloadedEntriesSize += int64(len(dataToSideload)) @@ -173,7 +174,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, enginepb.RaftIndex(ent.Index), enginepb.RaftIndex(ent.Index+1), 1<<20, ) if len(cachedSingleton) > 0 { @@ -204,7 +205,7 @@ func MaybeInlineSideloadedRaftCommand( return &ent, nil } - sideloadedData, err := sideloaded.Get(ctx, ent.Index, ent.Term) + sideloadedData, err := sideloaded.Get(ctx, enginepb.RaftIndex(ent.Index), enginepb.RaftTerm(ent.Term)) if err != nil { return nil, errors.Wrap(err, "loading sideloaded data") } @@ -251,7 +252,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 enginepb.RaftIndex, + term enginepb.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..2722c9b6478e 100644 --- a/pkg/kv/kvserver/logstore/sideload_disk.go +++ b/pkg/kv/kvserver/logstore/sideload_disk.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/storage/fs" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" @@ -87,7 +88,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 enginepb.RaftIndex, term enginepb.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 +112,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 enginepb.RaftIndex, term enginepb.RaftTerm, +) ([]byte, error) { filename := ss.filename(ctx, index, term) b, err := fs.ReadFile(ss.eng, filename) if oserror.IsNotExist(err) { @@ -119,16 +124,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 enginepb.RaftIndex, term enginepb.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 enginepb.RaftIndex, term enginepb.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 enginepb.RaftIndex, term enginepb.RaftTerm, +) (int64, error) { return ss.purgeFile(ctx, ss.filename(ctx, index, term)) } @@ -166,17 +177,17 @@ func (ss *DiskSideloadStorage) Clear(_ context.Context) error { // TruncateTo implements SideloadStorage. func (ss *DiskSideloadStorage) TruncateTo( - ctx context.Context, firstIndex uint64, + ctx context.Context, firstIndex enginepb.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 enginepb.RaftIndex, to enginepb.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 enginepb.RaftIndex, filename string) error { if index >= to { size, err := ss.fileSize(filename) if err != nil { @@ -220,13 +231,13 @@ func (ss *DiskSideloadStorage) possiblyTruncateTo( // BytesIfTruncatedFromTo implements SideloadStorage. func (ss *DiskSideloadStorage) BytesIfTruncatedFromTo( - ctx context.Context, from uint64, to uint64, + ctx context.Context, from enginepb.RaftIndex, to enginepb.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 enginepb.RaftIndex, filename string) error, ) error { matches, err := ss.eng.List(ss.dir) if oserror.IsNotExist(err) { @@ -253,7 +264,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(enginepb.RaftIndex(logIdx), match); err != nil { return errors.Wrapf(err, "matching pattern %q on dir %s", match, ss.dir) } } @@ -264,7 +275,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(_ enginepb.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..bb3b5d0639bd 100644 --- a/pkg/kv/kvserver/logstore/sideload_test.go +++ b/pkg/kv/kvserver/logstore/sideload_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -114,11 +115,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 enginepb.RaftIndex, term enginepb.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 +127,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 +193,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 := []enginepb.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 +204,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([]enginepb.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 []enginepb.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 +239,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 []enginepb.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 +287,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 := []enginepb.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 +324,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 := enginepb.RaftIndex(1); index < 5; index++ { if index == 3 { continue } @@ -332,7 +333,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 []enginepb.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..414cb99e71d2 100644 --- a/pkg/kv/kvserver/logstore/stateloader.go +++ b/pkg/kv/kvserver/logstore/stateloader.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "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/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" @@ -50,13 +51,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, +) (enginepb.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 enginepb.RaftIndex iter.SeekLT(storage.MakeMVCCMetadataKey(keys.RaftLogKeyFromPrefix(prefix, math.MaxUint64))) if ok, _ := iter.Valid(); ok { key := iter.UnsafeKey().Key @@ -154,13 +157,13 @@ func (sl StateLoader) SynthesizeHardState( readWriter storage.ReadWriter, oldHS raftpb.HardState, truncState roachpb.RaftTruncatedState, - raftAppliedIndex uint64, + raftAppliedIndex enginepb.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/BUILD.bazel b/pkg/kv/kvserver/loqrecovery/BUILD.bazel index f8fc26565f7f..24f0b94a44fa 100644 --- a/pkg/kv/kvserver/loqrecovery/BUILD.bazel +++ b/pkg/kv/kvserver/loqrecovery/BUILD.bazel @@ -35,6 +35,7 @@ go_library( "//pkg/server/serverpb", "//pkg/settings/cluster", "//pkg/storage", + "//pkg/storage/enginepb", "//pkg/util/contextutil", "//pkg/util/grpcutil", "//pkg/util/hlc", diff --git a/pkg/kv/kvserver/loqrecovery/collect.go b/pkg/kv/kvserver/loqrecovery/collect.go index 487c5d31e90b..85d091743f89 100644 --- a/pkg/kv/kvserver/loqrecovery/collect.go +++ b/pkg/kv/kvserver/loqrecovery/collect.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "go.etcd.io/raft/v3/raftpb" @@ -194,7 +195,7 @@ func visitStoreReplicas( NodeID: nodeID, Desc: desc, RaftAppliedIndex: rstate.RaftAppliedIndex, - RaftCommittedIndex: hstate.Commit, + RaftCommittedIndex: enginepb.RaftIndex(hstate.Commit), RaftLogDescriptorChanges: rangeUpdates, LocalAssumesLeaseholder: localIsLeaseholder, }) @@ -208,7 +209,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 enginepb.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/BUILD.bazel b/pkg/kv/kvserver/loqrecovery/loqrecoverypb/BUILD.bazel index 1aa1667ac2d6..4f892631e2ff 100644 --- a/pkg/kv/kvserver/loqrecovery/loqrecoverypb/BUILD.bazel +++ b/pkg/kv/kvserver/loqrecovery/loqrecoverypb/BUILD.bazel @@ -40,6 +40,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/roachpb", + "//pkg/storage/enginepb", "//pkg/util/keysutil", "//pkg/util/log/eventpb", "//pkg/util/log/logpb", diff --git a/pkg/kv/kvserver/loqrecovery/loqrecoverypb/recovery.go b/pkg/kv/kvserver/loqrecovery/loqrecoverypb/recovery.go index 6313e4825b05..942e7eb015ff 100644 --- a/pkg/kv/kvserver/loqrecovery/loqrecoverypb/recovery.go +++ b/pkg/kv/kvserver/loqrecovery/loqrecoverypb/recovery.go @@ -12,6 +12,7 @@ package loqrecoverypb import ( "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/keysutil" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/log/logpb" @@ -25,6 +26,9 @@ import ( // representation outside of tests. type RecoveryKey roachpb.RKey +// Force import of enginepb in this package. +var _ = enginepb.RaftIndex(0) + // MarshalYAML implements Marshaler interface. func (r RecoveryKey) MarshalYAML() (interface{}, error) { return roachpb.RKey(r).String(), nil diff --git a/pkg/kv/kvserver/loqrecovery/loqrecoverypb/recovery.proto b/pkg/kv/kvserver/loqrecovery/loqrecoverypb/recovery.proto index 9e78dd63e261..66eff1de75b8 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/storage/enginepb.RaftIndex"] ; + uint64 raft_committed_index = 5 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.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..895321f10e8b 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 enginepb.RaftIndex `yaml:"RangeAppliedIndex"` + RaftCommittedIndex enginepb.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 { + enginepb.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, enginepb.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 enginepb.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..146fdb7203b4 100644 --- a/pkg/kv/kvserver/raft_log_queue.go +++ b/pkg/kv/kvserver/raft_log_queue.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -374,8 +375,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 enginepb.RaftIndex + PendingSnapshotIndex enginepb.RaftIndex } func (input truncateDecisionInput) LogTooLarge() bool { @@ -388,13 +389,13 @@ func (input truncateDecisionInput) LogTooLarge() bool { // cluster data. type truncateDecision struct { Input truncateDecisionInput - CommitIndex uint64 + CommitIndex enginepb.RaftIndex - NewFirstIndex uint64 // first index of the resulting log after truncation + NewFirstIndex enginepb.RaftIndex // first index of the resulting log after truncation ChosenVia string } -func (td *truncateDecision) raftSnapshotsForIndex(index uint64) int { +func (td *truncateDecision) raftSnapshotsForIndex(index enginepb.RaftIndex) int { var n int for _, p := range td.Input.RaftStatus.Progress { if p.State != tracker.StateReplicate { @@ -412,7 +413,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 enginepb.RaftIndex(p.Match) < index && enginepb.RaftIndex(p.Next) <= index { n++ } } @@ -476,7 +477,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 enginepb.RaftIndex, chosenVia string) { if td.NewFirstIndex > index { td.NewFirstIndex = index td.ChosenVia = chosenVia @@ -502,7 +503,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 = enginepb.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 +546,7 @@ func computeTruncateDecision(input truncateDecisionInput) truncateDecision { if progress.State == tracker.StateProbe { decision.ProtectIndex(input.FirstIndex, truncatableIndexChosenViaProbingFollower) } else { - decision.ProtectIndex(progress.Match, truncatableIndexChosenViaFollowers) + decision.ProtectIndex(enginepb.RaftIndex(progress.Match), truncatableIndexChosenViaFollowers) } continue } @@ -553,7 +554,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(enginepb.RaftIndex(progress.Match), truncatableIndexChosenViaFollowers) } // Otherwise, we let it truncate to the committed index. @@ -613,7 +614,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 > enginepb.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..f75af7581293 100644 --- a/pkg/kv/kvserver/raft_log_queue_test.go +++ b/pkg/kv/kvserver/raft_log_queue_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -42,7 +43,7 @@ func TestShouldTruncate(t *testing.T) { defer log.Scope(t).Close(t) testCases := []struct { - truncatableIndexes uint64 + truncatableIndexes enginepb.RaftIndex raftLogSize int64 expected bool }{ @@ -77,12 +78,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 enginepb.RaftIndex progress []uint64 raftLogSize int64 - firstIndex uint64 - lastIndex uint64 - pendingSnapshot uint64 + firstIndex enginepb.RaftIndex + lastIndex enginepb.RaftIndex + pendingSnapshot enginepb.RaftIndex exp string }{ { @@ -183,7 +184,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 +249,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 := []enginepb.RaftIndex{100, 200, 300, 400, 500} + lastIndex := enginepb.RaftIndex(500) status.Commit = 300 for i, v := range progress { @@ -262,12 +263,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 +473,7 @@ func TestNewTruncateDecision(t *testing.T) { t.Fatal(err) } - getIndexes := func() (uint64, int, uint64, error) { + getIndexes := func() (enginepb.RaftIndex, int, enginepb.RaftIndex, error) { d, err := newTruncateDecision(ctx, r) if err != nil { return 0, 0, 0, err @@ -518,7 +519,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 enginepb.RaftIndex var numTruncatable int testutils.SucceedsSoon(t, func() error { var err error @@ -666,7 +667,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 enginepb.RaftIndex, now time.Time) { t.Helper() const anyRecipientStore roachpb.StoreID = 0 if _, maxIndex := r.getSnapshotLogTruncationConstraintsRLocked(anyRecipientStore, false /* initialOnly */); maxIndex != exp { @@ -718,7 +719,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 []enginepb.RaftIndex for i := 0; i < 10; i++ { args := incrementArgs([]byte("a"), int64(i)) @@ -913,7 +914,7 @@ func TestTruncateLogRecompute(t *testing.T) { } func waitForTruncationForTesting( - t *testing.T, r *Replica, newFirstIndex uint64, looselyCoupled bool, + t *testing.T, r *Replica, newFirstIndex enginepb.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..a36518f9fac9 100644 --- a/pkg/kv/kvserver/raft_log_truncator.go +++ b/pkg/kv/kvserver/raft_log_truncator.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "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" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -92,7 +93,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 enginepb.RaftIndex, +) enginepb.RaftIndex { p.mu.Lock() defer p.mu.Unlock() p.iterateLocked(func(_ int, trunc pendingTruncation) { @@ -167,14 +170,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 enginepb.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() enginepb.RaftIndex { // Reminder: RaftTruncatedState.Index is inclusive. return pt.Index + 1 } @@ -249,7 +252,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 enginepb.RaftIndex, ) (expectedFirstIndexWasAccurate bool) // Updates the stats related to the raft log size after the truncation is // enacted. @@ -260,7 +263,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 enginepb.RaftIndex) (freed int64, _ error) getStateLoader() stateloader.StateLoader // NB: Setting the persistent raft state is via the Engine exposed by // storeForTruncator. @@ -273,7 +276,7 @@ func (t *raftLogTruncator) addPendingTruncation( ctx context.Context, r replicaForTruncator, trunc roachpb.RaftTruncatedState, - raftExpectedFirstIndex uint64, + raftExpectedFirstIndex enginepb.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..eb0de5389e26 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 enginepb.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 enginepb.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 enginepb.RaftIndex, lastLogEntry enginepb.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 enginepb.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 = enginepb.RaftIndex(truncIndex) + r.writeRaftStateToEngine(t, eng, enginepb.RaftIndex(truncIndex), enginepb.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: enginepb.RaftIndex(truncIndex)}, enginepb.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, enginepb.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..f52b563e74f4 100644 --- a/pkg/kv/kvserver/raft_truncator_replica.go +++ b/pkg/kv/kvserver/raft_truncator_replica.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" ) // Implementation of the replicaForTruncator interface. @@ -34,7 +35,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 enginepb.RaftIndex, ) (expectedFirstIndexWasAccurate bool) { _, expectedFirstIndexAccurate := (*Replica)(r).handleTruncatedStateResult( ctx, trunc, expectedFirstIndexPreTruncation) @@ -64,7 +67,7 @@ func (r *raftTruncatorReplica) getPendingTruncs() *pendingLogTruncations { } func (r *raftTruncatorReplica) sideloadedBytesIfTruncatedFromTo( - ctx context.Context, from, to uint64, + ctx context.Context, from, to enginepb.RaftIndex, ) (freed int64, err error) { freed, _, err = r.raftMu.sideloaded.BytesIfTruncatedFromTo(ctx, from, to) return freed, err diff --git a/pkg/kv/kvserver/raftentry/BUILD.bazel b/pkg/kv/kvserver/raftentry/BUILD.bazel index 2bb8df2a34a6..69b55f1daf57 100644 --- a/pkg/kv/kvserver/raftentry/BUILD.bazel +++ b/pkg/kv/kvserver/raftentry/BUILD.bazel @@ -12,6 +12,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/roachpb", + "//pkg/storage/enginepb", "//pkg/util", "//pkg/util/metric", "//pkg/util/syncutil", @@ -31,6 +32,7 @@ go_test( embed = [":raftentry"], deps = [ "//pkg/roachpb", + "//pkg/storage/enginepb", "//pkg/util/leaktest", "@com_github_stretchr_testify//require", "@io_etcd_go_raft_v3//raftpb", diff --git a/pkg/kv/kvserver/raftentry/cache.go b/pkg/kv/kvserver/raftentry/cache.go index b3df6ada2af5..2c2ab97026bf 100644 --- a/pkg/kv/kvserver/raftentry/cache.go +++ b/pkg/kv/kvserver/raftentry/cache.go @@ -18,6 +18,7 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" "go.etcd.io/raft/v3/raftpb" @@ -99,11 +100,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 enginepb.RaftIndex) (bytesRemoved, entriesRemoved int32) + clearTo(hi enginepb.RaftIndex) (bytesRemoved, entriesRemoved int32) + get(index enginepb.RaftIndex) (raftpb.Entry, bool) + scan(ents []raftpb.Entry, lo, hi enginepb.RaftIndex, maxBytes uint64) ( + _ []raftpb.Entry, bytes uint64, nextIdx enginepb.RaftIndex, exceededMaxBytes bool) } // ringBuf implements rangeCache. @@ -188,7 +189,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 := enginepb.RaftIndex(ents[0].Index) bytesRemoved, entriesRemoved = p.truncateFrom(truncIdx) } if add { @@ -198,7 +199,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 enginepb.RaftIndex) { c.mu.Lock() p := c.getPartLocked(id, false /* create */, false /* recordUse */) if p == nil { @@ -214,7 +215,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 enginepb.RaftIndex) (e raftpb.Entry, ok bool) { c.metrics.Accesses.Inc(1) c.mu.Lock() p := c.getPartLocked(id, false /* create */, true /* recordUse */) @@ -238,8 +239,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 enginepb.RaftIndex, maxBytes uint64, +) (_ []raftpb.Entry, bytes uint64, nextIdx enginepb.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..34d4b4c8e126 100644 --- a/pkg/kv/kvserver/raftentry/cache_test.go +++ b/pkg/kv/kvserver/raftentry/cache_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/stretchr/testify/require" "go.etcd.io/raft/v3/raftpb" @@ -26,18 +27,18 @@ import ( const noLimit = math.MaxUint64 -func newEntry(index, size uint64) raftpb.Entry { - r := rand.New(rand.NewSource(int64(index * size))) +func newEntry(index enginepb.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 +46,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 enginepb.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 enginepb.RaftIndex) []raftpb.Entry { ents := newEntries(lo, hi, 9) c.Add(rangeID, ents, false) return ents @@ -70,9 +71,9 @@ func verifyGet( t *testing.T, c *Cache, rangeID roachpb.RangeID, - lo, hi uint64, + lo, hi enginepb.RaftIndex, expEnts []raftpb.Entry, - expNextIndex uint64, + expNextIndex enginepb.RaftIndex, allowEviction bool, ) { t.Helper() @@ -87,7 +88,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, enginepb.RaftIndex(e.Index)) if !ok { if allowEviction { break @@ -100,7 +101,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 ...enginepb.RaftIndex) { t.Helper() p := c.getPartLocked(rangeID, false /* create */, false /* recordUse */) if p == nil { @@ -111,7 +112,7 @@ func requireEqual(t *testing.T, c *Cache, rangeID roachpb.RangeID, idxs ...uint6 } b := &p.ringBuf it := first(b) - var act []uint64 + var act []enginepb.RaftIndex ok := it.valid(b) for ok { act = append(act, it.index(b)) @@ -126,7 +127,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 enginepb.RaftIndex, ents []raftpb.Entry, expNextIndex enginepb.RaftIndex) { t.Helper() verifyGet(t, c, rangeID, lo, hi, ents, expNextIndex, false) } @@ -220,21 +221,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 +422,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 := enginepb.RaftIndex(ents[offset].Index) + hi := lo + enginepb.RaftIndex(length) wg.Add(1) go func() { time.Sleep(time.Duration(rand.Intn(int(time.Microsecond)))) @@ -439,12 +440,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(enginepb.RaftIndex(lo), enginepb.RaftIndex(hi), 9) ents = append(ents[:offset], toAdd...) } else { lo := uint64(offset + 2) hi := lo + uint64(length) - toAdd = newEntries(lo, hi, 9) + toAdd = newEntries(enginepb.RaftIndex(lo), enginepb.RaftIndex(hi), 9) ents = toAdd } rangeData[r] = ents @@ -476,7 +477,7 @@ func TestConcurrentEvictions(t *testing.T) { if len(data) == 0 { continue } - c.Clear(r, data[len(data)-1].Index+1) + c.Clear(r, enginepb.RaftIndex(data[len(data)-1].Index+1)) } verifyMetrics(t, c, 0, int64(len(c.parts))*int64(partitionSize)) } @@ -576,7 +577,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, enginepb.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 +652,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, enginepb.RaftIndex(ents[0].Index)) }) } doAddAndGetToRange(1) doAddAndGetToRange(2) @@ -662,7 +663,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(enginepb.RaftIndex(i+1), 9) } b.ResetTimer() for i := 0; i < b.N; i++ { @@ -675,8 +676,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, enginepb.RaftIndex(len(ents)-10), noLimit) + c.Clear(rangeID, enginepb.RaftIndex(len(ents)-10)) } } @@ -684,7 +685,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(enginepb.RaftIndex(i+1), 9) } b.ResetTimer() for i := 0; i < b.N; i++ { @@ -692,6 +693,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, enginepb.RaftIndex(len(ents)-10)) } } diff --git a/pkg/kv/kvserver/raftentry/ring_buffer.go b/pkg/kv/kvserver/raftentry/ring_buffer.go index 96f94d172d6e..089d740a703c 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/storage/enginepb" "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) && enginepb.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, enginepb.RaftIndex(ents[0].Index), enginepb.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, enginepb.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 enginepb.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 enginepb.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 enginepb.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 enginepb.RaftIndex, hi enginepb.RaftIndex, maxBytes uint64, +) (_ []raftpb.Entry, bytes uint64, nextIdx enginepb.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 enginepb.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 enginepb.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) enginepb.RaftIndex { + return enginepb.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..948eb7584a04 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/storage/enginepb" "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 enginepb.RaftIndex + mb uint64 idxs []uint64 // full range is 10,11,12,13,14, each of size 10 scanBytes uint64 - nextIdx uint64 + nextIdx enginepb.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 ...enginepb.RaftIndex) { t.Helper() - var sl []uint64 + var sl []enginepb.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, enginepb.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..74e1b18e9573 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/storage/enginepb" "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() enginepb.RaftIndex { + return enginepb.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..f83ee90f5913 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, enginepb.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..6df3a44e0bc3 100644 --- a/pkg/kv/kvserver/raftlog/iter_test.go +++ b/pkg/kv/kvserver/raftlog/iter_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/stretchr/testify/require" @@ -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: enginepb.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 enginepb.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 enginepb.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 && enginepb.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 := enginepb.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 && enginepb.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 []enginepb.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 enginepb.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(enginepb.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, enginepb.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 enginepb.RaftIndex // firstIndex + var li enginepb.RaftIndex // lastIndex if n := len(tc.ents); n > 0 { - fi = tc.ents[0].Index - li = tc.ents[n-1].Index + fi = enginepb.RaftIndex(tc.ents[0].Index) + li = enginepb.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 enginepb.RaftIndex) (bool, error) Next() (bool, error) Entry() raftpb.Entry } -func consumeIter(it iter, lo uint64) ([]uint64, error) { +func consumeIter(it iter, lo enginepb.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..44871c0ce5fc 100644 --- a/pkg/kv/kvserver/raftlog/iterator.go +++ b/pkg/kv/kvserver/raftlog/iterator.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "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/iterutil" "go.etcd.io/raft/v3/raftpb" ) @@ -60,7 +61,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 enginepb.RaftIndex } // NewIterator initializes an Iterator that reads the raft log for the given @@ -107,7 +108,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 enginepb.RaftIndex) (bool, error) { it.iter.SeekGE(storage.MakeMVCCMetadataKey(it.prefixBuf.RaftLogKey(idx))) return it.load() } @@ -134,7 +135,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 enginepb.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/BUILD.bazel b/pkg/kv/kvserver/raftutil/BUILD.bazel index a236cb6574eb..900ece2bf970 100644 --- a/pkg/kv/kvserver/raftutil/BUILD.bazel +++ b/pkg/kv/kvserver/raftutil/BUILD.bazel @@ -8,6 +8,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/roachpb", + "//pkg/storage/enginepb", "@io_etcd_go_raft_v3//:raft", "@io_etcd_go_raft_v3//tracker", ], diff --git a/pkg/kv/kvserver/raftutil/util.go b/pkg/kv/kvserver/raftutil/util.go index 54d26a984401..a5f81a8d88f0 100644 --- a/pkg/kv/kvserver/raftutil/util.go +++ b/pkg/kv/kvserver/raftutil/util.go @@ -12,6 +12,7 @@ package raftutil import ( "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "go.etcd.io/raft/v3" "go.etcd.io/raft/v3/tracker" ) @@ -128,7 +129,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 enginepb.RaftIndex, replicaID roachpb.ReplicaID, ) ReplicaNeedsSnapshotStatus { if st == nil { // Testing only. @@ -158,7 +159,7 @@ func ReplicaMayNeedSnapshot( default: panic("unknown tracker.StateType") } - if progress.Match+1 < firstIndex { + if enginepb.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 145135dd25dc..aa75ddcc4341 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 enginepb.RaftIndex + lastTermNotDurable enginepb.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 enginepb.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..eb847d9db80a 100644 --- a/pkg/kv/kvserver/replica_app_batch.go +++ b/pkg/kv/kvserver/replica_app_batch.go @@ -510,11 +510,11 @@ func (b *replicaAppBatch) stageTrivialReplicatedEvalResult( ctx context.Context, cmd *replicatedCmd, ) { b.state.RaftAppliedIndex = cmd.Index() - b.state.RaftAppliedIndexTerm = cmd.Term + b.state.RaftAppliedIndexTerm = enginepb.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..74211332a360 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 enginepb.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..e52e8adab42d 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 enginepb.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..2bba35887337 100644 --- a/pkg/kv/kvserver/replica_application_state_machine_test.go +++ b/pkg/kv/kvserver/replica_application_state_machine_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -94,7 +95,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 +199,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 +286,7 @@ func TestReplicaStateMachineRaftLogTruncationLooselyCoupled(t *testing.T) { require.NoError(t, pErr.GoError()) } - raftLogSize, truncatedIndex := func() (_rls int64, truncIdx uint64) { + raftLogSize, truncatedIndex := func() (_rls int64, truncIdx enginepb.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 +318,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 +449,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..3b3c41b5bce6 100644 --- a/pkg/kv/kvserver/replica_closedts.go +++ b/pkg/kv/kvserver/replica_closedts.go @@ -14,10 +14,10 @@ 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" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -54,7 +54,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 +130,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 enginepb.LeaseSequence, ) { // We pass knownApplied = false because we don't know whether this lease // applied index has been applied locally yet. @@ -177,7 +177,7 @@ type sidetransportAccess struct { type sidetransportReceiver interface { GetClosedTimestamp( ctx context.Context, rangeID roachpb.RangeID, leaseholderNode roachpb.NodeID, - ) (hlc.Timestamp, ctpb.LAI) + ) (hlc.Timestamp, enginepb.LeaseSequence) HTML() string } @@ -186,7 +186,7 @@ type sidetransportReceiver interface { // lease applied index has been locally applied. type closedTimestamp struct { ts hlc.Timestamp - lai ctpb.LAI + lai enginepb.LeaseSequence } // regression returns whether the combination of the two closed timestamps @@ -226,7 +226,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 enginepb.LeaseSequence, knownApplied bool, ) closedTimestamp { st.mu.Lock() defer st.mu.Unlock() @@ -280,7 +280,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 enginepb.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 3913551895d3..02c021a56ccd 100644 --- a/pkg/kv/kvserver/replica_closedts_internal_test.go +++ b/pkg/kv/kvserver/replica_closedts_internal_test.go @@ -21,9 +21,9 @@ 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/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -51,7 +51,7 @@ func TestSideTransportClosed(t *testing.T) { curSet bool nextSet bool recSet bool - applied ctpb.LAI + applied enginepb.LeaseSequence sufficient hlc.Timestamp expClosed hlc.Timestamp @@ -443,10 +443,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 enginepb.LeaseSequence for atomic.LoadInt32(&done) == 0 { // Determine which lease applied index to use. - var lai ctpb.LAI + var lai enginepb.LeaseSequence switch rand.Intn(3) { case 0: lai = lastLAI @@ -500,7 +500,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, enginepb.LeaseSequence) { r.Lock() defer r.Unlock() return r.ts, r.lai @@ -522,10 +522,10 @@ func TestReplicaClosedTimestamp(t *testing.T) { for _, test := range []struct { name string - applied ctpb.LAI + applied enginepb.LeaseSequence raftClosed hlc.Timestamp sidetransportClosed hlc.Timestamp - sidetransportLAI ctpb.LAI + sidetransportLAI enginepb.LeaseSequence expClosed hlc.Timestamp }{ { @@ -569,7 +569,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..53c372a565fa 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" @@ -817,7 +818,7 @@ func waitForApplication( dialer *nodedialer.Dialer, rangeID roachpb.RangeID, replicas []roachpb.ReplicaDescriptor, - leaseIndex uint64, + leaseIndex enginepb.LeaseSequence, ) error { g := ctxgroup.WithContext(ctx) for _, repl := range replicas { @@ -2815,7 +2816,7 @@ func (r *Replica) sendSnapshotUsingDelegate( SenderQueueName: senderQueueName, SenderQueuePriority: senderQueuePriority, Type: snapType, - Term: status.Term, + Term: enginepb.RaftTerm(status.Term), DelegatedSender: sender, FirstIndex: appliedIndex, DescriptorGeneration: r.Desc().Generation, @@ -2936,7 +2937,7 @@ func (r *Replica) validateSnapshotDelegationRequest( // haven't woken up yet. return errors.Errorf("raft status not initialized") } - replTerm := status.Term + replTerm := enginepb.RaftTerm(status.Term) r.mu.RUnlock() // Delegate has a different term than the coordinator. This typically means @@ -3088,8 +3089,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: enginepb.RaftIndex(snap.RaftSnap.Metadata.Index), + Term: enginepb.RaftTerm(snap.RaftSnap.Metadata.Term), } // See comment on DeprecatedUsingAppliedStateKey for why we need to set this @@ -3108,7 +3109,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..f5253fd78011 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() enginepb.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 enginepb.RaftIndex) (enginepb.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() enginepb.LeaseSequence { return rec.i.GetLeaseAppliedIndex() } @@ -245,7 +245,7 @@ func (rec *SpanSetReplicaEvalContext) GetClosedTimestampOlderThanStorageSnapshot } // RevokeLease stops the replica from using its current lease. -func (rec *SpanSetReplicaEvalContext) RevokeLease(ctx context.Context, seq roachpb.LeaseSequence) { +func (rec *SpanSetReplicaEvalContext) RevokeLease(ctx context.Context, seq enginepb.LeaseSequence) { rec.i.RevokeLease(ctx, seq) } 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..dc3fa75b0be3 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 enginepb.RaftTerm, + index enginepb.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..4e417a3d87d4 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftutil" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -75,7 +76,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 enginepb.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 +98,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) enginepb.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 +140,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() enginepb.RaftIndex + leaseAppliedIndex() enginepb.LeaseSequence enqueueUpdateCheck() closedTimestampTarget() hlc.Timestamp leaderStatus(ctx context.Context, raftGroup proposerRaft) rangeLeaderInfo @@ -744,7 +745,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) { +) (enginepb.LeaseSequence, hlc.Timestamp, error) { // Assign a LeaseAppliedIndex (see checkForcedErr). These provide replay // protection. @@ -762,7 +763,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 enginepb.LeaseSequence if !p.Request.IsSingleRequestLeaseRequest() { b.assignedLAI++ lai = b.assignedLAI @@ -860,7 +861,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 enginepb.LeaseSequence, closedTimestamp hlc.Timestamp, ) error { buf := &b.scratchFooter buf.MaxLeaseIndex = lai @@ -888,7 +889,7 @@ func (b *propBuf) marshallLAIAndClosedTimestampToProposalLocked( return err } -func (b *propBuf) forwardAssignedLAILocked(v uint64) { +func (b *propBuf) forwardAssignedLAILocked(v enginepb.LeaseSequence) { if b.assignedLAI < v { b.assignedLAI = v } @@ -939,7 +940,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 enginepb.LeaseSequence, ) { if leaseOwned { b.forwardClosedTimestampLocked(appliedClosedTS) @@ -1151,11 +1152,11 @@ func (rp *replicaProposer) destroyed() destroyStatus { return rp.mu.destroyStatus } -func (rp *replicaProposer) firstIndex() uint64 { +func (rp *replicaProposer) firstIndex() enginepb.RaftIndex { return (*Replica)(rp).raftFirstIndexRLocked() } -func (rp *replicaProposer) leaseAppliedIndex() uint64 { +func (rp *replicaProposer) leaseAppliedIndex() enginepb.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..62bc61090cf5 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftutil" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -46,8 +47,8 @@ type testProposer struct { syncutil.RWMutex clock *hlc.Clock ds destroyStatus - fi uint64 - lai uint64 + fi enginepb.RaftIndex + lai enginepb.LeaseSequence enqueued int registered int @@ -148,11 +149,11 @@ func (t *testProposer) destroyed() destroyStatus { return t.ds } -func (t *testProposer) firstIndex() uint64 { +func (t *testProposer) firstIndex() enginepb.RaftIndex { return t.fi } -func (t *testProposer) leaseAppliedIndex() uint64 { +func (t *testProposer) leaseAppliedIndex() enginepb.LeaseSequence { return t.lai } @@ -357,12 +358,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, enginepb.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 enginepb.LeaseSequence for i, p := range proposals { if i != leaseReqIdx { lai++ @@ -649,7 +650,7 @@ func TestProposalBufferRejectUnsafeLeaseTransfer(t *testing.T) { ctx := context.Background() proposer := uint64(1) - proposerFirstIndex := uint64(5) + proposerFirstIndex := enginepb.RaftIndex(5) target := uint64(2) // Each subtest will try to propose a lease transfer in a different Raft @@ -659,7 +660,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 enginepb.RaftIndex expRejection bool expRejectionReason raftutil.ReplicaNeedsSnapshotStatus @@ -747,11 +748,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..6819b0408059 100644 --- a/pkg/kv/kvserver/replica_proposal_quota.go +++ b/pkg/kv/kvserver/replica_proposal_quota.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/quotapool" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -93,7 +94,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 = enginepb.RaftIndex(status.Applied) if r.mu.proposalQuota != nil { log.Fatal(ctx, "proposalQuota was not nil before becoming the leader") } @@ -138,12 +139,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 := enginepb.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 := enginepb.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 +204,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 enginepb.RaftIndex(progress.Match) < r.mu.proposalQuotaBaseIndex { return } if _, paused := r.mu.pausedFollowers[roachpb.ReplicaID(id)]; paused { @@ -214,13 +215,13 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( // See #79215. return } - if progress.Match > 0 && progress.Match < minIndex { - minIndex = progress.Match + if progress.Match > 0 && enginepb.RaftIndex(progress.Match) < minIndex { + minIndex = enginepb.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 && enginepb.RaftIndex(progress.Match) >= commitIndex { r.mu.lastReplicaAdded = 0 r.mu.lastReplicaAddedTime = time.Time{} } @@ -245,8 +246,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 + enginepb.RaftIndex(len(r.mu.quotaReleaseQueue)) + if releasableIndex != enginepb.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..10cd2797e266 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "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" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -66,7 +67,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 = enginepb.RaftIndex(util.ConstantWithMetamorphicTestRange( "raft-log-truncation-clearrange-threshold", 100000 /* default */, 1 /* min */, 1e6 /* max */)) ) @@ -233,7 +234,7 @@ func (r *Replica) evalAndPropose( // The remaining requests that skip a lease check (at the time of writing // ProbeRequest) will assign a zero lease sequence and thus won't be able // to mutate state. - var seq roachpb.LeaseSequence + var seq enginepb.LeaseSequence switch t := ba.Requests[0].GetInner().(type) { case *kvpb.RequestLeaseRequest: seq = t.PrevLease.Sequence @@ -356,7 +357,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 enginepb.LeaseSequence) { if pErr != nil { p.command.MaxLeaseIndex = prev } @@ -1464,8 +1465,8 @@ func (r *Replica) maybeCoalesceHeartbeat( RangeID: r.RangeID, ToReplicaID: toReplica.ReplicaID, FromReplicaID: fromReplica.ReplicaID, - Term: msg.Term, - Commit: msg.Commit, + Term: enginepb.RaftTerm(msg.Term), + Commit: enginepb.RaftIndex(msg.Commit), Quiesce: quiesce, LaggingFollowersOnQuiesce: lagging, LaggingFollowersOnQuiesceAccurate: quiesce, @@ -1783,7 +1784,7 @@ func (r *Replica) reportSnapshotStatus(ctx context.Context, to roachpb.ReplicaID } type snapTruncationInfo struct { - index uint64 + index enginepb.RaftIndex recipientStore roachpb.StoreID initial bool } @@ -1805,7 +1806,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()) { +) (enginepb.RaftIndex, func()) { r.mu.Lock() defer r.mu.Unlock() appliedIndex := r.mu.state.RaftAppliedIndex @@ -1854,7 +1855,7 @@ func (r *Replica) addSnapshotLogTruncationConstraint( // to new replicas are considered. func (r *Replica) getSnapshotLogTruncationConstraintsRLocked( recipientStore roachpb.StoreID, initialOnly bool, -) (_ []snapTruncationInfo, minSnapIndex uint64) { +) (_ []snapTruncationInfo, minSnapIndex enginepb.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..cae09cfaa3c1 100644 --- a/pkg/kv/kvserver/replica_raft_overload.go +++ b/pkg/kv/kvserver/replica_raft_overload.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" @@ -72,7 +73,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 enginepb.RaftIndex } type nonLiveReason byte @@ -129,7 +130,7 @@ func computeExpendableOverloadedFollowers( if pr.IsPaused() { nonLive[roachpb.ReplicaID(id)] = nonLiveReasonPaused } - if pr.Match < d.minLiveMatchIndex { + if enginepb.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..643987a4c86d 100644 --- a/pkg/kv/kvserver/replica_raft_overload_test.go +++ b/pkg/kv/kvserver/replica_raft_overload_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" @@ -53,7 +54,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 := enginepb.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 +68,7 @@ func TestReplicaRaftOverload_computeExpendableOverloadedFollowers(t *testing.T) arg.Scan(t, i, &id) switch arg.Key { case "min-live-match-index": - minLiveMatchIndex = id + minLiveMatchIndex = enginepb.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..f8d069fa89b4 100644 --- a/pkg/kv/kvserver/replica_raft_quiesce.go +++ b/pkg/kv/kvserver/replica_raft_quiesce.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -194,7 +195,7 @@ type quiescer interface { isRaftLeaderRLocked() bool raftSparseStatusRLocked() *raftSparseStatus raftBasicStatusRLocked() raft.BasicStatus - raftLastIndexRLocked() uint64 + raftLastIndexRLocked() enginepb.RaftIndex hasRaftReadyRLocked() bool hasPendingProposalsRLocked() bool hasPendingProposalQuotaRLocked() bool @@ -380,7 +381,7 @@ func shouldReplicaQuiesce( return nil, nil, false } lastIndex := q.raftLastIndexRLocked() - if status.Commit != lastIndex { + if enginepb.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..24c54838f617 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 = enginepb.RaftIndex(v) + d.ScanArgs(t, "term", &v) + prevTruncatedState.Term = enginepb.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: enginepb.RaftIndex(index), + Term: enginepb.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: enginepb.RaftIndex(index), + Term: enginepb.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 := []enginepb.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..ebc290866fa6 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -79,7 +80,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(enginepb.RaftIndex(lo), enginepb.RaftIndex(hi), maxBytes) +} + +func (r *replicaRaftStorage) TypedEntries( + lo, hi enginepb.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 +96,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 enginepb.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 +108,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(enginepb.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 enginepb.RaftIndex) (enginepb.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 +127,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 enginepb.RaftIndex) (enginepb.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 enginepb.RaftIndex) (enginepb.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() enginepb.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() (enginepb.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() enginepb.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() enginepb.RaftIndex { // TruncatedState is guaranteed to be non-nil. return r.mu.state.TruncatedState.Index + 1 } @@ -152,18 +171,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() (enginepb.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() enginepb.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() enginepb.LeaseSequence { r.mu.RLock() defer r.mu.RUnlock() return r.mu.state.LeaseAppliedIndex @@ -189,8 +213,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 +312,7 @@ type IncomingSnapshot struct { DataSize int64 snapType kvserverpb.SnapshotRequest_Type placeholder *ReplicaPlaceholder - raftAppliedIndex uint64 // logging only + raftAppliedIndex enginepb.RaftIndex // logging only } func (s IncomingSnapshot) String() string { @@ -336,8 +360,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 +593,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 +773,8 @@ func writeUnreplicatedSST( if err := sl.SetRaftTruncatedState( ctx, &unreplicatedSST, &roachpb.RaftTruncatedState{ - Index: meta.Index, - Term: meta.Term, + Index: enginepb.RaftIndex(meta.Index), + Term: enginepb.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_range_lease.go b/pkg/kv/kvserver/replica_range_lease.go index c81cc8c23651..40584f6b2b4b 100644 --- a/pkg/kv/kvserver/replica_range_lease.go +++ b/pkg/kv/kvserver/replica_range_lease.go @@ -57,6 +57,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftutil" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/retry" @@ -993,7 +994,7 @@ func (r *Replica) getLeaseRLocked() (roachpb.Lease, roachpb.Lease) { // RevokeLease stops the replica from using its current lease, if that lease // matches the provided lease sequence. All future calls to leaseStatus on this // node with the current lease will now return a PROSCRIBED status. -func (r *Replica) RevokeLease(ctx context.Context, seq roachpb.LeaseSequence) { +func (r *Replica) RevokeLease(ctx context.Context, seq enginepb.LeaseSequence) { r.mu.Lock() defer r.mu.Unlock() if r.mu.state.Lease.Sequence == seq { diff --git a/pkg/kv/kvserver/replica_rangefeed_test.go b/pkg/kv/kvserver/replica_rangefeed_test.go index 77281986930e..37d6f381577e 100644 --- a/pkg/kv/kvserver/replica_rangefeed_test.go +++ b/pkg/kv/kvserver/replica_rangefeed_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -859,7 +860,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 && enginepb.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..2f230749dc6c 100644 --- a/pkg/kv/kvserver/replica_rankings.go +++ b/pkg/kv/kvserver/replica_rankings.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/load" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "go.etcd.io/raft/v3" @@ -43,7 +44,7 @@ type CandidateReplica interface { RaftStatus() *raft.Status // GetFirstIndex returns the index of the first entry in the replica's Raft // log. - GetFirstIndex() uint64 + GetFirstIndex() enginepb.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..eb87e661fe53 100644 --- a/pkg/kv/kvserver/replica_sideload_test.go +++ b/pkg/kv/kvserver/replica_sideload_test.go @@ -24,6 +24,7 @@ import ( "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/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -205,7 +206,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, enginepb.RaftIndex(ent.Index), enginepb.RaftTerm(ent.Term)) require.NoError(t, err) require.Equal(t, origSSTData, sst) break @@ -229,7 +230,7 @@ func TestRaftSSTableSideloadingTruncation(t *testing.T) { const count = 10 - var indexes []uint64 + var indexes []enginepb.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 23ecf547c6f2..349e57a7eefc 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -1510,7 +1510,7 @@ func internalMergeArgs(key []byte, value roachpb.Value) kvpb.MergeRequest { } } -func truncateLogArgs(index uint64, rangeID roachpb.RangeID) kvpb.TruncateLogRequest { +func truncateLogArgs(index enginepb.RaftIndex, rangeID roachpb.RangeID) kvpb.TruncateLogRequest { return kvpb.TruncateLogRequest{ Index: index, RangeID: rangeID, @@ -6550,7 +6550,7 @@ func TestAppliedIndex(t *testing.T) { sc.TestingKnobs.DisableCanAckBeforeApplication = true tc.StartWithStoreConfig(ctx, t, stopper, sc) - var appliedIndex uint64 + var appliedIndex enginepb.RaftIndex var sum int64 for i := int64(1); i <= 10; i++ { args := incrementArgs([]byte("a"), i) @@ -7205,10 +7205,10 @@ func TestEntries(t *testing.T) { repl := tc.repl rangeID := repl.RangeID - var indexes []uint64 + var indexes []enginepb.RaftIndex - populateLogs := func(from, to int) []uint64 { - var newIndexes []uint64 + populateLogs := func(from, to int) []enginepb.RaftIndex { + var newIndexes []enginepb.RaftIndex for i := from; i < to; i++ { args := incrementArgs([]byte("a"), int64(i)) if _, pErr := tc.SendWrapped(args); pErr != nil { @@ -7237,8 +7237,8 @@ func TestEntries(t *testing.T) { indexes = append(indexes, populateLogs(0, 10)...) for i, tc := range []struct { - lo uint64 - hi uint64 + lo enginepb.RaftIndex + hi enginepb.RaftIndex maxBytes uint64 expResultCount int expCacheCount int @@ -7328,7 +7328,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 := enginepb.RaftIndex(ents[len(ents)-1].Index) < tc.hi-1 if hitLimit != expHitLimit { t.Errorf("%d: unexpected hit limit: %t", i, hitLimit) } @@ -7363,7 +7363,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 []enginepb.RaftIndex for i := 0; i < 10; i++ { args := incrementArgs([]byte("a"), int64(i)) @@ -7685,11 +7685,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 enginepb.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 enginepb.LeaseSequence) { if v := p.ctx.Value(magicKey{}); v != nil { if curAttempt := atomic.AddInt32(&c, 1); curAttempt == 1 { return wrongLeaseIndex @@ -8136,7 +8136,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) enginepb.LeaseSequence { if p == proposal && !assigned { assigned = true t.Logf("assigned wrong LAI %d", ai-1) @@ -8221,7 +8221,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) enginepb.LeaseSequence { if v := p.ctx.Value(magicKey{}); v != nil { flushAttempts := atomic.AddInt32(&curFlushAttempt, 1) switch flushAttempts { @@ -8231,7 +8231,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 := enginepb.LeaseSequence(1) return wrongLeaseIndex default: // Unexpected. Asserted against below. @@ -8325,7 +8325,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) enginepb.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. @@ -9900,7 +9900,7 @@ type testQuiescer struct { numProposals int pendingQuota bool status *raftSparseStatus - lastIndex uint64 + lastIndex enginepb.RaftIndex raftReady bool leaseStatus kvserverpb.LeaseStatus storeID roachpb.StoreID @@ -9928,7 +9928,7 @@ func (q *testQuiescer) raftBasicStatusRLocked() raft.BasicStatus { return q.status.BasicStatus } -func (q *testQuiescer) raftLastIndexRLocked() uint64 { +func (q *testQuiescer) raftLastIndexRLocked() enginepb.RaftIndex { return q.lastIndex } diff --git a/pkg/kv/kvserver/stateloader/BUILD.bazel b/pkg/kv/kvserver/stateloader/BUILD.bazel index 19b9627a766d..26a68af56083 100644 --- a/pkg/kv/kvserver/stateloader/BUILD.bazel +++ b/pkg/kv/kvserver/stateloader/BUILD.bazel @@ -34,6 +34,7 @@ go_test( deps = [ "//pkg/roachpb", "//pkg/storage", + "//pkg/storage/enginepb", "//pkg/testutils", "//pkg/util/leaktest", "//pkg/util/stop", diff --git a/pkg/kv/kvserver/stateloader/initial_test.go b/pkg/kv/kvserver/stateloader/initial_test.go index b940e98dcf71..420e6e3abd00 100644 --- a/pkg/kv/kvserver/stateloader/initial_test.go +++ b/pkg/kv/kvserver/stateloader/initial_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -33,19 +34,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 enginepb.RaftTerm + RaftAppliedIndex enginepb.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: enginepb.RaftIndex(tHS.Commit - 1), Err: "can't decrease HardState.Commit"}, + {OldHS: &tHS, RaftAppliedIndex: enginepb.RaftIndex(tHS.Commit), NewHS: tHS}, + {OldHS: &tHS, RaftAppliedIndex: enginepb.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: enginepb.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..22535c9173a9 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -200,7 +200,9 @@ func (rsl StateLoader) LoadMVCCStats( func (rsl StateLoader) SetRangeAppliedState( ctx context.Context, readWriter storage.ReadWriter, - appliedIndex, leaseAppliedIndex, appliedIndexTerm uint64, + appliedIndex enginepb.RaftIndex, + leaseAppliedIndex enginepb.LeaseSequence, + appliedIndexTerm enginepb.RaftTerm, newMS *enginepb.MVCCStats, raftClosedTimestamp hlc.Timestamp, asAlloc *enginepb.RangeAppliedState, // optional diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 231a3d6cdec1..4e19c83eb1c7 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -67,6 +67,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigstore" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/admission" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/contextutil" @@ -280,11 +281,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 enginepb.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, @@ -3610,7 +3615,7 @@ func (s *Store) WaitForSpanConfigSubscription(ctx context.Context) error { // registerLeaseholder registers the provided replica as a leaseholder in the // node's closed timestamp side transport. func (s *Store) registerLeaseholder( - ctx context.Context, r *Replica, leaseSeq roachpb.LeaseSequence, + ctx context.Context, r *Replica, leaseSeq enginepb.LeaseSequence, ) { if s.ctSender != nil { s.ctSender.RegisterLeaseholder(ctx, r, leaseSeq) 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..2269d69c9db8 100644 --- a/pkg/kv/kvserver/stores.go +++ b/pkg/kv/kvserver/stores.go @@ -19,10 +19,10 @@ 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" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/future" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -125,7 +125,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 enginepb.LeaseSequence, ) { if err := ls.VisitStores(func(s *Store) error { r := s.GetReplicaIfExists(rangeID) diff --git a/pkg/kv/kvserver/txnwait/queue.go b/pkg/kv/kvserver/txnwait/queue.go index 0d2267073300..4068f8f4fc90 100644 --- a/pkg/kv/kvserver/txnwait/queue.go +++ b/pkg/kv/kvserver/txnwait/queue.go @@ -243,7 +243,7 @@ func NewQueue(cfg Config) *Queue { // Enable allows transactions to be enqueued and waiting pushers // added. This method must be idempotent as it can be invoked multiple // times as range leases are updated for the same replica. -func (q *Queue) Enable(_ roachpb.LeaseSequence) { +func (q *Queue) Enable(_ enginepb.LeaseSequence) { q.mu.Lock() defer q.mu.Unlock() if q.mu.txns == nil { diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 67538a78fb87..3ebf34d03819 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -1737,12 +1737,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 { diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index 179059e6fd86..44af719eaa83 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -654,7 +654,7 @@ message Lease { // (see Lease.Equivalent). Two adjacent leases that are equivalent will have // the same sequence number and two adjacent leases that are not equivalent // will have different sequence numbers. - int64 sequence = 7 [(gogoproto.casttype) = "LeaseSequence"]; + int64 sequence = 7 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.LeaseSequence"]; // The type of acquisition event that result in this lease (transfer or // request). @@ -769,7 +769,7 @@ enum RangeClosedTimestampPolicy { // will return a RangeInfo with up-to-date information. message ClientRangeInfo { int64 descriptor_generation = 1 [(gogoproto.casttype) = "RangeGeneration"]; - int64 lease_sequence = 2 [(gogoproto.casttype) = "LeaseSequence"]; + int64 lease_sequence = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.LeaseSequence"]; RangeClosedTimestampPolicy closed_timestamp_policy = 3; // ExplicitlyRequested causes range info to be returned even if other fields // are up-to-date. diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 94645e3b7460..8492d454e3ec 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -1061,7 +1061,7 @@ func TestLeaseEqual(t *testing.T) { DeprecatedStartStasis *hlc.Timestamp ProposedTS *hlc.ClockTimestamp Epoch int64 - Sequence LeaseSequence + Sequence enginepb.LeaseSequence AcquisitionType LeaseAcquisitionType } // Verify that the lease structure does not change unexpectedly. If a compile diff --git a/pkg/roachpb/internal_raft.proto b/pkg/roachpb/internal_raft.proto index aa13c04b49e6..ff046b51c69a 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) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.RaftIndex"]; // The term corresponding to 'index'. - optional uint64 term = 2 [(gogoproto.nullable) = false]; + optional uint64 term = 2 [(gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.RaftTerm"]; } // RangeTombstone contains information about a replica that has been deleted. diff --git a/pkg/storage/enginepb/mvcc.go b/pkg/storage/enginepb/mvcc.go index 77378f5c0bba..cc97f0113469 100644 --- a/pkg/storage/enginepb/mvcc.go +++ b/pkg/storage/enginepb/mvcc.go @@ -368,3 +368,25 @@ var FormatBytesAsKey = func(k []byte) redact.RedactableString { var FormatBytesAsValue = func(v []byte) redact.RedactableString { return redact.Sprint(string(v)) } + +// 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/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index f815cb46d862..027bd78dd7b7 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -286,10 +286,12 @@ message RangeAppliedState { // raft_applied_index is the highest (and last) index applied to the Raft // state machine. - uint64 raft_applied_index = 1; + uint64 raft_applied_index = 1 [(gogoproto.casttype) = "RaftIndex"]; // lease_applied_index is the highest (and last) lease index applied to the // Raft state machine. - uint64 lease_applied_index = 2; + // NB: The LeaseSequence is a int64, but this field is a uint64 for + // compatibility reasons. + uint64 lease_applied_index = 2 [(gogoproto.casttype) = "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]; @@ -308,7 +310,7 @@ message RangeAppliedState { // 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; + uint64 raft_applied_index_term = 5 [(gogoproto.casttype) = "RaftTerm"]; } // MVCCWriteValueOp corresponds to a value being written outside of a diff --git a/pkg/testutils/lint/passes/redactcheck/redactcheck.go b/pkg/testutils/lint/passes/redactcheck/redactcheck.go index 9f57165bf266..28022c51ca42 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": { @@ -102,7 +101,6 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { "StreamID": {}, }, "github.com/cockroachdb/cockroach/pkg/roachpb": { - "LeaseSequence": {}, "NodeID": {}, "RangeGeneration": {}, "RangeID": {}, @@ -153,6 +151,9 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { "MVCCStatsDelta": {}, "TxnEpoch": {}, "TxnSeq": {}, + "RaftIndex": {}, + "RaftTerm": {}, + "LeaseSequence": {}, }, "github.com/cockroachdb/cockroach/pkg/util/hlc": { "ClockTimestamp": {},