diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt
index 14848a114bf1..15f8f2008bb1 100644
--- a/docs/generated/settings/settings-for-tenants.txt
+++ b/docs/generated/settings/settings-for-tenants.txt
@@ -240,6 +240,7 @@ sql.metrics.max_mem_txn_fingerprints integer 100000 the maximum number of transa
sql.metrics.statement_details.dump_to_logs boolean false dump collected statement statistics to node logs when periodically cleared
sql.metrics.statement_details.enabled boolean true collect per-statement query statistics
sql.metrics.statement_details.index_recommendation_collection.enabled boolean true generate an index recommendation for each fingerprint ID
+sql.metrics.statement_details.max_mem_reported_idx_recommendations integer 5000 the maximum number of reported index recommendation info stored in memory
sql.metrics.statement_details.plan_collection.enabled boolean true periodically save a logical plan for each fingerprint
sql.metrics.statement_details.plan_collection.period duration 5m0s the time until a new logical plan is collected
sql.metrics.statement_details.threshold duration 0s minimum execution time to cause statement statistics to be collected. If configured, no transaction stats are collected.
diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index 3de6c8914aa8..b3be03bba605 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -171,6 +171,7 @@
sql.metrics.statement_details.dump_to_logs | boolean | false | dump collected statement statistics to node logs when periodically cleared |
sql.metrics.statement_details.enabled | boolean | true | collect per-statement query statistics |
sql.metrics.statement_details.index_recommendation_collection.enabled | boolean | true | generate an index recommendation for each fingerprint ID |
+sql.metrics.statement_details.max_mem_reported_idx_recommendations | integer | 5000 | the maximum number of reported index recommendation info stored in memory |
sql.metrics.statement_details.plan_collection.enabled | boolean | true | periodically save a logical plan for each fingerprint |
sql.metrics.statement_details.plan_collection.period | duration | 5m0s | the time until a new logical plan is collected |
sql.metrics.statement_details.threshold | duration | 0s | minimum execution time to cause statement statistics to be collected. If configured, no transaction stats are collected. |
diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel
index 8f4eeb87346a..28f63c9354f3 100644
--- a/pkg/BUILD.bazel
+++ b/pkg/BUILD.bazel
@@ -342,6 +342,7 @@ ALL_TESTS = [
"//pkg/sql/gcjob/gcjobnotifier:gcjobnotifier_test",
"//pkg/sql/gcjob:gcjob_test",
"//pkg/sql/gcjob_test:gcjob_test_test",
+ "//pkg/sql/idxrecommendations:idxrecommendations_test",
"//pkg/sql/idxusage:idxusage_test",
"//pkg/sql/importer:importer_test",
"//pkg/sql/inverted:inverted_disallowed_imports_test",
@@ -1416,6 +1417,8 @@ GO_TARGETS = [
"//pkg/sql/gcjob:gcjob",
"//pkg/sql/gcjob:gcjob_test",
"//pkg/sql/gcjob_test:gcjob_test_test",
+ "//pkg/sql/idxrecommendations:idxrecommendations",
+ "//pkg/sql/idxrecommendations:idxrecommendations_test",
"//pkg/sql/idxusage:idxusage",
"//pkg/sql/idxusage:idxusage_test",
"//pkg/sql/importer:importer",
@@ -2523,6 +2526,7 @@ GET_X_DATA_TARGETS = [
"//pkg/sql/gcjob:get_x_data",
"//pkg/sql/gcjob/gcjobnotifier:get_x_data",
"//pkg/sql/gcjob_test:get_x_data",
+ "//pkg/sql/idxrecommendations:get_x_data",
"//pkg/sql/idxusage:get_x_data",
"//pkg/sql/importer:get_x_data",
"//pkg/sql/inverted:get_x_data",
diff --git a/pkg/roachpb/app_stats.go b/pkg/roachpb/app_stats.go
index 4078aa35cfb0..a7c7cfcc1265 100644
--- a/pkg/roachpb/app_stats.go
+++ b/pkg/roachpb/app_stats.go
@@ -155,7 +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.IndexRecommendations = other.IndexRecommendations
s.ExecStats.Add(other.ExecStats)
diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel
index c652503973a9..1c65bdc3c27b 100644
--- a/pkg/sql/BUILD.bazel
+++ b/pkg/sql/BUILD.bazel
@@ -350,6 +350,7 @@ go_library(
"//pkg/sql/faketreeeval",
"//pkg/sql/flowinfra",
"//pkg/sql/gcjob/gcjobnotifier",
+ "//pkg/sql/idxrecommendations",
"//pkg/sql/idxusage",
"//pkg/sql/inverted",
"//pkg/sql/lex",
diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go
index 1501120152d6..dda214fb668d 100644
--- a/pkg/sql/conn_executor.go
+++ b/pkg/sql/conn_executor.go
@@ -37,6 +37,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/clusterunique"
"github.com/cockroachdb/cockroach/pkg/sql/contention/txnidcache"
"github.com/cockroachdb/cockroach/pkg/sql/execstats"
+ "github.com/cockroachdb/cockroach/pkg/sql/idxrecommendations"
"github.com/cockroachdb/cockroach/pkg/sql/idxusage"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
@@ -305,6 +306,8 @@ type Server struct {
// TelemetryLoggingMetrics is used to track metrics for logging to the telemetry channel.
TelemetryLoggingMetrics *TelemetryLoggingMetrics
+ idxRecommendationsCache *idxrecommendations.IndexRecCache
+
mu struct {
syncutil.Mutex
connectionCount int64
@@ -393,6 +396,9 @@ func NewServer(cfg *ExecutorConfig, pool *mon.BytesMonitor) *Server {
txnIDCache: txnidcache.NewTxnIDCache(
cfg.Settings,
&serverMetrics.ContentionSubsystemMetrics),
+ idxRecommendationsCache: idxrecommendations.NewIndexRecommendationsCache(
+ cfg.Settings,
+ sqlstats.MaxMemReportedSampleIndexRecommendations),
}
telemetryLoggingMetrics := &TelemetryLoggingMetrics{}
diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go
index bec283f1be10..1f94819d6b76 100644
--- a/pkg/sql/conn_executor_exec.go
+++ b/pkg/sql/conn_executor_exec.go
@@ -1194,6 +1194,9 @@ func (ex *connExecutor) dispatchToExecutionEngine(
ex.extraTxnState.bytesRead += stats.bytesRead
ex.extraTxnState.rowsWritten += stats.rowsWritten
+ // Set index recommendations so it can be saved on statement statistics.
+ planner.instrumentation.SetIndexRecommendations(ctx, ex.server.idxRecommendationsCache, planner)
+
// Record the statement summary. This also closes the plan if the
// plan has not been closed earlier.
stmtFingerprintID = ex.recordStatementSummary(
diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go
index 1c32fe8dbb5c..dc666f8ae22e 100644
--- a/pkg/sql/executor_statement_metrics.go
+++ b/pkg/sql/executor_statement_metrics.go
@@ -15,6 +15,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
+ "github.com/cockroachdb/cockroach/pkg/sql/idxrecommendations"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessionphase"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
@@ -169,6 +170,7 @@ func (ex *connExecutor) recordStatementSummary(
PlanHash: planner.instrumentation.planGist.Hash(),
}
+ idxRecommendations := idxrecommendations.FormatIdxRecommendations(planner.instrumentation.indexRecommendations)
recordedStmtStats := sqlstats.RecordedStmtStats{
SessionID: ex.sessionID,
StatementID: planner.stmt.QueryID,
@@ -187,7 +189,7 @@ func (ex *connExecutor) recordStatementSummary(
Plan: planner.instrumentation.PlanForStats(ctx),
PlanGist: planner.instrumentation.planGist.String(),
StatementError: stmtErr,
- IndexRecommendations: planner.instrumentation.indexRecommendations,
+ IndexRecommendations: idxRecommendations,
Query: stmt.StmtNoConstants,
StartTime: phaseTimes.GetSessionPhaseTime(sessionphase.PlannerStartExecStmt),
EndTime: phaseTimes.GetSessionPhaseTime(sessionphase.PlannerEndExecStmt),
diff --git a/pkg/sql/idxrecommendations/BUILD.bazel b/pkg/sql/idxrecommendations/BUILD.bazel
new file mode 100644
index 000000000000..f1d774ed77bf
--- /dev/null
+++ b/pkg/sql/idxrecommendations/BUILD.bazel
@@ -0,0 +1,43 @@
+load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
+load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
+
+go_library(
+ name = "idxrecommendations",
+ srcs = [
+ "idx_recommendations.go",
+ "idx_recommendations_cache.go",
+ ],
+ importpath = "github.com/cockroachdb/cockroach/pkg/sql/idxrecommendations",
+ visibility = ["//visibility:public"],
+ deps = [
+ "//pkg/settings",
+ "//pkg/settings/cluster",
+ "//pkg/sql/sem/tree",
+ "//pkg/sql/sqlstats",
+ "//pkg/util/syncutil",
+ "//pkg/util/timeutil",
+ ],
+)
+
+go_test(
+ name = "idxrecommendations_test",
+ srcs = [
+ "idx_recommendations_cache_test.go",
+ "main_test.go",
+ ],
+ deps = [
+ ":idxrecommendations",
+ "//pkg/security/securityassets",
+ "//pkg/security/securitytest",
+ "//pkg/server",
+ "//pkg/sql/tests",
+ "//pkg/testutils/serverutils",
+ "//pkg/testutils/sqlutils",
+ "//pkg/testutils/testcluster",
+ "//pkg/util/leaktest",
+ "//pkg/util/log",
+ "@com_github_stretchr_testify//require",
+ ],
+)
+
+get_x_data(name = "get_x_data")
diff --git a/pkg/sql/idxrecommendations/idx_recommendations.go b/pkg/sql/idxrecommendations/idx_recommendations.go
new file mode 100644
index 000000000000..4c8fc9bdd18b
--- /dev/null
+++ b/pkg/sql/idxrecommendations/idx_recommendations.go
@@ -0,0 +1,69 @@
+// 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 idxrecommendations
+
+import (
+ "fmt"
+ "strings"
+
+ "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
+)
+
+// IdxRecommendations controls the generation of index recommendations
+// for specific statements, and update accordingly.
+type IdxRecommendations interface {
+ ShouldGenerateIndexRecommendation(
+ fingerprint string, planHash uint64, database string, stmtType tree.StatementType,
+ ) bool
+ UpdateIndexRecommendations(
+ fingerprint string,
+ planHash uint64,
+ database string,
+ stmtType tree.StatementType,
+ recommendations []string,
+ reset bool,
+ ) []string
+}
+
+// FormatIdxRecommendations received a list with recommendations info, e.g.:
+//{
+// "index recommendations: 2",
+// "1. type: index replacement",
+// "SQL commands: CREATE UNIQUE INDEX ON t1 (i) STORING (k); DROP INDEX t1@existing_t1_i;",
+// "2. type: index creation",
+// "SQL command: CREATE INDEX ON t2 (i) STORING (k);",
+//}
+// and returns a list of type and recommendations, e.g.:
+//{
+// "replacement : CREATE UNIQUE INDEX ON t1 (i) STORING (k); DROP INDEX t1@existing_t1_i;",
+// "creation : CREATE INDEX ON t2 (i) STORING (k);",
+//}
+func FormatIdxRecommendations(idxRec []string) []string {
+ recommendations := []string{}
+ if len(idxRec) == 0 {
+ return recommendations
+ }
+
+ var recType string
+ var recCommand string
+ var rec string
+
+ for i := 1; i < len(idxRec); i++ {
+ recType = strings.Split(idxRec[i], "type: index ")[1]
+ recCommand = strings.Replace(idxRec[i+1], " SQL command: ", "", 1)
+ recCommand = strings.Replace(recCommand, " SQL commands: ", "", 1)
+ rec = fmt.Sprintf("%s : %s", recType, recCommand)
+ recommendations = append(recommendations, rec)
+ i++
+ }
+
+ return recommendations
+}
diff --git a/pkg/sql/idxrecommendations/idx_recommendations_cache.go b/pkg/sql/idxrecommendations/idx_recommendations_cache.go
new file mode 100644
index 000000000000..6ff0de57e28f
--- /dev/null
+++ b/pkg/sql/idxrecommendations/idx_recommendations_cache.go
@@ -0,0 +1,233 @@
+// 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 idxrecommendations
+
+import (
+ "sync/atomic"
+ "time"
+
+ "github.com/cockroachdb/cockroach/pkg/settings"
+ "github.com/cockroachdb/cockroach/pkg/settings/cluster"
+ "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
+ "github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
+ "github.com/cockroachdb/cockroach/pkg/util/syncutil"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil"
+)
+
+// uniqueIndexRecInfoLimit is the limit on number of unique index
+// recommendations info we can store in memory.
+var uniqueIndexRecInfoLimit *settings.IntSetting
+
+type indexRecKey struct {
+ stmtNoConstants string
+ database string
+ planHash uint64
+}
+
+type indexRecInfo struct {
+ lastGeneratedTs time.Time
+ recommendations []string
+ executionCount int64
+}
+
+// IndexRecCache stores the map of index recommendations keys (fingerprint, database, planHash) and
+// information (lastGeneratedTs, recommendations, executionCount).
+type IndexRecCache struct {
+ st *cluster.Settings
+
+ // minExecCount is the minimum value for execution count that a statement
+ // must have before a recommendation is generated.
+ minExecCount int64
+
+ mu struct {
+ syncutil.RWMutex
+
+ // idxRecommendations stores index recommendations per indexRecKey.
+ idxRecommendations map[indexRecKey]indexRecInfo
+ }
+
+ atomic struct {
+ // uniqueIndexRecInfo is the number of unique index recommendations info
+ // we are storing in memory.
+ uniqueIndexRecInfo int64
+ }
+}
+
+// NewIndexRecommendationsCache creates a new map to be used as a cache for index recommendations.
+func NewIndexRecommendationsCache(
+ setting *cluster.Settings, uniqueIdxRecInfoLimit *settings.IntSetting,
+) *IndexRecCache {
+ idxRecCache := &IndexRecCache{
+ st: setting,
+ minExecCount: 5,
+ }
+ idxRecCache.mu.idxRecommendations = make(map[indexRecKey]indexRecInfo)
+ uniqueIndexRecInfoLimit = uniqueIdxRecInfoLimit
+ return idxRecCache
+}
+
+// ShouldGenerateIndexRecommendation implements IdxRecommendations interface.
+// It returns true if there was no generation in the past hour
+// and there is at least 5 executions of the same fingerprint/database/planHash combination.
+func (idxRec *IndexRecCache) ShouldGenerateIndexRecommendation(
+ fingerprint string, planHash uint64, database string, stmtType tree.StatementType,
+) bool {
+ if !idxRec.statementCanHaveRecommendation(stmtType) {
+ return false
+ }
+
+ idxKey := indexRecKey{
+ stmtNoConstants: fingerprint,
+ database: database,
+ planHash: planHash,
+ }
+ recInfo, found := idxRec.getOrCreateIndexRecommendation(idxKey)
+ // If we couldn't find or create, don't generate recommendations.
+ if !found {
+ return false
+ }
+
+ timeSinceLastGenerated := timeutil.Since(recInfo.lastGeneratedTs)
+ return recInfo.executionCount >= idxRec.minExecCount && timeSinceLastGenerated.Hours() >= 1
+}
+
+// UpdateIndexRecommendations implements IdxRecommendations interface.
+// It updates the values for index recommendations.
+// If reset is true, a new recommendation was generated, so reset the execution counter and
+// lastGeneratedTs, otherwise just increment the executionCount.
+func (idxRec *IndexRecCache) UpdateIndexRecommendations(
+ fingerprint string,
+ planHash uint64,
+ database string,
+ stmtType tree.StatementType,
+ recommendations []string,
+ reset bool,
+) []string {
+ if !idxRec.statementCanHaveRecommendation(stmtType) {
+ return recommendations
+ }
+
+ idxKey := indexRecKey{
+ stmtNoConstants: fingerprint,
+ database: database,
+ planHash: planHash,
+ }
+
+ if reset {
+ idxRec.setIndexRecommendations(idxKey, timeutil.Now(), recommendations, 0)
+ return recommendations
+ }
+
+ recInfo, found := idxRec.getOrCreateIndexRecommendation(idxKey)
+ if !found {
+ return recommendations
+ }
+
+ if recInfo.executionCount < idxRec.minExecCount {
+ idxRec.setIndexRecommendations(
+ idxKey,
+ recInfo.lastGeneratedTs,
+ recInfo.recommendations,
+ recInfo.executionCount+1,
+ )
+ }
+
+ return recInfo.recommendations
+}
+
+// statementCanHaveRecommendation returns true if that type of statement can have recommendations
+// generated for it. We only want to recommend if the statement is DML and recommendations are enabled.
+func (idxRec *IndexRecCache) statementCanHaveRecommendation(stmtType tree.StatementType) bool {
+ if !sqlstats.SampleIndexRecommendation.Get(&idxRec.st.SV) || stmtType != tree.TypeDML {
+ return false
+ }
+
+ return true
+}
+
+func (idxRec *IndexRecCache) getIndexRecommendation(key indexRecKey) (indexRecInfo, bool) {
+ idxRec.mu.RLock()
+ defer idxRec.mu.RUnlock()
+
+ recInfo, found := idxRec.mu.idxRecommendations[key]
+
+ return recInfo, found
+}
+
+func (idxRec *IndexRecCache) getOrCreateIndexRecommendation(key indexRecKey) (indexRecInfo, bool) {
+ recInfo, found := idxRec.getIndexRecommendation(key)
+ if found {
+ return recInfo, true
+ }
+
+ // If it was not found, check if a new entry can be created, without
+ // passing the limit of unique index recommendations from the cache.
+ limit := uniqueIndexRecInfoLimit.Get(&idxRec.st.SV)
+ incrementedCount :=
+ atomic.AddInt64(&idxRec.atomic.uniqueIndexRecInfo, int64(1))
+
+ if incrementedCount > limit {
+ // If we have exceeded limit of unique index recommendations try to delete older data.
+ deleted := idxRec.clearOldIdxRecommendations()
+ // Abort if no entries were deleted.
+ if deleted == 0 {
+ atomic.AddInt64(&idxRec.atomic.uniqueIndexRecInfo, -int64(1))
+ return indexRecInfo{}, false
+ }
+ }
+
+ idxRec.mu.Lock()
+ defer idxRec.mu.Unlock()
+ // For a new entry, we want the lastGeneratedTs to be in the past, in case we reach
+ // the execution count, we should generate new recommendations.
+ recInfo = indexRecInfo{
+ lastGeneratedTs: timeutil.Now().Add(-time.Hour),
+ recommendations: []string{},
+ executionCount: 0,
+ }
+ idxRec.mu.idxRecommendations[key] = recInfo
+
+ return recInfo, true
+}
+
+func (idxRec *IndexRecCache) setIndexRecommendations(
+ key indexRecKey, time time.Time, recommendations []string, execCount int64,
+) {
+ _, found := idxRec.getOrCreateIndexRecommendation(key)
+
+ if found {
+ idxRec.mu.Lock()
+ defer idxRec.mu.Unlock()
+
+ idxRec.mu.idxRecommendations[key] = indexRecInfo{
+ lastGeneratedTs: time,
+ recommendations: recommendations,
+ executionCount: execCount,
+ }
+ }
+}
+
+// clearOldIdxRecommendations clear entries that was last updated
+// more than a day ago. Returns the total deleted entries.
+func (idxRec *IndexRecCache) clearOldIdxRecommendations() int {
+ idxRec.mu.Lock()
+ defer idxRec.mu.Unlock()
+
+ deleted := 0
+ for key, value := range idxRec.mu.idxRecommendations {
+ if timeutil.Since(value.lastGeneratedTs).Hours() >= 24 {
+ delete(idxRec.mu.idxRecommendations, key)
+ deleted++
+ }
+ }
+ atomic.AddInt64(&idxRec.atomic.uniqueIndexRecInfo, int64(-deleted))
+ return deleted
+}
diff --git a/pkg/sql/idxrecommendations/idx_recommendations_cache_test.go b/pkg/sql/idxrecommendations/idx_recommendations_cache_test.go
new file mode 100644
index 000000000000..f2565b0a855e
--- /dev/null
+++ b/pkg/sql/idxrecommendations/idx_recommendations_cache_test.go
@@ -0,0 +1,132 @@
+// 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 idxrecommendations_test
+
+import (
+ "context"
+ "testing"
+
+ "github.com/cockroachdb/cockroach/pkg/sql/idxrecommendations"
+ "github.com/cockroachdb/cockroach/pkg/sql/tests"
+ "github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
+ "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
+ "github.com/cockroachdb/cockroach/pkg/util/leaktest"
+ "github.com/cockroachdb/cockroach/pkg/util/log"
+ "github.com/stretchr/testify/require"
+)
+
+func TestIndexRecommendationsStats(t *testing.T) {
+ defer leaktest.AfterTest(t)()
+ defer log.Scope(t).Close(t)
+
+ ctx := context.Background()
+ params, _ := tests.CreateTestServerParams()
+ testServer, sqlConn, _ := serverutils.StartServer(t, params)
+ defer func() {
+ require.NoError(t, sqlConn.Close())
+ testServer.Stopper().Stop(ctx)
+ }()
+
+ testConn := sqlutils.MakeSQLRunner(sqlConn)
+ testConn.Exec(t, "CREATE DATABASE idxrectest")
+ testConn.Exec(t, "USE idxrectest")
+ testConn.Exec(t, "CREATE TABLE t ( k INT PRIMARY KEY, v INT, FAMILY \"primary\" (k, v))")
+ testConn.Exec(t, "CREATE TABLE t1 (k INT, i INT, f FLOAT, s STRING)")
+ testConn.Exec(t, "CREATE TABLE t2 (k INT, i INT, s STRING)")
+ testConn.Exec(t, "CREATE UNIQUE INDEX existing_t1_i ON t1(i)")
+
+ testCases := []struct {
+ stmt string
+ fingerprint string
+ recommendations string
+ }{
+ {
+ stmt: "SELECT * FROM t WHERE v > 123",
+ fingerprint: "SELECT * FROM t WHERE v > _",
+ recommendations: "{\"creation : CREATE INDEX ON t (v);\"}",
+ },
+ {
+ stmt: "SELECT t1.k FROM t1 JOIN t2 ON t1.k = t2.k WHERE t1.i > 3 AND t2.i > 3",
+ fingerprint: "SELECT t1.k FROM t1 JOIN t2 ON t1.k = t2.k WHERE (t1.i > _) AND (t2.i > _)",
+ recommendations: "{\"replacement : CREATE UNIQUE INDEX ON t1 (i) STORING (k); DROP INDEX t1@existing_t1_i;\"," +
+ "\"creation : CREATE INDEX ON t2 (i) STORING (k);\"}",
+ },
+ }
+
+ var recommendations string
+ for i := 0; i < 8; i++ {
+ for _, tc := range testCases {
+ testConn.Exec(t, tc.stmt)
+ rows := testConn.QueryRow(t, "SELECT index_recommendations FROM CRDB_INTERNAL.STATEMENT_STATISTICS "+
+ " WHERE metadata ->> 'db' = 'idxrectest' AND metadata ->> 'query'=$1", tc.fingerprint)
+ rows.Scan(&recommendations)
+
+ expected := tc.recommendations
+ if i < 5 {
+ expected = "{}"
+ }
+ require.Equal(t, expected, recommendations)
+ }
+ }
+}
+
+func TestFormatIdxRecommendations(t *testing.T) {
+ testCases := []struct {
+ title string
+ fullRecInfo []string
+ formattedInfo []string
+ }{
+ {
+ title: "empty recommendation",
+ fullRecInfo: []string{},
+ formattedInfo: []string{},
+ },
+ {
+ title: "single recommendation with one command",
+ fullRecInfo: []string{
+ "index recommendations: 1",
+ " 1. type: index creation",
+ " SQL command: CREATE INDEX ON t2 (i) STORING (k);",
+ },
+ formattedInfo: []string{"creation : CREATE INDEX ON t2 (i) STORING (k);"},
+ },
+ {
+ title: "single recommendation with multiple commands",
+ fullRecInfo: []string{
+ "index recommendations: 1",
+ " 1. type: index replacement",
+ " SQL commands: CREATE UNIQUE INDEX ON t1 (i) STORING (k); DROP INDEX t1@existing_t1_i;",
+ },
+ formattedInfo: []string{"replacement : CREATE UNIQUE INDEX ON t1 (i) STORING (k); DROP INDEX t1@existing_t1_i;"},
+ },
+ {
+ title: "multiple recommendations",
+ fullRecInfo: []string{
+ "index recommendations: 2",
+ " 1. type: index replacement",
+ " SQL commands: CREATE UNIQUE INDEX ON t1 (i) STORING (k); DROP INDEX t1@existing_t1_i;",
+ " 2. type: index creation",
+ " SQL command: CREATE INDEX ON t2 (i) STORING (k);",
+ },
+ formattedInfo: []string{
+ "replacement : CREATE UNIQUE INDEX ON t1 (i) STORING (k); DROP INDEX t1@existing_t1_i;",
+ "creation : CREATE INDEX ON t2 (i) STORING (k);",
+ },
+ },
+ }
+
+ for _, tc := range testCases {
+ t.Run(tc.title, func(t *testing.T) {
+ actual := idxrecommendations.FormatIdxRecommendations(tc.fullRecInfo)
+ require.Equal(t, tc.formattedInfo, actual)
+ })
+ }
+}
diff --git a/pkg/sql/idxrecommendations/main_test.go b/pkg/sql/idxrecommendations/main_test.go
new file mode 100644
index 000000000000..709aa7334652
--- /dev/null
+++ b/pkg/sql/idxrecommendations/main_test.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 idxrecommendations_test
+
+import (
+ "os"
+ "testing"
+
+ "github.com/cockroachdb/cockroach/pkg/security/securityassets"
+ "github.com/cockroachdb/cockroach/pkg/security/securitytest"
+ "github.com/cockroachdb/cockroach/pkg/server"
+ "github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
+ "github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
+)
+
+func TestMain(m *testing.M) {
+ securityassets.SetLoader(securitytest.EmbeddedAssets)
+ serverutils.InitTestServerFactory(server.TestServerFactory)
+ serverutils.InitTestClusterFactory(testcluster.TestClusterFactory)
+ os.Exit(m.Run())
+}
diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go
index c5f0b4c1dfe9..88821b493e05 100644
--- a/pkg/sql/instrumentation.go
+++ b/pkg/sql/instrumentation.go
@@ -24,8 +24,10 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/execstats"
+ "github.com/cockroachdb/cockroach/pkg/sql/idxrecommendations"
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec"
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec/explain"
+ "github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder"
"github.com/cockroachdb/cockroach/pkg/sql/physicalplan"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
@@ -128,7 +130,7 @@ type instrumentationHelper struct {
costEstimate float64
// indexRecommendations is a string slice containing index recommendations for
- // the planned statement. This is only set for EXPLAIN statements.
+ // the planned statement.
indexRecommendations []string
// maxFullScanRows is the maximum number of rows scanned by a full scan, as
@@ -663,3 +665,63 @@ func (m execNodeTraceMetadata) annotateExplain(
return allRegions
}
+
+// SetIndexRecommendations check if we should generate a new index recommendation.
+// If true it will generate and update the idx recommendations cache,
+// if false, use the value on index recommendations cache and update its counter.
+func (ih *instrumentationHelper) SetIndexRecommendations(
+ ctx context.Context, idxRec *idxrecommendations.IndexRecCache, planner *planner,
+) {
+ opc := planner.optPlanningCtx
+ opc.reset(ctx)
+ stmtType := opc.p.stmt.AST.StatementType()
+
+ if idxRec.ShouldGenerateIndexRecommendation(
+ ih.fingerprint,
+ ih.planGist.Hash(),
+ planner.SessionData().Database,
+ stmtType,
+ ) {
+ f := opc.optimizer.Factory()
+ // EvalContext() has the context of the already closed span, so we need to update with the current context.
+ // The replacement of the context here isn't ideal, but the current implementation of contexts would need to change
+ // significantly to accommodate this case.
+ evalCtx := opc.p.EvalContext()
+ oldCtx := evalCtx.Context
+ evalCtx.Context = ctx
+ defer func() {
+ evalCtx.Context = oldCtx
+ }()
+
+ f.Init(evalCtx, &opc.catalog)
+ f.FoldingControl().AllowStableFolds()
+ bld := optbuilder.New(ctx, &opc.p.semaCtx, evalCtx, &opc.catalog, f, opc.p.stmt.AST)
+ err := bld.Build()
+ if err != nil {
+ log.Warningf(ctx, "unable to build memo: %s", err)
+ } else {
+ err = opc.makeQueryIndexRecommendation(ctx)
+ if err != nil {
+ log.Warningf(ctx, "unable to generate index recommendations: %s", err)
+ } else {
+ idxRec.UpdateIndexRecommendations(
+ ih.fingerprint,
+ ih.planGist.Hash(),
+ planner.SessionData().Database,
+ stmtType,
+ ih.indexRecommendations,
+ true,
+ )
+ }
+ }
+ } else {
+ ih.indexRecommendations = idxRec.UpdateIndexRecommendations(
+ ih.fingerprint,
+ ih.planGist.Hash(),
+ planner.SessionData().Database,
+ stmtType,
+ []string{},
+ false,
+ )
+ }
+}
diff --git a/pkg/sql/sqlstats/cluster_settings.go b/pkg/sql/sqlstats/cluster_settings.go
index dd487311c074..cd5920561fa7 100644
--- a/pkg/sql/sqlstats/cluster_settings.go
+++ b/pkg/sql/sqlstats/cluster_settings.go
@@ -168,3 +168,12 @@ var SampleIndexRecommendation = settings.RegisterBoolSetting(
"generate an index recommendation for each fingerprint ID",
true,
).WithPublic()
+
+// MaxMemReportedSampleIndexRecommendations specifies the maximum of unique index
+// recommendations info we store in memory.
+var MaxMemReportedSampleIndexRecommendations = settings.RegisterIntSetting(
+ settings.TenantWritable,
+ "sql.metrics.statement_details.max_mem_reported_idx_recommendations",
+ "the maximum number of reported index recommendation info stored in memory",
+ 5000,
+).WithPublic()
diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go
index fc351fb6b3ed..3a3cc7ebcf6b 100644
--- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go
+++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go
@@ -134,7 +134,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)
+ 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