-
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
storage: delay application of preemptive snapshots #35786
Conversation
This is a PR just to show some code to the interested parties. The real thing to look at here is the explanation and suggested strategy below. Don't review the code. ---- Learner replicas are full Replicas minus the right to vote (i.e they don't count for quorum). They are interesting to us because they allow us to phase out preemptive snapshots (via [delayed preemptive snaps] as a migration strategy; see there for all the things we don't like about preemptive snapshots), and because they can help us avoid spending more time in vulnerable configurations than we need to. To see an example of the latter, assume we're trying to upreplicate from three replicas to five replicas. As is, we need to add a fourth replica, wait for a preemptive snapshot, and add the fifth. We spend approximately the duration of one preemptive snapshot in an even replica configuration. In theory, we could send two preemptive snapshots first and then carry out two replica additions, but with learners, it would be much more straightforward and less error-prone. This doesn't solve the problems in cockroachdb#12768, but it helps avoid them. This PR shows the bare minimum of code changes to upreplicate using learner replicas and suggests further steps to make them a reality. Principally, to use learner replicas, we barely need to make any changes. Our current up-replication code is this: 1. send preemptive snapshot 1. run the ChangeReplicas txn, which adds a `ReplicaDescriptor` to the replicated range descriptor and, on commit, induces a Raft configuration change (`raftpb.ConfChangeAddNode`). The new up-replication code (note that the old one has to stick around, because compatibility): 1. run the ChangeReplicas txn, which adds a `ReplicaDescriptor` to the replicated range descriptor with the `Learner` flag set to true and, on commit, induces a Raft configuration change (`raftpb.ConfChangeAddLearnerNode`). 2. wait for the learner to have caught up or send it a Raft snapshot proactively (either works, just have to make sure not to duplicate work) 3. run a ChangeReplicas txn which removes the `Learner` flag from the `ReplicaDescriptor` and induces a Raft conf change of `raftpb.ConfChangeAddNode` (upgrading the learner). The existence of learners will need updates throughout the allocator so that it realizes that they don't count for quorum and are either upgraded or removed in a timely manner. None of that is in this POC. [delayed preemptive snaps]: cockroachdb#35786 Release note: None
e43fec7
to
95bc3e5
Compare
I pulled on this thread a little more. Unfortunately, just delaying preemptive snapshots isn't enough for what I wanted to accomplish. The basic motivation for getting rid of preemptive snapshots is to get rid of replicas without a replicaID, but preemptive snapshots are only one source of them. We also notably have a replica without a replicaID if a replica is fully added (after a delayed preemptive snap) and catching up on the log, but hasn't received the entry that adjusts the range descriptor to include the replica itself. In-memory, there will be a replicaID taken from incoming raft messages, but it won't survive a restart and so on restart the data looks like a preemptive snapshot again. We also use zero replicaIDs in a few other places, notably splits. When we acquire a split lock (i.e. the raftMu of the RHS) we pass a replicaID of zero to |
95bc3e5
to
ffbdd93
Compare
I think I found a buglet in Raft that was exposed by my latest (now pushed) version of these changes, and thankfully caught in Basically, when Raft applies a config change that adds a new peer, it doesn't seem to be reaching out to that peer proactively. It takes waiting for the next round of heartbeats (or next proposal) to do the job. With delayed snapshots, this doesn't just delay the catch-up of the newly added replica, but instead delays the creation of the replica in the first place, and that's what the test caught because it wasn't waiting for long enough. In the logs, I see this play out as follows:
(replicate queue goes off to do other things, and comes back):
What's happening here is that we add first a replica on n2, then later one on n3. But in the sequence of events for n2, we never see the delayed preemptive snapshot actually apply (which it would if the leader contacted the follower after the conf change). It takes another replica change to give the raft group work to do so that it'll actually contact the follower proactively. The test relies on Adding the line r.maybeSendAppend(id, true /* sendIfEmpty */) in this code fixes it: Before I go upstream, I'd like to hear from @bdarnell if I missed anything. |
ffbdd93
to
d12c4d8
Compare
d12c4d8
to
3bf3c93
Compare
Currently, in-memory Replica objects can end up having a replicaID zero. Roughly speaking, this is always the case when a Replica's range descriptor does not contain the Replica's store, though sometimes we do have a replicaID taken from incoming Raft messages (which then won't survive across a restart). We end up in this unnatural state mostly due to preemptive snapshots, which are a snapshot of the Range state before adding a certain replica, sent to the store that will house that replica once the configuration change to add it has completed. The range descriptor in the snapshot cannot yet assign the Replica a proper replicaID because none has been allocated yet (and this allocation has to be done in the replica change transaction, which hasn't started yet). Even when the configuration change completes and the leader starts "catching up" the preemptive snapshot and informs it of the replicaID, it will take a few moments until the Replica catches up to the log entry that actually updates the descriptor. If the node reboots before that log entry is durably applied, the replicaID will "restart" at zero until the leader contacts the Replica again. This suggests that preemptive snapshots introduce fundamental complexity which we'd like to avoid - as long as we use preemptive snapshots there will not be sanity in this department. This PR introduces a mechanism which delays the application of preemptive snapshots so that we apply them only when the first request *after* the completed configuration change comes in (at which point a replicaID is present). Superficially, this seems to solve the above problem (since the Replica will only be instantiated the moment a replicaID is known), though it doesn't do so across restarts. However, if we synchronously persisted (not done in this PR) the replicaID from incoming Raft messages whenever it changed, it seems that we should always be able to assign a replicaID when creating a Replica, even when dealing with descriptors that don't contain the replica itself (since there would've been a Raft message with a replicaID at some point, and we persist that). This roughly corresponds to persisting `Replica.lastToReplica`. We ultimately want to switch to learner replicas instead of preemptive snapshots. Learner replicas have the advantage that they are always represented in the replica descriptor, and so the snapshot that initializes them will be a proper Raft snapshot containing a descriptor containing the learner Replica itself. However, it's clear that we need to continue supporting preemptive snapshots in 19.2 due to the need to support mixed 19.1/19.2 clusters. This PR in conjunction with persisting the replicaID (and auxiliary work, for example on the split lock which currently also creates a replica with replicaID zero and which we know [has bugs]) should allow us to remove replicaID zero from the code base without waiting out the 19.1 release cycle. [has bugs]: cockroachdb#21146 Release note: None
3bf3c93
to
b8fc30a
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.
(and this allocation has to be done in the replica change
transaction, which hasn't started yet)
I think it would be possible to pre-allocate replica IDs. One CPut to the range descriptor to bump the next replica ID before the main transaction restarts, and then we could use that ID for the preemptive snapshots. I'm not 100% sure that's safe if there are multiple concurrent proposed schema changes, but that would be one way to get rid of replica ID zero without moving away from preemptive snapshots entirely.
Before I go upstream, I'd like to hear from @bdarnell if I missed anything.
Yeah, adding a maybeSendAppend there looks like a good idea.
This change seems like a step in the wrong direction - it moves us away from streaming snapshot application and increases memory pressure. What exactly is the problem being solved here? I understand the problems with preemptive snapshots, and we're planning to fix them by moving to learner replicas. What good does this intermediate state do for us and how long will it be in effect?
Reviewed 10 of 11 files at r1.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner, @nvanbenschoten, and @tbg)
pkg/storage/replica_raft.go, line 2134 at r1 (raw file):
// up a replica of `[a,b)'` by that time, the snapshot will be discarded by // Raft. If it has a replica of `[a,b)` (i.e. before the merge), the snapshot // again must be refused, for
Complete this sentence.
pkg/storage/store_snapshot.go, line 97 at r1 (raw file):
} maxSize := 128 * (1 << 20) // 128MB
Shoudn't this be a function of the zone config's range max size?
pkg/storage/txnwait/txnqueue.go, line 646 at r1 (raw file):
q.mu.Lock() // If the txn wait queue is not enabled or if the request is not // contained within the replica, do nothing. The request can fallp
Undo 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.
One CPut to the range descriptor to bump the next replica ID before the main transaction restarts, and then we could use that ID for the preemptive snapshots.
This isn't as easy as it sounds, we need to emit a commit trigger so that the in-memory copy of the range descriptor gets updated (we wouldn't want to let them diverge). Assuming we did that, we'll still be in an awkward spot. The preemptive snapshot will be instantiated with that replicaID, and now what? It's not in the descriptor, so its outgoing messages will be dropped. Similarly nobody can talk to it, but it does get to have a raft group now. I think this road ultimately doesn't lead to any good place, especially since we have our answer - learners - already.
This change seems like a step in the wrong direction - it moves us away from streaming snapshot application and increases memory pressure. What exactly is the problem being solved here?
The problem solved here (not fully solved, but getting there) is that we want to get rid of the Replica
state in which the replicaID is zero. The "feature" allowing that are learners (which we'll hopefully introduce in the 19.2 cycle as well) which leaves compatibility to be dealt with. We can "sit out" preemptive snapshots for a whole additional cycle but cleaning up the replica states is, in my opinion, the best tech debt reduction we can work on because it comes up with anything else in the store/replica area that we're discussing (pulling out the replication package, etc). That leads to wanting to receive preemptive snapshots without dealing with them throughout the codebase "very soon", so that we can get as close to removing replicaID zero as we can in this cycle (I'm hoping all the way). Applying delayed preemptive snapshots is a big step towards that.
This means that we wouldn't expect to actually use delayed preemptive snapshots outside of mixed version clusters (and baking it on master until we have learner replicas, which gives us confidence that they actually work). We're intentionally not focusing on scalability right now, so in this cycle we won't see larger snapshots than we have previously. In the next cycle, all things working out as expected, we can complete the transition into SST snapshots but either way the preemptive snapshot path will be dead because thanks to learners all snapshots will (look like) Raft snapshots and will apply with a replicaID immediately.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner, @bdarnell, @nvanbenschoten, and @tbg)
pkg/storage/store_snapshot.go, line 97 at r1 (raw file):
Previously, bdarnell (Ben Darnell) wrote…
Shoudn't this be a function of the zone config's range max size?
Yes and no. We only allow one incoming snapshot at any given point in time (granted, that could change in the cycle, though I don't anticipate it), so morally speaking the storage will only ever contain a single snapshot (with the exception of abandoned preemptive snaps, which stick around for ~20s). The exception are empty snapshots (or very small ones) for which we have special rules, and for which this storage even budgets based on size.
A single snapshot arriving will always be accepted into the storage, regardless of its size, evicting prior additions if necessary.
So even if you ran with 512mb ranges, things should just work unless many snapshots are sent in parallel. But we could make the max size a function of the maximum range size seen in the zone configs and add some padding to avoid early eviction of a single large snapshots by a subsequent very small snapshot.
Currently, in-memory Replica objects can end up having a replicaID zero.
Roughly speaking, this is always the case when a Replica's range
descriptor does not contain the Replica's store, though sometimes we do
have a replicaID taken from incoming Raft messages (which then won't
survive across a restart).
We end up in this unnatural state mostly due to preemptive snapshots,
which are a snapshot of the Range state before adding a certain replica,
sent to the store that will house that replica once the configuration
change to add it has completed. The range descriptor in the snapshot
cannot yet assign the Replica a proper replicaID because none has been
allocated yet (and this allocation has to be done in the replica change
transaction, which hasn't started yet).
Even when the configuration change completes and the leader starts
"catching up" the preemptive snapshot and informs it of the replicaID,
it will take a few moments until the Replica catches up to the log entry
that actually updates the descriptor. If the node reboots before that
log entry is durably applied, the replicaID will "restart" at zero until
the leader contacts the Replica again.
This suggests that preemptive snapshots introduce fundamental complexity
which we'd like to avoid - as long as we use preemptive snapshots there
will not be sanity in this department.
This PR introduces a mechanism which delays the application of
preemptive snapshots so that we apply them only when the first request
after the completed configuration change comes in (at which point a
replicaID is present).
Superficially, this seems to solve the above problem (since the Replica
will only be instantiated the moment a replicaID is known), though it
doesn't do so across restarts.
However, if we synchronously persisted (not done in this PR) the
replicaID from incoming Raft messages whenever it changed, it seems that
we should always be able to assign a replicaID when creating a Replica,
even when dealing with descriptors that don't contain the replica itself
(since there would've been a Raft message with a replicaID at some
point, and we persist that). This roughly corresponds to persisting
Replica.lastToReplica
.We ultimately want to switch to learner replicas instead of preemptive
snapshots. Learner replicas have the advantage that they are always
represented in the replica descriptor, and so the snapshot that
initializes them will be a proper Raft snapshot containing a descriptor
containing the learner Replica itself. However, it's clear that we need
to continue supporting preemptive snapshots in 19.2 due to the need to
support mixed 19.1/19.2 clusters.
This PR in conjunction with persisting the replicaID (and auxiliary
work, for example on the split lock which currently also creates a
replica with replicaID zero and which we know has bugs) should allow
us to remove replicaID zero from the code base without waiting out the
19.1 release cycle.
Release note: None