-
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
kvserver: self delegated snapshots #77329
Conversation
6825c25
to
5e1d397
Compare
1e3f174
to
83942c8
Compare
8498ae3
to
66f4f23
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.
I took a very shallow pass, but wanted to flush the comments I had. Things generally look good so there are only a few substantial points and the rest are stylistic.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @amygao9, @nvanbenschoten, and @tbg)
-- commits, line 16 at r1:
Let's remove the release justification since this is not going into 22.1.
pkg/kv/kvserver/raft_transport.go, line 438 at r2 (raw file):
return err } // check to ensure the header is valid.
same nit about correctly casing / punctuating comments as elsewhere.
pkg/kv/kvserver/raft_transport.go, line 465 at r2 (raw file):
// pass off the snapshot request to the new sender store. err = handler.SendDelegatedSnapshot(ctx, req)
Overloading this SendDelegatedSnapshot()
method name to be used on both the coordinator (leader/leaseholder) as well as the designated sender was fairly confusing to me. I'd suggest changing the method names on the coordinator side to be a bit more clear about the fact that the coordinator is simply delegating, and not actually sending (maybe just {D,d}elegateSnapshot()
?).
pkg/kv/kvserver/replica_command.go, line 2424 at r2 (raw file):
// TODO(amy): select a follower based on locality matching. func (r *Replica) getSenderReplica(ctx context.Context) (roachpb.ReplicaDescriptor, error) { log.Fatal(ctx, "Follower snapshots not implemented")
nit: error and log messages are all generally lower-cased, here and elsewhere.
pkg/kv/kvserver/replica_command.go, line 2556 at r2 (raw file):
return err } // check follower snapshots cluster setting.
nit: Let's have these comments (here and elsewhere) be well punctuated / properly cased.
pkg/kv/kvserver/replica_command.go, line 2564 at r2 (raw file):
} log.VEventf(ctx, 2, "New sender replica selected: %v", sender)
Let's be a bit more specific about what this sender replica is for. Generally, if someone is looking through logs / traces and they see this line, it should be relatively obvious what the line is about. So I have a mild preference towards changing this to something like
log.VEventf("delegating snapshot transmission for %v to %v", recipient, sender)
pkg/kv/kvserver/store.go, line 1203 at r1 (raw file):
s.metrics.registry.AddMetricStruct(s.txnWaitMetrics) s.snapshotApplySem = make(chan struct{}, cfg.concurrentSnapshotApplyLimit) s.snapshotSendSem = make(chan struct{}, cfg.concurrentSnapshotApplyLimit)
Let's introduce a separate attribute called concurrentSnapshotSendLimit
and use that instead of reusing the apply limit.
Additionally, this PR also addresses part of #58920 but I think we should discuss further to ensure that Raft snapshots will not queue up behind rebalancing snapshots, since with this single semaphore, they will.
pkg/kv/kvserver/store_raft.go, line 102 at r2 (raw file):
const name = "storage.Store: handle snapshot" return s.stopper.RunTaskWithErr( ctx, name, func(ctx context.Context) error {
Was this diff intentional?
pkg/kv/kvserver/store_snapshot.go, line 1249 at r2 (raw file):
// sendDelegatedSnapshot sends an outgoing delegated snapshot request via a // pre-opened GRPC stream. It sends the delegated snapshot request to the new
nit: Let's get rid of the "new" here and elsewhere.
pkg/kv/kvserver/store_snapshot.go, line 1265 at r2 (raw file):
resp, err := stream.Recv() if err != nil { storePool.throttle(throttleFailed, err.Error(), delegatedSender.StoreID)
I don't think we want to throttle the sender store if it rejects a request to send a delegated snapshot.
Essentially, this storePool.throttle
mechanism exists to ensure that stores that cannot receive snapshots (for any reason) aren't inundated with incoming snapshots (that may keep getting rejected). throttle
ing a store makes it such that we won't try to rebalance replicas to it for a short while, to avoid sending more snapshots to it. Therefore, it is a little weird to be throttling the sender of the snapshot here (even more so since, in this patch, we're self-delegating).
We may want to do something similar (in principle) in the future for potential sender stores that reject the request to send snapshots (based on the reason they're rejecting the delegated snapshot request), but for the first pass I think we should simply return the error up the stack and not do anything else.
pkg/kv/kvserver/store_snapshot.go, line 1310 at r2 (raw file):
switch resp.Status { case kvserverpb.SnapshotResponse_ERROR: // send the error message back to the leaseholder.
Note that this code is being executed on the leaseholder/leader, so this comment could be a little confusing. Consider rewording it or removing it.
66f4f23
to
15f1650
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.
I took a very shallow pass, but wanted to flush the comments I had. Things generally look good so there are only a few substantial points and the rest are stylistic.
TFTR! I made the changes you noted.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @nvanbenschoten, and @tbg)
Previously, aayushshah15 (Aayush Shah) wrote…
Let's remove the release justification since this is not going into 22.1.
Done.
pkg/kv/kvserver/raft_transport.go, line 438 at r2 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
same nit about correctly casing / punctuating comments as elsewhere.
Done.
pkg/kv/kvserver/raft_transport.go, line 465 at r2 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
Overloading this
SendDelegatedSnapshot()
method name to be used on both the coordinator (leader/leaseholder) as well as the designated sender was fairly confusing to me. I'd suggest changing the method names on the coordinator side to be a bit more clear about the fact that the coordinator is simply delegating, and not actually sending (maybe just{D,d}elegateSnapshot()
?).
Done. Changed the names on the coordinator side to {D,d}elegateSnapshot()
pkg/kv/kvserver/replica_command.go, line 2564 at r2 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
Let's be a bit more specific about what this sender replica is for. Generally, if someone is looking through logs / traces and they see this line, it should be relatively obvious what the line is about. So I have a mild preference towards changing this to something like
log.VEventf("delegating snapshot transmission for %v to %v", recipient, sender)
Done.
pkg/kv/kvserver/store.go, line 1203 at r1 (raw file):
this PR also addresses part of #58920 but I think we should discuss further to ensure that Raft snapshots will not queue up behind rebalancing snapshots, since with this single semaphore, they will.
Good point, I'll note this for our next meeting.
pkg/kv/kvserver/store_raft.go, line 102 at r2 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
Was this diff intentional?
Nope, I reverted it.
pkg/kv/kvserver/store_snapshot.go, line 1265 at r2 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
I don't think we want to throttle the sender store if it rejects a request to send a delegated snapshot.
Essentially, this
storePool.throttle
mechanism exists to ensure that stores that cannot receive snapshots (for any reason) aren't inundated with incoming snapshots (that may keep getting rejected).throttle
ing a store makes it such that we won't try to rebalance replicas to it for a short while, to avoid sending more snapshots to it. Therefore, it is a little weird to be throttling the sender of the snapshot here (even more so since, in this patch, we're self-delegating).We may want to do something similar (in principle) in the future for potential sender stores that reject the request to send snapshots (based on the reason they're rejecting the delegated snapshot request), but for the first pass I think we should simply return the error up the stack and not do anything else.
I see, that makes sense.
pkg/kv/kvserver/store_snapshot.go, line 1310 at r2 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
Note that this code is being executed on the leaseholder/leader, so this comment could be a little confusing. Consider rewording it or removing it.
Done.
07d31f5
to
94cd98c
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.
The patch looks good to me now, mod the things that came up today in our pod meeting. @nvanbenschoten, it'd be useful to have a pass from you on this, whenever you get a chance.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @amygao9, @nvanbenschoten, and @tbg)
pkg/kv/kvserver/raft_transport.go, line 420 at r4 (raw file):
} // DelegateRaftSnapshot handles incoming delegated snapshot requests and parses
passes*
pkg/kv/kvserver/raft_transport.go, line 469 at r4 (raw file):
return roachpb.NewStoreNotFoundError(req.DelegatedSender.StoreID) } // Acknowledge that the request has been accepted.
nit: there's a mild preference to have this acceptance be pulled into the followerSendSnapshot
method where the core logic lives.
pkg/kv/kvserver/store.go, line 1203 at r1 (raw file):
Previously, amygao9 (Amy Gao) wrote…
this PR also addresses part of #58920 but I think we should discuss further to ensure that Raft snapshots will not queue up behind rebalancing snapshots, since with this single semaphore, they will.
Good point, I'll note this for our next meeting.
Note for posterity: we decided offline in our pod meeting that we're going to have two separate semaphores on the sender side -- one for raft snapshots, and another for rebalancing snapshots.
pkg/kv/kvserver/store_raft.go, line 81 at r4 (raw file):
} // Throttle snapshot sending. cleanup, err := s.reserveSendSnapshot(ctx, req)
I'd also prefer if we pulled this reserveSendSnapshot
call into the followerSendSnapshot
method, just like the handshake acceptance comment I made above.
pkg/kv/kvserver/store_snapshot.go, line 491 at r3 (raw file):
const snapshotReservationWaitWarnThreshold = 32 * time.Second elapsed := timeutil.Since(tBegin) if elapsed > snapshotReservationWaitWarnThreshold && !buildutil.CrdbTestBuild {
Let's add an NB: ..
explaining why we avoid this log message in test builds. Let's say something about how some tests don't mock out all these objects being logged.
pkg/kv/kvserver/store_snapshot.go, line 1270 at r3 (raw file):
case kvserverpb.SnapshotResponse_ERROR: return errors.Errorf( "%s: remote couldn't accept %s with error: %s", delegatedSender,
sender* couldn't accept...
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 @aayushshah15, @nvanbenschoten, and @tbg)
pkg/kv/kvserver/raft_transport.go, line 420 at r4 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
passes*
Done.
pkg/kv/kvserver/store.go, line 1203 at r1 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
Note for posterity: we decided offline in our pod meeting that we're going to have two separate semaphores on the sender side -- one for raft snapshots, and another for rebalancing snapshots.
Just made the change to include two separate semaphores.
pkg/kv/kvserver/store_raft.go, line 81 at r4 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
I'd also prefer if we pulled this
reserveSendSnapshot
call into thefollowerSendSnapshot
method, just like the handshake acceptance comment I made above.
Done. I pulled both the reserveSendSnapshot
and acceptance response into the followerSendSnapshot
method, which will also be useful in the next patch for having senders decline delegated requests.
pkg/kv/kvserver/store_snapshot.go, line 491 at r3 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
Let's add an
NB: ..
explaining why we avoid this log message in test builds. Let's say something about how some tests don't mock out all these objects being logged.
Done.
pkg/kv/kvserver/store_snapshot.go, line 1270 at r3 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
sender* couldn't accept...
Done.
6092370
to
f6d4535
Compare
It's very cool to see this all falling into place. Nice job @amygao9! |
f885ede
to
666a762
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.
Thanks for the review :) I fixed up the throttling test and made all the changes and just need a final check from the two of you now
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @nvanbenschoten, and @tbg)
pkg/kv/kvserver/raft_transport.go, line 136 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Should this be called
SendDelegatedSnapshot
orHandleDelegatedSnapshot
?
Good point, I renamed it HandleDelegatedSnapshot
so it follows the same flow as the actual snapshot sending.
pkg/kv/kvserver/raft_transport.go, line 440 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Let's name this task something that differentiates it from the one in
RaftSnapshot
. How aboutprocessing snapshot delegation
and we change the other one toprocessing snapshot reception
.
Done.
pkg/kv/kvserver/raft_transport.go, line 485 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
In
RaftSnapshot
, we defer all of this to the handler and simplyreturn handler.HandleSnapshot(ctx, req.Header, stream)
. Should we do the same here?
Yep, to keep it uniform to the RaftSnapshot
methods I moved the logic here to handler.HandleSnapshot
and passed in the span created.
pkg/kv/kvserver/raft_transport.go, line 137 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: let's try to keep the commit history clean and ensure that each commit stands on its own. If changes like this should have been in the previous commit, let's patch them into that one.
Yes, I squashed the commits into one.
pkg/kv/kvserver/kvserverpb/raft.proto, line 207 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Instead of having a separate header here, do you think it would simplify logic to share the same header between the two snapshot requests?
Using the Snapshot Header
here would work too, and we could just leave some fields blank when sending the header in the DelegatedRequest
. @aayushshah15 mentioned that it might be more easily understandable to have a new header clearly show which fields are needed in a delegate request. Both of these options seem fine with me.
pkg/kv/kvserver/kvserverpb/raft.proto, line 212 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Could we add comments to these fields. It's not immediately clear who is "from" and who is "to". Maybe we need different terms, now that there are three actors at play here.
Done. Changed the naming to coordinator_replica
and recipient_replica
and added comments to make the terms more clear. I used the term coordinator replica
to represent the leaseholder in many comments - let me know if that part is confusing or if a different name would be better.
pkg/kv/kvserver/kvserverpb/raft.proto, line 220 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
I wonder if
DelegateSnapshotRequest
should be dictating the snapshot strategy or if we should just let the delegated sender make this choice. I think the latter because the strategy is an implementation detail of how the snapshot is sent.
I see, it makes sense to let the delegated sender make this choice then.
pkg/kv/kvserver/kvserverpb/raft.proto, line 226 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Same question here. Should this be passed through the
DelegateSnapshotRequest
or just computed on the delegated sender? I think the latter, because the size of the range may differ between the two replicas if the leaseholder is a few entries ahead of the delegated sender.
Yep, thats a good point! Done.
pkg/kv/kvserver/kvserverpb/raft.proto, line 228 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
This comment got cut off. Let's explain what this term is and why it's needed.
Done.
pkg/kv/kvserver/kvserverpb/raft.proto, line 235 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: add spacing between each message.
Done.
d7f2889
to
92572ba
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 7 of 9 files at r8, 2 of 2 files at r9, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @amygao9, and @tbg)
pkg/kv/kvserver/replica_command.go, line 2672 at r9 (raw file):
}, }, RangeSize: r.GetMVCCStats().Total(),
nit: we're grabbing r.GetMVCCStats()
twice. Can we store it in a local variable?
pkg/kv/kvserver/kvserverpb/raft.proto, line 207 at r5 (raw file):
Previously, amygao9 (Amy Gao) wrote…
Using the
Snapshot Header
here would work too, and we could just leave some fields blank when sending the header in theDelegatedRequest
. @aayushshah15 mentioned that it might be more easily understandable to have a new header clearly show which fields are needed in a delegate request. Both of these options seem fine with me.
Upon a re-read, I actually think we should remove this header entirely. It's used in SnapshotRequest
to separate control-plane metadata from data-plane data, but that distinction is not needed here. Do you see a reason to not inline all of these fields into the DelegateSnapshotRequest
?
pkg/kv/kvserver/replica_learner_test.go, line 263 at r9 (raw file):
scratch := tc.ScratchRange(t) replicationChange := make(chan error, 2) g := ctxgroup.WithContext(ctx)
Should we be waiting (g.Wait()
) for this ctxgroup to complete and testing the error?
pkg/kv/kvserver/replica_learner_test.go, line 323 at r9 (raw file):
atomic.StoreInt64(&activateBlocking, 0) <-blockIncomingSnapshots }
nit: new line after this.
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 @aayushshah15, @nvanbenschoten, and @tbg)
pkg/kv/kvserver/replica_command.go, line 2672 at r9 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: we're grabbing
r.GetMVCCStats()
twice. Can we store it in a local variable?
Done.
pkg/kv/kvserver/kvserverpb/raft.proto, line 207 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Upon a re-read, I actually think we should remove this header entirely. It's used in
SnapshotRequest
to separate control-plane metadata from data-plane data, but that distinction is not needed here. Do you see a reason to not inline all of these fields into theDelegateSnapshotRequest
?
Yeah, I agree that we don't need that same distinction here. The header here kind of separates the metadata needed for the snapshot, and the metadata needed for the delegate snapshot handshake. I'm not opposed to having the fields be inline though. @aayushshah15 do you have any thoughts on this?
pkg/kv/kvserver/replica_learner_test.go, line 263 at r9 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Should we be waiting (
g.Wait()
) for this ctxgroup to complete and testing the error?
Yep, I added the g.Wait()
to the end of the test and asserted on the error.
pkg/kv/kvserver/replica_learner_test.go, line 323 at r9 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: new line after this.
Done.
92572ba
to
ccef644
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 2 of 2 files at r10, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @amygao9, and @tbg)
pkg/kv/kvserver/replica_learner_test.go, line 325 at r10 (raw file):
// Wait for the goroutines to finish and assert the error is as expected. const msgRE = `descriptor changed`
Why do we expect this error? Is this what the test is intending to demonstrate, or just a byproduct of how the test is constructed? Could we add something to the comment about this?
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 @aayushshah15, @nvanbenschoten, and @tbg)
pkg/kv/kvserver/replica_learner_test.go, line 325 at r10 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Why do we expect this error? Is this what the test is intending to demonstrate, or just a byproduct of how the test is constructed? Could we add something to the comment about this?
The error was a byproduct of the test construction and it occurs due to the two add voter goroutines racing with each other. More specifically, it occurs when the first goroutine tries to promote the learner to a voter and realizes the second goroutine added a learner. Since the focus of the test was to show snapshot throttling, I changed the first goroutine to add a non-voter instead, so the test gracefully ends without an error.
6d0ef99
to
d592b42
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 @aayushshah15, @nvanbenschoten, and @tbg)
pkg/kv/kvserver/kvserverpb/raft.proto
line 207 at r5 (raw file):
Previously, amygao9 (Amy Gao) wrote…
Yeah, I agree that we don't need that same distinction here. The header here kind of separates the metadata needed for the snapshot, and the metadata needed for the delegate snapshot handshake. I'm not opposed to having the fields be inline though. @aayushshah15 do you have any thoughts on this?
I made the change to remove the header and have all the fields be in-line in this request, after discussing with Aayush.
d592b42
to
6452405
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.
The patch mod the one comment about generalizing the testing knob. Nice work. I'd suggest waiting for @nvanbenschoten to also take a final look before you merge.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @aayushshah15, @amygao9, and @nvanbenschoten)
pkg/kv/kvserver/store_snapshot.go
line 453 at r11 (raw file):
sem = s.raftSnapshotSendSem } if fn := s.cfg.TestingKnobs.CountSendSnapshotsThrottling; fn != nil {
Currently, this testing knob is a bit confusing. We should change this to 2, more general, testing knobs:
BeforeSendSnapshotThrottle
that's called in this location (i.e. before we throttle a snapshot) and is simply a closure (that the test defines).AfterSendSnapshotThrottle
that's called after we acquire a spot in the semaphore, and is also simply a closure.
PurgeOutdatedReplicasInterceptor
is an example of such a general testing knob.
That way, these testing knobs aren't intertwining test-specific logic with production code.
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 r11, all commit messages.
Reviewable status: complete! 2 of 0 LGTMs obtained (waiting on @aayushshah15, @amygao9, and @nvanbenschoten)
pkg/kv/kvserver/kvserverpb/raft.proto
line 241 at r11 (raw file):
// The Raft applied term of the coordinator replica. // The term is used during snapshot receiving to reject messages from an older term.
It looks like this comment is no longer correct (discussed in person). It used to be:
// 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.
5d1ffb9
to
df4ac33
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.
Thanks to both of you for the review!
Reviewable status: complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @aayushshah15 and @nvanbenschoten)
pkg/kv/kvserver/store_snapshot.go
line 453 at r11 (raw file):
Previously, aayushshah15 (Aayush Shah) wrote…
Currently, this testing knob is a bit confusing. We should change this to 2, more general, testing knobs:
BeforeSendSnapshotThrottle
that's called in this location (i.e. before we throttle a snapshot) and is simply a closure (that the test defines).AfterSendSnapshotThrottle
that's called after we acquire a spot in the semaphore, and is also simply a closure.
PurgeOutdatedReplicasInterceptor
is an example of such a general testing knob.That way, these testing knobs aren't intertwining test-specific logic with production code.
Done.
pkg/kv/kvserver/kvserverpb/raft.proto
line 241 at r11 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
It looks like this comment is no longer correct (discussed in person). It used to be:
// 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.
Done.
This commit adds a new rpc stream for sending raft message requests between replicas which allows for delegating snapshots. Currently this patch implements the leaseholder delegating to itself, but in future patches the leaseholder will be able to delegate snapshot sending to follower replicas. A new raft message request type of `DelegatedSnapshotRequest` includes a header of nessesary fields from a `SnapshotRequest` and the replica descriptor of the new sender replica. This allows the leaseholder to fill in snapshot metadata before delegating to the new sender store to generate the snapshot and transmit it to the recipient. Related to cockroachdb#42491 Release note: None
df4ac33
to
5018508
Compare
bors r+ |
Build succeeded: |
This commit adds a new rpc stream for sending raft message requests between
replicas which allows for delegating snapshots. Currently this patch implements
the leaseholder delegating to itself, but in future patches the leaseholder
will be able to delegate snapshot sending to follower replicas. A new raft
message request type of
DelegatedSnapshotRequest
includes a header ofnessesary fields from a
SnapshotRequest
and the replica descriptor of the newsender replica. This allows the leaseholder to fill in snapshot metadata before
delegating to the new sender store to generate the snapshot and transmit it to
the recipient.
Related to #42491
Release note: None