Skip to content

kv: avoid 1k entry readahead in each scan of uncached Raft log#66816

Merged
craig[bot] merged 1 commit intocockroachdb:masterfrom
nvb:nvanbenschoten/mvccIterStats2
Jul 1, 2021
Merged

kv: avoid 1k entry readahead in each scan of uncached Raft log#66816
craig[bot] merged 1 commit intocockroachdb:masterfrom
nvb:nvanbenschoten/mvccIterStats2

Conversation

@nvb
Copy link
Copy Markdown
Contributor

@nvb nvb commented Jun 24, 2021

Fixes #66682.

In #66682, we noticed a poor interaction between Raft and the MVCC API. When a Raft leader is catching up its followers, it often sends out large chunks of log entries. We prevent this from being overly expensive in the presence of large log entries by limiting the aggregate size of these entries to 32 KB (defaultRaftMaxSizePerMsg) at a time. This is accomplished through the through the maxBytes param that is passed throughout the raft.Storage implementation.

What we noticed was that even though we configure a maxBytes of 32 KB, no limit is passed through to iterateEntries's call to MVCCIterate. This seems harmless enough, because the iterator function passed to MVCCIterate terminates once the byte limit is exceeded. However, behind the scenes, MVCCIterate pulls in up to 1000 entries (maxKeysPerScan) from Pebble at a time, with no accompanying byte limit. So in a cluster with large log entries, where even just one is enough to exceed our budget, this ends up being very expensive.

In the customer cluster that we were looking at when we found #66682, each log entry was 131KB large. So each attempt to grab a new log entry read 131MB from the LSM. This was likely all cached, but even just unmarshaling this was expensive due to the memory copies doing so performs. This was so expensive that each call to grab a single entry took about 65ms. This was bad, and it became even worse when a follower sent 100 MsgAppResps in response to the 100 MsgApps sent by the leader (which is suboptimal but expected). If these all landed on the leader at once, it would spend 100*65ms = 6.5s processing messages in Store.processRequestQueue. This stall was long enough for another follower to call an election and for the leader to lose its leadership. Since two replicas were possible leaders and both were in this situation, it never resolved and leadership ping-ponged.

This commit fixes this issue by replacing the call to MVCCIterate in iterateEntries with direct use of an MVCC iterator. This provides more control over iteration and allows us to avoid the unwanted readahead policy. It also allows us to avoid some of the unnecessary cruft of MVCCIterate.

Rejected Alternatives

I briefly considered trying to adapt MVCCIterate to be more configurable through the MVCCScanOptions it is provided, but that seemed like a fraught task. This was primarily because the TargetBytes config is not quite what we want in this case, where we need to be precise about the sizes of these entries (see 9d46451) and don't want to count the key size of the MVCCMetadata wrapper size.

I also considered simply adding some conservative, non-configurable readahead byte limit to MVCCIterate to sit alongside its key limit. This may still be a good idea. But I also think we may want to restrict the cases where we even use the function.

Finally, I considered removing the readahead behavior in MVCCIterate entirely. I think it might have been added to avoid repeat CGo hops back when we had RocksDB. But now that we have Pebble and can manipulate an iterator cheaply, it can maybe be rejected. However, this would then require us to lift some logic from pebbleMVCCScanner into MVCCIterate, which I wasn't keen on doing.

Release note (bug fix): Catching up Raft followers on the Raft log is now more efficient in the presence of many large Raft log entries. This helps avoid situations where Raft leaders struggle to retain leadership while catching up their followers.

/cc. @cockroachdb/kv

@nvb nvb requested review from erikgrinaker and tbg June 24, 2021 04:54
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

Copy link
Copy Markdown
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

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

:lgtm:

I considered removing the readahead behavior in MVCCIterate entirely. I think it might have been added to avoid repeat CGo hops back when we had RocksDB. But now that we have Pebble and can manipulate an iterator cheaply, it can maybe be rejected. However, this would then require us to lift some logic from pebbleMVCCScanner into MVCCIterate, which I wasn't keen on doing.

This seems like a good idea, opened #66828 for it.

Reviewed 3 of 3 files at r1.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @nvanbenschoten and @tbg)


pkg/kv/kvserver/replica_raftstorage.go, line 256 at r1 (raw file):

			return err
		}
		if !ok || !iter.UnsafeKey().Less(storage.MakeMVCCMetadataKey(endKey)) {

nit: isn't this already enforced by the iterator bounds?


pkg/kv/kvserver/store_snapshot.go, line 395 at r1 (raw file):

		// preallocation is enough to run the server out of memory. Limit
		// the size of the buffer we will preallocate.
		preallocSize = maxPreallocSize

Why are we removing maxPreallocSize? Seems like this check might have been ineffectual because we didn't actually do anything with raftLogBytes before. Should we enforce a snapshot size limit elsewhere, or do we already do so?

@tbg tbg requested a review from erikgrinaker June 24, 2021 12:18
Copy link
Copy Markdown
Member

@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.

LGTM, had the same question as Erik about the upper bound though.

Reviewed 3 of 3 files at r1.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @erikgrinaker and @nvanbenschoten)


pkg/kv/kvserver/store_snapshot.go, line 395 at r1 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Why are we removing maxPreallocSize? Seems like this check might have been ineffectual because we didn't actually do anything with raftLogBytes before. Should we enforce a snapshot size limit elsewhere, or do we already do so?

This is mostly dead code and has been for multiple releases - we don't send any log entries with the snapshots any more:

usesReplicatedTruncatedState, err := storage.MVCCGetProto(
ctx, snap.EngineSnap, keys.RaftTruncatedStateLegacyKey(r.RangeID), hlc.Timestamp{}, nil, storage.MVCCGetOptions{},
)
if err != nil {
return errors.Wrap(err, "loading legacy truncated state")
}
canAvoidSendingLog := !usesReplicatedTruncatedState &&
snap.State.TruncatedState.Index < snap.State.RaftAppliedIndex

Also, as of 21.2 (i.e. master now), we actually know for sure it is dead code:

// TruncatedAndRangeAppliedStateMigration is part of the migration to stop
// using the legacy truncated state within KV. After the migration, we'll be
// using the unreplicated truncated state and the RangeAppliedState on all
// ranges. Callers that wish to assert on there no longer being any legacy
// will be able to do so after PostTruncatedAndRangeAppliedStateMigration is
// active. This lets remove any holdover code handling the possibility of
// replicated truncated state in 21.2.
//
// TODO(irfansharif): Do the above in 21.2.
TruncatedAndRangeAppliedStateMigration

I don't think this PR should rip it out (as we'll want to backport this one) but on master we shouldn't have to have this code around any longer at all.

Either way, this code is exceedingly unlikely to run in any adversarial scenarios because in reality it won't run at all - a range would've had to not have any log truncations for multiple releases.

I filed #66834 to track its removal.


pkg/storage/mvcc.go, line 2643 at r1 (raw file):

// MVCCIterate iterates over the key range [start,end). At each step of the
// iteration, f() is invoked with the current key/value pair. If f returns
// iterutil.StopIteration, the iteration stops and no error propagated. If f

stops with no error propagated.

Copy link
Copy Markdown
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.

I considered removing the readahead behavior in MVCCIterate entirely. I think it might have been added to avoid repeat CGo hops back when we had RocksDB. But now that we have Pebble and can manipulate an iterator cheaply, it can maybe be rejected. However, this would then require us to lift some logic from pebbleMVCCScanner into MVCCIterate, which I wasn't keen on doing.

This seems like a good idea, opened #66828 for it.

+1 to doing this. Or to removing MVCCIterate entirely. Now that we don't have to minimize cgo crossings I don't see the point of MVCCIterate vs using an MVCCIterator.

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @erikgrinaker and @nvanbenschoten)

@nvb nvb force-pushed the nvanbenschoten/mvccIterStats2 branch from e8522c3 to 903cecb Compare June 30, 2021 17:31
Copy link
Copy Markdown
Contributor Author

@nvb nvb left a comment

Choose a reason for hiding this comment

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

TFTRs! And thanks for opening #66828 Erik.

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @erikgrinaker)


pkg/kv/kvserver/replica_raftstorage.go, line 256 at r1 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

nit: isn't this already enforced by the iterator bounds?

I was also unsure, and have been in the past. I see this same pattern in various places that also pass an UpperBound to IterOptions, but I also don't know why it would be needed. I'll try to find out.


pkg/kv/kvserver/store_snapshot.go, line 395 at r1 (raw file):

Previously, tbg (Tobias Grieger) wrote…

This is mostly dead code and has been for multiple releases - we don't send any log entries with the snapshots any more:

usesReplicatedTruncatedState, err := storage.MVCCGetProto(
ctx, snap.EngineSnap, keys.RaftTruncatedStateLegacyKey(r.RangeID), hlc.Timestamp{}, nil, storage.MVCCGetOptions{},
)
if err != nil {
return errors.Wrap(err, "loading legacy truncated state")
}
canAvoidSendingLog := !usesReplicatedTruncatedState &&
snap.State.TruncatedState.Index < snap.State.RaftAppliedIndex

Also, as of 21.2 (i.e. master now), we actually know for sure it is dead code:

// TruncatedAndRangeAppliedStateMigration is part of the migration to stop
// using the legacy truncated state within KV. After the migration, we'll be
// using the unreplicated truncated state and the RangeAppliedState on all
// ranges. Callers that wish to assert on there no longer being any legacy
// will be able to do so after PostTruncatedAndRangeAppliedStateMigration is
// active. This lets remove any holdover code handling the possibility of
// replicated truncated state in 21.2.
//
// TODO(irfansharif): Do the above in 21.2.
TruncatedAndRangeAppliedStateMigration

I don't think this PR should rip it out (as we'll want to backport this one) but on master we shouldn't have to have this code around any longer at all.

Either way, this code is exceedingly unlikely to run in any adversarial scenarios because in reality it won't run at all - a range would've had to not have any log truncations for multiple releases.

I filed #66834 to track its removal.

Yeah, as you two noticed, the maxPreallocSize was useless because we never terminated iteration early. It was introduced in dda4bc9 for good reason but was made ineffectual in b2850e1. Since this code hasn't changed in over a year, this won't make the backport to v20.2 any more complicated, so I think it's fine to remove here.

Thanks for filing the issue Tobi.


pkg/storage/mvcc.go, line 2643 at r1 (raw file):

Previously, tbg (Tobias Grieger) wrote…

stops with no error propagated.

Done.

@nvb nvb force-pushed the nvanbenschoten/mvccIterStats2 branch from 903cecb to 8eebac2 Compare June 30, 2021 17:32
Fixes cockroachdb#66682.

In cockroachdb#66682, we noticed a poor interaction between Raft and the MVCC API.
When a Raft leader is catching up its followers, it often sends out
large chunks of log entries. We prevent this from being overly expensive
in the presence of large log entries by limiting the aggregate size of
these entries to 32 KB (`defaultRaftMaxSizePerMsg`) at a time. This is
accomplished through the through the `maxBytes` param that is passed
throughout the `raft.Storage` implementation.

What we noticed was that even though we configure a `maxBytes` of 32 KB,
no limit is passed through to `iterateEntries`'s call to `MVCCIterate`.
This seems harmless enough, because the iterator function passed to
`MVCCIterate` terminates once the byte limit is exceeded. However,
behind the scenes, `MVCCIterate` pulls in up to 1000 entries
(`maxKeysPerScan`) from Pebble at a time, with no accompanying byte
limit. So in a cluster with large log entries, where even just one is
enough to exceed our budget, this ends up being very expensive.

In the customer cluster that we were looking at when we found cockroachdb#66682,
each log entry was 131KB large. So each attempt to grab a new log entry
read 131MB from the LSM. This was likely all cached, but even just
unmarshaling this was expensive due to the memory copies doing so
performs. This was so expensive that each call to grab a single entry
took about 65ms. This was bad, and it became even worse when a follower
sent 100 MsgAppResps in response to the 100 MsgApps sent by the leader
(which is suboptimal but expected). If these all landed on the leader at
once, it would spend `100*65ms = 6.5s` processing messages in
`Store.processRequestQueue`. This stall was long enough for another
follower to call an election and for the leader to lose its leadership.
Since two replicas were possible leaders and both were in this
situation, it never resolved and leadership ping-ponged.

This commit fixes this issue by replacing the call to `MVCCIterate` in
`iterateEntries` with direct use of an MVCC iterator. This provides more
control over iteration and allows us to avoid the unwanted readahead
policy. It also allows us to avoid some of the unnecessary cruft of
`MVCCIterate`.

_### Rejected Alternatives

I briefly considered trying to adapt `MVCCIterate` to be more
configurable through the `MVCCScanOptions` it is provided, but that
seemed like a fraught task. This was primarily because the `TargetBytes`
config is not quite what we want in this case, where we need to be
precise about the sizes of these entries (see 9d46451) and don't want to
count the key size of the MVCCMetadata wrapper size.

I also considered simply adding some conservative, non-configurable
readahead byte limit to `MVCCIterate` to sit alongside its key limit.
This may still be a good idea. But I also think we may want to restrict
the cases where we even use the function.

Finally, I considered removing the readahead behavior in `MVCCIterate`
entirely. I think it might have been added to avoid repeat CGo hops back
when we had RocksDB. But now that we have Pebble and can manipulate an
iterator cheaply, it can maybe be rejected. However, this would then
require us to lift some logic from `pebbleMVCCScanner` into
`MVCCIterate`, which I wasn't keen on doing.

Release note (bug fix): Catching up Raft followers on the Raft log is
now more efficient in the presence of many large Raft log entries. This
helps avoid situations where Raft leaders struggle to retain leadership
while catching up their followers.
@nvb nvb force-pushed the nvanbenschoten/mvccIterStats2 branch from 8eebac2 to 69c934b Compare June 30, 2021 17:42
Copy link
Copy Markdown
Contributor Author

@nvb nvb left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @erikgrinaker and @tbg)


pkg/kv/kvserver/replica_raftstorage.go, line 256 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

I was also unsure, and have been in the past. I see this same pattern in various places that also pass an UpperBound to IterOptions, but I also don't know why it would be needed. I'll try to find out.

It does sound like this is unnecessary. Removed.

@nvb
Copy link
Copy Markdown
Contributor Author

nvb commented Jun 30, 2021

bors r+

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jul 1, 2021

Build succeeded:

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.

kvserver: limit MVCC scan in Raft Entries retrieval

5 participants