-
Notifications
You must be signed in to change notification settings - Fork 3.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
keys,kvserver: introduce RaftReplicaID #75761
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm looking for suggestions regarding:
- Narrowing the files touched and making it harder to forget to write RaftReplicaID. I could introduce a SetHardStateAndPossiblyReplicaID that takes additional parameters, which may reduce the code maintenance burden.
- Testing this change: any code pointers to existing tests would be appreciated.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker and @tbg)
0171df5
to
09a52de
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 13 of 13 files at r1, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker and @sumeerbhola)
pkg/kv/kvserver/replica.go, line 720 at r1 (raw file):
// TODO(sumeer): The preceding sentence is not consistent with the comment at // Replica.mu.replicaID that says "This value may never be 0". I wonder // whether this comment is stale since the only time we write to this field is
Yes, that has changed since this comment was written. I agree that lifting replicaID
out of under mu is a good idea.
pkg/kv/kvserver/replica_raft.go, line 796 at r1 (raw file):
return stats, expl, errors.Wrap(err, expl) } // It is possible that we have set HardState for the first time, in which
Could this be simpler & avoid having to carry state through the Replica
?
diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go
index 7fad551a2e..18941b0a0e 100644
--- a/pkg/kv/kvserver/replica_raft.go
+++ b/pkg/kv/kvserver/replica_raft.go
@@ -538,12 +538,16 @@ func (r *Replica) handleRaftReadyRaftMuLocked(
raftLogSize := r.mu.raftLogSize
leaderID := r.mu.leaderID
lastLeaderID := leaderID
+ var isFirstHardState bool
err := r.withRaftGroupLocked(true, func(raftGroup *raft.RawNode) (bool, error) {
numFlushed, err := r.mu.proposalBuf.FlushLockedWithRaftGroup(ctx, raftGroup)
if err != nil {
return false, err
}
if hasReady = raftGroup.HasReady(); hasReady {
+ if raftGroup.BasicStatus().HardState == (raftpb.HardState{}) {
+ isFirstHardState = true
+ }
rd = raftGroup.Ready()
}
// We unquiesce if we have a Ready (= there's work to do). We also have
I think this should work. BasicStatus()
isn't that expensive (no allocs iirc), and if it shows we could optimize it (do something that gives us just the HardState
).
pkg/kv/kvserver/replica_raftstorage.go, line 867 at r1 (raw file):
} // The HardState and RaftReplicaID should typically already exist for this
Nit: this comment talks both about HardState and ReplicaID, so move it before both.
pkg/kv/kvserver/store_create_replica.go, line 149 at r1 (raw file):
return nil, false, &roachpb.RaftGroupDeletedError{} } // It is possible that we have already created the HardState for an
Somewhat related & might address this problem, should we be writing a HardState & ReplicaID right here instead of in handleRaftReady? That way we get the invariant "every instantiated Replica has the ReplicaID persisted" (modulo snapshot application, where we will delete & atomically re-write it, so I guess it still holds).
pkg/kv/kvserver/store_split.go, line 107 at r1 (raw file):
// TODO(sumeer): we know HardState.Commit cannot advance since the RHS // cannot apply a snapshot yet. But what prevents a concurrent change to // HardState.{Term,Vote} that we would accidentally undo here.
We hold rightRepl.raftMu
(this is not totally clear from looking at this method, but look how we access rightRepl.raftMu.stateLoader
above, consider adding a comment; I think the lock is acquired in maybeAcquireSplitLock), and you need to hold that lock to mutate the HardState. We loaded the HardState just above, and are writing it again, so everything is still there.
pkg/kv/kvserver/stateloader/initial.go, line 119 at r1 (raw file):
// just write the RaftReplicaID again later. The invariant that HardState // and RaftReplicaID both exist in the store is not being violated. if err := Make(desc.RangeID).SetRaftReplicaID(ctx, readWriter, replicaID); err != nil {
tiny nit: you're calling Make(desc.RangeID)
twice, so just do
sl := Make(desc.RangeID)
// use it twice
pkg/kv/kvserver/stateloader/stateloader.go, line 443 at r1 (raw file):
) error { rid := roachpb.RaftReplicaID{ReplicaID: replicaID} // "Blind" because ms == nil and timestamp.IsEmpty().
Is there a reason to make this as cheap as possible & not to add assertions here? This shouldn't be perf-sensitive code as we're only doing it "once".
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker, @sumeerbhola, and @tbg)
pkg/kv/kvserver/store_split.go, line 107 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
We hold
rightRepl.raftMu
(this is not totally clear from looking at this method, but look how we accessrightRepl.raftMu.stateLoader
above, consider adding a comment; I think the lock is acquired in maybeAcquireSplitLock), and you need to hold that lock to mutate the HardState. We loaded the HardState just above, and are writing it again, so everything is still there.
maybeAcquireSplitLock
calls getOrCreateReplica
with the replicaID that we are trying to create. My reading of tryGetOrCreateReplica
is that if it finds a Replica
that is newer, it will return nil
cockroach/pkg/kv/kvserver/store_create_replica.go
Lines 118 to 124 in c4f15d6
if repl.mu.replicaID > replicaID { | |
// The sender is behind and is sending to an old replica. | |
// We could silently drop this message but this way we'll inform the | |
// sender that they may no longer exist. | |
repl.raftMu.Unlock() | |
return nil, false, &roachpb.RaftGroupDeletedError{} | |
} |
and we will then return here
https://github.com/cockroachdb/cockroach/blob/master/pkg/kv/kvserver/replica_raft.go#L1828-L1832 without acquiring any lock.
Then in the code here we will look up the Replica
using the RangeID and will find this newer Replica which isn't locked. Did I miss something?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'll defer to @tbg on the finer points here, and agree with his comments, but I'm on board with the general idea.
Reviewed 13 of 13 files at r1, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @sumeerbhola and @tbg)
pkg/kv/kvserver/replica_raft.go, line 796 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Could this be simpler & avoid having to carry state through the
Replica
?diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 7fad551a2e..18941b0a0e 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -538,12 +538,16 @@ func (r *Replica) handleRaftReadyRaftMuLocked( raftLogSize := r.mu.raftLogSize leaderID := r.mu.leaderID lastLeaderID := leaderID + var isFirstHardState bool err := r.withRaftGroupLocked(true, func(raftGroup *raft.RawNode) (bool, error) { numFlushed, err := r.mu.proposalBuf.FlushLockedWithRaftGroup(ctx, raftGroup) if err != nil { return false, err } if hasReady = raftGroup.HasReady(); hasReady { + if raftGroup.BasicStatus().HardState == (raftpb.HardState{}) { + isFirstHardState = true + } rd = raftGroup.Ready() } // We unquiesce if we have a Ready (= there's work to do). We also haveI think this should work.
BasicStatus()
isn't that expensive (no allocs iirc), and if it shows we could optimize it (do something that gives us just theHardState
).
+1, avoiding the additional state would be great.
pkg/kv/kvserver/stateloader/stateloader.go, line 442 at r1 (raw file):
ctx context.Context, writer storage.Writer, replicaID roachpb.ReplicaID, ) error { rid := roachpb.RaftReplicaID{ReplicaID: replicaID}
nit: do we need a Protobuf message for this? I don't see us ever writing anything more than a positive int32 here, which we could just as well do with PutUint32
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @sumeerbhola and @tbg)
pkg/kv/kvserver/store_split.go, line 107 at r1 (raw file):
Previously, sumeerbhola wrote…
maybeAcquireSplitLock
callsgetOrCreateReplica
with the replicaID that we are trying to create. My reading oftryGetOrCreateReplica
is that if it finds aReplica
that is newer, it will return nilcockroach/pkg/kv/kvserver/store_create_replica.go
Lines 118 to 124 in c4f15d6
if repl.mu.replicaID > replicaID { // The sender is behind and is sending to an old replica. // We could silently drop this message but this way we'll inform the // sender that they may no longer exist. repl.raftMu.Unlock() return nil, false, &roachpb.RaftGroupDeletedError{} }
and we will then return here
https://github.com/cockroachdb/cockroach/blob/master/pkg/kv/kvserver/replica_raft.go#L1828-L1832 without acquiring any lock.Then in the code here we will look up the
Replica
using the RangeID and will find this newer Replica which isn't locked. Did I miss something?
Ouch, yes, you are right. Throw in a rightRepl.raftMu.AssertHeld()
and hopefully some test will fail under race
. Then
if rightRepl == nil || rightRepl.isNewerThanSplit(split) {
if rightRepl != nil {
rightRepl.raftMu.Lock()
defer rightRepl.raftMu.Unlock() // is that even enough? Do we do more stuff with rightRepl in splitPostApply?
}
}
hopefully fixes that failure?
bd66e1f
to
a8b5c7d
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker and @tbg)
pkg/kv/kvserver/replica.go, line 720 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Yes, that has changed since this comment was written. I agree that lifting
replicaID
out of under mu is a good idea.
Added a TODO where replicaID is declared and changed the comment here.
I'll do that change in a followup PR.
pkg/kv/kvserver/replica_raft.go, line 796 at r1 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
+1, avoiding the additional state would be great.
Is the first HardState guaranteed to be empty? Or can Term be > 0 if this node has received something from the leader?
pkg/kv/kvserver/replica_raftstorage.go, line 867 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Nit: this comment talks both about HardState and ReplicaID, so move it before both.
Done
pkg/kv/kvserver/store_create_replica.go, line 149 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Somewhat related & might address this problem, should we be writing a HardState & ReplicaID right here instead of in handleRaftReady? That way we get the invariant "every instantiated Replica has the ReplicaID persisted" (modulo snapshot application, where we will delete & atomically re-write it, so I guess it still holds).
Hmm, I think this is basically the same kind of problem that introducing RaftReplicaID set out to solve. We can't write the HardState and RaftReplicaID here since we are can't be sure if the existing HardState is for the same ReplicaID or not (if it is for the same we should not overwrite the HardState). And we won't know this until we are writing RaftReplicaID everywhere.
I've added a comment.
This comment also implies we need a below raft migration that will ensure that every replica has populated the RaftReplicaID, the second step of which will purge the unmigrated replicas. My assumption here is that without this migration, we could have quiesced ranges that will continue to not have RaftReplicaID which will complicate future reasoning. Is my reasoning correct?
pkg/kv/kvserver/store_split.go, line 107 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Ouch, yes, you are right. Throw in a
rightRepl.raftMu.AssertHeld()
and hopefully some test will fail underrace
. Thenif rightRepl == nil || rightRepl.isNewerThanSplit(split) { if rightRepl != nil { rightRepl.raftMu.Lock() defer rightRepl.raftMu.Unlock() // is that even enough? Do we do more stuff with rightRepl in splitPostApply? } }
hopefully fixes that failure?
I'll record this in a separate issue.
pkg/kv/kvserver/stateloader/initial.go, line 119 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
tiny nit: you're calling
Make(desc.RangeID)
twice, so just dosl := Make(desc.RangeID) // use it twice
Done
pkg/kv/kvserver/stateloader/stateloader.go, line 442 at r1 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
nit: do we need a Protobuf message for this? I don't see us ever writing anything more than a positive int32 here, which we could just as well do with
PutUint32
.
I find it preferable to use a proto since we can evolve it later if needed. And this is not a performance sensitive path.
We have precedence for using a proto in the RangeTombstone
proto that has a single NextReplicaID
field.
pkg/kv/kvserver/stateloader/stateloader.go, line 443 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Is there a reason to make this as cheap as possible & not to add assertions here? This shouldn't be perf-sensitive code as we're only doing it "once".
Were the assertions you had in mind related to what value is already there?
If so, there are 2 difficulties:
- we have only a
Writer
here. This path is also taken when theWriter
is an sstable writer that we can't read from. - we are currently conservative and are willing to write the same ReplicaID more than once e.g. WriteInitialRangeState and
handleRaftReadyRaftMuLocked
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 6 of 6 files at r2, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker and @sumeerbhola)
pkg/kv/kvserver/replica_raft.go, line 796 at r1 (raw file):
Previously, sumeerbhola wrote…
Is the first HardState guaranteed to be empty? Or can Term be > 0 if this node has received something from the leader?
Note that what's empty is the raft group's HardState before handling the Ready. That HardState must reflect the persisted HardState (otherwise, it's not "hard" :-)); come to think of it, perhaps we have more the opposite problem, where even if there is a HardState it wouldn't be returned from BasicStatus() yet; don't know let's check.
If the node has received something from the leader, that will be reflected in rd.HardState
.
Note that while I feel confident in this, it's worth checking.
It still seems "right" to me to write the replicaID in tryGetOrCreateReplica
, as opposed to here. That initializes the on-disk state precisely when we instantiate the in-memory object. Doesn't get better than that and even if we don't "have" to write the replicaID unless there's also a nontrivial HardState, it sure seems like the right thing to do.
pkg/kv/kvserver/store_create_replica.go, line 149 at r1 (raw file):
Previously, sumeerbhola wrote…
Hmm, I think this is basically the same kind of problem that introducing RaftReplicaID set out to solve. We can't write the HardState and RaftReplicaID here since we are can't be sure if the existing HardState is for the same ReplicaID or not (if it is for the same we should not overwrite the HardState). And we won't know this until we are writing RaftReplicaID everywhere.
I've added a comment.This comment also implies we need a below raft migration that will ensure that every replica has populated the RaftReplicaID, the second step of which will purge the unmigrated replicas. My assumption here is that without this migration, we could have quiesced ranges that will continue to not have RaftReplicaID which will complicate future reasoning. Is my reasoning correct?
You're right, this is related to the original problem. But let's forget about the migration for a moment, wouldn't this be an attractive place to write the ReplicaID? If the incoming messages addresses a higher replicaID (i.e. the current Replica is too old and needs to be replaced), we hit the toTooOld
branch further up in this method on a previous attempt. So all we have to do here is write the replicaID (which is the entirety of the state for an uninitialized replica since the "empty" defaults work fine for HardState, TruncatedState, etc) to storage, done.
Regarding the migration, we need something slightly nonstandard here, I think. The tools we have now can only hit all replicas that are part of the range. We don't have a way to purge uninitialized replicaGC'able replicas. We need a version of PurgeReplicas that explicitly (also) targets uninitialized replicas and deletes those that don't have the ReplicaID persisted.
I do wonder if we can use an alternative approach here. Let's say v22.1, once fully migrated, starts writing the ReplicaID. In 22.2, we could add code at boot time that a) backfills the ReplicaID for all initialized ranges that are missing it b) deletes (without leaving a tombstone, to play it safe) any uninitialized replicas that don't have it. So "all code" except that startup logic has the invariant that all replicas have a ReplicaID persisted.
The remaining risk seems to be that we're replicaGC'ing an uninitialized replica and thus erasing a vote, and that a new uninit'ed replica is then created after the node restarts, which casts a new vote & causes split brain. I accept this as being in the realms of what's theoretically possible, but it's extremely unlikely, in a very strong sense of the word. For one, we're always going through learners and don't promote until caught up, so uninitialized replicas generally never get to vote. Second, even if their vote somehow mattered (perhaps we sent a learner snap which was not durably persisted - which I think we also know is impossible, but let's assume it - and then promoted the node & it immediately power-cycled, losing the snapshot) the fire-and-forget way in which raft votes are requested (in the same raft cycle) makes it extremely unlikely that the restarted node would then receive it.
Taking it a step further, can the 22.1 binary always, regardless of cluster version, write the ReplicaID (but not assume it is always present)? If someone goes into 22.1 but rolls back to 21.2, then they'll have some extra keys in the engine that are unused by CRDB, but I don't think this will matter much - the key is unreplicated, so nothing should look at it. I don't know if there's a particularly strong reason to do that, but it would further reduce the number of uninitialized replicas the 22.2 binary would ever purge, in effect multiplying the already-vanishingly-low probability of causing a split-brain by near-zero once more. I think the chances of something subtly going wrong with the always-write strategy outmatch the likelihood of preventing a split brain, so I'd be inclined to pass on this option.
pkg/kv/kvserver/store_create_replica.go, line 157 at r2 (raw file):
// has a newer ReplicaID than the one in the store, and a stored HardState // that is for the older ReplicaID. This seems harmless, but to be more // precise about the invariants we should remove the stale
rewrap
pkg/kv/kvserver/stateloader/stateloader.go, line 442 at r1 (raw file):
Previously, sumeerbhola wrote…
I find it preferable to use a proto since we can evolve it later if needed. And this is not a performance sensitive path.
We have precedence for using a proto in theRangeTombstone
proto that has a singleNextReplicaID
field.
FWIW, I'm fine with the proto here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker, @sumeerbhola, and @tbg)
pkg/kv/kvserver/store_create_replica.go, line 149 at r1 (raw file):
Thanks for the detailed thoughts!
Taking it a step further, can the 22.1 binary always, regardless of cluster version, write the ReplicaID (but not assume it is always present)? ... then they'll have some extra keys in the engine that are unused by CRDB, but I don't think this will matter much
I was planning to do this for the same reasons. Which is why this PR does not currently include a migration. I started thinking of a migration only last night.
But let's forget about the migration for a moment, wouldn't this be an attractive place to write the ReplicaID? If the incoming messages addresses a higher replicaID (i.e. the current Replica is too old and needs to be replaced), we hit the toTooOld branch further up in this method on a previous attempt. So all we have to do here is write the replicaID (which is the entirety of the state for an uninitialized replica since the "empty" defaults work fine for HardState, TruncatedState, etc) to storage, done.
I fully agree that this is very attractive. The thing I was worrying about is that we would have the latest ReplicaID written, but possibly a HardState from an old ReplicaID (guaranteed to have Commit=0, since that old replica never transitioned to initialized). Which made me think that etcd/raft must already be seeing that old HardState by using the raft.Storage
interface. And there isn't anything particularly wrong with that. Instead of a clean slate regarding term and vote, an existing HardState from an old ReplicaID forces this new replica into a narrower position regarding the term and vote. It is the same range, so the existing raft term must be >= the stored term. Even if it weren't the same range, and the stored term is higher than the raft group, all we are doing is forcing a fast forwarding of terms and forcing an election, and forcing a certain vote by this replica. It isn't violating anything this replica has said before.
In 22.2, we could add code at boot time that a) backfills the ReplicaID for all initialized ranges that are missing it b) deletes (without leaving a tombstone, to play it safe) any uninitialized replicas that don't have it.
So the remaining question is how we eliminate having the cases where there is a HardState but no RaftReplicaID. As you outlined, in 22.2 we will have ReplicasStorage.Init which can fix things up for initialized replicas on node startup. Since there is no coordination needed across the cluster, a "migration" seems unnecessary. I like the fact that you have outlined multiple defensible reasons on why deleting the uninitialized ones with no ReplicaID is reasonable.
So in conclusion:
- write RaftReplicaID unconditionally in 22.1, and write it here and not in handleRaftReadyRaftMuLocked. Justify why this is reasonable.
- document what we are going to do in 22.2 (for now in a code comment here) and why -- some version of what you wrote above.
Sound good?
bcf42cc
to
c131d96
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
TFTRs!
Also added tests. PTAL
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker, @sumeerbhola, and @tbg)
pkg/kv/kvserver/store_create_replica.go, line 149 at r1 (raw file):
Previously, sumeerbhola wrote…
Thanks for the detailed thoughts!
Taking it a step further, can the 22.1 binary always, regardless of cluster version, write the ReplicaID (but not assume it is always present)? ... then they'll have some extra keys in the engine that are unused by CRDB, but I don't think this will matter much
I was planning to do this for the same reasons. Which is why this PR does not currently include a migration. I started thinking of a migration only last night.
But let's forget about the migration for a moment, wouldn't this be an attractive place to write the ReplicaID? If the incoming messages addresses a higher replicaID (i.e. the current Replica is too old and needs to be replaced), we hit the toTooOld branch further up in this method on a previous attempt. So all we have to do here is write the replicaID (which is the entirety of the state for an uninitialized replica since the "empty" defaults work fine for HardState, TruncatedState, etc) to storage, done.
I fully agree that this is very attractive. The thing I was worrying about is that we would have the latest ReplicaID written, but possibly a HardState from an old ReplicaID (guaranteed to have Commit=0, since that old replica never transitioned to initialized). Which made me think that etcd/raft must already be seeing that old HardState by using the
raft.Storage
interface. And there isn't anything particularly wrong with that. Instead of a clean slate regarding term and vote, an existing HardState from an old ReplicaID forces this new replica into a narrower position regarding the term and vote. It is the same range, so the existing raft term must be >= the stored term. Even if it weren't the same range, and the stored term is higher than the raft group, all we are doing is forcing a fast forwarding of terms and forcing an election, and forcing a certain vote by this replica. It isn't violating anything this replica has said before.In 22.2, we could add code at boot time that a) backfills the ReplicaID for all initialized ranges that are missing it b) deletes (without leaving a tombstone, to play it safe) any uninitialized replicas that don't have it.
So the remaining question is how we eliminate having the cases where there is a HardState but no RaftReplicaID. As you outlined, in 22.2 we will have ReplicasStorage.Init which can fix things up for initialized replicas on node startup. Since there is no coordination needed across the cluster, a "migration" seems unnecessary. I like the fact that you have outlined multiple defensible reasons on why deleting the uninitialized ones with no ReplicaID is reasonable.
So in conclusion:
- write RaftReplicaID unconditionally in 22.1, and write it here and not in handleRaftReadyRaftMuLocked. Justify why this is reasonable.
- document what we are going to do in 22.2 (for now in a code comment here) and why -- some version of what you wrote above.
Sound good?
I've gone ahead and done this.
pkg/kv/kvserver/store_create_replica.go, line 157 at r2 (raw file):
Previously, tbg (Tobias Grieger) wrote…
rewrap
Done
pkg/kv/kvserver/replica_raft.go, line 796 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Note that what's empty is the raft group's HardState before handling the Ready. That HardState must reflect the persisted HardState (otherwise, it's not "hard" :-)); come to think of it, perhaps we have more the opposite problem, where even if there is a HardState it wouldn't be returned from BasicStatus() yet; don't know let's check.
If the node has received something from the leader, that will be reflected inrd.HardState
.
Note that while I feel confident in this, it's worth checking.It still seems "right" to me to write the replicaID in
tryGetOrCreateReplica
, as opposed to here. That initializes the on-disk state precisely when we instantiate the in-memory object. Doesn't get better than that and even if we don't "have" to write the replicaID unless there's also a nontrivial HardState, it sure seems like the right thing to do.
This change is reverted. tryGetOrCreateReplica
does the write of RaftReplicaID.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 10 of 10 files at r3, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker and @sumeerbhola)
pkg/kv/kvserver/store_create_replica.go, line 149 at r1 (raw file):
I was planning to do this for the same reasons. Which is why this PR does not currently include a migration. I started thinking of a migration only last night.
Sounds good, let's double check that 21.2 can handle this extra key. It's never read (I don't think we ever scan the /Local/<rangeID>/<unreplicated>
keyspace except in debug keys
which can handle it) and I think it's even cleared on replicaGC, see:
cockroach/pkg/kv/kvserver/rditer/replica_data_iter.go
Lines 139 to 146 in eec9dc3
} else { | |
prefixFn = keys.MakeRangeIDPrefix | |
} | |
sysRangeIDKey := prefixFn(rangeID) | |
return KeyRange{ | |
Start: sysRangeIDKey, | |
End: sysRangeIDKey.PrefixEnd(), | |
} |
Our mixed-version roachtests should also be exercising this case, as any replica movement on 21.2 populates the RaftReplicaID, and we have tests that roll back to 21.1 after.
pkg/kv/kvserver/store_create_replica.go, line 250 at r3 (raw file):
// RaftReplicaID. This removal will happen in ReplicasStorage.Init and // allow us to tighten invariants. Additionally, knowing the // ReplicaID for an unitialized range will allow a node to contact
Not really. We don't have a way to translate a ReplicaID to a RangeDescriptor (outside of doing a full scan of the meta ranges). But at least now we have some option to GC these replicas, where previously it was near-impossible.
pkg/kv/kvserver/store_split.go, line 108 at r3 (raw file):
// cannot apply a snapshot yet. But there could be a concurrent change // to HardState.{Term,Vote} that we would accidentally undo here, // because we are not actually holding the appropriate mutex
nit's nit: . and
// See:
// https:// ...
pkg/kv/kvserver/store_split.go, line 128 at r3 (raw file):
log.Fatalf(ctx, "%v", err) } // Write the RaftReplicaID for the RHS to maintain the invariant that any
nit: it's not really an invariant until you've booted 22.2 once.
pkg/kv/kvserver/stateloader/stateloader.go, line 455 at r3 (raw file):
} func (rsl StateLoader) LoadRaftReplicaID(
needs comment.
The RaftReplicaIDKey is an unreplicated range-id local key that contains the ReplicaID of the replica whose HardState is represented in the RaftHardStateKey. These two keys are removed atomically when we clear the range-id local keys for a replica. See store_create_replica.go for a detailed comment on correctness and version compatibility. We currently do not utilize this information on node restart to figure out whether we should cleanup stale uninitialized replicas. Doing such cleanup can wait until we implement and start using ReplicasStorage. The change here is meant to set us up to rely on RaftReplicaID from the next release onwards. Informs cockroachdb#75740 Release note: None
c131d96
to
0d6e8b6
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker, @sumeerbhola, and @tbg)
pkg/kv/kvserver/store_create_replica.go, line 149 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
I was planning to do this for the same reasons. Which is why this PR does not currently include a migration. I started thinking of a migration only last night.
Sounds good, let's double check that 21.2 can handle this extra key. It's never read (I don't think we ever scan the
/Local/<rangeID>/<unreplicated>
keyspace except indebug keys
which can handle it) and I think it's even cleared on replicaGC, see:cockroach/pkg/kv/kvserver/rditer/replica_data_iter.go
Lines 139 to 146 in eec9dc3
} else { prefixFn = keys.MakeRangeIDPrefix } sysRangeIDKey := prefixFn(rangeID) return KeyRange{ Start: sysRangeIDKey, End: sysRangeIDKey.PrefixEnd(), } Our mixed-version roachtests should also be exercising this case, as any replica movement on 21.2 populates the RaftReplicaID, and we have tests that roll back to 21.1 after.
Ack
pkg/kv/kvserver/store_create_replica.go, line 250 at r3 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Not really. We don't have a way to translate a ReplicaID to a RangeDescriptor (outside of doing a full scan of the meta ranges). But at least now we have some option to GC these replicas, where previously it was near-impossible.
I added "could" and "somehow" and hand-waved about broadcasting to all nodes in the cluster.
pkg/kv/kvserver/store_split.go, line 108 at r3 (raw file):
Previously, tbg (Tobias Grieger) wrote…
nit's nit: . and
// See:
// https:// ...
Done
pkg/kv/kvserver/store_split.go, line 128 at r3 (raw file):
Previously, tbg (Tobias Grieger) wrote…
nit: it's not really an invariant until you've booted 22.2 once.
Added more text to the comment
pkg/kv/kvserver/stateloader/stateloader.go, line 455 at r3 (raw file):
Previously, tbg (Tobias Grieger) wrote…
needs comment.
Done
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 6 of 6 files at r4, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker)
bors r+ |
Build failed (retrying...): |
Build failed (retrying...): |
bors r+ This got batched with a PR that didn't pass CI. |
Already running a review |
bors r- |
Canceled. |
bors p=999 |
Already running a review |
Build succeeded: |
80470: kvserver/loqrecovery: persist new replica ID in `RaftReplicaID` r=aliher1911 a=erikgrinaker I'm not certain that this is the cause of #75133 (wasn't able to reproduce), but it seems plausible. I think it's something that we'd need to fix anyway, but I'm not familiar with all of the nuance here, so I'd appreciate careful reviews. For reference, this was introduced in #75761. --- **cli: persist new replica ID in `unsafe-remove-dead-replicas`** The recently introduced local `RaftReplicaIDKey` was not updated when `unsafe-remove-dead-replicas` changed the replica's ID. This could lead to assertion failures. Touches #75133. Touches #79074. Release note: None **kvserver/loqrecovery: persist new replica ID in `RaftReplicaID`** The recently introduced local `RaftReplicaIDKey` was not updated when loss of quorum recovery changed the replica's ID. This could lead to assertion failures. Release note: None Co-authored-by: Erik Grinaker <[email protected]>
As of v22.2[^1], we always write the RaftReplicaID when creating a Replica or updating a snapshot. However, since this is persisted state that could've originated in older versions and not updated yet, we couldn't rely on a persisted ReplicaID yet. This commit adds code to the `(*Store).Start` boot sequence that - persists a RaftReplicaID for all initialized replicas (using the ReplicaID from the descriptor) - deletes all uninitialized replicas (since we don't know their ReplicaID at this point). The second item in theory violates Raft invariants, as uninitialized Replicas are allowed to vote (though they then cannot accept log entries). So in theory: - an uninitialized replica casts a decisive vote for a leader - it restarts - code in this commit removes the uninited replica (and its vote) - delayed MsgVote from another leader arrives - it casts another vote for the same term for a dueling leader - now there are two leaders in the same term. The above in addition presupposes that the two leaders cannot communicate with each other. Also, even if that is the case, since the two leaders cannot append to the uninitialized replica (it doesn't accept entries), we also need additional voters to return at the exact right time. Since an uninitialized replica without RaftReplicaID in is necessarily at least one release old, this is exceedingly unlikely and we will live with this theoretical risk. This commit also introduces a few assertions that make sure that we don't have overlapping initialized replicas (which would be detected at Store.Start time otherwise while inserting in the btree, but it's nice to catch this earlier) or duplicate RangeIDs. [^1]: cockroachdb#75761 Epic: CRDB-220 Release note: None
As of v22.1[^1], we always write the RaftReplicaID when creating a Replica or updating a snapshot. However, since this is persisted state that could've originated in older versions and not updated yet, we couldn't rely on a persisted ReplicaID yet. This commit adds code to the `(*Store).Start` boot sequence that - persists a RaftReplicaID for all initialized replicas (using the ReplicaID from the descriptor) - deletes all uninitialized replicas (since we don't know their ReplicaID at this point). The second item in theory violates Raft invariants, as uninitialized Replicas are allowed to vote (though they then cannot accept log entries). So in theory: - an uninitialized replica casts a decisive vote for a leader - it restarts - code in this commit removes the uninited replica (and its vote) - delayed MsgVote from another leader arrives - it casts another vote for the same term for a dueling leader - now there are two leaders in the same term. The above in addition presupposes that the two leaders cannot communicate with each other. Also, even if that is the case, since the two leaders cannot append to the uninitialized replica (it doesn't accept entries), we also need additional voters to return at the exact right time. Since an uninitialized replica without RaftReplicaID in is necessarily at least one release old, this is exceedingly unlikely and we will live with this theoretical risk. This commit also introduces a few assertions that make sure that we don't have overlapping initialized replicas (which would be detected at Store.Start time otherwise while inserting in the btree, but it's nice to catch this earlier) or duplicate RangeIDs. [^1]: cockroachdb#75761 Epic: CRDB-220 Release note: None
As of v22.1[^1], we always write the RaftReplicaID when creating a Replica or updating a snapshot. However, since this is persisted state that could've originated in older versions and not updated yet, we couldn't rely on a persisted ReplicaID yet. This commit adds code to the `(*Store).Start` boot sequence that - persists a RaftReplicaID for all initialized replicas (using the ReplicaID from the descriptor) - deletes all uninitialized replicas (since we don't know their ReplicaID at this point). The second item in theory violates Raft invariants, as uninitialized Replicas are allowed to vote (though they then cannot accept log entries). So in theory: - an uninitialized replica casts a decisive vote for a leader - it restarts - code in this commit removes the uninited replica (and its vote) - delayed MsgVote from another leader arrives - it casts another vote for the same term for a dueling leader - now there are two leaders in the same term. The above in addition presupposes that the two leaders cannot communicate with each other. Also, even if that is the case, since the two leaders cannot append to the uninitialized replica (it doesn't accept entries), we also need additional voters to return at the exact right time. Since an uninitialized replica without RaftReplicaID in is necessarily at least one release old, this is exceedingly unlikely and we will live with this theoretical risk. This commit also introduces a few assertions that make sure that we don't have overlapping initialized replicas (which would be detected at Store.Start time otherwise while inserting in the btree, but it's nice to catch this earlier) or duplicate RangeIDs. [^1]: cockroachdb#75761 Epic: CRDB-220 Release note: None
As of v22.1[^1], we always write the RaftReplicaID when creating a Replica or updating a snapshot. However, since this is persisted state that could've originated in older versions and not updated yet, we couldn't rely on a persisted ReplicaID yet. This commit adds code to the `(*Store).Start` boot sequence that - persists a RaftReplicaID for all initialized replicas (using the ReplicaID from the descriptor) - deletes all uninitialized replicas (since we don't know their ReplicaID at this point). The second item in theory violates Raft invariants, as uninitialized Replicas are allowed to vote (though they then cannot accept log entries). So in theory: - an uninitialized replica casts a decisive vote for a leader - it restarts - code in this commit removes the uninited replica (and its vote) - delayed MsgVote from another leader arrives - it casts another vote for the same term for a dueling leader - now there are two leaders in the same term. The above in addition presupposes that the two leaders cannot communicate with each other. Also, even if that is the case, since the two leaders cannot append to the uninitialized replica (it doesn't accept entries), we also need additional voters to return at the exact right time. Since an uninitialized replica without RaftReplicaID in is necessarily at least one release old, this is exceedingly unlikely and we will live with this theoretical risk. This commit also introduces a few assertions that make sure that we don't have overlapping initialized replicas (which would be detected at Store.Start time otherwise while inserting in the btree, but it's nice to catch this earlier) or duplicate RangeIDs. [^1]: cockroachdb#75761 Epic: CRDB-220 Release note: None
As of v22.1[^1], we always write the RaftReplicaID when creating a Replica or updating a snapshot. However, since this is persisted state that could've originated in older versions and not updated yet, we couldn't rely on a persisted ReplicaID yet. This commit adds code to the `(*Store).Start` boot sequence that - persists a RaftReplicaID for all initialized replicas (using the ReplicaID from the descriptor) - deletes all uninitialized replicas that don't have a RaftReplicaID (since we don't know their ReplicaID at this point). The second item in theory violates Raft invariants, as uninitialized Replicas are allowed to vote (though they then cannot accept log entries). So in theory: - an uninitialized replica casts a decisive vote for a leader - it restarts - code in this commit removes the uninited replica (and its vote) - delayed MsgVote from another leader arrives - it casts another vote for the same term for a dueling leader - now there are two leaders in the same term. The above in addition presupposes that the two leaders cannot communicate with each other. Also, even if that is the case, since the two leaders cannot append to the uninitialized replica (it doesn't accept entries), we also need additional voters to return at the exact right time. Since an uninitialized replica without RaftReplicaID in is necessarily at least one release old, this is exceedingly unlikely and we will live with this theoretical risk. This commit also introduces a few assertions that make sure that we don't have overlapping initialized replicas (which would be detected at Store.Start time otherwise while inserting in the btree, but it's nice to catch this earlier) or duplicate RangeIDs. [^1]: cockroachdb#75761 Epic: CRDB-220 Release note: None
As of v22.1[^1], we always write the RaftReplicaID when creating a Replica or updating a snapshot. However, since this is persisted state that could've originated in older versions and not updated yet, we couldn't rely on a persisted ReplicaID yet. This commit adds code to the `(*Store).Start` boot sequence that - persists a RaftReplicaID for all initialized replicas (using the ReplicaID from the descriptor) - deletes all uninitialized replicas that don't have a RaftReplicaID (since we don't know their ReplicaID at this point). The second item in theory violates Raft invariants, as uninitialized Replicas are allowed to vote (though they then cannot accept log entries). So in theory: - an uninitialized replica casts a decisive vote for a leader - it restarts - code in this commit removes the uninited replica (and its vote) - delayed MsgVote from another leader arrives - it casts another vote for the same term for a dueling leader - now there are two leaders in the same term. The above in addition presupposes that the two leaders cannot communicate with each other. Also, even if that is the case, since the two leaders cannot append to the uninitialized replica (it doesn't accept entries), we also need additional voters to return at the exact right time. Since an uninitialized replica without RaftReplicaID in is necessarily at least one release old, this is exceedingly unlikely and we will live with this theoretical risk. This commit also introduces a few assertions that make sure that we don't have overlapping initialized replicas (which would be detected at Store.Start time otherwise while inserting in the btree, but it's nice to catch this earlier) or duplicate RangeIDs. [^1]: cockroachdb#75761 Epic: CRDB-220 Release note: None
95513: kvstorage: complete RaftReplicaID migration r=pavelkalinnikov a=tbg As of v22.1[^1], we always write the RaftReplicaID when creating a Replica or updating a snapshot. However, since this is persisted state that could've originated in older versions and not updated yet, we couldn't rely on a persisted ReplicaID yet. This commit adds code to the `(*Store).Start` boot sequence that - persists a RaftReplicaID for all initialized replicas (using the ReplicaID from the descriptor) - deletes all uninitialized replicas lacking RaftReplicaID (since we don't know their ReplicaID at this point). The second item in theory violates Raft invariants, as uninitialized Replicas are allowed to vote (though they then cannot accept log entries). So in theory: - an uninitialized replica casts a decisive vote for a leader - it restarts - code in this commit removes the uninited replica (and its vote) - delayed MsgVote from another leader arrives - it casts another vote for the same term for a dueling leader - now there are two leaders in the same term. The above in addition presupposes that the two leaders cannot communicate with each other. Also, even if that is the case, since the two leaders cannot append to the uninitialized replica (it doesn't accept entries), we also need additional voters to return at the exact right time. Since an uninitialized replica without RaftReplicaID in is necessarily at least one release old, this is exceedingly unlikely and we will live with this theoretical risk. This PR also adds a first stab at a datadriven test harness for `kvstorage` which is likely to be of use for #93247. [^1]: #75761 Epic: [CRDB-220](https://cockroachlabs.atlassian.net/browse/CRDB-220) Release note: None Co-authored-by: Tobias Grieger <[email protected]>
The RaftReplicaIDKey is an unreplicated range-id local key that
contains the ReplicaID of the replica whose HardState is represented
in the RaftHardStateKey. These two keys are removed atomically when
we clear the range-id local keys for a replica. See
store_create_replica.go for a detailed comment on correctness
and version compatibility.
We currently do not utilize this information on node restart
to figure out whether we should cleanup stale uninitialized replicas.
Doing such cleanup can wait until we implement and start using
ReplicasStorage. The change here is meant to set us up to rely
on RaftReplicaID from the next release onwards.
Informs #75740
Release note: None