diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 690c76f8063e..bcca1bf17cfb 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -283,4 +283,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 22.1-26 set the active cluster version in the format '.' +version version 22.1-28 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 32142336f566..5d1e60b4d3a9 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -214,6 +214,6 @@ trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabledbooleantrueif set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion22.1-26set the active cluster version in the format '.' +versionversion22.1-28set the active cluster version in the format '.' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 4db9b95a8ccb..61ea8b14d8af 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -325,6 +325,9 @@ const ( // AlterSystemSQLInstancesAddLocality adds a locality column to the // system.sql_instances table. AlterSystemSQLInstancesAddLocality + // AlterSystemStatementStatisticsAddIndexRecommendations adds an + // index_recommendations column to the system.statement_statistics table. + AlterSystemStatementStatisticsAddIndexRecommendations // ************************************************* // Step (1): Add new versions here. @@ -558,6 +561,10 @@ var versionsSingleton = keyedVersions{ Key: AlterSystemSQLInstancesAddLocality, Version: roachpb.Version{Major: 22, Minor: 1, Internal: 26}, }, + { + Key: AlterSystemStatementStatisticsAddIndexRecommendations, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 28}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 70e586b01728..ce8e507e856e 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -57,11 +57,12 @@ func _() { _ = x[SystemPrivilegesTable-46] _ = x[EnablePredicateProjectionChangefeed-47] _ = x[AlterSystemSQLInstancesAddLocality-48] + _ = x[AlterSystemStatementStatisticsAddIndexRecommendations-49] } -const _Key_name = "V21_2Start22_1PebbleFormatBlockPropertyCollectorProbeRequestPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesMVCCIndexBackfillerEnableLeaseHolderRemovalLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocality" +const _Key_name = "V21_2Start22_1PebbleFormatBlockPropertyCollectorProbeRequestPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersEnablePebbleFormatVersionBlockPropertiesMVCCIndexBackfillerEnableLeaseHolderRemovalLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestampsEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalityAlterSystemStatementStatisticsAddIndexRecommendations" -var _Key_index = [...]uint16{0, 5, 14, 48, 60, 88, 118, 146, 167, 186, 220, 258, 292, 324, 360, 392, 428, 470, 510, 529, 553, 584, 602, 643, 673, 684, 715, 738, 762, 786, 808, 820, 846, 860, 881, 899, 904, 913, 928, 962, 996, 1018, 1038, 1057, 1090, 1109, 1129, 1150, 1185, 1219} +var _Key_index = [...]uint16{0, 5, 14, 48, 60, 88, 118, 146, 167, 186, 220, 258, 292, 324, 360, 392, 428, 470, 510, 529, 553, 584, 602, 643, 673, 684, 715, 738, 762, 786, 808, 820, 846, 860, 881, 899, 904, 913, 928, 962, 996, 1018, 1038, 1057, 1090, 1109, 1129, 1150, 1185, 1219, 1272} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/roachpb/app_stats.go b/pkg/roachpb/app_stats.go index fd511eb1790f..4078aa35cfb0 100644 --- a/pkg/roachpb/app_stats.go +++ b/pkg/roachpb/app_stats.go @@ -155,6 +155,7 @@ func (s *StatementStatistics) Add(other *StatementStatistics) { s.RowsWritten.Add(other.RowsWritten, s.Count, other.Count) s.Nodes = util.CombineUniqueInt64(s.Nodes, other.Nodes) s.PlanGists = util.CombineUniqueString(s.PlanGists, other.PlanGists) + s.IndexRecommendations = util.CombineUniqueString(s.IndexRecommendations, other.IndexRecommendations) s.ExecStats.Add(other.ExecStats) diff --git a/pkg/roachpb/app_stats.proto b/pkg/roachpb/app_stats.proto index 3b1b4f1653b9..a75efcf4e90e 100644 --- a/pkg/roachpb/app_stats.proto +++ b/pkg/roachpb/app_stats.proto @@ -104,12 +104,15 @@ message StatementStatistics { // Nodes is the ordered list of nodes ids on which the statement was executed. repeated int64 nodes = 24; - // plan_gists is list of a compressed version of plan that can be converted (lossily) + // plan_gists is the list of a compressed version of plan that can be converted (lossily) // back into a logical plan. // Each statement contain only one plan gist, but the same statement fingerprint id // can contain more than one value. repeated string plan_gists = 26; + // index_recommendations is the list of index recommendations generated for the statement fingerprint. + repeated string index_recommendations = 27; + // Note: be sure to update `sql/app_stats.go` when adding/removing fields here! reserved 13, 14, 17, 18, 19, 20; diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 850bff19392c..29ad561f9bfc 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -476,6 +476,7 @@ CREATE TABLE system.statement_statistics ( metadata JSONB NOT NULL, statistics JSONB NOT NULL, plan JSONB NOT NULL, + index_recommendations STRING[], crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8 INT4 NOT VISIBLE NOT NULL AS ( mod(fnv32(crdb_internal.datums_to_bytes(aggregated_ts, app_name, fingerprint_id, node_id, plan_hash, transaction_fingerprint_id)), 8:::INT8) @@ -495,7 +496,8 @@ CREATE TABLE system.statement_statistics ( agg_interval, metadata, statistics, - plan + plan, + index_recommendations, ) ) ` @@ -1998,6 +2000,7 @@ var ( ComputeExpr: &sqlStmtHashComputeExpr, Hidden: true, }, + {Name: "index_recommendations", ID: 12, Type: types.StringArray, Nullable: true}, }, []descpb.ColumnFamilyDescriptor{ { @@ -2006,9 +2009,9 @@ var ( ColumnNames: []string{ "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8", "aggregated_ts", "fingerprint_id", "transaction_fingerprint_id", "plan_hash", "app_name", "node_id", - "agg_interval", "metadata", "statistics", "plan", + "agg_interval", "metadata", "statistics", "plan", "index_recommendations", }, - ColumnIDs: []descpb.ColumnID{11, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, + ColumnIDs: []descpb.ColumnID{11, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 12}, DefaultColumnID: 0, }, }, diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 504beedfd496..f66a4693c62c 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1103,7 +1103,8 @@ CREATE TABLE crdb_internal.node_statement_statistics ( sample_plan JSONB, database_name STRING NOT NULL, exec_node_ids INT[] NOT NULL, - txn_fingerprint_id STRING + txn_fingerprint_id STRING, + index_recommendations STRING[] )`, populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { hasViewActivityOrViewActivityRedacted, err := p.HasViewActivityOrViewActivityRedactedRole(ctx) @@ -1156,6 +1157,13 @@ CREATE TABLE crdb_internal.node_statement_statistics ( } + indexRecommendations := tree.NewDArray(types.String) + for _, recommendation := range stats.Stats.IndexRecommendations { + if err := indexRecommendations.Append(tree.NewDString(recommendation)); err != nil { + return err + } + } + err := addRow( tree.NewDInt(tree.DInt(nodeID)), // node_id tree.NewDString(stats.Key.App), // application_name @@ -1199,6 +1207,7 @@ CREATE TABLE crdb_internal.node_statement_statistics ( tree.NewDString(stats.Key.Database), // database_name execNodeIDs, // exec_node_ids txnFingerprintID, // txn_fingerprint_id + indexRecommendations, // index_recommendations ) if err != nil { return err @@ -5368,7 +5377,8 @@ CREATE TABLE crdb_internal.cluster_statement_statistics ( metadata JSONB NOT NULL, statistics JSONB NOT NULL, sampled_plan JSONB NOT NULL, - aggregation_interval INTERVAL NOT NULL + aggregation_interval INTERVAL NOT NULL, + index_recommendations STRING[] );`, generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) { // TODO(azhng): we want to eventually implement memory accounting within the @@ -5399,7 +5409,7 @@ CREATE TABLE crdb_internal.cluster_statement_statistics ( curAggTs := s.ComputeAggregatedTs() aggInterval := s.GetAggregationInterval() - row := make(tree.Datums, 8 /* number of columns for this virtual table */) + row := make(tree.Datums, 9 /* number of columns for this virtual table */) worker := func(ctx context.Context, pusher rowPusher) error { return memSQLStats.IterateStatementStats(ctx, &sqlstats.IteratorOptions{ SortedAppNames: true, @@ -5434,6 +5444,13 @@ CREATE TABLE crdb_internal.cluster_statement_statistics ( duration.MakeDuration(aggInterval.Nanoseconds(), 0, 0), types.DefaultIntervalTypeMetadata) + indexRecommendations := tree.NewDArray(types.String) + for _, recommendation := range statistics.Stats.IndexRecommendations { + if err := indexRecommendations.Append(tree.NewDString(recommendation)); err != nil { + return err + } + } + row = row[:0] row = append(row, aggregatedTs, // aggregated_ts @@ -5445,6 +5462,7 @@ CREATE TABLE crdb_internal.cluster_statement_statistics ( tree.NewDJSON(statisticsJSON), // statistics tree.NewDJSON(plan), // plan aggInterval, // aggregation_interval + indexRecommendations, // index_recommendations ) return pusher.pushRow(row...) @@ -5470,7 +5488,8 @@ SELECT max(metadata) as metadata, crdb_internal.merge_statement_stats(array_agg(statistics)), max(sampled_plan), - aggregation_interval + aggregation_interval, + array_remove(array_agg(index_rec), null) AS index_recommendations FROM ( SELECT aggregated_ts, @@ -5481,7 +5500,8 @@ FROM ( metadata, statistics, sampled_plan, - aggregation_interval + aggregation_interval, + index_recommendations FROM crdb_internal.cluster_statement_statistics UNION ALL @@ -5494,10 +5514,12 @@ FROM ( metadata, statistics, plan, - agg_interval + agg_interval, + index_recommendations FROM system.statement_statistics ) +LEFT JOIN LATERAL unnest(index_recommendations) AS index_rec ON TRUE GROUP BY aggregated_ts, fingerprint_id, @@ -5511,6 +5533,7 @@ GROUP BY {Name: "transaction_fingerprint_id", Typ: types.Bytes}, {Name: "plan_hash", Typ: types.Bytes}, {Name: "app_name", Typ: types.String}, + {Name: "index_recommendations", Typ: types.StringArray}, {Name: "metadata", Typ: types.Jsonb}, {Name: "statistics", Typ: types.Jsonb}, {Name: "sampled_plan", Typ: types.Jsonb}, diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index 73751375cad1..9f7316663405 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -168,23 +168,24 @@ func (ex *connExecutor) recordStatementSummary( } recordedStmtStats := sqlstats.RecordedStmtStats{ - SessionID: ex.sessionID, - StatementID: planner.stmt.QueryID, - AutoRetryCount: automaticRetryCount, - RowsAffected: rowsAffected, - ParseLatency: parseLat, - PlanLatency: planLat, - RunLatency: runLat, - ServiceLatency: svcLat, - OverheadLatency: execOverhead, - BytesRead: stats.bytesRead, - RowsRead: stats.rowsRead, - RowsWritten: stats.rowsWritten, - Nodes: getNodesFromPlanner(planner), - StatementType: stmt.AST.StatementType(), - Plan: planner.instrumentation.PlanForStats(ctx), - PlanGist: planner.instrumentation.planGist.String(), - StatementError: stmtErr, + SessionID: ex.sessionID, + StatementID: planner.stmt.QueryID, + AutoRetryCount: automaticRetryCount, + RowsAffected: rowsAffected, + ParseLatency: parseLat, + PlanLatency: planLat, + RunLatency: runLat, + ServiceLatency: svcLat, + OverheadLatency: execOverhead, + BytesRead: stats.bytesRead, + RowsRead: stats.rowsRead, + RowsWritten: stats.rowsWritten, + Nodes: getNodesFromPlanner(planner), + StatementType: stmt.AST.StatementType(), + Plan: planner.instrumentation.PlanForStats(ctx), + PlanGist: planner.instrumentation.planGist.String(), + StatementError: stmtErr, + IndexRecommendations: planner.instrumentation.indexRecommendations, } stmtFingerprintID, err := diff --git a/pkg/sql/logictest/testdata/logic_test/create_statements b/pkg/sql/logictest/testdata/logic_test/create_statements index 96a07d1759ec..417969feaa7d 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -381,7 +381,8 @@ CREATE TABLE crdb_internal.cluster_statement_statistics ( metadata JSONB NOT NULL, statistics JSONB NOT NULL, sampled_plan JSONB NOT NULL, - aggregation_interval INTERVAL NOT NULL + aggregation_interval INTERVAL NOT NULL, + index_recommendations STRING[] ) CREATE TABLE crdb_internal.cluster_statement_statistics ( aggregated_ts TIMESTAMPTZ NOT NULL, fingerprint_id BYTES NOT NULL, @@ -391,7 +392,8 @@ CREATE TABLE crdb_internal.cluster_statement_statistics ( metadata JSONB NOT NULL, statistics JSONB NOT NULL, sampled_plan JSONB NOT NULL, - aggregation_interval INTERVAL NOT NULL + aggregation_interval INTERVAL NOT NULL, + index_recommendations STRING[] ) {} {} CREATE TABLE crdb_internal.cluster_transaction_statistics ( aggregated_ts TIMESTAMPTZ NOT NULL, @@ -1041,7 +1043,8 @@ CREATE TABLE crdb_internal.node_statement_statistics ( sample_plan JSONB NULL, database_name STRING NOT NULL, exec_node_ids INT8[] NOT NULL, - txn_fingerprint_id STRING NULL + txn_fingerprint_id STRING NULL, + index_recommendations STRING[] ) CREATE TABLE crdb_internal.node_statement_statistics ( node_id INT8 NOT NULL, application_name STRING NOT NULL, @@ -1084,7 +1087,8 @@ CREATE TABLE crdb_internal.node_statement_statistics ( sample_plan JSONB NULL, database_name STRING NOT NULL, exec_node_ids INT8[] NOT NULL, - txn_fingerprint_id STRING NULL + txn_fingerprint_id STRING NULL, + index_recommendations STRING[] ) {} {} CREATE TABLE crdb_internal.node_transaction_statistics ( node_id INT8 NOT NULL, @@ -1401,7 +1405,8 @@ CREATE VIEW crdb_internal.statement_statistics ( metadata, statistics, sampled_plan, - aggregation_interval + aggregation_interval, + index_recommendations ) AS SELECT aggregated_ts, fingerprint_id, @@ -1411,7 +1416,8 @@ CREATE VIEW crdb_internal.statement_statistics ( max(metadata) AS metadata, crdb_internal.merge_statement_stats(array_agg(statistics)), max(sampled_plan), - aggregation_interval + aggregation_interval, + array_remove(array_agg(index_rec), null) AS index_recommendations FROM ( SELECT @@ -1423,7 +1429,8 @@ CREATE VIEW crdb_internal.statement_statistics ( metadata, statistics, sampled_plan, - aggregation_interval + aggregation_interval, + index_recommendations FROM crdb_internal.cluster_statement_statistics UNION ALL @@ -1436,10 +1443,12 @@ CREATE VIEW crdb_internal.statement_statistics ( metadata, statistics, plan, - agg_interval + agg_interval, + index_recommendations FROM system.statement_statistics ) + LEFT JOIN LATERAL unnest(index_recommendations) AS index_rec ON TRUE GROUP BY aggregated_ts, fingerprint_id, @@ -1455,7 +1464,8 @@ CREATE VIEW crdb_internal.statement_statistics ( metadata, statistics, sampled_plan, - aggregation_interval + aggregation_interval, + index_recommendations ) AS SELECT aggregated_ts, fingerprint_id, @@ -1465,7 +1475,8 @@ CREATE VIEW crdb_internal.statement_statistics ( max(metadata) AS metadata, crdb_internal.merge_statement_stats(array_agg(statistics)), max(sampled_plan), - aggregation_interval + aggregation_interval, + array_remove(array_agg(index_rec), null) AS index_recommendations FROM ( SELECT @@ -1477,7 +1488,8 @@ CREATE VIEW crdb_internal.statement_statistics ( metadata, statistics, sampled_plan, - aggregation_interval + aggregation_interval, + index_recommendations FROM crdb_internal.cluster_statement_statistics UNION ALL @@ -1490,10 +1502,12 @@ CREATE VIEW crdb_internal.statement_statistics ( metadata, statistics, plan, - agg_interval + agg_interval, + index_recommendations FROM system.statement_statistics ) + LEFT JOIN LATERAL unnest(index_recommendations) AS index_rec ON TRUE GROUP BY aggregated_ts, fingerprint_id, diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 27938ecb7e01..37668ea0ac32 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -1913,6 +1913,7 @@ system public statement_statistics fingerprint_id system public statement_statistics node_id system public primary system public statement_statistics plan_hash system public primary system public statement_statistics transaction_fingerprint_id system public primary +system public statement_statistics index_recommendations system public primary system public table_statistics statisticID system public primary system public table_statistics tableID system public primary system public tenant_settings name system public primary diff --git a/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel b/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel index 816f6bc5206f..ec58e9b59a70 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel +++ b/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel @@ -41,6 +41,7 @@ go_library( "//pkg/sql/sqlstats/sslocal", "//pkg/sql/sqlstats/ssmemstorage", "//pkg/sql/sqlutil", + "//pkg/util", "//pkg/util/log", "//pkg/util/metric", "//pkg/util/mon", diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go index 04b23341bf5c..a3be9f3bb3a6 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding_test.go @@ -97,7 +97,8 @@ func TestSQLStatsJsonEncoding(t *testing.T) { "sqDiff": {{.Float}} }, "nodes": [{{joinInts .IntArray}}], - "planGists": [{{joinStrings .StringArray}}] + "planGists": [{{joinStrings .StringArray}}], + "indexRecommendations": [{{joinStrings .StringArray}}] }, "execution_statistics": { "cnt": {{.Int64}}, diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go index 240dee25204e..e0c1d7e963d2 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go @@ -290,6 +290,7 @@ func (s *innerStmtStats) jsonFields() jsonFields { {"rowsWritten", (*numericStats)(&s.RowsWritten)}, {"nodes", (*int64Array)(&s.Nodes)}, {"planGists", (*stringArray)(&s.PlanGists)}, + {"indexRecommendations", (*stringArray)(&s.IndexRecommendations)}, } } diff --git a/pkg/sql/sqlstats/persistedsqlstats/stmt_reader.go b/pkg/sql/sqlstats/persistedsqlstats/stmt_reader.go index b34d60684a12..d803735ea15c 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/stmt_reader.go +++ b/pkg/sql/sqlstats/persistedsqlstats/stmt_reader.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/errors" ) @@ -111,6 +112,7 @@ func (s *PersistedSQLStats) getFetchQueryForStmtStatsTable( "statistics", "plan", "agg_interval", + "index_recommendations", } // [1]: selection columns @@ -188,5 +190,12 @@ func rowToStmtStats(row tree.Datums) (*roachpb.CollectedStatementStatistics, err aggInterval := tree.MustBeDInterval(row[8]).Duration stats.AggregationInterval = time.Duration(aggInterval.Nanos()) + recommendations := tree.MustBeDArray(row[9]) + var indeRecommendations []string + for _, s := range recommendations.Array { + indeRecommendations = util.CombineUniqueString(indeRecommendations, []string{string(tree.MustBeDString(s))}) + } + stats.Stats.IndexRecommendations = indeRecommendations + return &stats, nil } diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go index 9691c0cc194b..4b742b7ef7f1 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go @@ -126,6 +126,7 @@ func (s *Container) RecordStatement( stats.mu.data.LastExecTimestamp = s.getTimeNow() stats.mu.data.Nodes = util.CombineUniqueInt64(stats.mu.data.Nodes, value.Nodes) stats.mu.data.PlanGists = util.CombineUniqueString(stats.mu.data.PlanGists, []string{value.PlanGist}) + stats.mu.data.IndexRecommendations = util.CombineUniqueString(stats.mu.data.IndexRecommendations, value.IndexRecommendations) // Note that some fields derived from tracing statements (such as // BytesSentOverNetwork) are not updated here because they are collected // on-demand. diff --git a/pkg/sql/sqlstats/ssprovider.go b/pkg/sql/sqlstats/ssprovider.go index 791c717ec2c5..479b099455cc 100644 --- a/pkg/sql/sqlstats/ssprovider.go +++ b/pkg/sql/sqlstats/ssprovider.go @@ -191,23 +191,24 @@ type Provider interface { // RecordedStmtStats stores the statistics of a statement to be recorded. type RecordedStmtStats struct { - SessionID clusterunique.ID - StatementID clusterunique.ID - AutoRetryCount int - RowsAffected int - ParseLatency float64 - PlanLatency float64 - RunLatency float64 - ServiceLatency float64 - OverheadLatency float64 - BytesRead int64 - RowsRead int64 - RowsWritten int64 - Nodes []int64 - StatementType tree.StatementType - Plan *roachpb.ExplainTreePlanNode - PlanGist string - StatementError error + SessionID clusterunique.ID + StatementID clusterunique.ID + AutoRetryCount int + RowsAffected int + ParseLatency float64 + PlanLatency float64 + RunLatency float64 + ServiceLatency float64 + OverheadLatency float64 + BytesRead int64 + RowsRead int64 + RowsWritten int64 + Nodes []int64 + StatementType tree.StatementType + Plan *roachpb.ExplainTreePlanNode + PlanGist string + StatementError error + IndexRecommendations []string } // RecordedTxnStats stores the statistics of a transaction to be recorded. diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.fixture.ts b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.fixture.ts index b1e1d3084d90..47124953d7e0 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.fixture.ts +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.fixture.ts @@ -94,6 +94,7 @@ const statementStats: Required = { squared_diffs: 0.005, }, plan_gists: ["Ais="], + index_recommendations: [""], exec_stats: execStats, last_exec_timestamp: { seconds: Long.fromInt(1599670292), diff --git a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.spec.ts b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.spec.ts index fc1306dc57a7..f35b12c121ce 100644 --- a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.spec.ts +++ b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.spec.ts @@ -275,6 +275,7 @@ function randomStats( }, nodes: [Long.fromInt(1), Long.fromInt(3), Long.fromInt(4)], plan_gists: ["Ais="], + index_recommendations: [""], }; } diff --git a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts index e047e04f4474..f579fb161829 100644 --- a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts +++ b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts @@ -139,6 +139,15 @@ export function addStatementStats( planGists = b.plan_gists; } + let indexRec: string[] = []; + if (a.index_recommendations && b.index_recommendations) { + indexRec = unique(a.index_recommendations.concat(b.index_recommendations)); + } else if (a.index_recommendations) { + indexRec = a.index_recommendations; + } else if (b.index_recommendations) { + indexRec = b.index_recommendations; + } + return { count: a.count.add(b.count), first_attempt_count: a.first_attempt_count.add(b.first_attempt_count), @@ -187,6 +196,7 @@ export function addStatementStats( : b.last_exec_timestamp, nodes: uniqueLong([...a.nodes, ...b.nodes]), plan_gists: planGists, + index_recommendations: indexRec, }; } diff --git a/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx b/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx index b6f463a87769..bebb26596b16 100644 --- a/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx +++ b/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx @@ -527,6 +527,7 @@ function makeStats(): Required { }, nodes: [Long.fromInt(1), Long.fromInt(2), Long.fromInt(3)], plan_gists: ["Ais="], + index_recommendations: [], }; } diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index f49d6b4aa375..fbbacd55b254 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -5,6 +5,7 @@ go_library( name = "upgrades", srcs = [ "alter_sql_instances_locality.go", + "alter_statement_statistics_index_recommendations.go", "alter_table_protected_timestamp_records.go", "comment_on_index_migration.go", "descriptor_utils.go", @@ -66,6 +67,7 @@ go_test( size = "large", srcs = [ "alter_sql_instances_locality_test.go", + "alter_statement_statistics_index_recommendations_test.go", "alter_table_protected_timestamp_records_test.go", "builtins_test.go", "comment_on_index_migration_external_test.go", diff --git a/pkg/upgrade/upgrades/alter_statement_statistics_index_recommendations.go b/pkg/upgrade/upgrades/alter_statement_statistics_index_recommendations.go new file mode 100644 index 000000000000..850dee9b44fa --- /dev/null +++ b/pkg/upgrade/upgrades/alter_statement_statistics_index_recommendations.go @@ -0,0 +1,42 @@ +// 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 upgrades + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/upgrade" +) + +const addIndexRecommendationsCol = ` +ALTER TABLE system.statement_statistics +ADD COLUMN IF NOT EXISTS "index_recommendations" STRING[] +FAMILY "primary" +` + +func alterSystemStatementStatisticsAddIndexRecommendations( + ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps, _ *jobs.Job, +) error { + op := operation{ + name: "add-statement-statistics-index-recommendations-col", + schemaList: []string{"index_rec"}, + query: addIndexRecommendationsCol, + schemaExistsFn: hasColumn, + } + if err := migrateTable(ctx, cs, d, op, keys.StatementStatisticsTableID, systemschema.StatementStatisticsTable); err != nil { + return err + } + return nil +} diff --git a/pkg/upgrade/upgrades/alter_statement_statistics_index_recommendations_test.go b/pkg/upgrade/upgrades/alter_statement_statistics_index_recommendations_test.go new file mode 100644 index 000000000000..893bf951faab --- /dev/null +++ b/pkg/upgrade/upgrades/alter_statement_statistics_index_recommendations_test.go @@ -0,0 +1,193 @@ +// 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 upgrades_test + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +func TestAlterSystemStatementStatisticsTable(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + clusterArgs := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: clusterversion.ByKey( + clusterversion.AlterSystemStatementStatisticsAddIndexRecommendations - 1), + }, + }, + }, + } + + var ( + ctx = context.Background() + + tc = testcluster.StartTestCluster(t, 1, clusterArgs) + s = tc.Server(0) + sqlDB = tc.ServerConn(0) + ) + defer tc.Stopper().Stop(ctx) + + var ( + validationSchemas = []upgrades.Schema{ + {Name: "index_recommendations", ValidationFn: upgrades.HasColumn}, + {Name: "primary", ValidationFn: upgrades.HasColumnFamily}, + } + ) + + // Inject the old copy of the descriptor. + upgrades.InjectLegacyTable(ctx, t, s, systemschema.StatementStatisticsTable, getDeprecatedStatementStatisticsDescriptor) + // Validate that the table statement_statistics has the old schema. + upgrades.ValidateSchemaExists( + ctx, + t, + s, + sqlDB, + keys.StatementStatisticsTableID, + systemschema.StatementStatisticsTable, + []string{}, + validationSchemas, + false, /* expectExists */ + ) + // Run the upgrade. + upgrades.Upgrade( + t, + sqlDB, + clusterversion.AlterSystemStatementStatisticsAddIndexRecommendations, + nil, /* done */ + false, /* expectError */ + ) + // Validate that the table has new schema. + upgrades.ValidateSchemaExists( + ctx, + t, + s, + sqlDB, + keys.StatementStatisticsTableID, + systemschema.StatementStatisticsTable, + []string{}, + validationSchemas, + true, /* expectExists */ + ) +} + +// getDeprecatedSqlInstancesDescriptor returns the system.sql_instances +// table descriptor that was being used before adding a new column in the +// current version. +func getDeprecatedStatementStatisticsDescriptor() *descpb.TableDescriptor { + sqlStmtHashComputeExpr := `mod(fnv32(crdb_internal.datums_to_bytes(aggregated_ts, app_name, fingerprint_id, node_id, plan_hash, transaction_fingerprint_id)), 8:::INT8)` + + return &descpb.TableDescriptor{ + Name: string(catconstants.StatementStatisticsTableName), + ID: keys.StatementStatisticsTableID, + ParentID: keys.SystemDatabaseID, + UnexposedParentSchemaID: keys.PublicSchemaID, + Version: 1, + Columns: []descpb.ColumnDescriptor{ + {Name: "aggregated_ts", ID: 1, Type: types.TimestampTZ, Nullable: false}, + {Name: "fingerprint_id", ID: 2, Type: types.Bytes, Nullable: false}, + {Name: "transaction_fingerprint_id", ID: 3, Type: types.Bytes, Nullable: false}, + {Name: "plan_hash", ID: 4, Type: types.Bytes, Nullable: false}, + {Name: "app_name", ID: 5, Type: types.String, Nullable: false}, + {Name: "node_id", ID: 6, Type: types.Int, Nullable: false}, + {Name: "agg_interval", ID: 7, Type: types.Interval, Nullable: false}, + {Name: "metadata", ID: 8, Type: types.Jsonb, Nullable: false}, + {Name: "statistics", ID: 9, Type: types.Jsonb, Nullable: false}, + {Name: "plan", ID: 10, Type: types.Jsonb, Nullable: false}, + { + Name: "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8", + ID: 11, + Type: types.Int4, + Nullable: false, + ComputeExpr: &sqlStmtHashComputeExpr, + Hidden: true, + }, + }, + NextColumnID: 12, + Families: []descpb.ColumnFamilyDescriptor{ + { + Name: "primary", + ID: 0, + ColumnNames: []string{ + "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8", + "aggregated_ts", "fingerprint_id", "transaction_fingerprint_id", "plan_hash", "app_name", "node_id", + "agg_interval", "metadata", "statistics", "plan", + }, + ColumnIDs: []descpb.ColumnID{11, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, + DefaultColumnID: 0, + }, + }, + NextFamilyID: 1, + PrimaryIndex: descpb.IndexDescriptor{ + Name: tabledesc.LegacyPrimaryKeyIndexName, + ID: 1, + Unique: true, + KeyColumnNames: []string{ + "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8", + "aggregated_ts", + "fingerprint_id", + "transaction_fingerprint_id", + "plan_hash", + "app_name", + "node_id", + }, + KeyColumnDirections: []catpb.IndexColumn_Direction{ + catpb.IndexColumn_ASC, + catpb.IndexColumn_ASC, + catpb.IndexColumn_ASC, + catpb.IndexColumn_ASC, + catpb.IndexColumn_ASC, + catpb.IndexColumn_ASC, + catpb.IndexColumn_ASC, + }, + KeyColumnIDs: []descpb.ColumnID{11, 1, 2, 3, 4, 5, 6}, + Version: descpb.StrictIndexColumnIDGuaranteesVersion, + Sharded: catpb.ShardedDescriptor{ + IsSharded: true, + Name: "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8", + ShardBuckets: 8, + ColumnNames: []string{ + "aggregated_ts", + "app_name", + "fingerprint_id", + "node_id", + "plan_hash", + "transaction_fingerprint_id", + }, + }, + }, + NextIndexID: 3, + Privileges: catpb.NewCustomSuperuserPrivilegeDescriptor(privilege.ReadWriteData, username.NodeUserName()), + NextMutationID: 1, + FormatVersion: 3, + } +} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index fd30653f7090..b3bb4950fceb 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -134,6 +134,12 @@ var upgrades = []upgrade.Upgrade{ NoPrecondition, alterSystemSQLInstancesAddLocality, ), + upgrade.NewTenantUpgrade( + "add column index_recommendations to table system.statement_statistics", + toCV(clusterversion.AlterSystemStatementStatisticsAddIndexRecommendations), + NoPrecondition, + alterSystemStatementStatisticsAddIndexRecommendations, + ), } func init() {