-
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: snapshot/log truncation/replica GC badness tracking issue #32046
Comments
This seems like a symptom rather than a root cause. Allowing more concurrency here can lead to death spirals when we overwhelm the receiver. You're much closer to this now than I am. Perhaps you have some interesting thoughts about something that can be changed. |
The problem I see here is not that there's throttling at the receiver, but that the throttling will back up the sender, who could spend the time sending snapshots to other nodes that need them. I don't fully understand why this semaphore gets that backed up (lack of introspection is one part of the problem), but if we sent 64mb snapshots at 8mb/s we'd clock in at 8s/snapshot. Being backed up after 3-4 of them would easily do it, and how much you can be backed up by scales by the number of nodes, which is typically 8-11 in my experiments. |
Ah, I missed that we're only refusing preemptive snapshots. That defuses the situation, though the check doesn't need to compute the Raft log size in the first place (assuming we trust our computed number, which ... well we don't). |
This could be merge related. I don't think the merge queue has any protection against causing this situation. |
Merge queue is off, but hold on to that thought. Also, if you want to get rid of a follower, our strategy now is to upreplicate then downreplicate. This means upreplicating when there's a lame duck in the group is necessary. We just have to make sure that we're not losing the preemptive snapshot, which brings us back to reigning in the overly aggressive replicaGC. I have a (ticks-based) WIP in my latest round of experiments. |
This could be addressed with either a deadline (to unblock the thread and allow it to try a different range/replica) or more parallelism on the sender (run the queue with a higher concurrency, but then limit it back down with a sender-side semaphore once the recipient has accepted the snapshot reservation). There's also #14768: Preemptive snapshots are limited to 2MiB/sec while raft snapshots get 8MiB/sec. However, we don't queue them separately, so a higher-priority raft snapshot may be blocked until a throttled preemptive snapshot has completed. |
This probably isn't going to be close to the code that will eventually get checked in, but I wanted to get the conversation started. I don't have concrete evidence that this problem is a root cause of \cockroachdb#32046, however I want to address it (at the very least for cockroachdb#31875). I have to dig in more, but what I'm seeing in various import/tpch flavor tests is that the split-scatter phase is extremely aggressive in splitting and then downreplicating overreplicated ranges. For example, r100 might have descriptor [n1,n2,n3,n4] and will rapidly be split (and its LHS and RHS split again, multiple times) while, say, n4 is removed. I think that in this kind of situation, needing one Raft snapshot quickly implies needing ~O(splits) Raft snapshots. This is because splitting a range on which one replica requires a Raft snapshot you end up with two ranges that do. The implication is that we don't want to need Raft snapshots (and perhaps also: we want to go easy on splitting ranges for which one replica already needs a snapshot). On a recent "successful" run of tpccbench/nodes=11/cpus=32, a spike in pending snapshots from zero to 5k (resolved within minutes) was observed. A run of import/tpch/nodes=8 typically shows a rapid increase from zero to ~1k which only dissipates after the import returns. This variation may be random, or it may indicate that the import test is a lot more aggressive for some reason. I have to look into the details, but the following script results in a number of Raft snapshots (dozens). This may already be fixed by other PRs such as cockroachdb#31875, though. Easy to verify. ---- An upreplication begins by sending a preemptive snapshot, followed by a transaction which "officially" adds the new member to the the Raft group. This leaves a (typically small) window during which the replicaGC queue could pick up the preemptive snapshot and delete it. This is unfortunate as it leaves the range in a fragile state, with one follower requiring a Raft snapshot to catch up. This commit introduces a heuristic that holds off on GC'ing replicas that look like preemptive snapshots until they've been around for a while. Release note: None
This makes it a lot easier to log descriptive debug messages indicating how a truncation decision was arrived at, and in particular allows pointing the finger at truncations that lead to Raft snapshots, which is relevant in the context of cockroachdb#32046. Release note: None
This makes it a lot easier to log descriptive debug messages indicating how a truncation decision was arrived at, and in particular allows pointing the finger at truncations that lead to Raft snapshots, which is relevant in the context of cockroachdb#32046. Release note: None
32137: storage: refactor log truncation index computation r=petermattis a=tschottdorf This makes it a lot easier to log descriptive debug messages indicating how a truncation decision was arrived at, and in particular allows pointing the finger at truncations that lead to Raft snapshots, which is relevant in the context of #32046. Release note: None Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
The log size truncation is definitely still janky. We know it can undercount, but the behavior here is rather the opposite: we set the "max raft log size" to the size of the replica, which is really small (74KiB). The Raft log size includes the overhead of the proposal, etc, which the replica size doesn't eat. So we see an endless stream of premature proposals. I don't think any of these really cause a snapshot (because the latest log entries likely have been sent to the client at this point) but it seems silly, especially since this is a condition I want to be able to keep logging.
|
The writes to that range are the node statuses, which are large (tens of ks) and inline (i.e. each write replaces a previous write). So the replica size is mostly constant, whereas the Raft log grows fairly quickly. Under the configured behavior, it's doing the right thing, but I would argue that it shouldn't be truncating all the way up to the quorum index quite as aggressively. Truncating to the joint commit index (taking into account live nodes only) would fare better. I added a bullet to that effect in the initial list. |
Another bad-looking thing:
Added a bullet for this as well. |
cc @nvanbenschoten. I'm going to run some kv95 experiments in which I vary the 64kb threshold in both directions to see if there are any effects on performance in doing so. ---- Whenever the "max raft log size" is exceeded, log truncations become more aggressive in that they aim at the quorum commit index, potentially cutting off followers (which then need Raft snapshots). The effective threshold log size is 4mb for replicas larger than 4mb and the replica size otherwise. This latter case can be problematic since replicas can be persistently small despite having steady log progress (for example, range 4 receives node status updates which are large inline puts). If in such a range a follower falls behind just slightly, it'll need a snapshot. This isn't in itself the biggest deal since the snapshot is fairly rare (the required log entries are usually already on in transit to the follower) and would be small, but it's not ideal. Always use a 4mb threshold instead. Note that we also truncate the log to the minimum replicated index if the log size is above 64kb. This is similarly aggressive but respects followers (until they fall behind by 4mb or more). My expectation is that this will not functionally change anything. It might leave behind a little bit more Raft log on quiescent ranges, but I think the solution here is performing "one last truncation" for ranges that are quiescent to make sure they shed the remainder of their Raft log. Touches cockroachdb#32046. Release note: None
Whenever the "max raft log size" is exceeded, log truncations become more aggressive in that they aim at the quorum commit index, potentially cutting off followers (which then need Raft snapshots). The effective threshold log size is 4mb for replicas larger than 4mb and the replica size otherwise. This latter case can be problematic since replicas can be persistently small despite having steady log progress (for example, range 4 receives node status updates which are large inline puts). If in such a range a follower falls behind just slightly, it'll need a snapshot. This isn't in itself the biggest deal since the snapshot is fairly rare (the required log entries are usually already on in transit to the follower) and would be small, but it's not ideal. Always use a 4mb threshold instead. Note that we also truncate the log to the minimum replicated index if the log size is above 64kb. This is similarly aggressive but respects followers (until they fall behind by 4mb or more). My expectation is that this will not functionally change anything. It might leave behind a little bit more Raft log on quiescent ranges, but I think the solution here is performing "one last truncation" for ranges that are quiescent to make sure they shed the remainder of their Raft log. Touches cockroachdb#32046. Release note: None
32437: storage: truncate aggressively only after 4mb of logs r=nvanbenschoten,petermattis a=tbg cc @nvanbenschoten. I'm going to run some kv95 experiments in which I vary the 64kb threshold in both directions to see if there are any effects on performance in doing so. ---- Whenever the "max raft log size" is exceeded, log truncations become more aggressive in that they aim at the quorum commit index, potentially cutting off followers (which then need Raft snapshots). The effective threshold log size is 4mb for replicas larger than 4mb and the replica size otherwise. This latter case can be problematic since replicas can be persistently small despite having steady log progress (for example, range 4 receives node status updates which are large inline puts). If in such a range a follower falls behind just slightly, it'll need a snapshot. This isn't in itself the biggest deal since the snapshot is fairly rare (the required log entries are usually already on in transit to the follower) and would be small, but it's not ideal. Always use a 4mb threshold instead. Note that we also truncate the log to the minimum replicated index if the log size is above 64kb. This is similarly aggressive but respects followers (until they fall behind by 4mb or more). My expectation is that this will not functionally change anything. It might leave behind a little bit more Raft log on quiescent ranges, but I think the solution here is performing "one last truncation" for ranges that are quiescent to make sure they shed the remainder of their Raft log. Touches #32046. Release note: None Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
This makes it a lot easier to log descriptive debug messages indicating how a truncation decision was arrived at, and in particular allows pointing the finger at truncations that lead to Raft snapshots, which is relevant in the context of cockroachdb#32046. Release note: None
Whenever the "max raft log size" is exceeded, log truncations become more aggressive in that they aim at the quorum commit index, potentially cutting off followers (which then need Raft snapshots). The effective threshold log size is 4mb for replicas larger than 4mb and the replica size otherwise. This latter case can be problematic since replicas can be persistently small despite having steady log progress (for example, range 4 receives node status updates which are large inline puts). If in such a range a follower falls behind just slightly, it'll need a snapshot. This isn't in itself the biggest deal since the snapshot is fairly rare (the required log entries are usually already on in transit to the follower) and would be small, but it's not ideal. Always use a 4mb threshold instead. Note that we also truncate the log to the minimum replicated index if the log size is above 64kb. This is similarly aggressive but respects followers (until they fall behind by 4mb or more). My expectation is that this will not functionally change anything. It might leave behind a little bit more Raft log on quiescent ranges, but I think the solution here is performing "one last truncation" for ranges that are quiescent to make sure they shed the remainder of their Raft log. Touches cockroachdb#32046. Release note: None
We know there can be a backlog of Raft snapshots at the beginning of the test. This isn't ideal, but we know about it and have cockroachdb#32046 tracking it. Closes cockroachdb#32859. Release note: None
33011: roachtest: don't fail tests based on slow health checker r=petermattis a=tbg We know there can be a backlog of Raft snapshots at the beginning of the test. This isn't ideal, but we know about it and have #32046 tracking it. Closes #32859. Release note: None Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
I have too many investigations across the various roachtest import/restore failures. This is an authoritative list of problems that I don't want to lose track of.
From #30261 (comment)
From #31409 (comment) and surroundings:
Higher up in the thread: #31409 (comment)
we have a mechanism that refuses Raft snapshots based on log size, which is a recipe for disaster as reducing log size needs a truncation which needs quorum. We should never ever refuse Raft snapshots. It only aborts preemptive snapshots, but the mechanism should be updated so that snapshot aborted == raft log queue would truncate, so that we can just add to the queue reactively.assorted:
Likely there are some rough edges around the anomalous case of SST ingestion where one entry already blows the raft log max size. Example
from #32046 (comment):
#32437 via @petermattis
The text was updated successfully, but these errors were encountered: