-
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: fix Raft log size accounting #31914
Conversation
b8e4f92
to
51f40c5
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.
Thanks for pushing this over the finish line.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale)
pkg/cmd/roachtest/tpcc.go, line 362 at r1 (raw file):
if _, err := db.ExecContext(ctx, `SET CLUSTER SETTING kv.range_merge.queue_enabled = false`); err != nil { return err }
Perhaps revert the changes in this file which I made to debug tpcc-bench stuff.
@benesch might have an opinion about disabling the merge queue for tpcc bench runs until the merge issues are sorted out.
pkg/storage/replica.go, line 6584 at r1 (raw file):
// Gossip the cluster ID from all replicas of the first range; there // is no expiration on the cluster ID. if false && log.V(1) {
There are a bunch of log changes such as this that should be reverted.
pkg/storage/replica.go, line 6617 at r1 (raw file):
log.Event(ctx, "gossiping sentinel and first range") if log.V(1) { //log.Infof(ctx, "gossiping sentinel from store %d, r%d", r.store.StoreID(), r.RangeID)
Revert?
pkg/storage/replica.go, line 6624 at r1 (raw file):
log.Errorf(ctx, "failed to gossip sentinel: %s", err) } if false && log.V(1) {
Revert?
pkg/storage/replica_proposal.go, line 573 at r1 (raw file):
log.Errorf(ctx, "while removing sideloaded files during log truncation: %s", err) } else { rResult.RaftLogDelta -= size
I trust that you agree this is copacetic. This was the part of the change that I was most unsure about.
I ran a few roachtests to verify that things are stable now. These two look good (another two running now):
But I got this suspicious failure:
This originates in cockroach/pkg/roachpb/metadata.go Lines 222 to 234 in a050d56
So we have a replica that has a NodeID and StoreID but a zero replicaID either in cockroach/pkg/storage/replica_command.go Line 348 in 62ea444
or in the other call to PS merges are turned off in this testing. My best bet for this error is that cockroach/pkg/storage/replica_command.go Line 173 in 62ea444
|
Filed that error as a separate issue #31918. |
51f40c5
to
1eb4852
Compare
We were accounting for sideloaded payloads (SSTs) when adding them to the log, but were omitting them during truncations. As a result, the tracked raft log size would permanently skyrocket which in turn would lead to extremely aggressive truncations and resulted in pathological amounts of Raft snapshots. I'm still concerned about this logic as we're now relying on numbers obtained from the file system to match exactly a prior in-mem computation, and there may be other bugs that cause a divergence. But this fixes the blatant obvious one, so it's a step in the right direction. The added tests highlight a likely omission in the sideloaded storage code which doesn't access the file system through the RocksDB env as it seems like it should, filed as cockroachdb#31913. At this point it's unclear whether it fixes the below issues, but at the very least it seems to address a major problem they encountered: Touches cockroachdb#31732. Touches cockroachdb#31740. Touches cockroachdb#30261. Touches cockroachdb#31768. Touches cockroachdb#31745. Release note (bug fix): avoid a performance degradation related to overly aggressive Raft log truncations that could occur during RESTORE or IMPORT operations.
1eb4852
to
e808caf
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.
Reviewed 5 of 9 files at r1, 4 of 4 files at r2.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale)
pkg/storage/replica_proposal.go, line 573 at r1 (raw file):
Previously, petermattis (Peter Mattis) wrote…
I trust that you agree this is copacetic. This was the part of the change that I was most unsure about.
Are you concerned about anything in particular? This seems fine to me. We could compute this upstream, but it doesn't seem better, and we don't need this to be equivalent between replicas anyway.
One problem could be that we might be removing more than we added. For example, if a bunch of data gets sideloaded and then the cluster restarts, the first truncation after the restart will get lots of bytes here "for free". This is likely going to result in a raft log size of zero, and it would be a little off (too small) from that point on. (This isn't a problem that needs sideloaded payloads, it's basically by design). This can only result in a de facto size that is off by a factor of two (i.e. truncating 2x as late as configured), so it's probably not a big deal.
It'll be painful to get all of this "correct". We can't simply recompute the size of the remaining log here as there might be more sideloaded payloads. Besides, sideloaded payloads for older terms may coexist with new ones.
I just wrote a comment to deal with residual space in Raft logs for idle ranges here:
cockroach/pkg/storage/raft_log_queue.go
Lines 38 to 41 in e808caf
// RaftLogQueueStaleThreshold is the minimum threshold for stale raft log | |
// entries. A stale entry is one which all replicas of the range have | |
// progressed past and thus is no longer needed and can be truncated. | |
RaftLogQueueStaleThreshold = 100 |
Perhaps when performing such a truncation, we could do a little more work to recompute the exact length. I hope it's never going to be worth it though.
pkg/cmd/roachtest/tpcc.go, line 362 at r1 (raw file):
Previously, petermattis (Peter Mattis) wrote…
Perhaps revert the changes in this file which I made to debug tpcc-bench stuff.
@benesch might have an opinion about disabling the merge queue for tpcc bench runs until the merge issues are sorted out.
Not intentional. Sorry for not catching that.
pkg/storage/replica.go, line 6584 at r1 (raw file):
Previously, petermattis (Peter Mattis) wrote…
There are a bunch of log changes such as this that should be reverted.
Not intentional. Sorry for not catching that.
pkg/storage/replica.go, line 6617 at r1 (raw file):
Previously, petermattis (Peter Mattis) wrote…
Revert?
Not intentional. Sorry for not catching that.
pkg/storage/replica.go, line 6624 at r1 (raw file):
Previously, petermattis (Peter Mattis) wrote…
Revert?
Not intentional. Sorry for not catching that.
Another one just passed (and yet another one within 20 minutes of completion and looking good)
Really hoping to see less import tests fail tonight. |
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.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale)
pkg/storage/replica_proposal.go, line 573 at r1 (raw file):
Previously, tschottdorf (Tobias Schottdorf) wrote…
Are you concerned about anything in particular? This seems fine to me. We could compute this upstream, but it doesn't seem better, and we don't need this to be equivalent between replicas anyway.
One problem could be that we might be removing more than we added. For example, if a bunch of data gets sideloaded and then the cluster restarts, the first truncation after the restart will get lots of bytes here "for free". This is likely going to result in a raft log size of zero, and it would be a little off (too small) from that point on. (This isn't a problem that needs sideloaded payloads, it's basically by design). This can only result in a de facto size that is off by a factor of two (i.e. truncating 2x as late as configured), so it's probably not a big deal.
It'll be painful to get all of this "correct". We can't simply recompute the size of the remaining log here as there might be more sideloaded payloads. Besides, sideloaded payloads for older terms may coexist with new ones.I just wrote a comment to deal with residual space in Raft logs for idle ranges here:
cockroach/pkg/storage/raft_log_queue.go
Lines 38 to 41 in e808caf
// RaftLogQueueStaleThreshold is the minimum threshold for stale raft log // entries. A stale entry is one which all replicas of the range have // progressed past and thus is no longer needed and can be truncated. RaftLogQueueStaleThreshold = 100
Perhaps when performing such a truncation, we could do a little more work to recompute the exact length. I hope it's never going to be worth it though.
No particular concerns, but this is the only place where we're computing RaftLogDelta
below Raft.
Ack. I'm going to merge but would like to hear if @nvanbenschoten has any concerns and would rather move this upstream of Raft. bors r=petermattis |
31914: storage: fix Raft log size accounting r=petermattis a=tschottdorf We were accounting for sideloaded payloads (SSTs) when adding them to the log, but were omitting them during truncations. As a result, the tracked raft log size would permanently skyrocket which in turn would lead to extremely aggressive truncations and resulted in pathological amounts of Raft snapshots. I'm still concerned about this logic as we're now relying on numbers obtained from the file system to match exactly a prior in-mem computation, and there may be other bugs that cause a divergence. But this fixes the blatant obvious one, so it's a step in the right direction. The added tests highlight a likely omission in the sideloaded storage code which doesn't access the file system through the RocksDB env as it seems like it should, filed as #31913. At this point it's unclear whether it fixes the below issues, but at the very least it seems to address a major problem they encountered: Touches #31732. Touches #31740. Touches #30261. Touches #31768. Touches #31745. Release note (bug fix): avoid a performance degradation related to overly aggressive Raft log truncations that could occur during RESTORE or IMPORT operations. Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
Build succeeded |
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.
I don't think this can be moved upstream into Raft because etcd/raft
has no concept of log truncation. My understanding is that it assumes that logs are persisted forever.
Reviewed 4 of 9 files at r1, 4 of 4 files at r2.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 2 stale)
pkg/storage/replica_proposal.go, line 573 at r1 (raw file):
Besides, sideloaded payloads for older terms may coexist with new ones.
That's interesting. Don't we run into the same issue that we do here when a new term overwrites old sideloaded entries? Should we be doing something similar to this here:
cockroach/pkg/storage/replica.go
Lines 4333 to 4340 in 0368172
firstPurge := rd.Entries[0].Index // first new entry written | |
purgeTerm := rd.Entries[0].Term - 1 | |
lastPurge := prevLastIndex // old end of the log, include in deletion | |
for i := firstPurge; i <= lastPurge; i++ { | |
err := r.raftMu.sideloaded.Purge(ctx, i, purgeTerm) | |
if err != nil && errors.Cause(err) != errSideloadedFileNotFound { | |
const expl = "while purging index %d" | |
return stats, expl, errors.Wrapf(err, expl, i) |
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.
Sorry, I didn't mean "into etcd/raft" but "into proposal evaluation".
Reviewable status: complete! 1 of 0 LGTMs obtained (and 1 stale)
pkg/storage/replica_proposal.go, line 573 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Besides, sideloaded payloads for older terms may coexist with new ones.
That's interesting. Don't we run into the same issue that we do here when a new term overwrites old sideloaded entries? Should we be doing something similar to this here:
cockroach/pkg/storage/replica.go
Lines 4333 to 4340 in 0368172
firstPurge := rd.Entries[0].Index // first new entry written purgeTerm := rd.Entries[0].Term - 1 lastPurge := prevLastIndex // old end of the log, include in deletion for i := firstPurge; i <= lastPurge; i++ { err := r.raftMu.sideloaded.Purge(ctx, i, purgeTerm) if err != nil && errors.Cause(err) != errSideloadedFileNotFound { const expl = "while purging index %d" return stats, expl, errors.Wrapf(err, expl, i)
Absolutely, let me look into that.
Ah, I see. I don't have any big concerns about what we do here. If computing this during proposal evaluation means that we would need to peek at each Raft log entry (in Go) then I'd say it's a non-starter. If we could just look directly at the sideloaded storage, that might be a workable solution. It would be nice to avoid looking at the file size on each replica beneath Raft and hoping they're all the same, even if it is ok for them to diverge. It would also be nice to keep this out of the single-threaded Raft loop. |
…ries This follows up on a comment of @nvanbenschoten in cockroachdb#31914. Unfortunately, it seems really hairy to come up with a test for this, so a bit of refactoring will be needed. Release note: None
31881: exec: distinct manages its own scratch column r=jordanlewis a=jordanlewis Previously, distinct relied on its input batch to have a scratch boolean column for working. It's unnecessary - instead, manage the scratch boolean directly during construction. Release note: None 31926: storage: adjust raft log size correctly when replacing sideloaded entries r=nvanbenschoten a=tschottdorf This follows up on a comment of @nvanbenschoten in #31914 which highlighted yet another potential (though hopefully rare) source of raft log size not being reduced correctly. Release note: None Co-authored-by: Jordan Lewis <jordanthelewis@gmail.com> Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
…ries This follows up on a comment of @nvanbenschoten in cockroachdb#31914. Unfortunately, it seems really hairy to come up with a test for this, so a bit of refactoring will be needed. Release note: None
We were accounting for sideloaded payloads (SSTs) when adding them to
the log, but were omitting them during truncations. As a result, the
tracked raft log size would permanently skyrocket which in turn would
lead to extremely aggressive truncations and resulted in pathological
amounts of Raft snapshots.
I'm still concerned about this logic as we're now relying on numbers
obtained from the file system to match exactly a prior in-mem
computation, and there may be other bugs that cause a divergence. But
this fixes the blatant obvious one, so it's a step in the right
direction.
The added tests highlight a likely omission in the sideloaded storage
code which doesn't access the file system through the RocksDB env as it
seems like it should, filed as #31913.
At this point it's unclear whether it fixes the below issues, but at the
very least it seems to address a major problem they encountered:
Touches #31732.
Touches #31740.
Touches #30261.
Touches #31768.
Touches #31745.
Release note (bug fix): avoid a performance degradation related to
overly aggressive Raft log truncations that could occur during RESTORE
or IMPORT operations.