Skip to content

Comments

kv: enable delegate snapshots#83991

Merged
craig[bot] merged 1 commit intocockroachdb:masterfrom
andrewbaptist:follower_snapshots
Feb 3, 2023
Merged

kv: enable delegate snapshots#83991
craig[bot] merged 1 commit intocockroachdb:masterfrom
andrewbaptist:follower_snapshots

Conversation

@andrewbaptist
Copy link

@andrewbaptist andrewbaptist commented Jul 7, 2022

kvserver: delegate snapshots to followers

Fixes: #42491

This commit allows a snapshot to be sent by a follower instead of the
leader of a range. The follower(s) are chosen based on locality to the
final recipient of the snapshot. If the follower is not able to
quickly send the snapshot, the attempt is aborted and the leader sends
the snapshot instead.

By choosing a delegate rather than sending the snapshot directly, WAN
traffic can be minimized. Additionally the snapshot will likely be
delivered faster.

There are two settings that control this feature. The first,
kv.snapshot_delegation.num_follower, controls how many followers
the snapshot is attempted to be delegated through. If set to 0, then
snapshot delegation is disabled. The second,
kv.snapshot_delegation_queue.enabled, controls whether delegated
snapshots will queue on the delegate or return failure immediately. This
is useful to prevent a delegation request from spending a long time
waiting before it is sent.

Before the snapshot is sent from the follower checks are done to
verify that the delegate is able to send a snapshot that will be valid
for the recipient. If not the request is rerouted to the leader.

Release note (performance improvement): Adds delegated snapshots which can reduce WAN traffic for snapshot movement.

@cockroach-teamcity
Copy link
Member

This change is Reviewable

@andrewbaptist
Copy link
Author

This is the restore of the initial PR from amygao9@7c525d9 - this is still a WIP

@andrewbaptist andrewbaptist force-pushed the follower_snapshots branch 9 times, most recently from 94241ca to 1a6329c Compare July 12, 2022 17:43
@andrewbaptist andrewbaptist changed the title Initial commit kv: complete delegate snapshots Jul 12, 2022
@andrewbaptist andrewbaptist force-pushed the follower_snapshots branch 5 times, most recently from 2e76245 to 89ad698 Compare July 25, 2022 19:22
@andrewbaptist andrewbaptist changed the title kv: complete delegate snapshots kv: enable delegate snapshots Jul 26, 2022
@andrewbaptist andrewbaptist force-pushed the follower_snapshots branch 8 times, most recently from 432c493 to f8d5d89 Compare August 1, 2022 18:25
@andrewbaptist andrewbaptist force-pushed the follower_snapshots branch 2 times, most recently from 7e6fa1d to 5878898 Compare September 28, 2022 13:05
Copy link
Author

@andrewbaptist andrewbaptist left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @herkolategan, @nvanbenschoten, and @smg260)


pkg/kv/kvserver/replica_command.go line 2861 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

I'm still curious about this. Are there descriptor generation changes that don't necessarily mean this, but we're ok with false positives?

Just to record what we discussed and added a comment to it. This check probably is a little overly strict, which means some delegated snapshots may be rejected unnecessarily. This is not really a problem and should be pretty rare. We could likely remove this check in the future.


pkg/kv/kvserver/replica_command.go line 3034 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

but that risks holding the constraint for a longer window

The leaseholder also doesn't know which index to bound log truncation up to. It may have applied further than the delegated sender, and it doesn't want to truncate the log any higher than the index that the delegated snapshot is being sent.

I think this is why the RPC protocol was streaming before. We had envisioned a handshake where the leaseholder would ask the delegate for its applied index, install a constraint at that index (rechecking that it hadn't truncated above this already), then allow the delegated snapshot to proceed.

After discussing options on this I added the log truncation constraint up front on the coordinator and there is a risk of holding this longer than strictly required. There is a path to change this if necessary in the future, but it is unlikely to be a concern for most systems. https://cockroachlabs.atlassian.net/wiki/spaces/~6268113f52310b0068ffd245/pages/2869854482/Delegate+snapshot+overview
There are a few ways to solve this, but I don't think the streaming RPC is correct since the data required is disjoint for the different parts of the flow. There is a risk with the latest change that a delegated requests will fail since it won't have the latest applied index from the leaseholder. If that occurs often it would be worth changing to retrying on the delegate since it will "eventually" catch up in most cases.


pkg/kv/kvserver/replica_command.go line 2591 at r6 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

s/desc/replID/

Done


pkg/kv/kvserver/replica_command.go line 2611 at r6 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

small nit: the 0 is unnecessary.

Done


pkg/kv/kvserver/replica_command.go line 2619 at r6 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Can we use ReplicaSet.GetReplicaDescriptorByID here?

Done

@andrewbaptist andrewbaptist force-pushed the follower_snapshots branch 2 times, most recently from dd822da to d5bcbc3 Compare January 24, 2023 15:39
@andrewbaptist andrewbaptist requested a review from nvb January 24, 2023 17:09
@andrewbaptist andrewbaptist force-pushed the follower_snapshots branch 4 times, most recently from d312c2c to ac66f07 Compare January 27, 2023 15:04
Copy link
Contributor

@nvb nvb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 18 of 20 files at r7, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @andrewbaptist, @herkolategan, and @smg260)


pkg/kv/kvserver/replica_command.go line 2821 at r7 (raw file):

		Term:                 status.Term,
		DelegatedSender:      sender,
		FirstIndex:           appliedIndex,

Could you add a comment here about why you are setting FirstIndex to the value of appliedIndex on the leaseholder? It's subtle and could benefit from a discussion about the consequences and plans for future improvement.


pkg/kv/kvserver/replica_command.go line 2880 at r7 (raw file):

	// If the generation has changed, this snapshot may be useless, so don't
	// attempt to send it.
	//NB: This is an overly strict check. If other delegates are added to this

nit: missing a space after // on each line.

@andrewbaptist andrewbaptist force-pushed the follower_snapshots branch 3 times, most recently from dad9c3b to 0e75e33 Compare January 31, 2023 22:17
@andrewbaptist andrewbaptist requested a review from a team January 31, 2023 22:17
Copy link
Contributor

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @andrewbaptist, @herkolategan, @nvanbenschoten, and @smg260)


-- commits line 29 at r8:
nit: consider adding more words to the release note for this awesome work so that the average user can understand it

Copy link
Author

@andrewbaptist andrewbaptist left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @herkolategan, @nvanbenschoten, and @smg260)


pkg/kv/kvserver/replica_command.go line 2821 at r7 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Could you add a comment here about why you are setting FirstIndex to the value of appliedIndex on the leaseholder? It's subtle and could benefit from a discussion about the consequences and plans for future improvement.

Done


pkg/kv/kvserver/replica_command.go line 2880 at r7 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

nit: missing a space after // on each line.

Done

Copy link
Contributor

@nvb nvb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @andrewbaptist, @herkolategan, and @smg260)


pkg/kv/kvserver/replica_command.go line 2821 at r7 (raw file):

Previously, andrewbaptist (Andrew Baptist) wrote…

Done

Did you miss a git push?

Copy link
Author

@andrewbaptist andrewbaptist left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 2 of 4 files at r8.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @AlexTalks, @andreimatei, @herkolategan, @nvanbenschoten, and @smg260)


-- commits line 29 at r8:

Previously, andreimatei (Andrei Matei) wrote…

nit: consider adding more words to the release note for this awesome work so that the average user can understand it

Thanks, I added some more to the release notes. I will likely publish a short blog no this as well when 23.1 comes out!


pkg/kv/kvserver/replica_command.go line 2821 at r7 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Did you miss a git push?

I'm not sure what happened - I rewrote and pushed again.

Copy link
Contributor

@nvb nvb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm:

Reviewed 1 of 1 files at r10, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @AlexTalks, @andreimatei, @andrewbaptist, @herkolategan, and @smg260)

Fixes: cockroachdb#42491

This commit allows a snapshot to be sent by a follower instead of the
leader of a range. The follower(s) are chosen based on locality to the
final recipient of the snapshot.  If the follower is not able to
quickly send the snapshot, the attempt is aborted and the leader sends
the snapshot instead.

By choosing a delegate rather than sending the snapshot directly, WAN
traffic can be minimized. Additionally the snapshot will likely be
delivered faster.

There are two settings that control this feature. The first,
`kv.snapshot_delegation.num_follower`, controls how many followers
the snapshot is attempted to be delegated through. If set to 0, then
snapshot delegation is disabled. The second,
`kv.snapshot_delegation_queue.enabled`, controls whether delegated
snapshots will queue on the delegate or return failure immediately. This
is useful to prevent a delegation request from spending a long time
waiting before it is sent.

Before the snapshot is sent from the follower checks are done to
verify that the delegate is able to send a snapshot that will be valid
for the recipient. If not the request is rerouted to the leader.

Release note (performance improvement): Adds delegated snapshots which
can reduce WAN traffic for snapshot movement. If there is another
replica for this range with a closer locality than the delegate, the
leaseholder will attempt to have that delegate send the snapshot. This
is particularly useful in the case of a decommission of a node where
most snapshots are transferred to another replica in the same locality.
@andrewbaptist
Copy link
Author

bors r=nvanbenschoten

@craig
Copy link
Contributor

craig bot commented Feb 3, 2023

Build succeeded:

@craig craig bot merged commit f148476 into cockroachdb:master Feb 3, 2023
@shralex
Copy link
Contributor

shralex commented Feb 3, 2023

congrats!!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Send Raft snapshots between follower replicas

5 participants