obsservice: export statement insights to obs service#114022
obsservice: export statement insights to obs service#114022craig[bot] merged 1 commit intocockroachdb:masterfrom
Conversation
02df3a4 to
3b52445
Compare
e197e43 to
d954ec7
Compare
d954ec7 to
d09f5e3
Compare
abarganier
left a comment
There was a problem hiding this comment.
This has been an awesome exercise to learn more about how the insights feature works! I wonder if we can export continuously instead of at the flush interval 🤔 let me know what you think. Nice work!
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @maryliag and @xinhaoz)
pkg/sql/sqlstats/insights/registry.go line 278 at r3 (raw file):
} // TODO(maryliag): add information about Contention Events
I think CI is failing with those nil pointer panics because of StatementInsightsStatistics.ContentionEvents.
This is the line in the generated code that panics:
func (m *StatementInsightsStatistics) MarshalToSizedBuffer(dAtA []byte) (int, error) {
...
// Triggers a nil pointer dereference panic
if len(m.ContentionEvents) > 0 {
Can we update the field in the proto to remove the (gogoproto.nullable = true) tag to fix?
(This assumes a check like len(m.ContentionEvents) > 0 would suffice)
pkg/sql/sqlstats/insights/store.go line 67 at r3 (raw file):
} var fromPool *obspb.StatementInsightsStatistics defer pool.Put(fromPool)
nit: do we need to declare the var & defer the returning of the var to the pool outside of the below for loop?
I'm having a bit of trouble understanding how this defer pool.Put(fromPool) call plays together with the pool.Put(fromPool) call inside the loop, since it seems like the var fromPool is never used outside of the loop.
pkg/sql/sqlstats/insights/store.go line 86 at r3 (raw file):
}) for _, key := range keysToDelete {
Since we hold the lock as a writer (as opposed to RLock()), it seems like keysToDelete should encompass everything in the cache, no?
If so, would it be faster to just drop the entire cache and replace it with a new one, in the same way we originally created it when first initialized the lockingStore? e.g.:
// Reset the cache with a fresh one.
s.mu.insights = cache.NewUnorderedCache(cache.Config{
Policy: cache.CacheFIFO,
ShouldEvict: func(size int, key, value interface{}) bool {
return int64(size) > ExecutionInsightsCapacity.Get(&st.SV)
},
OnEvicted: func(_, value interface{}) {
releaseInsight(value.(*Insight))
},
})
Then, we no longer need to allocated/track keysToDelete (as an additional bonus).
pkg/sql/sqlstats/persistedsqlstats/flush.go line 109 at r3 (raw file):
// Exporting Insights to Observability Service. exportInsights := insights.SQLInsightsStatsExportEnabled.Get(&s.SQLStats.GetClusterSettings().SV)
After learning more about the insights code, how would you feel about doing this exporting continuously, instead of at the SQL Stats flush interval?
For example, once the lockingRegistry observes a transaction, it pushes the insight to the lockingStore:
This appears to clear the lockingRegistry cache of all the statements with the same session ID associated with that txn, so it seems like the insight that it pushes to the lockingStore are "complete", is that right?
So could we just push the insight to the events exporter when we call (*lockingRegistry).AddInsight()?
With statement stats we wanted to use the events exporter at the flush interval so that we could benefit from the pre-aggregation happening during those ~10 minutes, but since Insights don't aggregate in a similar way, is it preferable to just export continuously? LMK what you think!
d09f5e3 to
f935eb8
Compare
maryliag
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @abarganier and @xinhaoz)
pkg/sql/sqlstats/insights/registry.go line 278 at r3 (raw file):
Previously, abarganier (Alex Barganier) wrote…
I think CI is failing with those nil pointer panics because of
StatementInsightsStatistics.ContentionEvents.This is the line in the generated code that panics:
func (m *StatementInsightsStatistics) MarshalToSizedBuffer(dAtA []byte) (int, error) { ... // Triggers a nil pointer dereference panic if len(m.ContentionEvents) > 0 {Can we update the field in the proto to remove the
(gogoproto.nullable = true)tag to fix?(This assumes a check like
len(m.ContentionEvents) > 0would suffice)
Ah nice! Thanks for catching this
pkg/sql/sqlstats/insights/store.go line 67 at r3 (raw file):
Previously, abarganier (Alex Barganier) wrote…
nit: do we need to declare the var & defer the returning of the var to the pool outside of the below
forloop?I'm having a bit of trouble understanding how this
defer pool.Put(fromPool)call plays together with thepool.Put(fromPool)call inside the loop, since it seems like the varfromPoolis never used outside of the loop.
initially I though the panic was caused here, so this was me playing with it and trying to see if would stop the panic. I can move things back to just inside the loop
pkg/sql/sqlstats/insights/store.go line 86 at r3 (raw file):
Previously, abarganier (Alex Barganier) wrote…
Since we hold the lock as a writer (as opposed to
RLock()), it seems likekeysToDeleteshould encompass everything in the cache, no?If so, would it be faster to just drop the entire cache and replace it with a new one, in the same way we originally created it when first initialized the
lockingStore? e.g.:// Reset the cache with a fresh one. s.mu.insights = cache.NewUnorderedCache(cache.Config{ Policy: cache.CacheFIFO, ShouldEvict: func(size int, key, value interface{}) bool { return int64(size) > ExecutionInsightsCapacity.Get(&st.SV) }, OnEvicted: func(_, value interface{}) { releaseInsight(value.(*Insight)) }, })Then, we no longer need to allocated/track
keysToDelete(as an additional bonus).
Good point! Forgot that we had the lock here. In my mind there was a chance something new could have been added between the export and the reset, but that won't be possible with the lock indeed!
abarganier
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @maryliag and @xinhaoz)
pkg/sql/sqlstats/insights/store.go line 86 at r3 (raw file):
Previously, maryliag (Marylia Gutierrez) wrote…
Good point! Forgot that we had the lock here. In my mind there was a chance something new could have been added between the export and the reset, but that won't be possible with the lock indeed!
Hm on second glance, we might have to call .Clear() on the cache instead. Otherwise, the OnEvicted func won't get called on the elements in the cache, so they won't get returned to the pool.
c8147cc to
0a481a6
Compare
maryliag
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @abarganier and @xinhaoz)
pkg/sql/sqlstats/insights/store.go line 86 at r3 (raw file):
Previously, abarganier (Alex Barganier) wrote…
Hm on second glance, we might have to call
.Clear()on the cache instead. Otherwise, theOnEvictedfunc won't get called on the elements in the cache, so they won't get returned to the pool.
I changed to use Clear()
pkg/sql/sqlstats/persistedsqlstats/flush.go line 109 at r3 (raw file):
Previously, abarganier (Alex Barganier) wrote…
After learning more about the insights code, how would you feel about doing this exporting continuously, instead of at the SQL Stats flush interval?
For example, once the
lockingRegistryobserves a transaction, it pushes the insight to thelockingStore:This appears to clear the
lockingRegistrycache of all the statements with the same session ID associated with that txn, so it seems like the insight that it pushes to thelockingStoreare "complete", is that right?So could we just push the insight to the events exporter when we call
(*lockingRegistry).AddInsight()?With statement stats we wanted to use the events exporter at the flush interval so that we could benefit from the pre-aggregation happening during those ~10 minutes, but since Insights don't aggregate in a similar way, is it preferable to just export continuously? LMK what you think!
I'm doing some testing to see how insights can get hooked to the exporter.
0a481a6 to
4e51b69
Compare
rafiss
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @abarganier, @maryliag, and @xinhaoz)
pkg/sql/sqlstats/insights/insights.go line 104 at r6 (raw file):
" was not set at node startup, enabling this setting will have no effect until the "+ "node is restarted with the flag set.", false)
nit: if the setting is supposed to be documented + visible to users, this needs a settings.WithPublic option as the last parameter
pkg/sql/sqlstats/insights/registry.go line 203 at r6 (raw file):
) { if other == nil { other = &obspb.StatementInsightsStatistics{}
i think this may be problematic - if the other variable is reassigned here, then that means it becomes a different reference than the one that was passed in. i think the solution has to be to make sure it is always non-nil when passed in. and you can still a nil check here, and have this function return an error if a nil value was passed in
285ffe5 to
b65cd4e
Compare
maryliag
left a comment
There was a problem hiding this comment.
Made the changes to export once an Insight is detected, so no longer using the Flush system.
PTAL!
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @abarganier, @rafiss, and @xinhaoz)
pkg/sql/sqlstats/insights/insights.go line 104 at r6 (raw file):
Previously, rafiss (Rafi Shamim) wrote…
nit: if the setting is supposed to be documented + visible to users, this needs a
settings.WithPublicoption as the last parameter
We're only using now while under development, once is ready for prod this flag will be updated to true and we don't want users changing back to false, so leaving as not public on purpose.
pkg/sql/sqlstats/insights/registry.go line 203 at r6 (raw file):
Previously, rafiss (Rafi Shamim) wrote…
i think this may be problematic - if the
othervariable is reassigned here, then that means it becomes a different reference than the one that was passed in. i think the solution has to be to make sure it is always non-nil when passed in. and you can still a nil check here, and have this function return an error if a nil value was passed in
Turns out I was getting some null values because of a mistake I made on another part of the code, now is no longer happening, but I decided to keep the check here just in case, with a log message
b65cd4e to
10b0c5d
Compare
abarganier
left a comment
There was a problem hiding this comment.
Nice! I think exporting per-insight will help amortize the network usage here, which will be especially helpful once we hook the events exporter up to the SQL stats flush. Appreciate you being open to the suggestion 🙏
My main comment is an idea on how we might be able to cut down on some of the plumbing - the remaining few are just small nits. Let me know what you think!
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @maryliag, @rafiss, and @xinhaoz)
pkg/sql/sqlstats/ssprovider.go line 52 at r7 (raw file):
RecordTransaction(ctx context.Context, key appstatspb.TransactionFingerprintID, value RecordedTxnStats,
I wonder if all the extra plumbing is necessary. Seems like we didn't previously need an insightsReader in RecordTransaction, because we just used the sink that was provided to the registry at initialization. Seems like ExportInsight() could be part of the sink interface?
Then, we might even be able to remove the eventsExporter param in RecordTransaction here. Instead, we could provide the lockingStore with an events exporter at initialization. Looks like the events exporter is easily accessible from where we initialize the insights provider, which is only one hop away from where we initialize the store.
cockroach/pkg/sql/conn_executor.go
Lines 413 to 417 in bab4335
pkg/sql/sqlstats/insights/registry.go line 186 at r7 (raw file):
r.sink.AddInsight(insight) // Exporting Insights to Observability Service. if SQLInsightsStatsExportEnabled.Get(&r.causes.st.SV) {
Nice! One thing I'm curious about - is this in the synchronous path of txn execution? Just wondering if it could have any notable impact on txn latency when the cluster setting is enabled. If it is, it might be useful to do a little perf testing to understand the impact.
I created https://cockroachlabs.atlassian.net/browse/CC-26271 to track this work. Of course, no need for premature optimization! Just noting it as something for us to do down the line once the prototype is up and running 🙂
pkg/sql/sqlstats/insights/registry.go line 295 at r7 (raw file):
// TODO(maryliag): add information about Contention Events // and Idle/Parse/Run Latencies. other.ContentionEvents = []*obspb.ContentionEvent{}
nit: I don't think we need to allocate a new slice here. By removing [(gogoproto.nullable) = true]; from the field definition, I believe the zero value of other.ContentionEvents will be an empty slice by default.
pkg/sql/sqlstats/insights/store.go line 86 at r3 (raw file):
Previously, maryliag (Marylia Gutierrez) wrote…
I changed to use
Clear()
No more cache evictions required! 🙌
pkg/sql/sqlstats/insights/store.go line 67 at r7 (raw file):
statBytes, e := protoutil.Marshal(fromPool) if e != nil { err = e
nit: can we make the anonymous function return an error? e.g.:
err := func() error {
...
if statBytes, err := protoutil.Marshal(fromPool); err != nil {
return err
}
...
}()
if err != nil {
return err
}
This way, we don't continue iterating if we fail to Marshal one of the statement insights. I'd expect a failure to Marshal indicates something serious has wrong, and that the others are likely to fail to Marshal as well. It might be best to fail fast.
abarganier
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @maryliag, @rafiss, and @xinhaoz)
pkg/sql/sqlstats/ssprovider.go line 52 at r7 (raw file):
Previously, abarganier (Alex Barganier) wrote…
I wonder if all the extra plumbing is necessary. Seems like we didn't previously need an
insightsReaderinRecordTransaction, because we just used thesinkthat was provided to the registry at initialization. Seems likeExportInsight()could be part of thesinkinterface?Then, we might even be able to remove the
eventsExporterparam inRecordTransactionhere. Instead, we could provide thelockingStorewith an events exporter at initialization. Looks like the events exporter is easily accessible from where we initialize the insights provider, which is only one hop away from where we initialize the store.cockroach/pkg/sql/conn_executor.go
Lines 413 to 417 in bab4335
One extra thing to note here - I see we use a compositeSink for the registry. We want to be sure we only call the events exporter once per-event, so iterating each store that compositeSink maintains a reference and calling ExportEvent is probably not a good idea as it could lead to duplicates
Do we really need the compositeSink? Its only use just has one underlying store. Maybe we can get rid of it altogether if it's not being used for anything 🤷♂️
65abf52 to
3427745
Compare
maryliag
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @abarganier, @rafiss, and @xinhaoz)
pkg/sql/sqlstats/insights/registry.go line 186 at r7 (raw file):
Previously, abarganier (Alex Barganier) wrote…
Nice! One thing I'm curious about - is this in the synchronous path of txn execution? Just wondering if it could have any notable impact on txn latency when the cluster setting is enabled. If it is, it might be useful to do a little perf testing to understand the impact.
I created https://cockroachlabs.atlassian.net/browse/CC-26271 to track this work. Of course, no need for premature optimization! Just noting it as something for us to do down the line once the prototype is up and running 🙂
This part will happen async.
Added per results to the PR description. I used both YCSB and the Insights workload, to simulate a normal workload and a bad one. The tests showed no performance degradation.
pkg/sql/sqlstats/insights/registry.go line 295 at r7 (raw file):
Previously, abarganier (Alex Barganier) wrote…
nit: I don't think we need to allocate a new slice here. By removing
[(gogoproto.nullable) = true];from the field definition, I believe the zero value ofother.ContentionEventswill be an empty slice by default.
Done
pkg/sql/sqlstats/insights/store.go line 67 at r7 (raw file):
Previously, abarganier (Alex Barganier) wrote…
nit: can we make the anonymous function return an error? e.g.:
err := func() error { ... if statBytes, err := protoutil.Marshal(fromPool); err != nil { return err } ... }() if err != nil { return err }This way, we don't continue iterating if we fail to Marshal one of the statement insights. I'd expect a failure to Marshal indicates something serious has wrong, and that the others are likely to fail to Marshal as well. It might be best to fail fast.
Done
pkg/sql/sqlstats/ssprovider.go line 52 at r7 (raw file):
Previously, abarganier (Alex Barganier) wrote…
One extra thing to note here - I see we use a
compositeSinkfor the registry. We want to be sure we only call the events exporter once per-event, so iterating each store thatcompositeSinkmaintains a reference and callingExportEventis probably not a good idea as it could lead to duplicatesDo we really need the
compositeSink? Its only use just has one underlyingstore. Maybe we can get rid of it altogether if it's not being used for anything 🤷♂️
Removed compositeSink which was not really used.
Since I'm not using flush, I moved the events exporter and pool to the insights system, so a lot less plumbing is required now.
PTAL!
pkg/sql/sqlstats/persistedsqlstats/flush.go line 109 at r3 (raw file):
Previously, maryliag (Marylia Gutierrez) wrote…
I'm doing some testing to see how insights can get hooked to the exporter.
Changes made to export the event on each detection.
3427745 to
9261cf3
Compare
When an Insights is detected, we store it in memory and also export it to Observability Service. This behaviour is controlled by a new cluster setting `sql.insights.export.enabled`. If the value is disabled, no data is exported. Epic: CC-25978 Release note: None
9261cf3 to
78ce9b1
Compare
abarganier
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 1 of 0 LGTMs obtained (waiting on @maryliag, @rafiss, and @xinhaoz)
pkg/sql/sqlstats/insights/registry.go line 186 at r7 (raw file):
Previously, maryliag (Marylia Gutierrez) wrote…
This part will happen async.
Added per results to the PR description. I used both YCSB and the Insights workload, to simulate a normal workload and a bad one. The tests showed no performance degradation.
Awesome, glad to see! Thank you for the performance tests 🙌
pkg/sql/sqlstats/insights/store.go line 67 at r7 (raw file):
Previously, maryliag (Marylia Gutierrez) wrote…
Done
Thank you!
pkg/sql/sqlstats/ssprovider.go line 52 at r7 (raw file):
Previously, maryliag (Marylia Gutierrez) wrote…
Removed
compositeSinkwhich was not really used.Since I'm not using flush, I moved the events exporter and pool to the insights system, so a lot less plumbing is required now.
PTAL!
Looks great, thanks for being open to my suggestions!
rafiss
left a comment
There was a problem hiding this comment.
Reviewed 1 of 6 files at r4, 3 of 37 files at r7, 12 of 25 files at r8, 6 of 6 files at r9, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @maryliag and @xinhaoz)
|
Thank you for all the reviews and suggestions! bors r+ |
|
Build succeeded: |
When an Insights is detected, we store it in memory and
also export it to Observability Service.
This behaviour is controlled by a new cluster setting
sql.insights.export.enabled. If the value is disabled, nodata is exported.
Epic: CC-25978
Running YCSB for 10min (with Export Enable and Disabled):
Running the Insights Workload (with Export Enable and Disabled) which is mostly statements that will be detected as having an Insight
Release note: None