From 77740ac72f8e86d2206906517137394a2a3744b0 Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Thu, 22 Sep 2022 22:26:02 -0700 Subject: [PATCH] sql: add telemetry for statistics forecast usage Add a few fields to the sampled_query telemetry events that will help us measure how useful table statistics forecasting is in practice. Fixes: #86356 Release note (ops change): Add five new fields to the sampled_query telemetry events: - `ScanCount`: Number of scans in the query plan. - `ScanWithStatsCount`: Number of scans using statistics (including forecasted statistics) in the query plan. - `ScanWithStatsForecastCount`: Number of scans using forecasted statistics in the query plan. - `TotalScanRowsWithoutForecastsEstimate`: Total number of rows read by all scans in the query, as estimated by the optimizer without using forecasts. - `NanosSinceStatsForecasted`: The greatest quantity of nanoseconds that have passed since the forecast time (or until the forecast time, if it is in the future, in which case it will be negative) for any table with forecasted stats scanned by this query. --- docs/generated/eventlog.md | 5 + pkg/sql/exec_log.go | 6 + pkg/sql/instrumentation.go | 14 + pkg/sql/opt/exec/execbuilder/builder.go | 14 + pkg/sql/opt/exec/execbuilder/relational.go | 48 ++- pkg/sql/opt/exec/factory.go | 15 + pkg/sql/plan_opt.go | 3 + pkg/sql/telemetry_logging_test.go | 274 ++++++++++++++++++ pkg/util/log/eventpb/json_encode_generated.go | 45 +++ pkg/util/log/eventpb/telemetry.proto | 20 ++ 10 files changed, 438 insertions(+), 6 deletions(-) diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index 06aac039b359..7ec74e57bdfb 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -2816,6 +2816,11 @@ contains common SQL event/execution details. | `KVRowsRead` | The number of rows read at the KV layer for this query. | no | | `NetworkMessages` | The number of network messages sent by nodes for this query. | no | | `IndexRecommendations` | Generated index recommendations for this query. | no | +| `ScanCount` | The number of scans in the query plan. | no | +| `ScanWithStatsCount` | The number of scans using statistics (including forecasted statistics) in the query plan. | no | +| `ScanWithStatsForecastCount` | The number of scans using forecasted statistics in the query plan. | no | +| `TotalScanRowsWithoutForecastsEstimate` | Total number of rows read by all scans in the query, as estimated by the optimizer without using forecasts. | no | +| `NanosSinceStatsForecasted` | The greatest quantity of nanoseconds that have passed since the forecast time (or until the forecast time, if it is in the future, in which case it will be negative) for any table with forecasted stats scanned by this query. | no | #### Common fields diff --git a/pkg/sql/exec_log.go b/pkg/sql/exec_log.go index 87eaa24a5250..4ce38aa6d11a 100644 --- a/pkg/sql/exec_log.go +++ b/pkg/sql/exec_log.go @@ -457,6 +457,12 @@ func (p *planner) maybeLogStatementInternal( KVRowsRead: stats.KVRowsRead, NetworkMessages: stats.NetworkMessages, IndexRecommendations: indexRecs, + + ScanCount: int64(p.curPlan.instrumentation.scanCounts[exec.ScanCount]), + ScanWithStatsCount: int64(p.curPlan.instrumentation.scanCounts[exec.ScanWithStatsCount]), + ScanWithStatsForecastCount: int64(p.curPlan.instrumentation.scanCounts[exec.ScanWithStatsForecastCount]), + TotalScanRowsWithoutForecastsEstimate: p.curPlan.instrumentation.totalScanRowsWithoutForecasts, + NanosSinceStatsForecasted: int64(p.curPlan.instrumentation.nanosSinceStatsForecasted), } p.logOperationalEventsOnlyExternally(ctx, isCopy, &sampledQuery) } else { diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go index 0f4c81e1be96..af7764464f97 100644 --- a/pkg/sql/instrumentation.go +++ b/pkg/sql/instrumentation.go @@ -155,6 +155,11 @@ type instrumentationHelper struct { // as estimated by the optimizer. totalScanRows float64 + // totalScanRowsWithoutForecasts is the total number of rows read by all scans + // in the query, as estimated by the optimizer without using forecasts. (If + // forecasts were not used, this should be the same as totalScanRows.) + totalScanRowsWithoutForecasts float64 + // outputRows is the number of rows output by the query, as estimated by the // optimizer. outputRows float64 @@ -167,6 +172,12 @@ type instrumentationHelper struct { // passed since stats were collected on any table scanned by this query. nanosSinceStatsCollected time.Duration + // nanosSinceStatsForecasted is the greatest quantity of nanoseconds that have + // passed since the forecast time (or until the forecast time, if it is in the + // future, in which case it will be negative) for any table with forecasted + // stats scanned by this query. + nanosSinceStatsForecasted time.Duration + // joinTypeCounts records the number of times each type of logical join was // used in the query. joinTypeCounts map[descpb.JoinType]int @@ -174,6 +185,9 @@ type instrumentationHelper struct { // joinAlgorithmCounts records the number of times each type of join algorithm // was used in the query. joinAlgorithmCounts map[exec.JoinAlgorithm]int + + // scanCounts records the number of times scans were used in the query. + scanCounts [exec.NumScanCountTypes]int } // outputMode indicates how the statement output needs to be populated (for diff --git a/pkg/sql/opt/exec/execbuilder/builder.go b/pkg/sql/opt/exec/execbuilder/builder.go index 8f52c284a7a9..8b5df3e4a60f 100644 --- a/pkg/sql/opt/exec/execbuilder/builder.go +++ b/pkg/sql/opt/exec/execbuilder/builder.go @@ -140,10 +140,21 @@ type Builder struct { // as estimated by the optimizer. TotalScanRows float64 + // TotalScanRowsWithoutForecasts is the total number of rows read by all scans + // in the query, as estimated by the optimizer without using forecasts. (If + // forecasts were not used, this should be the same as TotalScanRows.) + TotalScanRowsWithoutForecasts float64 + // NanosSinceStatsCollected is the maximum number of nanoseconds that have // passed since stats were collected on any table scanned by this query. NanosSinceStatsCollected time.Duration + // NanosSinceStatsForecasted is the greatest quantity of nanoseconds that have + // passed since the forecast time (or until the forecast time, if the it is in + // the future, in which case it will be negative) for any table with + // forecasted stats scanned by this query. + NanosSinceStatsForecasted time.Duration + // JoinTypeCounts records the number of times each type of logical join was // used in the query. JoinTypeCounts map[descpb.JoinType]int @@ -152,6 +163,9 @@ type Builder struct { // was used in the query. JoinAlgorithmCounts map[exec.JoinAlgorithm]int + // ScanCounts records the number of times scans were used in the query. + ScanCounts [exec.NumScanCountTypes]int + // wrapFunctionOverride overrides default implementation to return resolvable // function reference for function with specified function name. // The default can be overridden by calling SetBuiltinFuncWrapper method to provide diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index 257a99804dc1..3c9919e19d78 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -14,6 +14,7 @@ import ( "bytes" "context" "fmt" + "math" "strings" "github.com/cockroachdb/cockroach/pkg/server/telemetry" @@ -726,7 +727,8 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) { // Save if we planned a full table/index scan on the builder so that the // planner can be made aware later. We only do this for non-virtual tables. - stats := scan.Relational().Stats + relProps := scan.Relational() + stats := relProps.Stats if !tab.IsVirtualTable() && isUnfiltered { large := !stats.Available || stats.RowCount > b.evalCtx.SessionData().LargeFullScanRows if scan.Index == cat.PrimaryIndex { @@ -741,16 +743,50 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) { } } - // Save the total estimated number of rows scanned and the time since stats - // were collected. + // Save some instrumentation info. + b.ScanCounts[exec.ScanCount]++ if stats.Available { b.TotalScanRows += stats.RowCount - if tab.StatisticCount() > 0 { - // The first stat is the most recent one. - nanosSinceStatsCollected := timeutil.Since(tab.Statistic(0).CreatedAt()) + b.ScanCounts[exec.ScanWithStatsCount]++ + + // The first stat is the most recent one. Check if it was a forecast. + var first int + if first < tab.StatisticCount() && tab.Statistic(first).IsForecast() { + if b.evalCtx.SessionData().OptimizerUseForecasts { + b.ScanCounts[exec.ScanWithStatsForecastCount]++ + + // Calculate time since the forecast (or negative time until the forecast). + nanosSinceStatsForecasted := timeutil.Since(tab.Statistic(first).CreatedAt()) + if nanosSinceStatsForecasted.Abs() > b.NanosSinceStatsForecasted.Abs() { + b.NanosSinceStatsForecasted = nanosSinceStatsForecasted + } + } + // Find the first non-forecast stat. + for first < tab.StatisticCount() && tab.Statistic(first).IsForecast() { + first++ + } + } + + if first < tab.StatisticCount() { + tabStat := tab.Statistic(first) + + nanosSinceStatsCollected := timeutil.Since(tabStat.CreatedAt()) if nanosSinceStatsCollected > b.NanosSinceStatsCollected { b.NanosSinceStatsCollected = nanosSinceStatsCollected } + + // Calculate another row count estimate using these (non-forecast) + // stats. If forecasts were not used, this should be the same as + // stats.RowCount. + rowCountWithoutForecast := float64(tabStat.RowCount()) + rowCountWithoutForecast *= stats.Selectivity.AsFloat() + minCardinality, maxCardinality := relProps.Cardinality.Min, relProps.Cardinality.Max + if rowCountWithoutForecast > float64(maxCardinality) && maxCardinality != math.MaxUint32 { + rowCountWithoutForecast = float64(maxCardinality) + } else if rowCountWithoutForecast < float64(minCardinality) { + rowCountWithoutForecast = float64(minCardinality) + } + b.TotalScanRowsWithoutForecasts += rowCountWithoutForecast } } diff --git a/pkg/sql/opt/exec/factory.go b/pkg/sql/opt/exec/factory.go index 5fa4d4435bca..989a2db9cb2f 100644 --- a/pkg/sql/opt/exec/factory.go +++ b/pkg/sql/opt/exec/factory.go @@ -386,3 +386,18 @@ const ( ZigZagJoin NumJoinAlgorithms ) + +// ScanCountType is the type of count of scan operations in a query. +type ScanCountType int + +const ( + // ScanCount is the count of all scans in a query. + ScanCount ScanCountType = iota + // ScanWithStatsCount is the count of scans with statistics in a query. + ScanWithStatsCount + // ScanWithStatsForecastCount is the count of scans which used forecasted + // statistics in a query. + ScanWithStatsForecastCount + // NumScanCountTypes is the total number of types of counts of scans. + NumScanCountTypes +) diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go index 00c0792785f8..227b583db900 100644 --- a/pkg/sql/plan_opt.go +++ b/pkg/sql/plan_opt.go @@ -656,9 +656,12 @@ func (opc *optPlanningCtx) runExecBuilder( containsMutation = bld.ContainsMutation planTop.instrumentation.maxFullScanRows = bld.MaxFullScanRows planTop.instrumentation.totalScanRows = bld.TotalScanRows + planTop.instrumentation.totalScanRowsWithoutForecasts = bld.TotalScanRowsWithoutForecasts planTop.instrumentation.nanosSinceStatsCollected = bld.NanosSinceStatsCollected + planTop.instrumentation.nanosSinceStatsForecasted = bld.NanosSinceStatsForecasted planTop.instrumentation.joinTypeCounts = bld.JoinTypeCounts planTop.instrumentation.joinAlgorithmCounts = bld.JoinAlgorithmCounts + planTop.instrumentation.scanCounts = bld.ScanCounts planTop.instrumentation.RecordExplainPlan(explainPlan) } diff --git a/pkg/sql/telemetry_logging_test.go b/pkg/sql/telemetry_logging_test.go index 9e58752fa0f6..5be7972051cb 100644 --- a/pkg/sql/telemetry_logging_test.go +++ b/pkg/sql/telemetry_logging_test.go @@ -12,6 +12,7 @@ package sql import ( "context" + "encoding/json" "fmt" "math" "regexp" @@ -897,3 +898,276 @@ func TestTelemetryLogJoinTypesAndAlgorithms(t *testing.T) { } } } + +// TestTelemetryScanCounts tests that scans with and without forecasted +// statistics are counted correctly. It also tests that other statistics +// forecasting telemetry is counted correctly. +func TestTelemetryScanCounts(t *testing.T) { + defer leaktest.AfterTest(t)() + sc := log.ScopeWithoutShowLogs(t) + defer sc.Close(t) + + cleanup := logtestutils.InstallTelemetryLogFileSink(sc, t) + defer cleanup() + + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + db := sqlutils.MakeSQLRunner(sqlDB) + defer s.Stopper().Stop(context.Background()) + + db.Exec(t, "SET CLUSTER SETTING sql.telemetry.query_sampling.enabled = true;") + db.Exec(t, "SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false;") + db.Exec(t, "CREATE TABLE d (d PRIMARY KEY) AS SELECT generate_series(10, 16);") + db.Exec(t, "CREATE TABLE e (e PRIMARY KEY) AS SELECT generate_series(0, 19);") + db.Exec(t, "CREATE TABLE f (f PRIMARY KEY) AS SELECT generate_series(5, 8) * 2;") + db.Exec(t, `ALTER TABLE e INJECT STATISTICS '[ + { + "avg_size": 1, + "columns": [ + "e" + ], + "created_at": "2017-08-05 00:00:00.000000", + "distinct_count": 20, + "histo_buckets": [ + { + "distinct_range": 0, + "num_eq": 1, + "num_range": 0, + "upper_bound": "0" + }, + { + "distinct_range": 18, + "num_eq": 1, + "num_range": 18, + "upper_bound": "20" + } + ], + "histo_col_type": "INT8", + "histo_version": 2, + "name": "__auto__", + "null_count": 0, + "row_count": 20 + } +]';`) + db.Exec(t, `ALTER TABLE f INJECT STATISTICS '[ + { + "avg_size": 1, + "columns": [ + "f" + ], + "created_at": "2017-05-07 00:00:00.000000", + "distinct_count": 1, + "histo_buckets": [ + { + "distinct_range": 0, + "num_eq": 0, + "num_range": 0, + "upper_bound": "1" + }, + { + "distinct_range": 1, + "num_eq": 0, + "num_range": 1, + "upper_bound": "11" + } + ], + "histo_col_type": "INT8", + "histo_version": 2, + "name": "__auto__", + "null_count": 0, + "row_count": 1 + }, + { + "avg_size": 1, + "columns": [ + "f" + ], + "created_at": "2017-05-08 00:00:00.000000", + "distinct_count": 2, + "histo_buckets": [ + { + "distinct_range": 0, + "num_eq": 0, + "num_range": 0, + "upper_bound": "3" + }, + { + "distinct_range": 2, + "num_eq": 0, + "num_range": 2, + "upper_bound": "13" + } + ], + "histo_col_type": "INT8", + "histo_version": 2, + "name": "__auto__", + "null_count": 0, + "row_count": 2 + }, + { + "avg_size": 1, + "columns": [ + "f" + ], + "created_at": "2017-05-09 00:00:00.000000", + "distinct_count": 3, + "histo_buckets": [ + { + "distinct_range": 0, + "num_eq": 0, + "num_range": 0, + "upper_bound": "5" + }, + { + "distinct_range": 3, + "num_eq": 0, + "num_range": 3, + "upper_bound": "15" + } + ], + "histo_col_type": "INT8", + "histo_version": 2, + "name": "__auto__", + "null_count": 0, + "row_count": 3 + } +]';`) + + testData := []struct { + query string + logStmt string + scanCount float64 + scanWithStatsCount float64 + scanWithStatsForecastCount float64 + totalScanRowsEstimate float64 + totalScanRowsWithoutForecastsEstimate float64 + }{ + { + query: "SELECT 1", + logStmt: "SELECT ‹1›", + }, + { + query: "SELECT * FROM d WHERE true", + logStmt: `SELECT * FROM \"\".\"\".d WHERE ‹true›`, + + scanCount: 1, + }, + { + query: "SELECT * FROM e WHERE true", + logStmt: `SELECT * FROM \"\".\"\".e WHERE ‹true›`, + + scanCount: 1, + scanWithStatsCount: 1, + totalScanRowsEstimate: 20, + totalScanRowsWithoutForecastsEstimate: 20, + }, + { + query: "SELECT * FROM f WHERE true", + logStmt: `SELECT * FROM \"\".\"\".f WHERE ‹true›`, + + scanCount: 1, + scanWithStatsCount: 1, + scanWithStatsForecastCount: 1, + totalScanRowsEstimate: 4, + totalScanRowsWithoutForecastsEstimate: 3, + }, + { + query: "SELECT * FROM d INNER HASH JOIN e ON d = e INNER HASH JOIN f ON e = f", + logStmt: `SELECT * FROM \"\".\"\".d INNER HASH JOIN \"\".\"\".e ON d = e INNER HASH JOIN \"\".\"\".f ON e = f`, + + scanCount: 3, + scanWithStatsCount: 2, + scanWithStatsForecastCount: 1, + totalScanRowsEstimate: 24, + totalScanRowsWithoutForecastsEstimate: 23, + }, + } + + for _, tc := range testData { + db.Exec(t, tc.query) + } + + log.Flush() + + entries, err := log.FetchEntriesFromFiles( + 0, + math.MaxInt64, + 10000, + regexp.MustCompile(`"EventType":"sampled_query"`), + log.WithMarkedSensitiveData, + ) + + if err != nil { + t.Fatal(err) + } + + if len(entries) == 0 { + t.Fatal(errors.Newf("no entries found")) + } + + t.Log("testcases") +cases: + for _, tc := range testData { + for i := len(entries) - 1; i >= 0; i-- { + if strings.Contains(entries[i].Message, tc.logStmt) { + var entry map[string]interface{} + if err := json.Unmarshal([]byte(entries[i].Message), &entry); err != nil { + t.Error(err) + continue cases + } + get := func(key string) float64 { + if val, ok := entry[key]; ok { + return val.(float64) + } + return 0 + } + + if get("ScanCount") != tc.scanCount { + t.Errorf( + "query `%s` expected ScanCount %v, was: %v", + tc.query, tc.scanCount, get("ScanCount"), + ) + } + if get("ScanWithStatsCount") != tc.scanWithStatsCount { + t.Errorf( + "query `%s` expected ScanWithStatsCount %v, was: %v", + tc.query, tc.scanWithStatsCount, get("ScanWithStatsCount"), + ) + } + if get("ScanWithStatsForecastCount") != tc.scanWithStatsForecastCount { + t.Errorf( + "query `%s` expected ScanWithStatsForecastCount %v, was: %v", + tc.query, tc.scanWithStatsForecastCount, get("ScanWithStatsForecastCount"), + ) + } + if get("TotalScanRowsEstimate") != tc.totalScanRowsEstimate { + t.Errorf( + "query `%s` expected TotalScanRowsEstimate %v, was: %v", + tc.query, tc.totalScanRowsEstimate, get("TotalScanRowsEstimate"), + ) + } + if get("TotalScanRowsWithoutForecastsEstimate") != tc.totalScanRowsWithoutForecastsEstimate { + t.Errorf( + "query `%s` expected TotalScanRowsWithoutForecastsEstimate %v, was: %v", + tc.query, tc.totalScanRowsWithoutForecastsEstimate, get("TotalScanRowsWithoutForecastsEstimate"), + ) + } + if tc.scanWithStatsForecastCount > 0 { + if get("NanosSinceStatsForecasted") <= 0 { + t.Errorf( + "query `%s` expected NanosSinceStatsForecasted > 0, was: %v", + tc.query, get("NanosSinceStatsForecasted"), + ) + } + if get("NanosSinceStatsForecasted") >= get("NanosSinceStatsCollected") { + t.Errorf( + "query `%s` expected NanosSinceStatsForecasted < NanosSinceStatsCollected: %v, %v", + tc.query, get("NanosSinceStatsForecasted"), get("NanosSinceStatsCollected"), + ) + } + } + continue cases + } + } + t.Errorf("couldn't find log entry containing `%s`", tc.logStmt) + } +} diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index 6758dcf7470e..14ae69555ae4 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -4256,6 +4256,51 @@ func (m *SampledQuery) AppendJSONFields(printComma bool, b redact.RedactableByte b = append(b, ']') } + if m.ScanCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"ScanCount\":"...) + b = strconv.AppendInt(b, int64(m.ScanCount), 10) + } + + if m.ScanWithStatsCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"ScanWithStatsCount\":"...) + b = strconv.AppendInt(b, int64(m.ScanWithStatsCount), 10) + } + + if m.ScanWithStatsForecastCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"ScanWithStatsForecastCount\":"...) + b = strconv.AppendInt(b, int64(m.ScanWithStatsForecastCount), 10) + } + + if m.TotalScanRowsWithoutForecastsEstimate != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TotalScanRowsWithoutForecastsEstimate\":"...) + b = strconv.AppendFloat(b, float64(m.TotalScanRowsWithoutForecastsEstimate), 'f', -1, 64) + } + + if m.NanosSinceStatsForecasted != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"NanosSinceStatsForecasted\":"...) + b = strconv.AppendInt(b, int64(m.NanosSinceStatsForecasted), 10) + } + return printComma, b } diff --git a/pkg/util/log/eventpb/telemetry.proto b/pkg/util/log/eventpb/telemetry.proto index 77e7b619a158..281bb827fbb4 100644 --- a/pkg/util/log/eventpb/telemetry.proto +++ b/pkg/util/log/eventpb/telemetry.proto @@ -164,6 +164,26 @@ message SampledQuery { // Generated index recommendations for this query. repeated string index_recommendations = 45 [(gogoproto.jsontag) = ',omitempty', (gogoproto.moretags) = "redact:\"nonsensitive\""]; + // The number of scans in the query plan. + int64 scan_count = 46 [(gogoproto.jsontag) = ",omitempty"]; + + // The number of scans using statistics (including forecasted statistics) in + // the query plan. + int64 scan_with_stats_count = 47 [(gogoproto.jsontag) = ",omitempty"]; + + // The number of scans using forecasted statistics in the query plan. + int64 scan_with_stats_forecast_count = 48 [(gogoproto.jsontag) = ",omitempty"]; + + // Total number of rows read by all scans in the query, as estimated by the + // optimizer without using forecasts. + double total_scan_rows_without_forecasts_estimate = 49 [(gogoproto.jsontag) = ",omitempty"]; + + // The greatest quantity of nanoseconds that have passed since the forecast + // time (or until the forecast time, if it is in the future, in which case it + // will be negative) for any table with forecasted stats scanned by this + // query. + int64 nanos_since_stats_forecasted = 50 [(gogoproto.jsontag) = ",omitempty"]; + reserved 12; }