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: aggregate iterator stats #99726

Merged
merged 2 commits into from
Apr 3, 2023

Conversation

jbowens
Copy link
Collaborator

@jbowens jbowens commented Mar 27, 2023

Aggregate the iterator stats across all of an engine's iterators. Expose seven
new timeseries metrics for visibility into the behavior of storage engine iterators:

  • storage.iterator.block-load.bytes
  • storage.iterator.block-load.cached-bytes
  • storage.iterator.block-load.read-duration
  • storage.iterator.external.seeks
  • storage.iterator.external.steps
  • storage.iterator.internal.seeks
  • storage.iterator.internal.steps

Close #95790.
Epic: None
Release note (ops change): Introduces seven new timeseries metrics for better
visibility into the behavior of storage engine iterators and their internals.

@jbowens jbowens added the backport-23.1.x Flags PRs that need to be backported to 23.1 label Mar 27, 2023
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@jbowens jbowens force-pushed the aggregate-iterator-stats branch 2 times, most recently from a9f4953 to d725ca9 Compare March 28, 2023 14:57
@jbowens jbowens marked this pull request as ready for review March 28, 2023 15:27
@jbowens jbowens requested a review from a team March 28, 2023 15:27
@jbowens jbowens requested review from a team as code owners March 28, 2023 15:27
@jbowens jbowens requested review from bananabrick and tbg March 28, 2023 15:27
Copy link
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.

Looks good! Had some comments, the most significant ones are

  • lock contention, which we might be catching a lot of here, and
  • whether these metrics should all be counters.

An additional expert review on the smaller code changes when passing the stats around wouldn't hurt, though it looked straightforward enough.

@@ -2421,6 +2470,13 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics {
RdbLevelScore: rdbLevelScore,
RdbWriteStalls: metric.NewGauge(metaRdbWriteStalls),
RdbWriteStallNanos: metric.NewGauge(metaRdbWriteStallNanos),
IterBlockBytes: metric.NewGauge(metaBlockBytes),
Copy link
Member

Choose a reason for hiding this comment

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

I was expecting them all to be counters. They are all gauges. What's going on?


// AggregatedIteratorStats holds cumulative stats, collected and summed over all
// of an engine's iterators.
type AggregatedIteratorStats struct {
Copy link
Member

Choose a reason for hiding this comment

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

It would be helpful to reference this type in the help text for each of the metrics as a pointer for more detailed semantics of what's tracked within each.

// directions performed on pebble.Iterators.
ExternalSteps int
// InternalSeeks is the total count of steps in forward and backward
// directions performed on Pebble's internal iterator. If this is high
Copy link
Member

Choose a reason for hiding this comment

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

"If this is high..." could you copy this to the metric help text?

// Nexts, Prevs, NextPrefix, etc) in forward and backward directions
// performed on pebble's internal iterator. If this is high relative to
// ExternalSteps, it's a good indication that there's an accumulation of
// garbage within the LSM (NOT MVCC garbage).
Copy link
Member

Choose a reason for hiding this comment

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

ditto about "If this is high..."

// reportStats is a function that should be invoked when the iterator is
// closed or its stats reset. It's intended to be used with
// Pebble.aggregateIterStats to sum iterator stats across all the iterators
// during the lifetime of the Engine.
Copy link
Member

Choose a reason for hiding this comment

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

and it can't be nil, right?

Copy link
Collaborator Author

@jbowens jbowens left a comment

Choose a reason for hiding this comment

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

lock contention, which we might be catching a lot of here, and
whether these metrics should all be counters.

Yeah, these two are related. I believe using a mutex without a standard metric.Counter (which internally uses atomic intrinsics) will be faster and have less contention. The critical section here is so tiny that the cost of synchronization itself should dominate.

A single atomic increment intrinsic is faster than a mutex lock, add, and mutex unlock, but my understanding is that atomic intrinsics pay the cost of synchronization (eg, write barriers, cache line invalidation) per increment. Since we have seven counters to increment, the mutex overhead gets amortized over the seven increments.

eg, running this benchmark on my gceworker:

goos: linux
goarch: amd64
pkg: github.com/mutex
cpu: Intel(R) Xeon(R) CPU @ 2.30GHz
BenchmarkInc
BenchmarkInc/atomic
BenchmarkInc/atomic/parallelism=1
BenchmarkInc/atomic/parallelism=1-24         	25120298	        47.57 ns/op
BenchmarkInc/atomic/parallelism=2
BenchmarkInc/atomic/parallelism=2-24         	 4960356	       255.0 ns/op
BenchmarkInc/atomic/parallelism=4
BenchmarkInc/atomic/parallelism=4-24         	 2159403	       554.2 ns/op
BenchmarkInc/atomic/parallelism=8
BenchmarkInc/atomic/parallelism=8-24         	 1000000	      1140 ns/op
BenchmarkInc/atomic/parallelism=16
BenchmarkInc/atomic/parallelism=16-24        	  476132	      2577 ns/op
BenchmarkInc/atomic/parallelism=32
BenchmarkInc/atomic/parallelism=32-24        	  229970	      5171 ns/op
BenchmarkInc/atomic/parallelism=64
BenchmarkInc/atomic/parallelism=64-24        	  117676	     11069 ns/op
BenchmarkInc/mutex
BenchmarkInc/mutex/parallelism=1
BenchmarkInc/mutex/parallelism=1-24          	49485908	        24.13 ns/op
BenchmarkInc/mutex/parallelism=2
BenchmarkInc/mutex/parallelism=2-24          	19753113	        62.67 ns/op
BenchmarkInc/mutex/parallelism=4
BenchmarkInc/mutex/parallelism=4-24          	 5321437	       229.2 ns/op
BenchmarkInc/mutex/parallelism=8
BenchmarkInc/mutex/parallelism=8-24          	 1689697	       701.5 ns/op
BenchmarkInc/mutex/parallelism=16
BenchmarkInc/mutex/parallelism=16-24         	  693480	      1733 ns/op
BenchmarkInc/mutex/parallelism=32
BenchmarkInc/mutex/parallelism=32-24         	  313561	      4045 ns/op
BenchmarkInc/mutex/parallelism=64
BenchmarkInc/mutex/parallelism=64-24         	  144046	      8279 ns/op

Also, today all metrics returned from Engine.GetMetrics() are gauges because all the metrics reported by Pebble are cumulative, not deltas, even if internally they're monotonically increasing counters.

I goofed and wasn't propagating the metrics in Engine.GetMetrics(), so I made that change too.

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


pkg/storage/engine.go line 1047 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

It would be helpful to reference this type in the help text for each of the metrics as a pointer for more detailed semantics of what's tracked within each.

Done.


pkg/storage/engine.go line 1074 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

"If this is high..." could you copy this to the metric help text?

Done.


pkg/storage/engine.go line 1082 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

ditto about "If this is high..."

Done.


pkg/storage/pebble_iterator.go line 52 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

and it can't be nil, right?

Done.

@nicktrav
Copy link
Collaborator

nicktrav commented Mar 28, 2023

Thoughts on adding some / all of these metrics to the telemetry event pipeline? Would they be helpful for observing long term trends or identifying areas for investigation / improvement?

Could certainly be added in a follow-up.

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

eg, running this benchmark on my gceworker:

I dug a little deeper on this benchmark. I don't think a parallelism of 64 is sufficient - since this is in the read hot path, we have to think about 1000s of ops/sec, so I extended to high concurrencies (and thinned it out a bit).
Second, I think the benchmark was doing it wrong? It was running parallelism*b.N operations but the benchmark harness assumed it was b.N. So we'd really have to divide all reported ops/sec numbers by parallelism. I cleaned that all up a bit (using b.RunParallel in the benchmark and running only b.N ops each). I also added a sharded mutex (GOMAXPROCS shards). The resulting numbers (gceworker) are... surprising, the standard mutex actually beats the other options and actually gets ... better? It's almost as though queuing up behind a single mutex is more efficient than shorter queues across multiple mutexes. I wonder if there's something smart going on in the scheduler, or something dumb in my code.

image.png

https://github.com/tbg/goplay/blob/master/mutex/mutex_test.go
https://docs.google.com/spreadsheets/d/1B9g1ZfzjhpZefJr9RlZWKJ3XvNiokHW10M8q-Eb7--A/edit?usp=sharing

I suppose if this new mutex does cause an issue it is best observed in roachperf rather than embarking on discussion too focused on microbenchmarks.

Reviewed 5 of 5 files at r1.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @bananabrick and @jbowens)


pkg/kv/kvserver/metrics.go line 2473 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

I was expecting them all to be counters. They are all gauges. What's going on?

Could you add a comment that these are cumulative values, but tracked as gauges because the cumulative counters mostly live in pebble (though this isn't always true, it's more convenient to stick to the pattern throughout), at the top of this block (i.e. not on each metric)?

I also wonder (not for this PR) if we can't use counters here regardless. A gauge and a counter get different prometheus metric types, and there are consequences of this type. For example, plotting a counter on grafana, it will remind you that this is a counter and propose looking at the rate(). Having a gauge that is really a counter is unintuitive.

I think the only reason we're using gauges here is because we don't have a .Update method on Counter. I think this could be added and all of these metrics ported. Filed #99922, perhaps mention it in the comment requested above.

@@ -1338,13 +1357,13 @@ func (p *Pebble) NewMVCCIterator(iterKind MVCCIterKind, opts IterOptions) MVCCIt
return maybeWrapInUnsafeIter(iter)
}

iter := newPebbleIterator(p.db, opts, StandardDurability)
iter := newPebbleIterator(p.db, opts, StandardDurability, p.aggregateIterStats)
Copy link
Member

Choose a reason for hiding this comment

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

Drive-by comment: does passing this method reference to newPebbleIterator result in a new heap allocation? If so, consider replacing func(IteratorStats) with an interface that is implemented by Pebble. p *Pebble is already on the heap, so that approach would not heap allocation.

Copy link
Member

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

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

Not that SetParallelism sets the number of goroutines used by RunParallel to p*GOMAXPROCS so the parallelism you are looking at is way too high

I have a feeling that the atomic would fare relatively worse if the values were not packed close together in 1-2 cachelines.

It might be more intuitive to show a graph of throughput (total number of ops per second). That one should grow ~linearly up to the number of CPUs then level off and start going down a bit. I think it would also show that the number is so much higher than what we're actually doing in CRDB that this is mostly academic 😂

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @bananabrick, @jbowens, and @tbg)

Copy link
Collaborator Author

@jbowens jbowens left a comment

Choose a reason for hiding this comment

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

It was running parallelism*b.N operations but the benchmark harness assumed it was b.N. So we'd really have to divide all reported ops/sec numbers by parallelism

Makes sense, but it doesn't impact the relative numbers between mutex and atomics.

I don't think a parallelism of 64 is sufficient - since this is in the read hot path, we have to think about 1000s of ops/sec,

There would be need to millions of iterators closing in parallel to see 1000s of iterators aggregating stats in parallel. Since the critical section is tiny, my expectation is that higher parallelism beyond the expected number of CPU threads won't matter. The probability of any individual aggregateIterStats call being context switched away while in the critical section is incredibly small, and IIRC the golang mutex has an optimistic spin-loop fast path for very short critical sections like this.

I think we see the effect of the cache line invalidation on every individual atomic increment. At the extreme parallelism counts, every increment can expect a cache miss. In contrast, the mutex suffers the cost of the write barrier and cache miss once, on mutex acquisition.

FWIW, iterator construction also acquires an Engine-wide RWMutex in order to acquire the current version of the LSM. We've never seen contention there. (We typically see it on the block cache shards, which see much higher ops/sec and are sharded 2*num procs: cockroachdb/pebble#1997).

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @bananabrick, @nvanbenschoten, and @tbg)


pkg/kv/kvserver/metrics.go line 2473 at r2 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Could you add a comment that these are cumulative values, but tracked as gauges because the cumulative counters mostly live in pebble (though this isn't always true, it's more convenient to stick to the pattern throughout), at the top of this block (i.e. not on each metric)?

I also wonder (not for this PR) if we can't use counters here regardless. A gauge and a counter get different prometheus metric types, and there are consequences of this type. For example, plotting a counter on grafana, it will remind you that this is a counter and propose looking at the rate(). Having a gauge that is really a counter is unintuitive.

I think the only reason we're using gauges here is because we don't have a .Update method on Counter. I think this could be added and all of these metrics ported. Filed #99922, perhaps mention it in the comment requested above.

Added a comment here, and updated the existing comment above the struct comments to reference #99922.


pkg/storage/pebble.go line 1360 at r4 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Drive-by comment: does passing this method reference to newPebbleIterator result in a new heap allocation? If so, consider replacing func(IteratorStats) with an interface that is implemented by Pebble. p *Pebble is already on the heap, so that approach would not heap allocation.

Good catch, it does. Fixed.

@tbg

This comment was marked as outdated.

@tbg
Copy link
Member

tbg commented Mar 29, 2023

^-- hiding the above, @RaduBerinde pointed out the numbers are still garbage - it's hard to get something useful out of the go bench harness, @jbowens' initial attempt was still the best shot. I went looking upstream too but they only have a very basic mutex benchmark. I suppose it doesn't matter, since we're confident there isn't going to be noticeable contention. Sorry about the noise!

Copy link
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 (but note the eng-wide ask to not bors anything right now)

Reviewed 10 of 10 files at r5, 11 of 11 files at r6, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @bananabrick and @nvanbenschoten)

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:

Reviewed 1 of 5 files at r1, 8 of 10 files at r5, 11 of 11 files at r6, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @bananabrick and @nvanbenschoten)

@jbowens
Copy link
Collaborator Author

jbowens commented Mar 31, 2023

TFTRs!

bors r+

@craig
Copy link
Contributor

craig bot commented Mar 31, 2023

Build failed (retrying...):

@craig
Copy link
Contributor

craig bot commented Mar 31, 2023

Build failed (retrying...):

@craig
Copy link
Contributor

craig bot commented Mar 31, 2023

Build failed (retrying...):

@craig
Copy link
Contributor

craig bot commented Mar 31, 2023

Build failed (retrying...):

@craig
Copy link
Contributor

craig bot commented Apr 1, 2023

Build failed:

Previously, the EngineIterator interface exposed a GetRawIter method to
retrieve the underlying Pebble iterator. This existed only to facilitate
iterator clones and was intended to only ever be consumed by the storage
package itself.

This commit replaces GetRawIter with a CloneContext method that returns an
opaque CloneContext type that contains the raw iterator. This prevents external
packages from directly using the raw Pebble iterator. It also prepares for the
introduction of aggregation of iterator stats, providing an envelope for
propagating information on where to aggregate iterator stats when the clonee
iterator closes.

Epic: None
Release note: None
Aggregate the iterator stats across all of an engine's iterators. Expose seven
new timeseries metrics for visibility into the behavior of storage engine iterators:

  - storage.iterator.block-load.bytes
  - storage.iterator.block-load.cached-bytes
  - storage.iterator.block-load.read-duration
  - storage.iterator.external.seeks
  - storage.iterator.external.steps
  - storage.iterator.internal.seeks
  - storage.iterator.internal.steps

Close cockroachdb#95790.
Epic: None
Release note (ops change): Introduces seven new timeseries metrics for better
visibility into the behavior of storage engine iterators and their internals.
@jbowens
Copy link
Collaborator Author

jbowens commented Apr 2, 2023

bors r+

@craig
Copy link
Contributor

craig bot commented Apr 2, 2023

Build failed:

@jbowens

This comment was marked as duplicate.

@craig
Copy link
Contributor

craig bot commented Apr 3, 2023

Build failed:

@jbowens

This comment was marked as duplicate.

@craig
Copy link
Contributor

craig bot commented Apr 3, 2023

Build failed:

@jbowens
Copy link
Collaborator Author

jbowens commented Apr 3, 2023

bors retry

@craig
Copy link
Contributor

craig bot commented Apr 3, 2023

Build succeeded:

@craig craig bot merged commit bebd2dd into cockroachdb:master Apr 3, 2023
@jbowens jbowens deleted the aggregate-iterator-stats branch April 3, 2023 15:52
@@ -223,11 +223,11 @@ func BenchmarkMVCCGet_Pebble(b *testing.B) {
ctx := context.Background()
for _, batch := range []bool{false, true} {
b.Run(fmt.Sprintf("batch=%t", batch), func(b *testing.B) {
for _, numVersions := range []int{1, 10, 100} {
for _, numVersions := range []int{10} {
Copy link
Member

Choose a reason for hiding this comment

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

@jbowens did you intend to merge this change to the benchmark?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
backport-23.1.x Flags PRs that need to be backported to 23.1
Projects
None yet
Development

Successfully merging this pull request may close these issues.

storage: consider aggregating iterator stats
7 participants