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: add limits to skipped data iteration #68467

Merged
merged 3 commits into from Oct 5, 2021

Conversation

aliher1911
Copy link
Contributor

@aliher1911 aliher1911 commented Aug 5, 2021

Previously when iterating engine using MVCCIncrementalIterator caller
could skip large amounts of non-matching data which would result in
"unbounded" resource usage.
This is becoming a problem for resource constrained clusters where low
priority requests like export that are used by backups to interfere with
high priority workloads. If we want to throttle backups we need to be able
to limit how many underlying operations we want to perform per request.
This change adds an optional iteration limit to the iterator. Once the
limit is reached, iterator will return an error. Error will provide a
resume key to continue iteration in next request.

Release note: None

Fixes #68234

@aliher1911 aliher1911 requested a review from a team as a code owner August 5, 2021 12:25
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@aliher1911
Copy link
Contributor Author

aliher1911 commented Aug 5, 2021

This PR needs #68102 to land first because it would only work once we have an ability to stop on arbitrary key mid export.

I will update this one once it is ready. Regardless of its readiness, this functionality is distinct and could be reviewed independently and I don't want to put it in the stack of changes in previous PR.

Upd: rebased over necessary change on master already.

@aliher1911 aliher1911 requested review from a team and dt and removed request for a team August 5, 2021 12:33
@aliher1911 aliher1911 force-pushed the add_mvcc_iterator_limits branch 3 times, most recently from 6fffb7e to b513076 Compare August 6, 2021 09:52
@aliher1911 aliher1911 force-pushed the add_mvcc_iterator_limits branch 3 times, most recently from 7bf1107 to e18e0ff Compare August 13, 2021 13:20
@aliher1911 aliher1911 requested a review from a team August 13, 2021 13:20
@aliher1911 aliher1911 requested a review from a team as a code owner August 13, 2021 13:20
@aliher1911
Copy link
Contributor Author

@dt David do you think having an iteration limit should be a tenant/caller responsibility or a cluster responsibility? We have similar limit maxIntentCount which limits how many intents could be collected during export to have some protection for memory overuse and it is defined as a cluster setting because it protects servers from oom-ing. For iteration limits that throttle export resource usage on different dimension, should we stick to the same approach or rather let caller pass the limit explicitly? IIRC we wanted to give cpu constrained clients an ability to spread export requests, but it looks like it is more of a cluster level rather than request level limit.
I added it as export method argument, but having second thoughts before drilling it through export request api.

@aliher1911 aliher1911 force-pushed the add_mvcc_iterator_limits branch 3 times, most recently from 46bcac8 to 1edc5dd Compare August 16, 2021 11:50
Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewed 5 of 10 files at r2, 5 of 5 files at r4, 1 of 5 files at r5.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @aliher1911, @dt, and @sumeerbhola)


pkg/storage/engine.go, line 362 at r4 (raw file):

// ExportOptions contains options provided to export operation.
type ExportOptions struct {
	// StartKey determines the start of exported interval (inclusive).

nit: ... start of the exported ...


pkg/storage/engine.go, line 363 at r4 (raw file):

type ExportOptions struct {
	// StartKey determines the start of exported interval (inclusive).
	// StartKey.Timestamp is either empty which represent starting from potential

nit: represents starting from a potential ...


pkg/storage/engine.go, line 365 at r4 (raw file):

	// StartKey.Timestamp is either empty which represent starting from potential
	// intent and continuing to versions or non-empty, which represents starting
	// from particular version.

nit: from a particular version.


pkg/storage/engine.go, line 367 at r4 (raw file):

	// from particular version.
	StartKey MVCCKey
	// EndKey determines end of exported interval (exclusive).

nit: determines the end ...


pkg/storage/engine.go, line 369 at r4 (raw file):

	// EndKey determines end of exported interval (exclusive).
	EndKey roachpb.Key
	// StartTS and EndTS determine exported time range as (startTS, endTS]

nit: missing period.


pkg/storage/engine.go, line 387 at r4 (raw file):

	// If StopMidKey is false, once function reaches targetSize it would continue
	// adding all versions until it reaches next key or end of range. If true, it
	// would stop immediately when targetSize is reached and return a next versions

nit: ... the next versions ...


pkg/storage/mvcc_incremental_iterator.go, line 130 at r5 (raw file):

	// Resume key is not necessarily a valid iteration key as we could stop in between
	// eligible keys.
	MaxAllowedIterations int64

I'm wary of introducing resource control settings that cannot be easily understood in terms of real resources.
maxIntentCount was not quite a real resource but could be understood as a limit on the return size.

Ideally, we would want to limit the cpu time and IO time spent in executing an operation that scans data. This would also fit in well with the current admission control which functions better when the execution of a BatchRequest has a bounded size. But we don't have cpu information from golang and I can't see when we we would have IO time information either.

But I think this could still be something more tangible that we can use not just for exports, but also for other scans and gets (requests for non-background operations could use a higher limit). We have some counts in pebble.IteratorStats but they have multiple dimensions like step/seek, forward/reverse and it would be nicer to have a single dimension that was more relatable. I am thinking number of ssblock bytes "read" (we'd count all the bytes for an ssblock when loading it into the iterator) would be a good metric. If the setting puts a 100MB limit on it, it means something real. We don't expose this value via IteratorStats, or track it in the low-level sstable iterators but it can be added. One downside is that it does not count the bytes iterated in the memtable -- I don't think that matters in production settings at all since the memtable is tiny relative to the rest of the store.
@jbowens @itsbilal for other opinions.

@aliher1911 aliher1911 requested a review from a team as a code owner August 17, 2021 10:58
Copy link
Member

@dt dt 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 (waiting on @aliher1911, @itsbilal, @jbowens, and @sumeerbhola)


pkg/storage/mvcc_incremental_iterator.go, line 130 at r5 (raw file):

Previously, sumeerbhola wrote…

I'm wary of introducing resource control settings that cannot be easily understood in terms of real resources.
maxIntentCount was not quite a real resource but could be understood as a limit on the return size.

Ideally, we would want to limit the cpu time and IO time spent in executing an operation that scans data. This would also fit in well with the current admission control which functions better when the execution of a BatchRequest has a bounded size. But we don't have cpu information from golang and I can't see when we we would have IO time information either.

But I think this could still be something more tangible that we can use not just for exports, but also for other scans and gets (requests for non-background operations could use a higher limit). We have some counts in pebble.IteratorStats but they have multiple dimensions like step/seek, forward/reverse and it would be nicer to have a single dimension that was more relatable. I am thinking number of ssblock bytes "read" (we'd count all the bytes for an ssblock when loading it into the iterator) would be a good metric. If the setting puts a 100MB limit on it, it means something real. We don't expose this value via IteratorStats, or track it in the low-level sstable iterators but it can be added. One downside is that it does not count the bytes iterated in the memtable -- I don't think that matters in production settings at all since the memtable is tiny relative to the rest of the store.
@jbowens @itsbilal for other opinions.

I think like the idea of limiting on block-bytes loaded, but as far as a KV-level work limit, I think capping the number of keys compared for inclusion in the returned result seems reasonable; we already have limits on number of keys returned by KV, so limiting number of keys that we will examined by KV for return, even if they aren't ultimately returned, doesn't seem like too much of a conceptual stretch?

Ultimately I think both limits would be desirable -- limiting block bytes loaded helps cap the IO and storage CPU footprint of the request, but if for example we're in some very compressed, valueless index-key blocks, I could see wanting to limit the number of iterations, not just loaded block bytes, to also limit KV's cpu time too.

Copy link
Contributor Author

@aliher1911 aliher1911 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 (waiting on @aliher1911, @itsbilal, @jbowens, and @sumeerbhola)


pkg/storage/mvcc_incremental_iterator.go, line 130 at r5 (raw file):

Previously, dt (David Taylor) wrote…

I think like the idea of limiting on block-bytes loaded, but as far as a KV-level work limit, I think capping the number of keys compared for inclusion in the returned result seems reasonable; we already have limits on number of keys returned by KV, so limiting number of keys that we will examined by KV for return, even if they aren't ultimately returned, doesn't seem like too much of a conceptual stretch?

Ultimately I think both limits would be desirable -- limiting block bytes loaded helps cap the IO and storage CPU footprint of the request, but if for example we're in some very compressed, valueless index-key blocks, I could see wanting to limit the number of iterations, not just loaded block bytes, to also limit KV's cpu time too.

My 2p on what I see our goal here.
We want to limit resources used by export requests to minimize impact on higher priority requests. The most obvious resources are CPU and IO in this case, but also a time that we held range locks. I'm not sure what is the memory impact at the moment. Depending on range content we could have many small values that we need to go through which could incur extra CPU load when skipping data or large payloads where we can hit higher IO use.

We also need a way to expose that to MVCCIncrementalIterator which is currently agnostic of underlying storage and delegates work to storage iterators.
Current approach with just counting how many times we stepped avoid adding extra methods to iterators, but it is a bit ugly and only acting as a proxy of underlying complexity.

A better approach may be to get a "counters" object when constructing iterator in reader.NewMVCCIterator() and then just ask that object if we reached target?

The second aspect that I don't particularly like is that we explicitly pass limits which couples caller with iterator while a better way could be to have some reading "profile" that would be configured separately and could have as many resource limits as we currently have so that we don't have to change the whole call stack if we find a way to expose something more useful.

Copy link
Collaborator

@sumeerbhola sumeerbhola 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 (waiting on @itsbilal, @jbowens, and @sumeerbhola)


pkg/storage/mvcc_incremental_iterator.go, line 130 at r5 (raw file):

The most obvious resources are CPU and IO in this case, but also a time that we held range locks.

How about we measure walltime. Unless the system is very overloaded, it should be a good proxy for cpu+io time.

The second aspect that I don't particularly like is that we explicitly pass limits which couples caller with iterator while a better way could be to have some reading "profile" that would be configured separately and could have as many resource limits as we currently have so that we don't have to change the whole call stack if we find a way to expose something more useful.

This seems worthwhile. We could add something like a ResourceTimeLimiter struct, which for now would only do walltime, and pass it through MVCCIncrementalIteratOptions/MVCCScanOptions/MVCCGetOptions (like we do with the mon.BoundAccount in the latter two). This would be called every N loop iterations to amortize the cost of fetching the "resource time", to check if some resource time had been exceeded. Later we could add other dimensions to the ResourceTimeLimiter.

@aliher1911 aliher1911 removed the request for review from a team August 24, 2021 13:30
@aliher1911 aliher1911 force-pushed the add_mvcc_iterator_limits branch 2 times, most recently from 21365be to bfceb08 Compare September 1, 2021 10:11
Copy link
Contributor Author

@aliher1911 aliher1911 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 (waiting on @itsbilal, @jbowens, and @sumeerbhola)


pkg/storage/mvcc_incremental_iterator.go, line 130 at r5 (raw file):

Previously, sumeerbhola wrote…

The most obvious resources are CPU and IO in this case, but also a time that we held range locks.

How about we measure walltime. Unless the system is very overloaded, it should be a good proxy for cpu+io time.

The second aspect that I don't particularly like is that we explicitly pass limits which couples caller with iterator while a better way could be to have some reading "profile" that would be configured separately and could have as many resource limits as we currently have so that we don't have to change the whole call stack if we find a way to expose something more useful.

This seems worthwhile. We could add something like a ResourceTimeLimiter struct, which for now would only do walltime, and pass it through MVCCIncrementalIteratOptions/MVCCScanOptions/MVCCGetOptions (like we do with the mon.BoundAccount in the latter two). This would be called every N loop iterations to amortize the cost of fetching the "resource time", to check if some resource time had been exceeded. Later we could add other dimensions to the ResourceTimeLimiter.

I gave it a go and it looks ok I think. There's still an issue with ensuring that we always advance, but it is currently mitigated by throttling checks so that we always move N times before first check is done.
Maybe we could pass current key to Exhausted() so that it could check or have this check in iterator itself which would require saving start key and comparing current to start once we hit the limit.
If there would be other checks, limiter might need a view of underlying iterator stats injected or the whole creation sequence turned upside down.

@aliher1911 aliher1911 force-pushed the add_mvcc_iterator_limits branch 2 times, most recently from 0b3478e to 5c1e92b Compare September 14, 2021 15:28
Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewed 3 of 11 files at r10, 2 of 7 files at r12, 1 of 532 files at r14, 2 of 529 files at r15, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @aliher1911, @dt, @itsbilal, @jbowens, and @sumeerbhola)


pkg/storage/mvcc_incremental_iterator.go, line 41 at r15 (raw file):

//
// If iteration resource limit is requested, iterator would return an error as
// soon as limit is reached. The error will contain a resume key which could be

why would this be outside the requested span?


pkg/storage/mvcc_incremental_iterator.go, line 132 at r15 (raw file):

// ClockCheckEveryNIterations defines for how many iterations we could cache
// current time when performing iteration wall clock time limiting.
const ClockCheckEveryNIterations = 100

can this be package private?


pkg/storage/mvcc_incremental_iterator.go, line 240 at r15 (raw file):

	// StartKey must also be populated with resume key. This is needed to ensure progress
	// for cases when initial seek would exhaust resources and that subsequent call would
	// restart from further position.

I didn't understand this comment. Is this trying to say that we don't want to stop until we are past startKey? Given this is a SimpleMVCCIterator the first call on this needs to be SeekGE. We can keep a notFirstCallToSeekGE bool in MVCCIncrementalIterator and and pass it to advance().

Something like

func (...) SeekGE(...) {
   ...
   ignoreLimiter := !i.notFirstCallToSeekGE
   i.notFirstCallToSeekGE = false
   i.advance(ignoreLimiter)
}

I think this is better than doing additional key comparisons, and doesn't require additional state in options.


pkg/storage/mvcc_incremental_iterator.go, line 242 at r15 (raw file):

	// restart from further position.
	// Note that resume key is not necessarily a valid iteration key as we could stop in
	// between eligible keys because of timestamp range limits.

because of resource limits.

@tbg tbg removed the request for review from a team September 15, 2021 08:19
@aliher1911 aliher1911 force-pushed the add_mvcc_iterator_limits branch 3 times, most recently from ac7f201 to b25cbef Compare September 15, 2021 17:59
Copy link
Contributor Author

@aliher1911 aliher1911 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 (waiting on @dt, @itsbilal, @jbowens, and @sumeerbhola)


pkg/storage/mvcc_incremental_iterator.go, line 41 at r15 (raw file):

Previously, sumeerbhola wrote…

why would this be outside the requested span?

If we don't use TBI then when we do Next on underlying iter it may move to the mvcckey that is out of bounds for iterator. We then do advance which will start iterating until it hits next valid mvcckey, but resource limiter could cut it short so we end up with the key that is not eligible for inclusion into results.
With TBI it should always be within limits I think.


pkg/storage/mvcc_incremental_iterator.go, line 132 at r15 (raw file):

Previously, sumeerbhola wrote…

can this be package private?

Done.


pkg/storage/mvcc_incremental_iterator.go, line 240 at r15 (raw file):

Previously, sumeerbhola wrote…

I didn't understand this comment. Is this trying to say that we don't want to stop until we are past startKey? Given this is a SimpleMVCCIterator the first call on this needs to be SeekGE. We can keep a notFirstCallToSeekGE bool in MVCCIncrementalIterator and and pass it to advance().

Something like

func (...) SeekGE(...) {
   ...
   ignoreLimiter := !i.notFirstCallToSeekGE
   i.notFirstCallToSeekGE = false
   i.advance(ignoreLimiter)
}

I think this is better than doing additional key comparisons, and doesn't require additional state in options.

I like that, it looks cleaner at least on the outside.


pkg/storage/mvcc_incremental_iterator.go, line 242 at r15 (raw file):

Previously, sumeerbhola wrote…

because of resource limits.

Done.

Copy link
Contributor Author

@aliher1911 aliher1911 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 (waiting on @dt, @itsbilal, @jbowens, and @sumeerbhola)


pkg/storage/mvcc_incremental_iterator.go, line 41 at r15 (raw file):

Previously, aliher1911 (Oleg) wrote…

If we don't use TBI then when we do Next on underlying iter it may move to the mvcckey that is out of bounds for iterator. We then do advance which will start iterating until it hits next valid mvcckey, but resource limiter could cut it short so we end up with the key that is not eligible for inclusion into results.
With TBI it should always be within limits I think.

I was thinking about it and it could be fixed by adding checks before error is raised and adjusting the key in the following way - if timestamp is lower than requested, move to the next key and use latest requested timestamp, if timestamp is higher, move to the highest requested without changing key, if the key moved outside of requested range as a result of adjustment, don't err and finish gracefully.
That would duplicate the logic we have within the loop. Do you think this added complexity is justified for consistency?

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewed all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @aliher1911, @dt, @itsbilal, @jbowens, and @sumeerbhola)


pkg/storage/mvcc_incremental_iterator.go, line 41 at r15 (raw file):

If we don't use TBI then when we do Next on underlying iter it may move to the mvcckey that is out of bounds for iterator.

We set an upper bound for the iter, yes? So I am not sure why that would happen.

We then do advance which will start iterating until it hits next valid mvcckey, but resource limiter could cut it short so we end up with the key that is not eligible for inclusion into results.

But I think I agree that the general structure of this change is error prone. The MVCCIncrementalIterator is maintaining some invariants that are true at the end of each seek and next* function. By checking for resource limits in the middle of these functions after some work is done, we are inviting complexity regarding those invariants. An easy fix would be lift the resource limit checking to the start of each seek and next* function and avoid doing the checking for the first seek, to ensure forward progress.
That brings me to a related concern: the code in pebble.go isn't necessarily trivial since it has to deal with the fact that MVCCIncrementalIterator can complain about resource limits at any call to next*. There are 2 additional users of MVCCIncrementalIterator in `catchup_scan.go and mvcc.go. The former is probably not one that will use resource limits, but the latter eventually could. Can you look at the code to see if it would be simpler to lift the resource limit checking into the callers? With the current approach we can't avoid modifying the callsites, so why not give them full control. That way if they don't want the resource checking to happen before some particular next call since it is harder to maintain some resumption invariant, they can do so.

Let me know if you want to discuss synchronously.

@aliher1911 aliher1911 force-pushed the add_mvcc_iterator_limits branch 6 times, most recently from aa30b60 to 0d78564 Compare September 29, 2021 15:07
@aliher1911
Copy link
Contributor Author

Summarizing: we had an offline conversation with @sumeerbhola and he made fair points that time bound iterator should reduce severity of effects when we need to skip over large amounts of data. Without such requirement, code that tracks resources could be pulled out on level above so that export itself could check how long we iterate and stop if needed.
This should be good enough until we see any evidence that we are really blocked by iterator making slow Next() calls.

Second consideration is that mvcc_incremental_iterator maintains some invariants regarding its state and what it returns and breaking out of loop violates them. Having such loosely defined component is not good and it would impact planned work to rewrite it in near future.

Based on that discussion I made changes to pull code out to pebble part.

@sumeerbhola
Copy link
Collaborator

Thanks for summarizing. Adding to the considerations in the previous comment: the clients of MVCCIncrementalIterator are also not always able to stop and resume at arbitrary points in the iteration. e.g. if stopMidKey is false in pebbleExportToSst. By lifting the logic up into the client, they can take into account their own stopping constraints.

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Generally looks good. A few minor comments

pkg/storage/engine.go Outdated Show resolved Hide resolved
pkg/storage/resource_limiter.go Show resolved Hide resolved
pkg/storage/resource_limiter_test.go Outdated Show resolved Hide resolved
pkg/storage/pebble.go Show resolved Hide resolved
Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

lgtm

pkg/storage/pebble.go Show resolved Hide resolved
Number of arguments to ExportMVCCToSst is too large. This diff moves
them into a struct to improve readability.

Release note: None

Release justification:
Previously when exporting data from pebble iterator could spend unbouded
time skipping entries regardless of export size limits.
This is becoming a problem for resource constrained clusters where low
priority requests like export that are used by backups to interfere with
high priority workloads. If we want to throttle backups we need to be able
to limit how many underlying operations we want to perform per request.
This change adds an optional iteration limit to export. Once the
limit is reached, export will end its current chunk and return a resume
span even if desired size is not reached.
Current limiter uses wall clock time to stop interation.

Release note: None
Export requests could iterate unbounded amount of data in storage
this diff adds kv.bulk_sst.max_request_time hidden cluster setting
to limit how for long export could run irrespective of how much data
is actually exported.

Release note: None
@aliher1911
Copy link
Contributor Author

bors r=sumeerbhola

@craig
Copy link
Contributor

craig bot commented Oct 5, 2021

Build succeeded:

@craig craig bot merged commit cf4fe62 into cockroachdb:master Oct 5, 2021
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.

Expose number of keys mvcc_incremental_iterator skips over for better resource accounting
4 participants