From 493a43ba0738282bc9568bec3d014c1a79dfc664 Mon Sep 17 00:00:00 2001 From: Matthew Todd Date: Thu, 18 Aug 2022 11:34:08 -0400 Subject: [PATCH] insights: identify statements with high retry counts Addresses #85827. This change undoes the previous "concerns" concept from #85345 and replaces it with a "problems" field.[^1][^2] We then mark slow statements retried more than 10 times as having a "HighRetryCount" problem.[^3] Slow statements without any specifically identified problems will have "Unknown" in their list of problems. We will add support for further problem detection in future commits. [^1]: Every execution insight we offer happens because a statement was slow, so the "SlowExecution" concern was the only one we'd ever have. [^2]: The protocol buffer changes are safe because we haven't deployed this code anywhere yet. [^3]: This threshold may be configured via the `sql.insights.high_retry_count.threshold` cluster setting. Release justification: Category 2: Bug fixes and low-risk updates to new functionality Release note (ops change): The new `sql.insights.high_retry_count.threshold` cluster setting may be used to configure how many times a slow statement (as identified by the execution insights system) must have been retried to be marked as having a high retry count. --- .../settings/settings-for-tenants.txt | 1 + docs/generated/settings/settings.html | 1 + pkg/sql/crdb_internal.go | 9 +++ .../testdata/logic_test/create_statements | 4 + pkg/sql/sqlstats/insights/BUILD.bazel | 2 + pkg/sql/sqlstats/insights/detector.go | 40 ++++------ pkg/sql/sqlstats/insights/detector_test.go | 80 +++++++++---------- pkg/sql/sqlstats/insights/insights.go | 10 +++ pkg/sql/sqlstats/insights/insights.proto | 33 ++++++-- pkg/sql/sqlstats/insights/problems.go | 29 +++++++ pkg/sql/sqlstats/insights/problems_test.go | 48 +++++++++++ pkg/sql/sqlstats/insights/registry.go | 24 +++--- pkg/sql/sqlstats/insights/registry_test.go | 6 +- 13 files changed, 200 insertions(+), 87 deletions(-) create mode 100644 pkg/sql/sqlstats/insights/problems.go create mode 100644 pkg/sql/sqlstats/insights/problems_test.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 78fb5a83d6b1..b007a76772a9 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -233,6 +233,7 @@ sql.distsql.temp_storage.workmem byte size 64 MiB maximum amount of memory in by sql.guardrails.max_row_size_err byte size 512 MiB maximum size of row (or column family if multiple column families are in use) that SQL can write to the database, above which an error is returned; use 0 to disable sql.guardrails.max_row_size_log byte size 64 MiB maximum size of row (or column family if multiple column families are in use) that SQL can write to the database, above which an event is logged to SQL_PERF (or SQL_INTERNAL_PERF if the mutating statement was internal); use 0 to disable sql.insights.execution_insights_capacity integer 1000 the size of the per-node store of execution insights +sql.insights.high_retry_count.threshold integer 10 the number of retries a slow statement must have undergone for its high retry count to be highlighted as a potential problem sql.insights.latency_threshold duration 100ms amount of time after which an executing statement is considered slow. Use 0 to disable. sql.log.slow_query.experimental_full_table_scans.enabled boolean false when set to true, statements that perform a full table/index scan will be logged to the slow query log even if they do not meet the latency threshold. Must have the slow query log enabled for this setting to have any effect. sql.log.slow_query.internal_queries.enabled boolean false when set to true, internal queries which exceed the slow query log threshold are logged to a separate log. Must have the slow query log enabled for this setting to have any effect. diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 79b50190d8b0..bc7bf3d24e7b 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -164,6 +164,7 @@ sql.guardrails.max_row_size_logbyte size64 MiBmaximum size of row (or column family if multiple column families are in use) that SQL can write to the database, above which an event is logged to SQL_PERF (or SQL_INTERNAL_PERF if the mutating statement was internal); use 0 to disable sql.hash_sharded_range_pre_split.maxinteger16max pre-split ranges to have when adding hash sharded index to an existing table sql.insights.execution_insights_capacityinteger1000the size of the per-node store of execution insights +sql.insights.high_retry_count.thresholdinteger10the number of retries a slow statement must have undergone for its high retry count to be highlighted as a potential problem sql.insights.latency_thresholdduration100msamount of time after which an executing statement is considered slow. Use 0 to disable. sql.log.slow_query.experimental_full_table_scans.enabledbooleanfalsewhen set to true, statements that perform a full table/index scan will be logged to the slow query log even if they do not meet the latency threshold. Must have the slow query log enabled for this setting to have any effect. sql.log.slow_query.internal_queries.enabledbooleanfalsewhen set to true, internal queries which exceed the slow query log threshold are logged to a separate log. Must have the slow query log enabled for this setting to have any effect. diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 43b0400e4be1..3e81b5b1541c 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -6338,6 +6338,7 @@ CREATE TABLE crdb_internal.%s ( txn_fingerprint_id BYTES NOT NULL, stmt_id STRING NOT NULL, stmt_fingerprint_id BYTES NOT NULL, + problems STRING[] NOT NULL, query STRING NOT NULL, status STRING NOT NULL, start_time TIMESTAMP NOT NULL, @@ -6396,6 +6397,13 @@ func populateExecutionInsights( return } for _, insight := range response.Insights { + problems := tree.NewDArray(types.String) + for _, problem := range insight.Problems { + if errProblem := problems.Append(tree.NewDString(problem.String())); err != nil { + err = errors.CombineErrors(err, errProblem) + } + } + startTimestamp, errTimestamp := tree.MakeDTimestamp(insight.Statement.StartTime, time.Nanosecond) if errTimestamp != nil { err = errors.CombineErrors(err, errTimestamp) @@ -6442,6 +6450,7 @@ func populateExecutionInsights( tree.NewDBytes(tree.DBytes(sqlstatsutil.EncodeUint64ToBytes(uint64(insight.Transaction.FingerprintID)))), tree.NewDString(hex.EncodeToString(insight.Statement.ID.GetBytes())), tree.NewDBytes(tree.DBytes(sqlstatsutil.EncodeUint64ToBytes(uint64(insight.Statement.FingerprintID)))), + problems, tree.NewDString(insight.Statement.Query), tree.NewDString(insight.Statement.Status), startTimestamp, diff --git a/pkg/sql/logictest/testdata/logic_test/create_statements b/pkg/sql/logictest/testdata/logic_test/create_statements index 87085087c55f..7c528e201e62 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -249,6 +249,7 @@ CREATE TABLE crdb_internal.cluster_execution_insights ( txn_fingerprint_id BYTES NOT NULL, stmt_id STRING NOT NULL, stmt_fingerprint_id BYTES NOT NULL, + problems STRING[] NOT NULL, query STRING NOT NULL, status STRING NOT NULL, start_time TIMESTAMP NOT NULL, @@ -272,6 +273,7 @@ CREATE TABLE crdb_internal.cluster_execution_insights ( txn_fingerprint_id BYTES NOT NULL, stmt_id STRING NOT NULL, stmt_fingerprint_id BYTES NOT NULL, + problems STRING[] NOT NULL, query STRING NOT NULL, status STRING NOT NULL, start_time TIMESTAMP NOT NULL, @@ -963,6 +965,7 @@ CREATE TABLE crdb_internal.node_execution_insights ( txn_fingerprint_id BYTES NOT NULL, stmt_id STRING NOT NULL, stmt_fingerprint_id BYTES NOT NULL, + problems STRING[] NOT NULL, query STRING NOT NULL, status STRING NOT NULL, start_time TIMESTAMP NOT NULL, @@ -986,6 +989,7 @@ CREATE TABLE crdb_internal.node_execution_insights ( txn_fingerprint_id BYTES NOT NULL, stmt_id STRING NOT NULL, stmt_fingerprint_id BYTES NOT NULL, + problems STRING[] NOT NULL, query STRING NOT NULL, status STRING NOT NULL, start_time TIMESTAMP NOT NULL, diff --git a/pkg/sql/sqlstats/insights/BUILD.bazel b/pkg/sql/sqlstats/insights/BUILD.bazel index 9748d4f2e143..185208c69f6b 100644 --- a/pkg/sql/sqlstats/insights/BUILD.bazel +++ b/pkg/sql/sqlstats/insights/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "detector.go", "ingester.go", "insights.go", + "problems.go", "registry.go", ], embed = [":insights_go_proto"], @@ -35,6 +36,7 @@ go_test( "detector_test.go", "ingester_test.go", "insights_test.go", + "problems_test.go", "registry_test.go", ], embed = [":insights"], diff --git a/pkg/sql/sqlstats/insights/detector.go b/pkg/sql/sqlstats/insights/detector.go index fe9e975a73e4..fa8c658f357a 100644 --- a/pkg/sql/sqlstats/insights/detector.go +++ b/pkg/sql/sqlstats/insights/detector.go @@ -20,7 +20,7 @@ import ( type detector interface { enabled() bool - examine(*Statement) []Concern + isSlow(*Statement) bool } var _ detector = &compositeDetector{} @@ -40,20 +40,14 @@ func (a compositeDetector) enabled() bool { return false } -func (a compositeDetector) examine(statement *Statement) (concerns []Concern) { - set := make(map[Concern]struct{}) - +func (a compositeDetector) isSlow(statement *Statement) bool { + // Because some detectors may need to observe all statements to build up + // their baseline sense of what "normal" is, we avoid short-circuiting. + result := false for _, d := range a.detectors { - for _, concern := range d.examine(statement) { - set[concern] = struct{}{} - } + result = d.isSlow(statement) || result } - - for concern := range set { - concerns = append(concerns, concern) - } - - return concerns + return result } var desiredQuantiles = map[float64]float64{0.5: 0.05, 0.99: 0.001} @@ -74,23 +68,21 @@ func (d anomalyDetector) enabled() bool { return AnomalyDetectionEnabled.Get(&d.settings.SV) } -func (d *anomalyDetector) examine(stmt *Statement) (concerns []Concern) { +func (d *anomalyDetector) isSlow(stmt *Statement) (decision bool) { if !d.enabled() { - return concerns + return } d.withFingerprintLatencySummary(stmt, func(latencySummary *quantile.Stream) { latencySummary.Insert(stmt.LatencyInSeconds) p50 := latencySummary.Query(0.5) p99 := latencySummary.Query(0.99) - if stmt.LatencyInSeconds >= p99 && + decision = stmt.LatencyInSeconds >= p99 && stmt.LatencyInSeconds >= 2*p50 && - stmt.LatencyInSeconds >= AnomalyDetectionLatencyThreshold.Get(&d.settings.SV).Seconds() { - concerns = append(concerns, Concern_SlowExecution) - } + stmt.LatencyInSeconds >= AnomalyDetectionLatencyThreshold.Get(&d.settings.SV).Seconds() }) - return concerns + return } func (d *anomalyDetector) withFingerprintLatencySummary( @@ -146,10 +138,6 @@ func (l latencyThresholdDetector) enabled() bool { return LatencyThreshold.Get(&l.st.SV) > 0 } -func (l latencyThresholdDetector) examine(s *Statement) (concerns []Concern) { - if l.enabled() && s.LatencyInSeconds >= LatencyThreshold.Get(&l.st.SV).Seconds() { - concerns = append(concerns, Concern_SlowExecution) - } - - return concerns +func (l latencyThresholdDetector) isSlow(s *Statement) bool { + return l.enabled() && s.LatencyInSeconds >= LatencyThreshold.Get(&l.st.SV).Seconds() } diff --git a/pkg/sql/sqlstats/insights/detector_test.go b/pkg/sql/sqlstats/insights/detector_test.go index 785b3c516216..1900f9647d46 100644 --- a/pkg/sql/sqlstats/insights/detector_test.go +++ b/pkg/sql/sqlstats/insights/detector_test.go @@ -37,38 +37,32 @@ func TestAnyDetector(t *testing.T) { require.True(t, detector.enabled()) }) - t.Run("examine is nil without any detectors", func(t *testing.T) { + t.Run("isSlow is false without any detectors", func(t *testing.T) { detector := &compositeDetector{} - require.Empty(t, detector.examine(&Statement{})) + require.False(t, detector.isSlow(&Statement{})) }) - t.Run("examine is nil without any concerned detectors", func(t *testing.T) { + t.Run("isSlow is false without any concerned detectors", func(t *testing.T) { detector := &compositeDetector{[]detector{&fakeDetector{}, &fakeDetector{}}} - require.Empty(t, detector.examine(&Statement{})) + require.False(t, detector.isSlow(&Statement{})) }) - t.Run("examine combines detector concerns (first)", func(t *testing.T) { - detector := &compositeDetector{[]detector{ - &fakeDetector{stubExamine: []Concern{Concern_SlowExecution}}, - &fakeDetector{}, - }} - require.Equal(t, []Concern{Concern_SlowExecution}, detector.examine(&Statement{})) + t.Run("isSlow is true with at least one concerned detector", func(t *testing.T) { + detector := &compositeDetector{[]detector{&fakeDetector{stubIsSlow: true}, &fakeDetector{}}} + require.True(t, detector.isSlow(&Statement{})) }) - t.Run("examine combines detector concerns (second)", func(t *testing.T) { - detector := &compositeDetector{[]detector{ - &fakeDetector{}, - &fakeDetector{stubExamine: []Concern{Concern_SlowExecution}}, - }} - require.Equal(t, []Concern{Concern_SlowExecution}, detector.examine(&Statement{})) - }) + t.Run("isSlow consults all detectors without short-circuiting", func(t *testing.T) { + // Detector implementations may wish to observe all statements, to + // build up their baseline sense of what "usual" is. To short-circuit + // would deny them that chance. + d1 := &fakeDetector{stubIsSlow: true} + d2 := &fakeDetector{stubIsSlow: true} - t.Run("examine uniqs detector concerns", func(t *testing.T) { - detector := &compositeDetector{[]detector{ - &fakeDetector{stubExamine: []Concern{Concern_SlowExecution}}, - &fakeDetector{stubExamine: []Concern{Concern_SlowExecution}}, - }} - require.Equal(t, []Concern{Concern_SlowExecution}, detector.examine(&Statement{})) + detector := &compositeDetector{[]detector{d1, d2}} + detector.isSlow(&Statement{}) + require.True(t, d1.isSlowCalled, "the first detector should be consulted") + require.True(t, d2.isSlowCalled, "the second detector should be consulted") }) } @@ -85,7 +79,7 @@ func TestLatencyQuantileDetector(t *testing.T) { require.True(t, d.enabled()) }) - t.Run("examine", func(t *testing.T) { + t.Run("isSlow", func(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() AnomalyDetectionEnabled.Override(ctx, &st.SV, true) @@ -95,28 +89,30 @@ func TestLatencyQuantileDetector(t *testing.T) { name string seedLatency time.Duration candidateLatency time.Duration - concerns []Concern + isSlow bool }{{ name: "false with normal latency", seedLatency: 100 * time.Millisecond, candidateLatency: 100 * time.Millisecond, + isSlow: false, }, { name: "true with higher latency", seedLatency: 100 * time.Millisecond, candidateLatency: 200 * time.Millisecond, - concerns: []Concern{Concern_SlowExecution}, + isSlow: true, }, { name: "false with higher latency under interesting threshold", seedLatency: 10 * time.Millisecond, candidateLatency: 20 * time.Millisecond, + isSlow: false, }} for _, test := range tests { t.Run(test.name, func(t *testing.T) { d := newAnomalyDetector(st, NewMetrics()) for i := 0; i < 1000; i++ { - d.examine(&Statement{LatencyInSeconds: test.seedLatency.Seconds()}) + d.isSlow(&Statement{LatencyInSeconds: test.seedLatency.Seconds()}) } - require.Equal(t, test.concerns, d.examine(&Statement{LatencyInSeconds: test.candidateLatency.Seconds()})) + require.Equal(t, test.isSlow, d.isSlow(&Statement{LatencyInSeconds: test.candidateLatency.Seconds()})) }) } }) @@ -173,7 +169,7 @@ func TestLatencyQuantileDetector(t *testing.T) { d := newAnomalyDetector(st, metrics) // Show the detector `test.fingerprints` distinct fingerprints. for i := 0; i < test.fingerprints; i++ { - d.examine(&Statement{ + d.isSlow(&Statement{ LatencyInSeconds: AnomalyDetectionLatencyThreshold.Get(&st.SV).Seconds(), FingerprintID: roachpb.StmtFingerprintID(i), }) @@ -192,7 +188,7 @@ func BenchmarkLatencyQuantileDetector(b *testing.B) { AnomalyDetectionEnabled.Override(context.Background(), &settings.SV, true) d := newAnomalyDetector(settings, NewMetrics()) for i := 0; i < b.N; i++ { - d.examine(&Statement{ + d.isSlow(&Statement{ LatencyInSeconds: random.Float64(), }) } @@ -213,39 +209,41 @@ func TestLatencyThresholdDetector(t *testing.T) { require.True(t, detector.enabled()) }) - t.Run("examine nil when disabled", func(t *testing.T) { + t.Run("isSlow false when disabled", func(t *testing.T) { st := cluster.MakeTestingClusterSettings() LatencyThreshold.Override(context.Background(), &st.SV, 0) detector := latencyThresholdDetector{st: st} - require.Empty(t, detector.examine(&Statement{LatencyInSeconds: 1})) + require.False(t, detector.isSlow(&Statement{LatencyInSeconds: 1})) }) - t.Run("examine nil when fast enough", func(t *testing.T) { + t.Run("isSlow false when fast enough", func(t *testing.T) { st := cluster.MakeTestingClusterSettings() LatencyThreshold.Override(context.Background(), &st.SV, 1*time.Second) detector := latencyThresholdDetector{st: st} - require.Empty(t, detector.examine(&Statement{LatencyInSeconds: 0.5})) + require.False(t, detector.isSlow(&Statement{LatencyInSeconds: 0.5})) }) - t.Run("examine slow beyond threshold", func(t *testing.T) { + t.Run("isSlow true beyond threshold", func(t *testing.T) { st := cluster.MakeTestingClusterSettings() LatencyThreshold.Override(context.Background(), &st.SV, 1*time.Second) detector := latencyThresholdDetector{st: st} - require.Equal(t, []Concern{Concern_SlowExecution}, detector.examine(&Statement{LatencyInSeconds: 1})) + require.True(t, detector.isSlow(&Statement{LatencyInSeconds: 1})) }) } type fakeDetector struct { - stubEnabled bool - stubExamine []Concern + stubEnabled bool + stubIsSlow bool + isSlowCalled bool } -func (f fakeDetector) enabled() bool { +func (f *fakeDetector) enabled() bool { return f.stubEnabled } -func (f fakeDetector) examine(_ *Statement) []Concern { - return f.stubExamine +func (f *fakeDetector) isSlow(*Statement) bool { + f.isSlowCalled = true + return f.stubIsSlow } var _ detector = &fakeDetector{} diff --git a/pkg/sql/sqlstats/insights/insights.go b/pkg/sql/sqlstats/insights/insights.go index fd6cf1d76507..6245ea99a59c 100644 --- a/pkg/sql/sqlstats/insights/insights.go +++ b/pkg/sql/sqlstats/insights/insights.go @@ -78,6 +78,16 @@ var AnomalyDetectionMemoryLimit = settings.RegisterByteSizeSetting( 1024*1024, ) +// HighRetryCountThreshold sets the number of times a slow statement must have +// been retried to be marked as having a high retry count. +var HighRetryCountThreshold = settings.RegisterIntSetting( + settings.TenantWritable, + "sql.insights.high_retry_count.threshold", + "the number of retries a slow statement must have undergone for its high retry count to be highlighted as a potential problem", + 10, + settings.NonNegativeInt, +).WithPublic() + // Metrics holds running measurements of various outliers-related runtime stats. type Metrics struct { // Fingerprints measures the number of statement fingerprints being monitored for diff --git a/pkg/sql/sqlstats/insights/insights.proto b/pkg/sql/sqlstats/insights/insights.proto index 7634c5a17ae6..1db387f27bfd 100644 --- a/pkg/sql/sqlstats/insights/insights.proto +++ b/pkg/sql/sqlstats/insights/insights.proto @@ -16,13 +16,30 @@ import "gogoproto/gogo.proto"; import "google/protobuf/timestamp.proto"; import "google/protobuf/duration.proto"; -enum Concern { - // SlowExecution is for statement executions that either take longer than a predetermined - // threshold, configured by the sql.insights.latency_threshold - // cluster setting, or that take significantly "longer than usual" for their statement - // fingerprint, based on some heuristics we're developing, which can be enabled by the - // sql.insights.anomaly_detection.enabled cluster setting. - SlowExecution = 0; +enum Problem { + // We were unable to identify specific reasons why this statement was slow. + Unknown = 0; + + // This statement was slow because we picked the wrong plan, possibly due to + // outdated statistics, the statement using different literals or search + // conditions, or a change in the database schema. + PlanRegression = 1; + + // This statement was slow because a good plan was not available, whether + // due to outdated statistics or missing indexes. + SuboptimalPlan = 2; + + // This statement was slow because of contention. + HighWaitTime = 3; + + // This statement was slow because of being retried multiple times, again due + // to contention. The "high" threshold may be configured by the + // `sql.insights.high_retry_count.threshold` cluster setting. + HighRetryCount = 4; + + // This statement execution failed completely, due to contention, resource + // saturation, or syntax errors. + FailedExecution = 5; } message Session { @@ -74,5 +91,5 @@ message Insight { Transaction transaction = 2; Statement statement = 3; - repeated Concern concerns = 4; + repeated Problem problems = 4; } diff --git a/pkg/sql/sqlstats/insights/problems.go b/pkg/sql/sqlstats/insights/problems.go new file mode 100644 index 000000000000..268c3e501192 --- /dev/null +++ b/pkg/sql/sqlstats/insights/problems.go @@ -0,0 +1,29 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package insights + +import "github.com/cockroachdb/cockroach/pkg/settings/cluster" + +type problems struct { + st *cluster.Settings +} + +func (p *problems) examine(stmt *Statement) (result []Problem) { + if stmt.Retries >= HighRetryCountThreshold.Get(&p.st.SV) { + result = append(result, Problem_HighRetryCount) + } + + if len(result) == 0 { + result = append(result, Problem_Unknown) + } + + return +} diff --git a/pkg/sql/sqlstats/insights/problems_test.go b/pkg/sql/sqlstats/insights/problems_test.go new file mode 100644 index 000000000000..d4b1031b1a2f --- /dev/null +++ b/pkg/sql/sqlstats/insights/problems_test.go @@ -0,0 +1,48 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package insights + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/stretchr/testify/require" +) + +func TestProblems(t *testing.T) { + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + p := &problems{st: st} + HighRetryCountThreshold.Override(ctx, &st.SV, 10) + + testCases := []struct { + name string + statement *Statement + problems []Problem + }{ + { + name: "unknown", + statement: &Statement{}, + problems: []Problem{Problem_Unknown}, + }, + { + name: "high retry count", + statement: &Statement{Retries: 10}, + problems: []Problem{Problem_HighRetryCount}, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + require.ElementsMatch(t, tc.problems, p.examine(tc.statement)) + }) + } +} diff --git a/pkg/sql/sqlstats/insights/registry.go b/pkg/sql/sqlstats/insights/registry.go index 6c5fb1531f2b..b624db0e0266 100644 --- a/pkg/sql/sqlstats/insights/registry.go +++ b/pkg/sql/sqlstats/insights/registry.go @@ -25,6 +25,7 @@ import ( // exposes the set of currently retained outliers. type registry struct { detector detector + problems *problems // Note that this single mutex places unnecessary constraints on outlier // detection and reporting. We will develop a higher-throughput system @@ -49,7 +50,11 @@ func newRegistry(st *cluster.Settings, metrics Metrics) Registry { detector: compositeDetector{detectors: []detector{ latencyThresholdDetector{st: st}, newAnomalyDetector(st, metrics), - }}} + }}, + problems: &problems{ + st: st, + }, + } r.mu.statements = make(map[clusterunique.ID][]*Statement) r.mu.outliers = cache.NewUnorderedCache(config) return r @@ -77,23 +82,24 @@ func (r *registry) ObserveTransaction(sessionID clusterunique.ID, transaction *T statements := r.mu.statements[sessionID] delete(r.mu.statements, sessionID) - concerns := make(map[clusterunique.ID][]Concern, len(statements)) - - hasConcerns := false + slowStatements := make(map[clusterunique.ID]struct{}) for _, s := range statements { - concerns[s.ID] = r.detector.examine(s) - if len(concerns[s.ID]) > 0 { - hasConcerns = true + if r.detector.isSlow(s) { + slowStatements[s.ID] = struct{}{} } } - if hasConcerns { + if len(slowStatements) > 0 { for _, s := range statements { + var p []Problem + if _, ok := slowStatements[s.ID]; ok { + p = r.problems.examine(s) + } r.mu.outliers.Add(s.ID, &Insight{ Session: &Session{ID: sessionID}, Transaction: transaction, Statement: s, - Concerns: concerns[s.ID], + Problems: p, }) } } diff --git a/pkg/sql/sqlstats/insights/registry_test.go b/pkg/sql/sqlstats/insights/registry_test.go index d1d6c9bc451e..c3316a148cad 100644 --- a/pkg/sql/sqlstats/insights/registry_test.go +++ b/pkg/sql/sqlstats/insights/registry_test.go @@ -47,7 +47,7 @@ func TestRegistry(t *testing.T) { Session: session, Transaction: transaction, Statement: statement, - Concerns: []Concern{Concern_SlowExecution}, + Problems: []Problem{Problem_Unknown}, }} var actual []*Insight @@ -121,12 +121,12 @@ func TestRegistry(t *testing.T) { Session: session, Transaction: transaction, Statement: statement, - Concerns: []Concern{Concern_SlowExecution}, + Problems: []Problem{Problem_Unknown}, }, { Session: otherSession, Transaction: otherTransaction, Statement: otherStatement, - Concerns: []Concern{Concern_SlowExecution}, + Problems: []Problem{Problem_Unknown}, }} var actual []*Insight registry.IterateInsights(