cockroachdb / cockroach

CockroachDB — the cloud native, distributed SQL database designed for high availability, effortless scale, and control over data placement.
https://www.cockroachlabs.com
Other
30.21k stars 3.82k forks source link

storage: widening Raft snapshot spanning merge may clobber newer range via subsumption #36611

Closed tbg closed 4 years ago

tbg commented 5 years ago

I was thinking about the safety properties of preemptive snapshots, learner snapshots, and Raft snapshots (for https://github.com/cockroachdb/cockroach/pull/35786 and https://github.com/cockroachdb/cockroach/pull/35787) with and without merges today and I arrived at an example which I think we mishandle. The TL;DR is that I think that we'll accept a Raft snapshot that extends an initialized replica and "overwrites" a newer right hand side that was merged in but then split out again (and that may have new data).

This problem would be avoided if (edit: this statement about generations is wrong) ~we checked the generations of all subsumed replicas before handing the snapshot to Raft (i.e. in canApplySnapshot), dropping the snapshot unless they're all smaller than the snapshot's generation. The problem would also be avoided if~ we didn't allow existing replicas to change their replicaID in place. That is, upon trying to apply the last snapshot, the existing replica would be gc'ed before checking whether the snapshot could be applied to a new, uninitialized replica. This uninitialized replica would then reject the snapshot based on its overlapping another replica. In both cases the result would be another snapshot being sent.

I'd appreciate if someone (@bdarnell or @nvanbenschoten?) gave this a close reading to check my understanding.

tbg commented 5 years ago

On a meta level, I think the comment that "Raft snapshots to initialized replicas can't be refused" which first originated in https://github.com/cockroachdb/cockroach/pull/28683/files isn't quite right. We can always refuse a Raft snapshot, the question is whether there will ever be a snapshot for that range that we won't drop.

I think there's really only one situation in which refusing the snapshot would end us in a loop, and it's the situation outlined in https://github.com/cockroachdb/cockroach/pull/28533/commits/b728fcd384cf2e6497f7aa84c959aad1e2edfd9a:

Raft snapshots can "widen" an existing replica. Consider a merge where one of the LHS replicas goes offline immediately before the merge commits. Before this replica comes back online, the leader truncates the log. When this range comes back online, it will receive a Raft snapshot that postdates the merge application.

To apply this snapshot, the receiving replica needs to susume its copy of the RHS during snapshot application. Note that it is guaranteed to have a copy of the RHS, as merges cannot commit unless the ranges are collocated and the replica GC queue is careful to never GC a replica that could be required for subsumption.

Note that in this scenario the replicaID didn't change.

tbg commented 5 years ago

I was wrong in thinking that the generations are helpful here. They generally only make sense when comparing generations for the same rangeID. This is because the RHS of a split always starts out at generation zero. If the RHS inherited the generation of the LHS, maybe something can work but I haven't thought about it.

bdarnell commented 5 years ago

That looks right to me.

My initial sense is that the in-place changes of replica ID have always been a little sketchy and we should aim to eliminate them as much as possible. We should try to go through a GC/snapshot cycle instead of reusing the data that is already there. (much of the need for in-place replica ID changes came from a time when the GC/snapshot processes were more expensive, and replicas that needed GC would linger for a longer time)

nvanbenschoten commented 5 years ago

If the RHS inherited the generation of the LHS, maybe something can work but I haven't thought about it.

I figured this was already how this worked. I would expect that both sides of a split would get the generation lhs.gen + 1 and the result of a merge would get the generation max(lhs.gen, rhs.gen)+1. This way, we could compare the generation for any key ranges that overlap to determine who is new and who is stale. We could then immediately GC stale Ranges and reject stale snapshots. I haven't thought about this enought to understand the challenges here though.

tbg commented 5 years ago

I sent PR #36654 to change the semantics -- I hope we can get this in for 19.1. Let's discuss there.

tbg commented 5 years ago

cc @danhhz, this is the issue I talked about yesterday but couldn't find on the spot

tbg commented 5 years ago

Here's the example in schematic form, I find this easier to follow along with.

   a      b      c      d
S1
n1 |--1----------|-2----|
n2 |--1--(rmvd)--|
n3 |--1----------|-2----|

S2
n1 |--1----------|-2----|
n2 |--1--(rmvd)--|
n3 |--1----------|-2----|

S3
n1 |--1-----------------|
n2 |--1--(rmvd)--|        <-- r1@S3 snap incoming
n3 |--1-----------------|

S4
n1 |--1----------|-3----|
n2 |--1--(rmvd)--|        <-- r1@S3 snap incoming
n3 |--1----------|-3----|

S5
n1 |--1----------|-3----|
n2 |--1--(rmvd)--|-3----| <-- r1@S3 snap incoming
n3 |--1----------|-3----|

S6
n1 |--1----------|-3----|
n2 |--r1@S3-------------| <-- data lost from r3
n3 |--1----------|-3----|

The code that erroneously accepts r1@S3 is

https://github.com/cockroachdb/cockroach/blob/f8be5091ee4324542d18b2f6bb93ef9e6ec139eb/pkg/storage/store_snapshot.go#L459-L472

Looking at the example one could argue that maybe the creation of r3@S3 should have gotten blocked instead, but it can only be blocked once we know that we're receiving r1@S3 which we simply may not know for a while (for the usual distributed systems reasons).

nvanbenschoten commented 4 years ago

I believe this issue was solved by @ajwerner somewhere in the combination of 20201150f4fc83d826045f23c07abba73b1756bc and b2850e12dfb1d436fcf9af522cad142af7501bbc. We no longer allow replicas to change their replica IDs in place (as of 19.2) and we also no longer support preemptive snapshots (as of 20.1).

Should we close this?

ajwerner commented 4 years ago

Yes