Skip to content
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: truncate aggressively only after 4mb of logs #32437

Merged
merged 1 commit into from
Nov 19, 2018

Conversation

tbg
Copy link
Member

@tbg tbg commented Nov 16, 2018

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

@tbg tbg requested a review from a team November 16, 2018 23:59
@cockroach-teamcity
Copy link
Member

This change is Reviewable

Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm: glad to see this change! The old behavior was always surpising. I'm interested in hearing the results of your testing.

Reviewed 1 of 1 files at r1.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained

Copy link
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm:

Reviewable status: :shipit: complete! 2 of 0 LGTMs obtained


pkg/storage/raft_log_queue.go, line 108 at r1 (raw file):

	// and end transaction operations. If the estimated raft log size becomes
	// larger than the replica size, we're better off recovering the replica
	// using a snapshot.

This comment needs updating. It also explains the rationale behind the old heuristic. A snapshot can be cheaper than sending Raft log entries, though applying Raft log entries can be done in parallel while snapshots are serialized.

Another thought of something that can be done in this area is to use a size-based quota system for concurrent snapshot application. Instead of limiting the number of concurrent snapshots based on count, we'd limit based on bytes so that a large number of tiny snapshots could be allowed concurrently.

@tbg tbg force-pushed the fix/truncation-by-size branch from 5c8b086 to 50a619c Compare November 19, 2018 09:38
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
Copy link
Member Author

@tbg tbg left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TFTRs! Holding off on the merge until I see benchmark parity.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 2 stale)


pkg/storage/raft_log_queue.go, line 108 at r1 (raw file):

Previously, petermattis (Peter Mattis) wrote…

This comment needs updating. It also explains the rationale behind the old heuristic. A snapshot can be cheaper than sending Raft log entries, though applying Raft log entries can be done in parallel while snapshots are serialized.

Another thought of something that can be done in this area is to use a size-based quota system for concurrent snapshot application. Instead of limiting the number of concurrent snapshots based on count, we'd limit based on bytes so that a large number of tiny snapshots could be allowed concurrently.

Comment updated.

I agree that the snapshot queue needs a reworking independently of this change. Added your remark to #32046 (comment).

Just to be clear though, I want to reach a state in which we never cause Raft snapshots when they're not necessary irrespective of whether they get the system in a bad state or not. Think of this as a stability metric driving my particular string of investigations right now. Any snapshot we're not expecting to see is a problem. (There may be diminishing returns at some point, but I don't think I'm there yet).

@tbg
Copy link
Member Author

tbg commented Nov 19, 2018

@nvanbenschoten here are some (bad, because I used an individual cluster for each run) numbers:

kv0-master
_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total
  600.0s        0        7123527        11872.5     16.2     12.6     46.1     71.3    671.1  write
kv0-pr
_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total
  600.0s        0        7182234        11970.4     16.0     12.6     44.0     71.3    142.6  write

kv95-master
_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total
  600.0s        0       17496355        29160.5      5.7      4.5     18.9     35.7    167.8  read
  600.0s        0         919924         1533.2     16.5     14.7     37.7     54.5    209.7  write
kv95-pr
_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total
  600.0s        0       20327845        33879.6      4.9      3.5     15.7     35.7    285.2  read
  600.0s        0        1071319         1785.5     15.1     13.1     37.7     54.5    302.0  write

I might've gotten lucky with the machines for the PR runs. I wouldn't really expect to see a perf difference.

Going to run some more where I vary the stale trunc threshold, but going to merge this first (once it passes CI).

@tbg
Copy link
Member Author

tbg commented Nov 19, 2018

bors r=nvanbenschoten,petermattis

craig bot pushed a commit that referenced this pull request Nov 19, 2018
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 <[email protected]>
@tbg
Copy link
Member Author

tbg commented Nov 19, 2018

1-node numbers from my gceworker also show that if anything, things have gotten better.

95-master
_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total
  600.0s        0       24932240        41553.7      0.8      0.7      1.4      5.0     58.7  read
  600.0s        0        1313142         2188.6      6.5      5.8     12.6     18.9     56.6  write
95-pr
_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total
  600.0s        0       25414365        42357.2      0.8      0.7      1.4      4.7     54.5  read
  600.0s        0        1336055         2226.8      6.4      5.5     12.1     18.9     60.8  write

0-master
_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total
  600.0s        0        5772116         9620.2      5.0      4.7      7.9     11.5     62.9  write
0-pr
  600.0s        0        6245009        10408.3      4.6      4.2      7.6     12.1     48.2  write

@tbg
Copy link
Member Author

tbg commented Nov 19, 2018

PS @nvanbenschoten it's quite likely I didn't do exactly what you had in mind. By running the roachtest, I think my table gets pre-split into 1000 pieces, so each individual Raft log will see only a small amount of activity. Is that what you wanted or was I supposed to run this against a single instance? I unfortunately lost track of where we talked about this initially.

@craig
Copy link
Contributor

craig bot commented Nov 19, 2018

Build succeeded

@craig craig bot merged commit 53cecf1 into cockroachdb:master Nov 19, 2018
@nvanbenschoten
Copy link
Member

I wasn't even considering a perf benchmark because we shouldn't be snapshotting at all during steady-state load. I was more interested in whether this helped with some of your other testing around snapshots and overly aggressive log truncation.

@tbg
Copy link
Member Author

tbg commented Nov 20, 2018

Oh, sorry. No, it won't help with that, not beyond this fix. Ok, nothing to do then.

@tbg tbg deleted the fix/truncation-by-size branch November 20, 2018 09:01
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants