-
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: Raft entry cache grows (inefficiently) to huge size #13231
Comments
For the record, the range that this is affecting is an ordinary data range in the block_writer table, not a special system range. One range has over 2M raft log entries, compared to under 100 for other ranges. If the cluster had survived a little longer, this range could have split and the problem could be distributed over more nodes. The only theory I have for why this problem started appearing on gamma is that it's been running binaries with modified compaction settings, which might have slowed things down when the cluster reaches a certain size (without letting it survive long enough to split enough time). |
Reading all of the Raft log to count the number of pending entries is a bit silly. One short term fix would be to special case |
I was wrong about this being the only place we load an unbounded number of entries at once - we also do it with committed entries that need to be applied. Changing that will be a bit trickier, since the code currently assumes that after a Ready cycle the new applied index can be set to the previous commit index. I don't think there's any deep obstacle to changing this, though. |
This is good to change in any case as I've noticed in the past that processing a large number of Raft entries in a single ready call can take a significant period of time causing missed heartbeats and other fun. |
Unless I'm missing something, the I think we should fix both of the places in |
It is too late in the cycle to consider fixing @dianasaur323 or @nstewart This is a small to medium size stability task. |
kk added to airtable and marked you as an owner :). Currently it's a medium to be conservative. |
@bdarnell @petermattis could I have a try for this? I'm very interested in this part. :) |
@a6802739 Go ahead, but probably worthwhile to sketch out what your plan is before implementing as you'll likely need to modify |
@petermattis, Thank you very much. Here is my understanding, I'm not sure if it's accurate. when we load unbounded number of raft entries from raft state machine, we should first add these raft entries to the cache, but the cache will evict them because of exceeding the maxBytes? So What we want to do is chunk the And if we modify |
I'm not sure what you mean by chunking There are 4 places where Raft reads an unbounded number of log entries:
We have to submit a patch upstream to |
This issue is less critical than it used to be, thanks to the introduction of That said, I think I'd address the
Finally, this issue was originally about some inefficiencies in the |
@petermattis , Thank you for your explanation.
From my understanding, it means we should add a interface
About this, should we just limit the real number of Entries we iterate from Rocksdb according to the left size of @bdarnell Do you mean we have no need to concern about this, Or we just need to fix the case in And what do you mean for |
I don't think we have an urgent need to do any of this, but if we do something we should fix
Instead of introducing a new
to something like this
Adding a new |
@bdarnell, Yeah, I understand what I have to do now. So for |
Maybe. As I said, it's not clear whether this is safe. (and I'd probably add a field to |
@bdarnell, Thank you very much. So we could just add a field like |
Yes, I think so. |
@bdarnell Is there anything left to do here? |
No, I think we can close it. |
Each store has a cache for raft entries, configured by default to be 16MB. On gamma, we see that on one node (at a time), we have over a gigabyte of raft entries in memory. I believe that not all of these entries are in the cache, but they are all being held in place by the cache, because
Replica.Entries
allocates one large array ofraftpb.Entry
objects, so that a reference to any one of them keeps the whole array alive.Additionally, whenever we load a large array of entries, we try to add them all to the cache, inserting them all one by one and then evicting all but the last 16MB. This is actually the bigger concern in the gamma cluster at this time, since this process takes long enough (and blocks the server) so that it loses leases and never makes any progress.
Why do we load this monolithic block of entries? Whenever a new node becomes leader, it loads all uncommitted entries to see if there are any config changes. This is the one time in which we load raft entries without any chunking. It would be easy to add chunking here. In addition, we may want to change the behavior of the raft.Entries method to break up the arrays that it uses when adding entries to the cache (trading off the allocation overhead of smaller allocations vs the wasted memory of sibling array entries). Finally, this problem is also a result of our inability to throttle incoming raft entries. The log appears to keep growing beyond our ability to process it.
The text was updated successfully, but these errors were encountered: