Skip to content

Commit

Permalink
raft: move StateType from raft into raftpb
Browse files Browse the repository at this point in the history
We'll need this to avoid a circular dependency when we use StateType
in the FortificationTracker.

Epic: none

Release note: None
  • Loading branch information
arulajmani committed Oct 10, 2024
1 parent d11fa04 commit 0286472
Show file tree
Hide file tree
Showing 43 changed files with 515 additions and 508 deletions.
2 changes: 1 addition & 1 deletion pkg/kv/kvclient/kvcoord/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,7 @@ go_test(
"//pkg/kv/kvserver/tscache",
"//pkg/kv/kvserver/txnwait",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer",
"//pkg/raft",
"//pkg/raft/raftpb",
"//pkg/roachpb",
"//pkg/rpc",
"//pkg/rpc/nodedialer",
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvclient/kvcoord/dist_sender_server_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/raft"
"github.com/cockroachdb/cockroach/pkg/raft/raftpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
Expand Down Expand Up @@ -4771,7 +4771,7 @@ func TestPartialPartition(t *testing.T) {
store, err := sl.GetStores().(*kvserver.Stores).GetStore(sl.GetFirstStoreID())
require.NoError(t, err)
status := store.LookupReplica(roachpb.RKey(scratchKey)).RaftStatus()
if status == nil || status.RaftState != raft.StateLeader {
if status == nil || status.RaftState != raftpb.StateLeader {
return errors.Newf("Leader leaseholder split %v", status)
}
return nil
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1917,7 +1917,7 @@ func (r *mockRepl) RaftStatus() *raft.Status {
raftStatus := &raft.Status{
Progress: make(map[raftpb.PeerID]tracker.Progress),
}
raftStatus.RaftState = raft.StateLeader
raftStatus.RaftState = raftpb.StateLeader
for i := int32(1); i <= r.replicationFactor; i++ {
state := tracker.StateReplicate
if _, ok := r.replsInNeedOfSnapshot[roachpb.ReplicaID(i)]; ok {
Expand Down Expand Up @@ -8290,7 +8290,7 @@ func TestFilterBehindReplicas(t *testing.T) {
Progress: make(map[raftpb.PeerID]tracker.Progress),
}
status.Lead = c.leader
status.RaftState = raft.StateLeader
status.RaftState = raftpb.StateLeader
status.Commit = c.commit
var replicas []roachpb.ReplicaDescriptor
for j, v := range c.progress {
Expand Down Expand Up @@ -8363,7 +8363,7 @@ func TestFilterUnremovableReplicas(t *testing.T) {
// Use an invalid replica ID for the leader. TestFilterBehindReplicas covers
// valid replica IDs.
status.Lead = 99
status.RaftState = raft.StateLeader
status.RaftState = raftpb.StateLeader
status.Commit = c.commit
var replicas []roachpb.ReplicaDescriptor
for j, v := range c.progress {
Expand Down Expand Up @@ -8421,7 +8421,7 @@ func TestSimulateFilterUnremovableReplicas(t *testing.T) {
// Use an invalid replica ID for the leader. TestFilterBehindReplicas covers
// valid replica IDs.
status.Lead = 99
status.RaftState = raft.StateLeader
status.RaftState = raftpb.StateLeader
status.Commit = c.commit
var replicas []roachpb.ReplicaDescriptor
for j, v := range c.progress {
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/allocator/plan/replicate.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/rac2"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/raft"
"github.com/cockroachdb/cockroach/pkg/raft/raftpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -534,7 +535,7 @@ func (rp ReplicaPlanner) findRemoveVoter(
lastReplAdded = 0
}
raftStatus := repl.RaftStatus()
if raftStatus == nil || raftStatus.RaftState != raft.StateLeader {
if raftStatus == nil || raftStatus.RaftState != raftpb.StateLeader {
// If requested, assume all replicas are up-to-date.
if rp.knobs.AllowVoterRemovalWhenNotLeader {
candidates = allocatorimpl.FilterUnremovableReplicasWithoutRaftStatus(
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/allocator_impl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -303,7 +303,7 @@ func TestAllocatorRebalanceTarget(t *testing.T) {
Progress: make(map[raftpb.PeerID]tracker.Progress),
}
status.Lead = 1
status.RaftState = raft.StateLeader
status.RaftState = raftpb.StateLeader
status.Commit = 10
for _, replica := range replicas {
status.Progress[raftpb.PeerID(replica.ReplicaID)] = tracker.Progress{
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/asim/state/impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -1203,7 +1203,7 @@ func (s *state) RaftStatus(rangeID RangeID, storeID StoreID) *raft.Status {
// TODO(kvoli): The raft leader will always be the current leaseholder
// here. This should change to enable testing this scenario.
status.Lead = raftpb.PeerID(leader.ReplicaID())
status.RaftState = raft.StateLeader
status.RaftState = raftpb.StateLeader
status.Commit = 2
// TODO(kvoli): A replica is never behind on their raft log, this should
// change to enable testing this scenario where replicas fall behind. e.g.
Expand Down
3 changes: 1 addition & 2 deletions pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb"
"github.com/cockroachdb/cockroach/pkg/raft"
"github.com/cockroachdb/cockroach/pkg/raft/raftpb"
"github.com/cockroachdb/cockroach/pkg/raft/tracker"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down Expand Up @@ -431,7 +430,7 @@ func TestTransferLeaseDuringJointConfigWithDeadIncomingVoter(t *testing.T) {
require.NoError(t, repl0.RaftReportUnreachable(4))
// Check the Raft progress.
s := repl0.RaftStatus()
require.Equal(t, raft.StateLeader, s.RaftState)
require.Equal(t, raftpb.StateLeader, s.RaftState)
p := s.Progress
require.Len(t, p, 4)
require.Contains(t, p, raftpb.PeerID(4))
Expand Down
Loading

0 comments on commit 0286472

Please sign in to comment.