diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md
index 07c045dd0e93..1083e6d2c065 100644
--- a/docs/generated/eventlog.md
+++ b/docs/generated/eventlog.md
@@ -2205,6 +2205,32 @@ An event of type `drop_role` is recorded when a role is dropped.
Events in this category are logged to the `TELEMETRY` channel.
+### `captured_index_usage_stats`
+
+An event of type `captured_index_usage_stats`
+
+
+| Field | Description | Sensitive |
+|--|--|--|
+| `TotalReadCount` | TotalReadCount is the number of times this index has been read from. | no |
+| `LastRead` | LastRead is the timestamp that this index was last being read from. | yes |
+| `TableID` | TableID is the ID of the table this index is created on. This is same as descpb.TableID and is unique within the cluster. | no |
+| `IndexID` | IndexID is the ID of the index within the scope of the given table. | no |
+| `DatabaseName` | | yes |
+| `TableName` | | yes |
+| `IndexName` | | yes |
+| `IndexType` | | yes |
+| `IsUnique` | | no |
+| `IsInverted` | | no |
+
+
+#### Common fields
+
+| Field | Description | Sensitive |
+|--|--|--|
+| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no |
+| `EventType` | The type of the event. | no |
+
### `sampled_query`
An event of type `sampled_query` is the SQL query event logged to the telemetry channel. It
diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt
index 06b6a0fa1eeb..e9fc3c41991b 100644
--- a/docs/generated/settings/settings-for-tenants.txt
+++ b/docs/generated/settings/settings-for-tenants.txt
@@ -159,6 +159,7 @@ sql.stats.multi_column_collection.enabled boolean true multi-column statistics c
sql.stats.persisted_rows.max integer 1000000 maximum number of rows of statement and transaction statistics that will be persisted in the system tables
sql.stats.post_events.enabled boolean false if set, an event is logged for every CREATE STATISTICS job
sql.stats.response.max integer 20000 the maximum number of statements and transaction stats returned in a CombinedStatements request
+sql.telemetry.capture_index_usage_stats.enabled boolean true enable/disable capturing index usage statistics to the telemetry logging channel
sql.telemetry.query_sampling.enabled boolean false when set to true, executed queries will emit an event on the telemetry logging channel
sql.temp_object_cleaner.cleanup_interval duration 30m0s how often to clean up orphaned temporary objects
sql.temp_object_cleaner.wait_interval duration 30m0s how long after creation a temporary object will be cleaned up
diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index ce5220a62d0e..da114f1ccb62 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -164,6 +164,7 @@
sql.stats.persisted_rows.max | integer | 1000000 | maximum number of rows of statement and transaction statistics that will be persisted in the system tables |
sql.stats.post_events.enabled | boolean | false | if set, an event is logged for every CREATE STATISTICS job |
sql.stats.response.max | integer | 20000 | the maximum number of statements and transaction stats returned in a CombinedStatements request |
+sql.telemetry.capture_index_usage_stats.enabled | boolean | true | enable/disable capturing index usage statistics to the telemetry logging channel |
sql.telemetry.query_sampling.enabled | boolean | false | when set to true, executed queries will emit an event on the telemetry logging channel |
sql.temp_object_cleaner.cleanup_interval | duration | 30m0s | how often to clean up orphaned temporary objects |
sql.temp_object_cleaner.wait_interval | duration | 30m0s | how long after creation a temporary object will be cleaned up |
diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel
index d7c76e06e858..0ee3a4aac34e 100644
--- a/pkg/BUILD.bazel
+++ b/pkg/BUILD.bazel
@@ -287,6 +287,7 @@ ALL_TESTS = [
"//pkg/sql/rowexec:rowexec_test",
"//pkg/sql/rowflow:rowflow_test",
"//pkg/sql/scanner:scanner_test",
+ "//pkg/sql/scheduledlogging:scheduledlogging_test",
"//pkg/sql/schemachange:schemachange_test",
"//pkg/sql/schemachanger/scbuild:scbuild_test",
"//pkg/sql/schemachanger/scexec:scexec_test",
diff --git a/pkg/base/testing_knobs.go b/pkg/base/testing_knobs.go
index 85b2fc4e762c..17effc9aa24e 100644
--- a/pkg/base/testing_knobs.go
+++ b/pkg/base/testing_knobs.go
@@ -19,29 +19,30 @@ type ModuleTestingKnobs interface {
// TestingKnobs contains facilities for controlling various parts of the
// system for testing.
type TestingKnobs struct {
- Store ModuleTestingKnobs
- KVClient ModuleTestingKnobs
- RangeFeed ModuleTestingKnobs
- SQLExecutor ModuleTestingKnobs
- SQLLeaseManager ModuleTestingKnobs
- SQLSchemaChanger ModuleTestingKnobs
- SQLNewSchemaChanger ModuleTestingKnobs
- SQLTypeSchemaChanger ModuleTestingKnobs
- GCJob ModuleTestingKnobs
- PGWireTestingKnobs ModuleTestingKnobs
- StartupMigrationManager ModuleTestingKnobs
- DistSQL ModuleTestingKnobs
- SQLEvalContext ModuleTestingKnobs
- NodeLiveness ModuleTestingKnobs
- Server ModuleTestingKnobs
- TenantTestingKnobs ModuleTestingKnobs
- JobsTestingKnobs ModuleTestingKnobs
- BackupRestore ModuleTestingKnobs
- Streaming ModuleTestingKnobs
- MigrationManager ModuleTestingKnobs
- IndexUsageStatsKnobs ModuleTestingKnobs
- SQLStatsKnobs ModuleTestingKnobs
- SpanConfig ModuleTestingKnobs
- SQLLivenessKnobs ModuleTestingKnobs
- TelemetryLoggingKnobs ModuleTestingKnobs
+ Store ModuleTestingKnobs
+ KVClient ModuleTestingKnobs
+ RangeFeed ModuleTestingKnobs
+ SQLExecutor ModuleTestingKnobs
+ SQLLeaseManager ModuleTestingKnobs
+ SQLSchemaChanger ModuleTestingKnobs
+ SQLNewSchemaChanger ModuleTestingKnobs
+ SQLTypeSchemaChanger ModuleTestingKnobs
+ GCJob ModuleTestingKnobs
+ PGWireTestingKnobs ModuleTestingKnobs
+ StartupMigrationManager ModuleTestingKnobs
+ DistSQL ModuleTestingKnobs
+ SQLEvalContext ModuleTestingKnobs
+ NodeLiveness ModuleTestingKnobs
+ Server ModuleTestingKnobs
+ TenantTestingKnobs ModuleTestingKnobs
+ JobsTestingKnobs ModuleTestingKnobs
+ BackupRestore ModuleTestingKnobs
+ Streaming ModuleTestingKnobs
+ MigrationManager ModuleTestingKnobs
+ IndexUsageStatsKnobs ModuleTestingKnobs
+ SQLStatsKnobs ModuleTestingKnobs
+ SpanConfig ModuleTestingKnobs
+ SQLLivenessKnobs ModuleTestingKnobs
+ TelemetryLoggingKnobs ModuleTestingKnobs
+ CapturedIndexUsageStatsKnobs ModuleTestingKnobs
}
diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel
index 71e41737ac27..44ee88903134 100644
--- a/pkg/server/BUILD.bazel
+++ b/pkg/server/BUILD.bazel
@@ -21,6 +21,7 @@ go_library(
"doc.go",
"drain.go",
"grpc_server.go",
+ "import_ts.go",
"index_usage_stats.go",
"init.go",
"init_handshake.go",
@@ -145,6 +146,7 @@ go_library(
"//pkg/sql/physicalplan",
"//pkg/sql/querycache",
"//pkg/sql/roleoption",
+ "//pkg/sql/scheduledlogging",
"//pkg/sql/schemachanger/scexec",
"//pkg/sql/schemachanger/scjob",
"//pkg/sql/sem/tree",
@@ -404,6 +406,7 @@ go_test(
"@org_golang_google_grpc//:go_default_library",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//credentials",
+ "@org_golang_google_grpc//metadata",
"@org_golang_google_grpc//status",
"@org_golang_x_crypto//bcrypt",
],
diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go
index 868adf90dbc7..8325b525e515 100644
--- a/pkg/server/server_sql.go
+++ b/pkg/server/server_sql.go
@@ -68,6 +68,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire"
"github.com/cockroachdb/cockroach/pkg/sql/querycache"
+ "github.com/cockroachdb/cockroach/pkg/sql/scheduledlogging"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
@@ -735,6 +736,9 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
if telemetryLoggingKnobs := cfg.TestingKnobs.TelemetryLoggingKnobs; telemetryLoggingKnobs != nil {
execCfg.TelemetryLoggingTestingKnobs = telemetryLoggingKnobs.(*sql.TelemetryLoggingTestingKnobs)
}
+ if capturedIndexUsageStatsKnobs := cfg.TestingKnobs.CapturedIndexUsageStatsKnobs; capturedIndexUsageStatsKnobs != nil {
+ execCfg.CaptureIndexUsageStatsKnobs = capturedIndexUsageStatsKnobs.(*scheduledlogging.CaptureIndexUsageStatsTestingKnobs)
+ }
statsRefresher := stats.MakeRefresher(
cfg.Settings,
@@ -1124,6 +1128,7 @@ func (s *SQLServer) preStart(
scheduledjobs.ProdJobSchedulerEnv,
)
+ scheduledlogging.Start(ctx, stopper, s.execCfg.DB, s.execCfg.Settings, s.internalExecutor, s.execCfg.CaptureIndexUsageStatsKnobs)
return nil
}
diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel
index 5798b2857311..02ed91a0d87c 100644
--- a/pkg/sql/BUILD.bazel
+++ b/pkg/sql/BUILD.bazel
@@ -343,6 +343,7 @@ go_library(
"//pkg/sql/rowenc",
"//pkg/sql/rowexec",
"//pkg/sql/rowinfra",
+ "//pkg/sql/scheduledlogging",
"//pkg/sql/schemachange",
"//pkg/sql/schemachanger/scbuild",
"//pkg/sql/schemachanger/scexec",
diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go
index 39f72b5495be..e815b1c53060 100644
--- a/pkg/sql/exec_util.go
+++ b/pkg/sql/exec_util.go
@@ -72,6 +72,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/querycache"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
+ "github.com/cockroachdb/cockroach/pkg/sql/scheduledlogging"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
@@ -1102,6 +1103,7 @@ type ExecutorConfig struct {
StreamingTestingKnobs *StreamingTestingKnobs
SQLStatsTestingKnobs *sqlstats.TestingKnobs
TelemetryLoggingTestingKnobs *TelemetryLoggingTestingKnobs
+ CaptureIndexUsageStatsKnobs *scheduledlogging.CaptureIndexUsageStatsTestingKnobs
// HistogramWindowInterval is (server.Config).HistogramWindowInterval.
HistogramWindowInterval time.Duration
diff --git a/pkg/sql/scheduledlogging/BUILD.bazel b/pkg/sql/scheduledlogging/BUILD.bazel
new file mode 100644
index 000000000000..f68335c6a942
--- /dev/null
+++ b/pkg/sql/scheduledlogging/BUILD.bazel
@@ -0,0 +1,51 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
+
+go_library(
+ name = "scheduledlogging",
+ srcs = ["captured_index_usage_stats.go"],
+ importpath = "github.com/cockroachdb/cockroach/pkg/sql/scheduledlogging",
+ visibility = ["//visibility:public"],
+ deps = [
+ "//pkg/kv",
+ "//pkg/roachpb:with-mocks",
+ "//pkg/security",
+ "//pkg/settings",
+ "//pkg/settings/cluster",
+ "//pkg/sql/sem/tree",
+ "//pkg/sql/sessiondata",
+ "//pkg/sql/sqlutil",
+ "//pkg/util/log",
+ "//pkg/util/log/eventpb",
+ "//pkg/util/stop",
+ "//pkg/util/timeutil",
+ "@com_github_cockroachdb_errors//:errors",
+ ],
+)
+
+go_test(
+ name = "scheduledlogging_test",
+ srcs = [
+ "captured_index_usage_stats_test.go",
+ "main_test.go",
+ ],
+ embed = [":scheduledlogging"],
+ deps = [
+ "//pkg/base",
+ "//pkg/security",
+ "//pkg/security/securitytest",
+ "//pkg/server",
+ "//pkg/settings/cluster",
+ "//pkg/testutils",
+ "//pkg/testutils/serverutils",
+ "//pkg/testutils/sqlutils",
+ "//pkg/testutils/testcluster",
+ "//pkg/util/leaktest",
+ "//pkg/util/log",
+ "//pkg/util/log/channel",
+ "//pkg/util/log/logconfig",
+ "//pkg/util/randutil",
+ "//pkg/util/syncutil",
+ "@com_github_cockroachdb_errors//:errors",
+ "@com_github_stretchr_testify//require",
+ ],
+)
diff --git a/pkg/sql/scheduledlogging/captured_index_usage_stats.go b/pkg/sql/scheduledlogging/captured_index_usage_stats.go
new file mode 100644
index 000000000000..e559fd82e1b6
--- /dev/null
+++ b/pkg/sql/scheduledlogging/captured_index_usage_stats.go
@@ -0,0 +1,314 @@
+// 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 scheduledlogging
+
+import (
+ "context"
+ "fmt"
+ "time"
+
+ "github.com/cockroachdb/cockroach/pkg/kv"
+ "github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
+ "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/sessiondata"
+ "github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
+ "github.com/cockroachdb/cockroach/pkg/util/log"
+ "github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
+ "github.com/cockroachdb/cockroach/pkg/util/stop"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil"
+ "github.com/cockroachdb/errors"
+)
+
+var telemetryCaptureIndexUsageStatsEnabled = settings.RegisterBoolSetting(
+ "sql.telemetry.capture_index_usage_stats.enabled",
+ "enable/disable capturing index usage statistics to the telemetry logging channel",
+ true,
+).WithPublic()
+
+var telemetryCaptureIndexUsageStatsInterval = settings.RegisterDurationSetting(
+ "sql.telemetry.capture_index_usage_stats.interval",
+ "the scheduled interval time between capturing index usage statistics when capturing index usage statistics is enabled",
+ 8*time.Hour,
+ settings.NonNegativeDuration,
+).WithSystemOnly()
+
+var telemetryCaptureIndexUsageStatsStatusCheckEnabledInterval = settings.RegisterDurationSetting(
+ "sql.telemetry.capture_index_usage_stats.check_enabled_interval",
+ "the scheduled interval time between checks to see if index usage statistics has been enabled",
+ 10*time.Minute,
+ settings.NonNegativeDuration,
+).WithSystemOnly()
+
+var telemetryCaptureIndexUsageStatsLoggingDelay = settings.RegisterDurationSetting(
+ "sql.telemetry.capture_index_usage_stats.logging_delay",
+ "the time delay between emitting individual index usage stats logs, this is done to "+
+ "mitigate the log-line limit of 10 logs per second on the telemetry pipeline",
+ 500*time.Millisecond,
+ settings.NonNegativeDuration,
+).WithSystemOnly()
+
+// CaptureIndexUsageStatsTestingKnobs provides hooks and knobs for unit tests.
+type CaptureIndexUsageStatsTestingKnobs struct {
+ // getLoggingDuration allows tests to override the duration of the index
+ // usage stats logging operation.
+ getLoggingDuration func() time.Duration
+ // getOverlapDuration allows tests to override the duration until the next
+ // scheduled interval in the case that the logging duration exceeds the
+ // default scheduled interval duration.
+ getOverlapDuration func() time.Duration
+}
+
+// ModuleTestingKnobs implements base.ModuleTestingKnobs interface.
+func (*CaptureIndexUsageStatsTestingKnobs) ModuleTestingKnobs() {}
+
+// CaptureIndexUsageStatsLoggingScheduler is responsible for logging index usage stats
+// on a scheduled interval.
+type CaptureIndexUsageStatsLoggingScheduler struct {
+ db *kv.DB
+ st *cluster.Settings
+ ie sqlutil.InternalExecutor
+ knobs *CaptureIndexUsageStatsTestingKnobs
+ currentCaptureStartTime time.Time
+}
+
+func (s *CaptureIndexUsageStatsLoggingScheduler) getLoggingDuration() time.Duration {
+ if s.knobs != nil && s.knobs.getLoggingDuration != nil {
+ return s.knobs.getLoggingDuration()
+ }
+ return timeutil.Since(s.currentCaptureStartTime)
+}
+
+func (s *CaptureIndexUsageStatsLoggingScheduler) durationOnOverlap() time.Duration {
+ if s.knobs != nil && s.knobs.getOverlapDuration != nil {
+ return s.knobs.getOverlapDuration()
+ }
+ // If the logging duration overlaps into the next scheduled interval, start
+ // the next scheduled interval immediately instead of waiting.
+ return 0 * time.Second
+}
+
+func (s *CaptureIndexUsageStatsLoggingScheduler) durationUntilNextInterval() time.Duration {
+ // If telemetry is disabled, return the disabled interval duration.
+ if !telemetryCaptureIndexUsageStatsEnabled.Get(&s.st.SV) {
+ return telemetryCaptureIndexUsageStatsStatusCheckEnabledInterval.Get(&s.st.SV)
+ }
+ // If the previous logging operation took longer than or equal to the set
+ // schedule interval, schedule the next interval immediately.
+ if s.getLoggingDuration() >= telemetryCaptureIndexUsageStatsInterval.Get(&s.st.SV) {
+ return s.durationOnOverlap()
+ }
+ // Otherwise, schedule the next interval normally.
+ return telemetryCaptureIndexUsageStatsInterval.Get(&s.st.SV)
+}
+
+// Start starts the capture index usage statistics logging scheduler.
+func Start(
+ ctx context.Context,
+ stopper *stop.Stopper,
+ db *kv.DB,
+ cs *cluster.Settings,
+ ie sqlutil.InternalExecutor,
+ knobs *CaptureIndexUsageStatsTestingKnobs,
+) {
+ scheduler := CaptureIndexUsageStatsLoggingScheduler{
+ db: db,
+ st: cs,
+ ie: ie,
+ knobs: knobs,
+ }
+ scheduler.start(ctx, stopper)
+}
+
+func (s *CaptureIndexUsageStatsLoggingScheduler) start(ctx context.Context, stopper *stop.Stopper) {
+ _ = stopper.RunAsyncTask(ctx, "capture-index-usage-stats", func(ctx context.Context) {
+ // Start the scheduler immediately.
+ for timer := time.NewTimer(0 * time.Second); ; timer.Reset(s.durationUntilNextInterval()) {
+ select {
+ case <-stopper.ShouldQuiesce():
+ timer.Stop()
+ return
+ case <-timer.C:
+ s.currentCaptureStartTime = timeutil.Now()
+ if !telemetryCaptureIndexUsageStatsEnabled.Get(&s.st.SV) {
+ continue
+ }
+
+ err := captureIndexUsageStats(ctx, s.ie, stopper, telemetryCaptureIndexUsageStatsLoggingDelay.Get(&s.st.SV))
+ if err != nil {
+ log.Warningf(ctx, "error capturing index usage stats: %+v", err)
+ }
+ }
+ }
+ })
+}
+
+func captureIndexUsageStats(
+ ctx context.Context,
+ ie sqlutil.InternalExecutor,
+ stopper *stop.Stopper,
+ loggingDelay time.Duration,
+) error {
+ allDatabaseNames, err := getAllDatabaseNames(ctx, ie)
+ if err != nil {
+ return err
+ }
+
+ // Capture index usage statistics for each database.
+ var ok bool
+ expectedNumDatums := 9
+ var allCapturedIndexUsageStats []eventpb.EventPayload
+ for _, databaseName := range allDatabaseNames {
+ // Omit index usage statistics on the default databases 'system',
+ // 'defaultdb', and 'postgres'.
+ if databaseName == "system" || databaseName == "defaultdb" || databaseName == "postgres" {
+ continue
+ }
+ stmt := fmt.Sprintf(`
+ SELECT
+ ti.descriptor_name as table_name,
+ ti.descriptor_id as table_id,
+ ti.index_name,
+ ti.index_id,
+ ti.index_type,
+ ti.is_unique,
+ ti.is_inverted,
+ total_reads,
+ last_read
+ FROM %s.crdb_internal.index_usage_statistics AS us
+ JOIN %s.crdb_internal.table_indexes ti
+ ON us.index_id = ti.index_id
+ AND us.table_id = ti.descriptor_id
+ ORDER BY total_reads ASC;
+ `, databaseName, databaseName)
+
+ it, err := ie.QueryIteratorEx(
+ ctx,
+ "capture-index-usage-stats",
+ nil,
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()},
+ stmt,
+ )
+ if err != nil {
+ return err
+ }
+
+ for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) {
+ var row tree.Datums
+ if err != nil {
+ return err
+ }
+ if row = it.Cur(); row == nil {
+ return errors.New("unexpected null row while capturing index usage stats")
+ }
+
+ if row.Len() != expectedNumDatums {
+ return errors.Newf("expected %d columns, received %d while capturing index usage stats", expectedNumDatums, row.Len())
+ }
+
+ tableName := tree.MustBeDString(row[0])
+ tableID := tree.MustBeDInt(row[1])
+ indexName := tree.MustBeDString(row[2])
+ indexID := tree.MustBeDInt(row[3])
+ indexType := tree.MustBeDString(row[4])
+ isUnique := tree.MustBeDBool(row[5])
+ isInverted := tree.MustBeDBool(row[6])
+ totalReads := tree.MustBeDInt(row[7])
+ lastRead := time.Time{}
+ if row[8] != tree.DNull {
+ lastRead = tree.MustBeDTimestampTZ(row[8]).Time
+ }
+
+ capturedIndexStats := &eventpb.CapturedIndexUsageStats{
+ TableID: uint32(roachpb.TableID(tableID)),
+ IndexID: uint32(roachpb.IndexID(indexID)),
+ TotalReadCount: uint64(totalReads),
+ LastRead: lastRead.String(),
+ DatabaseName: databaseName,
+ TableName: string(tableName),
+ IndexName: string(indexName),
+ IndexType: string(indexType),
+ IsUnique: bool(isUnique),
+ IsInverted: bool(isInverted),
+ }
+
+ allCapturedIndexUsageStats = append(allCapturedIndexUsageStats, capturedIndexStats)
+ }
+ if err = it.Close(); err != nil {
+ return err
+ }
+ }
+ logIndexUsageStatsWithDelay(ctx, allCapturedIndexUsageStats, stopper, loggingDelay)
+ return nil
+}
+
+// logIndexUsageStatsWithDelay logs an eventpb.EventPayload at each
+// telemetryCaptureIndexUsageStatsLoggingDelay to avoid exceeding the 10
+// log-line per second limit per node on the telemetry logging pipeline.
+// Currently, this log-line limit is only shared with 1 other telemetry event,
+// SampledQuery, which now has a logging frequency of 8 logs per second.
+func logIndexUsageStatsWithDelay(
+ ctx context.Context, events []eventpb.EventPayload, stopper *stop.Stopper, delay time.Duration,
+) {
+
+ // Log the first event immediately.
+ timer := time.NewTimer(0 * time.Second)
+ for len(events) > 0 {
+ select {
+ case <-stopper.ShouldQuiesce():
+ timer.Stop()
+ return
+ case <-timer.C:
+ event := events[0]
+ log.StructuredEvent(ctx, event)
+ events = events[1:]
+ // Apply a delay to subsequent events.
+ timer.Reset(delay)
+ }
+ }
+ timer.Stop()
+}
+
+func getAllDatabaseNames(ctx context.Context, ie sqlutil.InternalExecutor) ([]string, error) {
+ var allDatabaseNames []string
+ var ok bool
+ var expectedNumDatums = 1
+
+ it, err := ie.QueryIteratorEx(
+ ctx,
+ "get-all-db-names",
+ nil,
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
+ `SELECT database_name FROM [SHOW DATABASES]`,
+ )
+ if err != nil {
+ return []string{}, err
+ }
+
+ // We have to make sure to close the iterator since we might return from the
+ // for loop early (before Next() returns false).
+ defer func() { err = errors.CombineErrors(err, it.Close()) }()
+ for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) {
+ var row tree.Datums
+ if row = it.Cur(); row == nil {
+ return []string{}, errors.New("unexpected null row while capturing index usage stats")
+ }
+ if row.Len() != expectedNumDatums {
+ return []string{}, errors.Newf("expected %d columns, received %d while capturing index usage stats", expectedNumDatums, row.Len())
+ }
+
+ databaseName := string(tree.MustBeDString(row[0]))
+ allDatabaseNames = append(allDatabaseNames, databaseName)
+ }
+ return allDatabaseNames, nil
+}
diff --git a/pkg/sql/scheduledlogging/captured_index_usage_stats_test.go b/pkg/sql/scheduledlogging/captured_index_usage_stats_test.go
new file mode 100644
index 000000000000..a5bfb1360a47
--- /dev/null
+++ b/pkg/sql/scheduledlogging/captured_index_usage_stats_test.go
@@ -0,0 +1,328 @@
+// 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 scheduledlogging
+
+import (
+ "context"
+ "math"
+ "regexp"
+ "sort"
+ "strings"
+ "testing"
+ "time"
+
+ "github.com/cockroachdb/cockroach/pkg/base"
+ "github.com/cockroachdb/cockroach/pkg/settings/cluster"
+ "github.com/cockroachdb/cockroach/pkg/testutils"
+ "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/cockroachdb/cockroach/pkg/util/log/channel"
+ "github.com/cockroachdb/cockroach/pkg/util/log/logconfig"
+ "github.com/cockroachdb/cockroach/pkg/util/syncutil"
+ "github.com/cockroachdb/errors"
+ "github.com/stretchr/testify/require"
+)
+
+type stubDurations struct {
+ syncutil.RWMutex
+ loggingDuration time.Duration
+ overlapDuration time.Duration
+}
+
+func (s *stubDurations) setLoggingDuration(d time.Duration) {
+ s.Lock()
+ defer s.Unlock()
+ s.loggingDuration = d
+}
+
+func (s *stubDurations) getLoggingDuration() time.Duration {
+ s.RLock()
+ defer s.RUnlock()
+ return s.loggingDuration
+}
+
+func (s *stubDurations) setOverlapDuration(d time.Duration) {
+ s.Lock()
+ defer s.Unlock()
+ s.overlapDuration = d
+}
+
+func (s *stubDurations) getOverlapDuration() time.Duration {
+ s.RLock()
+ defer s.RUnlock()
+ return s.overlapDuration
+}
+
+func installTelemetryLogFileSink(t *testing.T, sc *log.TestLogScope) func() {
+ // Enable logging channels.
+ log.TestingResetActive()
+ cfg := logconfig.DefaultConfig()
+ // Make a sink for just the session log.
+ cfg.Sinks.FileGroups = map[string]*logconfig.FileSinkConfig{
+ "telemetry": {
+ Channels: logconfig.SelectChannels(channel.TELEMETRY),
+ }}
+ dir := sc.GetDirectory()
+ require.NoError(t, cfg.Validate(&dir), "expected no errors validating log config")
+ cleanup, err := log.ApplyConfig(cfg)
+ if err != nil {
+ t.Fatal(err)
+ }
+
+ return cleanup
+}
+
+func TestCaptureIndexUsageStats(t *testing.T) {
+ defer leaktest.AfterTest(t)()
+ sc := log.ScopeWithoutShowLogs(t)
+ defer sc.Close(t)
+
+ cleanup := installTelemetryLogFileSink(t, sc)
+ defer cleanup()
+
+ sd := stubDurations{}
+ sd.setLoggingDuration(1 * time.Second)
+ sd.setOverlapDuration(10 * time.Second)
+ stubScheduleInterval := 20 * time.Second
+ stubScheduleCheckEnabledInterval := 1 * time.Second
+ stubLoggingDelay := 0 * time.Second
+
+ // timeBuffer is a short time buffer to account for delays in the schedule
+ // timings when running tests. The time buffer is smaller than the difference
+ // between each schedule interval to ensure that there is no overlap.
+ timeBuffer := 5 * time.Second
+
+ settings := cluster.MakeTestingClusterSettings()
+ // Configure capture index usage statistics to be disabled. This is to test
+ // whether the disabled interval works correctly. We start in a disabled
+ // state, once the disabled interval expires, we check whether we have
+ // transitioned to an enabled state, if we have, we check that the expected
+ // logs have been emitted.
+ telemetryCaptureIndexUsageStatsEnabled.Override(context.Background(), &settings.SV, false)
+ // Configure the schedule interval at which we capture index usage
+ // statistics.
+ telemetryCaptureIndexUsageStatsInterval.Override(context.Background(), &settings.SV, stubScheduleInterval)
+ // Configure the schedule interval at which we check whether capture index
+ // usage statistics has been enabled.
+ telemetryCaptureIndexUsageStatsStatusCheckEnabledInterval.Override(context.Background(), &settings.SV, stubScheduleCheckEnabledInterval)
+ // Configure the delay between each emission of index usage stats logs.
+ telemetryCaptureIndexUsageStatsLoggingDelay.Override(context.Background(), &settings.SV, stubLoggingDelay)
+
+ s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{
+ Settings: settings,
+ Knobs: base.TestingKnobs{
+ CapturedIndexUsageStatsKnobs: &CaptureIndexUsageStatsTestingKnobs{
+ getLoggingDuration: sd.getLoggingDuration,
+ getOverlapDuration: sd.getOverlapDuration,
+ },
+ },
+ })
+
+ defer s.Stopper().Stop(context.Background())
+
+ db := sqlutils.MakeSQLRunner(sqlDB)
+
+ // Create test databases.
+ db.Exec(t, "CREATE DATABASE test")
+ db.Exec(t, "CREATE DATABASE test2")
+
+ // Create a table for each database.
+ db.Exec(t, "CREATE TABLE test.test_table (num INT PRIMARY KEY, letter char)")
+ db.Exec(t, "CREATE TABLE test2.test2_table (num INT PRIMARY KEY, letter char)")
+
+ // Create an index on each created table (each table now has two indices:
+ // primary and this one)
+ db.Exec(t, "CREATE INDEX ON test.test_table (letter)")
+ db.Exec(t, "CREATE INDEX ON test2.test2_table (letter)")
+
+ // Check that telemetry log file contains all the entries we're expecting, at the scheduled intervals.
+
+ // Enable capture of index usage stats.
+ telemetryCaptureIndexUsageStatsEnabled.Override(context.Background(), &s.ClusterSettings().SV, true)
+
+ expectedTotalNumEntriesInSingleInterval := 4
+ expectedNumberOfIndividualIndexEntriesInSingleInterval := 1
+
+ // Expect index usage statistics logs once the schedule disabled interval has passed.
+ // Assert that we have the expected number of total logs and expected number
+ // of logs for each index.
+ testutils.SucceedsWithin(t, func() error {
+ return checkNumTotalEntriesAndNumIndexEntries(
+ expectedTotalNumEntriesInSingleInterval,
+ expectedNumberOfIndividualIndexEntriesInSingleInterval,
+ )
+ }, stubScheduleCheckEnabledInterval+timeBuffer)
+
+ // Verify that a second schedule has run after the enabled interval has passed.
+ // Expect number of total entries to hold 2 times the number of entries in a
+ // single interval.
+ expectedTotalNumEntriesAfterTwoIntervals := expectedTotalNumEntriesInSingleInterval * 2
+ // Expect number of individual index entries to hold 2 times the number of
+ // entries in a single interval.
+ expectedNumberOfIndividualIndexEntriesAfterTwoIntervals := expectedNumberOfIndividualIndexEntriesInSingleInterval * 2
+ // Set the logging duration for the next run to be longer than the schedule
+ // interval duration.
+ stubLoggingDuration := stubScheduleInterval * 2
+ sd.setLoggingDuration(stubLoggingDuration)
+
+ // Expect index usage statistics logs once the schedule enabled interval has passed.
+ // Assert that we have the expected number of total logs and expected number
+ // of logs for each index.
+ testutils.SucceedsWithin(t, func() error {
+ return checkNumTotalEntriesAndNumIndexEntries(
+ expectedTotalNumEntriesAfterTwoIntervals,
+ expectedNumberOfIndividualIndexEntriesAfterTwoIntervals,
+ )
+ }, stubScheduleInterval+timeBuffer)
+
+ // Verify that a third schedule has run after the overlap duration has passed.
+ // Expect number of total entries to hold 3 times the number of entries in a
+ // single interval.
+ expectedTotalNumEntriesAfterThreeIntervals := expectedTotalNumEntriesInSingleInterval * 3
+ // Expect number of individual index entries to hold 3 times the number of
+ // entries in a single interval.
+ expectedNumberOfIndividualIndexEntriesAfterThreeIntervals := expectedNumberOfIndividualIndexEntriesInSingleInterval * 3
+
+ // Assert that we have the expected number of total logs and expected number
+ // of logs for each index.
+ testutils.SucceedsWithin(t, func() error {
+ return checkNumTotalEntriesAndNumIndexEntries(
+ expectedTotalNumEntriesAfterThreeIntervals,
+ expectedNumberOfIndividualIndexEntriesAfterThreeIntervals,
+ )
+ }, sd.getOverlapDuration()+timeBuffer)
+ // Stop capturing index usage statistics.
+ telemetryCaptureIndexUsageStatsEnabled.Override(context.Background(), &settings.SV, false)
+
+ // Iterate through entries, ensure that the timestamp difference between each
+ // schedule is as expected.
+ startTimestamp := int64(0)
+ endTimestamp := int64(math.MaxInt64)
+ maxEntries := 10000
+ entries, err := log.FetchEntriesFromFiles(
+ startTimestamp,
+ endTimestamp,
+ maxEntries,
+ regexp.MustCompile(`"EventType":"captured_index_usage_stats"`),
+ log.WithMarkedSensitiveData,
+ )
+
+ require.NoError(t, err, "expected no error fetching entries from files")
+
+ // Sort slice by timestamp, ascending order.
+ sort.Slice(entries, func(a int, b int) bool {
+ return entries[a].Time < entries[b].Time
+ })
+
+ testData := []time.Duration{
+ 0 * time.Second,
+ // the difference in number of seconds between first and second schedule
+ stubScheduleInterval,
+ // the difference in number of seconds between second and third schedule
+ sd.getOverlapDuration(),
+ }
+
+ var (
+ previousTimestamp = int64(0)
+ currentTimestamp = int64(0)
+ )
+
+ // Check the timestamp differences between schedules.
+ for idx, expectedDuration := range testData {
+ entriesLowerBound := idx * expectedTotalNumEntriesInSingleInterval
+ entriesUpperBound := (idx + 1) * expectedTotalNumEntriesInSingleInterval
+ scheduleEntryBlock := entries[entriesLowerBound:entriesUpperBound]
+ // Take the first log entry from the schedule.
+ currentTimestamp = scheduleEntryBlock[0].Time
+ // If this is the first iteration, initialize the previous timestamp.
+ if idx == 0 {
+ previousTimestamp = currentTimestamp
+ }
+
+ nanoSecondDiff := currentTimestamp - previousTimestamp
+ // We allow for integer division to remove any miscellaneous nanosecond
+ // delay from the logging.
+ secondDiff := nanoSecondDiff / 1e9
+ actualDuration := time.Duration(secondDiff) * time.Second
+ require.Equal(t, expectedDuration, actualDuration)
+ previousTimestamp = currentTimestamp
+ }
+}
+
+// checkNumTotalEntriesAndNumIndexEntries is a helper function that verifies that
+// we are getting the correct number of total log entries and correct number of
+// log entries for each index. Also checks that each log entry contains a node_id
+// field, used to filter node-duplicate logs downstream.
+func checkNumTotalEntriesAndNumIndexEntries(
+ expectedTotalEntries int, expectedIndividualIndexEntries int,
+) error {
+ // Fetch log entries.
+ entries, err := log.FetchEntriesFromFiles(
+ 0,
+ math.MaxInt64,
+ 10000,
+ regexp.MustCompile(`"EventType":"captured_index_usage_stats"`),
+ log.WithMarkedSensitiveData,
+ )
+
+ if err != nil {
+ return err
+ }
+
+ // Assert that we have the correct number of entries.
+ if expectedTotalEntries != len(entries) {
+ return errors.Newf("expected %d total entries, got %d", expectedTotalEntries, len(entries))
+ }
+
+ var (
+ numEntriesForTestTablePrimaryKeyIndex int
+ numEntriesForTestTableLetterIndex int
+ numEntriesForTest2TablePrimaryKeyIndex int
+ numEntriesForTest2TableLetterIndex int
+ )
+
+ for _, e := range entries {
+ if strings.Contains(e.Message, `"TableName":"‹test_table›","IndexName":"‹primary›"`) {
+ numEntriesForTestTablePrimaryKeyIndex++
+ }
+ if strings.Contains(e.Message, `"TableName":"‹test_table›","IndexName":"‹test_table_letter_idx›"`) {
+ numEntriesForTestTableLetterIndex++
+ }
+ if strings.Contains(e.Message, `"TableName":"‹test2_table›","IndexName":"‹primary›"`) {
+ numEntriesForTest2TablePrimaryKeyIndex++
+ }
+ if strings.Contains(e.Message, `"TableName":"‹test2_table›","IndexName":"‹test2_table_letter_idx›"`) {
+ numEntriesForTest2TableLetterIndex++
+ }
+ // Check that the entry has a tag for a node ID of 1.
+ if !strings.Contains(e.Tags, `n1`) {
+ return errors.Newf("expected the entry's tags to include n1, but include got %s", e.Tags)
+ }
+ }
+
+ // Assert that we have the correct number index usage statistic entries for
+ // each index we created across the tables in each database.
+ if expectedIndividualIndexEntries != numEntriesForTestTablePrimaryKeyIndex {
+ return errors.Newf("expected %d test_table primary key index entries, got %d", expectedIndividualIndexEntries, numEntriesForTestTablePrimaryKeyIndex)
+ }
+ if expectedIndividualIndexEntries != numEntriesForTestTableLetterIndex {
+ return errors.Newf("expected %d test_table letter index entries, got %d", expectedIndividualIndexEntries, numEntriesForTestTableLetterIndex)
+ }
+ if expectedIndividualIndexEntries != numEntriesForTest2TablePrimaryKeyIndex {
+ return errors.Newf("expected %d test2_table primary key index entries, got %d", expectedIndividualIndexEntries, numEntriesForTest2TablePrimaryKeyIndex)
+ }
+ if expectedIndividualIndexEntries != numEntriesForTest2TableLetterIndex {
+ return errors.Newf("expected %d test2_table letter index entries, got %d", expectedIndividualIndexEntries, numEntriesForTest2TableLetterIndex)
+ }
+ return nil
+}
diff --git a/pkg/sql/scheduledlogging/main_test.go b/pkg/sql/scheduledlogging/main_test.go
new file mode 100644
index 000000000000..6f02005f4e59
--- /dev/null
+++ b/pkg/sql/scheduledlogging/main_test.go
@@ -0,0 +1,31 @@
+// 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 scheduledlogging_test
+
+import (
+ "os"
+ "testing"
+
+ "github.com/cockroachdb/cockroach/pkg/security"
+ "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"
+ "github.com/cockroachdb/cockroach/pkg/util/randutil"
+)
+
+func TestMain(m *testing.M) {
+ security.SetAssetLoader(securitytest.EmbeddedAssets)
+ randutil.SeedForTests()
+ serverutils.InitTestServerFactory(server.TestServerFactory)
+ serverutils.InitTestClusterFactory(testcluster.TestClusterFactory)
+ os.Exit(m.Run())
+}
diff --git a/pkg/sql/telemetry_logging.go b/pkg/sql/telemetry_logging.go
index 87493c8cc919..b37b2245bfc9 100644
--- a/pkg/sql/telemetry_logging.go
+++ b/pkg/sql/telemetry_logging.go
@@ -21,11 +21,13 @@ import (
// Default value used to designate the maximum frequency at which events
// are logged to the telemetry channel.
-const defaultMaxEventFrequency = 10
+const defaultMaxEventFrequency = 8
var telemetryMaxEventFrequency = settings.RegisterIntSetting(
"sql.telemetry.query_sampling.max_event_frequency",
- "the max event frequency at which we sample queries for telemetry",
+ "the max event frequency at which we sample queries for telemetry, "+
+ "note that this value shares a log-line limit of 10 logs per second on the "+
+ "telemetry pipeline with all other telemetry events",
defaultMaxEventFrequency,
settings.NonNegativeInt,
)
diff --git a/pkg/util/log/eventpb/eventlog_channels_generated.go b/pkg/util/log/eventpb/eventlog_channels_generated.go
index 64364eca4974..f32ff2b798cb 100644
--- a/pkg/util/log/eventpb/eventlog_channels_generated.go
+++ b/pkg/util/log/eventpb/eventlog_channels_generated.go
@@ -254,6 +254,9 @@ func (m *CreateRole) LoggingChannel() logpb.Channel { return logpb.Channel_USER_
// LoggingChannel implements the EventPayload interface.
func (m *DropRole) LoggingChannel() logpb.Channel { return logpb.Channel_USER_ADMIN }
+// LoggingChannel implements the EventPayload interface.
+func (m *CapturedIndexUsageStats) LoggingChannel() logpb.Channel { return logpb.Channel_TELEMETRY }
+
// LoggingChannel implements the EventPayload interface.
func (m *SampledQuery) LoggingChannel() logpb.Channel { return logpb.Channel_TELEMETRY }
diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go
index fb9e9d468c6e..560b3843a5be 100644
--- a/pkg/util/log/eventpb/json_encode_generated.go
+++ b/pkg/util/log/eventpb/json_encode_generated.go
@@ -552,6 +552,117 @@ func (m *AlterTypeOwner) AppendJSONFields(printComma bool, b redact.RedactableBy
return printComma, b
}
+// AppendJSONFields implements the EventPayload interface.
+func (m *CapturedIndexUsageStats) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) {
+
+ printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b)
+
+ if m.TotalReadCount != 0 {
+ if printComma {
+ b = append(b, ',')
+ }
+ printComma = true
+ b = append(b, "\"TotalReadCount\":"...)
+ b = strconv.AppendUint(b, uint64(m.TotalReadCount), 10)
+ }
+
+ if m.LastRead != "" {
+ if printComma {
+ b = append(b, ',')
+ }
+ printComma = true
+ b = append(b, "\"LastRead\":\""...)
+ b = append(b, redact.StartMarker()...)
+ b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.LastRead)))))
+ b = append(b, redact.EndMarker()...)
+ b = append(b, '"')
+ }
+
+ if m.TableID != 0 {
+ if printComma {
+ b = append(b, ',')
+ }
+ printComma = true
+ b = append(b, "\"TableID\":"...)
+ b = strconv.AppendUint(b, uint64(m.TableID), 10)
+ }
+
+ if m.IndexID != 0 {
+ if printComma {
+ b = append(b, ',')
+ }
+ printComma = true
+ b = append(b, "\"IndexID\":"...)
+ b = strconv.AppendUint(b, uint64(m.IndexID), 10)
+ }
+
+ if m.DatabaseName != "" {
+ if printComma {
+ b = append(b, ',')
+ }
+ printComma = true
+ b = append(b, "\"DatabaseName\":\""...)
+ b = append(b, redact.StartMarker()...)
+ b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.DatabaseName)))))
+ b = append(b, redact.EndMarker()...)
+ b = append(b, '"')
+ }
+
+ if m.TableName != "" {
+ if printComma {
+ b = append(b, ',')
+ }
+ printComma = true
+ b = append(b, "\"TableName\":\""...)
+ b = append(b, redact.StartMarker()...)
+ b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.TableName)))))
+ b = append(b, redact.EndMarker()...)
+ b = append(b, '"')
+ }
+
+ if m.IndexName != "" {
+ if printComma {
+ b = append(b, ',')
+ }
+ printComma = true
+ b = append(b, "\"IndexName\":\""...)
+ b = append(b, redact.StartMarker()...)
+ b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.IndexName)))))
+ b = append(b, redact.EndMarker()...)
+ b = append(b, '"')
+ }
+
+ if m.IndexType != "" {
+ if printComma {
+ b = append(b, ',')
+ }
+ printComma = true
+ b = append(b, "\"IndexType\":\""...)
+ b = append(b, redact.StartMarker()...)
+ b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.IndexType)))))
+ b = append(b, redact.EndMarker()...)
+ b = append(b, '"')
+ }
+
+ if m.IsUnique {
+ if printComma {
+ b = append(b, ',')
+ }
+ printComma = true
+ b = append(b, "\"IsUnique\":true"...)
+ }
+
+ if m.IsInverted {
+ if printComma {
+ b = append(b, ',')
+ }
+ printComma = true
+ b = append(b, "\"IsInverted\":true"...)
+ }
+
+ return printComma, b
+}
+
// AppendJSONFields implements the EventPayload interface.
func (m *CertsReload) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) {
diff --git a/pkg/util/log/eventpb/telemetry.pb.go b/pkg/util/log/eventpb/telemetry.pb.go
index 558491c1512c..6a5a06768c6e 100644
--- a/pkg/util/log/eventpb/telemetry.pb.go
+++ b/pkg/util/log/eventpb/telemetry.pb.go
@@ -70,39 +70,102 @@ func (m *SampledQuery) XXX_DiscardUnknown() {
var xxx_messageInfo_SampledQuery proto.InternalMessageInfo
+// CapturedIndexUsageStats
+type CapturedIndexUsageStats struct {
+ CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""`
+ // TotalReadCount is the number of times this index has been read from.
+ TotalReadCount uint64 `protobuf:"varint,2,opt,name=total_read_count,json=totalReadCount,proto3" json:"total_read_count,omitempty"`
+ // LastRead is the timestamp that this index was last being read from.
+ LastRead string `protobuf:"bytes,3,opt,name=last_read,json=lastRead,proto3" json:",omitempty"`
+ // TableID is the ID of the table this index is created on. This is same as
+ // descpb.TableID and is unique within the cluster.
+ TableID uint32 `protobuf:"varint,4,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
+ // IndexID is the ID of the index within the scope of the given table.
+ IndexID uint32 `protobuf:"varint,5,opt,name=index_id,json=indexId,proto3" json:"index_id,omitempty"`
+ DatabaseName string `protobuf:"bytes,6,opt,name=database_name,json=databaseName,proto3" json:",omitempty"`
+ TableName string `protobuf:"bytes,7,opt,name=table_name,json=tableName,proto3" json:",omitempty"`
+ IndexName string `protobuf:"bytes,8,opt,name=index_name,json=indexName,proto3" json:",omitempty"`
+ IndexType string `protobuf:"bytes,9,opt,name=index_type,json=indexType,proto3" json:",omitempty"`
+ IsUnique bool `protobuf:"varint,10,opt,name=is_unique,json=isUnique,proto3" json:",omitempty"`
+ IsInverted bool `protobuf:"varint,11,opt,name=is_inverted,json=isInverted,proto3" json:",omitempty"`
+}
+
+func (m *CapturedIndexUsageStats) Reset() { *m = CapturedIndexUsageStats{} }
+func (m *CapturedIndexUsageStats) String() string { return proto.CompactTextString(m) }
+func (*CapturedIndexUsageStats) ProtoMessage() {}
+func (*CapturedIndexUsageStats) Descriptor() ([]byte, []int) {
+ return fileDescriptor_3d317b4ad74be4f7, []int{1}
+}
+func (m *CapturedIndexUsageStats) XXX_Unmarshal(b []byte) error {
+ return m.Unmarshal(b)
+}
+func (m *CapturedIndexUsageStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+ b = b[:cap(b)]
+ n, err := m.MarshalToSizedBuffer(b)
+ if err != nil {
+ return nil, err
+ }
+ return b[:n], nil
+}
+func (m *CapturedIndexUsageStats) XXX_Merge(src proto.Message) {
+ xxx_messageInfo_CapturedIndexUsageStats.Merge(m, src)
+}
+func (m *CapturedIndexUsageStats) XXX_Size() int {
+ return m.Size()
+}
+func (m *CapturedIndexUsageStats) XXX_DiscardUnknown() {
+ xxx_messageInfo_CapturedIndexUsageStats.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_CapturedIndexUsageStats proto.InternalMessageInfo
+
func init() {
proto.RegisterType((*SampledQuery)(nil), "cockroach.util.log.eventpb.SampledQuery")
+ proto.RegisterType((*CapturedIndexUsageStats)(nil), "cockroach.util.log.eventpb.CapturedIndexUsageStats")
}
func init() { proto.RegisterFile("util/log/eventpb/telemetry.proto", fileDescriptor_3d317b4ad74be4f7) }
var fileDescriptor_3d317b4ad74be4f7 = []byte{
- // 391 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x91, 0x3d, 0x8f, 0xd3, 0x30,
- 0x18, 0x80, 0x63, 0x5a, 0x8a, 0x30, 0xe1, 0x90, 0x22, 0x90, 0xa2, 0x4a, 0x38, 0x51, 0x97, 0x0b,
- 0x12, 0x4a, 0x80, 0x1b, 0x90, 0x18, 0x03, 0x37, 0xc1, 0x92, 0x1c, 0x13, 0x4b, 0x94, 0x3a, 0xaf,
- 0x82, 0x75, 0x4e, 0x9c, 0xc6, 0x6f, 0x4e, 0xd7, 0x7f, 0xc1, 0xaf, 0x42, 0x1d, 0x3b, 0xde, 0x14,
- 0x41, 0xba, 0xdd, 0xc8, 0x2f, 0x40, 0x49, 0x33, 0x50, 0x95, 0xaf, 0xc9, 0xb6, 0xde, 0xe7, 0x79,
- 0x64, 0xd9, 0xd4, 0x6d, 0x50, 0xc8, 0x40, 0xaa, 0x3c, 0x80, 0x2b, 0x28, 0xb1, 0x5a, 0x06, 0x08,
- 0x12, 0x0a, 0xc0, 0x7a, 0xed, 0x57, 0xb5, 0x42, 0x65, 0xcd, 0xb9, 0xe2, 0x97, 0xb5, 0x4a, 0xf9,
- 0x67, 0xbf, 0x67, 0x7d, 0xa9, 0x72, 0x7f, 0x64, 0xe7, 0x8f, 0x73, 0x95, 0xab, 0x01, 0x0b, 0xfa,
- 0xdd, 0xde, 0x98, 0x3f, 0x3d, 0x6a, 0x0e, 0xab, 0x1e, 0xc7, 0xa7, 0x47, 0x63, 0xbd, 0x92, 0x49,
- 0xda, 0x64, 0x02, 0x93, 0x5f, 0xc1, 0xc5, 0xd7, 0x09, 0x35, 0x2f, 0xd2, 0xa2, 0x92, 0x90, 0x45,
- 0x0d, 0xd4, 0x6b, 0xeb, 0x23, 0x9d, 0x71, 0x55, 0x14, 0xaa, 0xb4, 0x89, 0x4b, 0xbc, 0x07, 0xaf,
- 0x7c, 0xff, 0xcf, 0x77, 0xf3, 0xdf, 0x0e, 0xe4, 0x79, 0x7f, 0x7a, 0x07, 0x98, 0x0a, 0xa9, 0x43,
- 0x73, 0xd3, 0x3a, 0xc6, 0xb6, 0x75, 0xc8, 0x6d, 0xeb, 0x18, 0xf1, 0xd8, 0xb2, 0x22, 0x3a, 0xd1,
- 0x2b, 0x69, 0xdf, 0x19, 0x92, 0x2f, 0xff, 0x9d, 0xbc, 0x88, 0x3e, 0xfc, 0xa5, 0xda, 0xb7, 0xac,
- 0x98, 0x4e, 0xe1, 0x1a, 0xb8, 0x3d, 0x19, 0x9a, 0x2f, 0xfe, 0xaf, 0x79, 0x0d, 0xfc, 0xf7, 0xc9,
- 0xa1, 0x65, 0xbd, 0xa6, 0x8f, 0xf4, 0xa5, 0xa8, 0x2a, 0xc8, 0x92, 0x55, 0x03, 0xb5, 0x00, 0x6d,
- 0x4f, 0x5d, 0xe2, 0x4d, 0xc3, 0x93, 0xdb, 0xd6, 0xa1, 0xcf, 0x55, 0x21, 0x10, 0x8a, 0x0a, 0xd7,
- 0xf1, 0xc9, 0x88, 0x45, 0x7b, 0xca, 0x3a, 0xa3, 0x0f, 0xb9, 0xd2, 0x98, 0x80, 0x46, 0x51, 0xa4,
- 0x08, 0xf6, 0x5d, 0x97, 0x78, 0xe4, 0x48, 0x33, 0x7b, 0xe8, 0x7c, 0x64, 0xac, 0xf7, 0xd4, 0xcc,
- 0x84, 0xc6, 0x5a, 0x2c, 0x1b, 0x14, 0xaa, 0xb4, 0x67, 0x2e, 0xf1, 0xee, 0x87, 0xa7, 0x87, 0xce,
- 0x8f, 0xd6, 0x79, 0x52, 0x43, 0x96, 0x72, 0x7c, 0xb3, 0x28, 0x55, 0xa9, 0xa1, 0xd4, 0x02, 0xc5,
- 0x15, 0x2c, 0xe2, 0x03, 0x39, 0x7c, 0xb6, 0xf9, 0xce, 0x8c, 0x4d, 0xc7, 0xc8, 0xb6, 0x63, 0xe4,
- 0xa6, 0x63, 0xe4, 0x5b, 0xc7, 0xc8, 0x97, 0x1d, 0x33, 0xb6, 0x3b, 0x66, 0xdc, 0xec, 0x98, 0xf1,
- 0xe9, 0xde, 0xf8, 0x1c, 0xcb, 0xd9, 0xf0, 0xf5, 0x67, 0x3f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xfd,
- 0xec, 0x21, 0x49, 0x98, 0x02, 0x00, 0x00,
+ // 601 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0xd4, 0xc1, 0x6e, 0xd3, 0x4c,
+ 0x10, 0x07, 0xf0, 0xec, 0xd7, 0xb4, 0x49, 0x36, 0x69, 0x3f, 0x64, 0x81, 0xb0, 0x2a, 0xe1, 0x44,
+ 0x39, 0x50, 0x23, 0xc0, 0x01, 0x7a, 0x40, 0xe2, 0x98, 0xb6, 0x87, 0x08, 0x84, 0x14, 0xb7, 0xbd,
+ 0x70, 0xb1, 0x36, 0xde, 0x51, 0x58, 0xd5, 0xf6, 0x3a, 0xde, 0x71, 0xd5, 0xbc, 0x05, 0x4f, 0xc3,
+ 0x23, 0xa0, 0x1e, 0x7b, 0xec, 0xc9, 0x82, 0xf4, 0xd6, 0x23, 0x4f, 0x80, 0x76, 0xe3, 0x0a, 0x2a,
+ 0x87, 0xc2, 0x85, 0x53, 0x92, 0x9d, 0xdf, 0xfc, 0xb5, 0x9e, 0x89, 0x4c, 0x7b, 0x39, 0x8a, 0x68,
+ 0x10, 0xc9, 0xe9, 0x00, 0x4e, 0x21, 0xc1, 0x74, 0x32, 0x40, 0x88, 0x20, 0x06, 0xcc, 0xe6, 0x5e,
+ 0x9a, 0x49, 0x94, 0xd6, 0x76, 0x28, 0xc3, 0x93, 0x4c, 0xb2, 0xf0, 0xa3, 0xa7, 0xad, 0x17, 0xc9,
+ 0xa9, 0x57, 0xda, 0xed, 0xfb, 0x53, 0x39, 0x95, 0x86, 0x0d, 0xf4, 0xb7, 0x65, 0xc7, 0xf6, 0xa3,
+ 0x4a, 0xa6, 0xf9, 0x54, 0x65, 0x79, 0xa7, 0x52, 0x56, 0xb3, 0x28, 0x60, 0x39, 0x17, 0x18, 0xfc,
+ 0x0a, 0xfb, 0x5f, 0xd6, 0x68, 0xe7, 0x90, 0xc5, 0x69, 0x04, 0x7c, 0x9c, 0x43, 0x36, 0xb7, 0x8e,
+ 0xe8, 0x46, 0x28, 0xe3, 0x58, 0x26, 0x36, 0xe9, 0x11, 0xb7, 0xfd, 0xca, 0xf3, 0x7e, 0x7f, 0x37,
+ 0x6f, 0xcf, 0xc8, 0x03, 0xfd, 0x6b, 0x1f, 0x90, 0x89, 0x48, 0x0d, 0x3b, 0xe7, 0x45, 0xb7, 0x76,
+ 0x51, 0x74, 0xc9, 0x75, 0xd1, 0xad, 0xf9, 0x65, 0x96, 0x35, 0xa6, 0x6b, 0x6a, 0x16, 0xd9, 0xff,
+ 0x99, 0xc8, 0x97, 0x7f, 0x8e, 0x3c, 0x1c, 0xbf, 0xbb, 0x23, 0x55, 0x67, 0x59, 0x3e, 0xad, 0xc3,
+ 0x19, 0x84, 0xf6, 0x9a, 0xc9, 0x7c, 0xf1, 0x77, 0x99, 0x67, 0x10, 0xae, 0x8e, 0x34, 0x59, 0xd6,
+ 0x6b, 0xfa, 0xbf, 0x3a, 0x11, 0x69, 0x0a, 0x3c, 0x98, 0xe5, 0x90, 0x09, 0x50, 0x76, 0xbd, 0x47,
+ 0xdc, 0xfa, 0x70, 0xeb, 0xba, 0xe8, 0xd2, 0x67, 0x32, 0x16, 0x08, 0x71, 0x8a, 0x73, 0x7f, 0xab,
+ 0x64, 0xe3, 0xa5, 0xb2, 0x76, 0xe9, 0x66, 0x28, 0x15, 0x06, 0xa0, 0x50, 0xc4, 0x0c, 0xc1, 0x5e,
+ 0xef, 0x11, 0x97, 0x54, 0xda, 0x3a, 0x1a, 0x1d, 0x94, 0xc6, 0x7a, 0x4b, 0x3b, 0x5c, 0x28, 0xcc,
+ 0xc4, 0x24, 0x47, 0x21, 0x13, 0x7b, 0xa3, 0x47, 0xdc, 0xd6, 0x70, 0xe7, 0x76, 0xcf, 0xf7, 0xa2,
+ 0xfb, 0x20, 0x03, 0xce, 0x42, 0x7c, 0xd3, 0x4f, 0x64, 0xa2, 0x20, 0x51, 0x02, 0xc5, 0x29, 0xf4,
+ 0xfd, 0x5b, 0xcd, 0xfd, 0xcf, 0x75, 0xfa, 0x70, 0x8f, 0xa5, 0x98, 0x67, 0xc0, 0x47, 0x09, 0x87,
+ 0xb3, 0x63, 0xc5, 0xa6, 0x70, 0x88, 0x0c, 0xd5, 0x3f, 0xda, 0xa9, 0x4b, 0xef, 0xa1, 0x44, 0x16,
+ 0x05, 0x19, 0x30, 0x1e, 0x84, 0x32, 0x4f, 0xd0, 0x2c, 0xb8, 0xee, 0x6f, 0x99, 0x73, 0x1f, 0x18,
+ 0xdf, 0xd3, 0xa7, 0xd6, 0x53, 0xda, 0x8a, 0x98, 0x42, 0x03, 0xcd, 0xbe, 0x5a, 0x95, 0xc9, 0x34,
+ 0x35, 0xd0, 0x1d, 0xd6, 0x63, 0xda, 0x44, 0x36, 0x89, 0x20, 0x10, 0xdc, 0x0c, 0x7f, 0x73, 0xd8,
+ 0x5e, 0x14, 0xdd, 0xc6, 0x91, 0x3e, 0x1b, 0xed, 0xfb, 0x0d, 0x53, 0x1c, 0x19, 0x27, 0xf4, 0x73,
+ 0x6a, 0xb7, 0xfe, 0xd3, 0x99, 0x67, 0xd7, 0xce, 0x14, 0x47, 0x5c, 0xaf, 0x86, 0x33, 0x64, 0x13,
+ 0xa6, 0x20, 0x48, 0x58, 0x0c, 0xe5, 0x98, 0x2b, 0xab, 0xb9, 0x41, 0xef, 0x59, 0x0c, 0xd6, 0x73,
+ 0x4a, 0x97, 0x97, 0x30, 0x1d, 0x8d, 0x95, 0x1d, 0x2d, 0x23, 0x6e, 0xf8, 0xf2, 0x2e, 0x86, 0x37,
+ 0x57, 0x73, 0x23, 0x6e, 0x73, 0x9c, 0xa7, 0x60, 0xb7, 0xee, 0xe0, 0x47, 0xf3, 0x14, 0xf4, 0xf8,
+ 0x84, 0x0a, 0xf2, 0x44, 0xcc, 0x72, 0xb0, 0x69, 0x8f, 0xb8, 0xcd, 0xea, 0xf8, 0x84, 0x3a, 0x36,
+ 0x75, 0x6b, 0x40, 0xdb, 0x42, 0x05, 0x22, 0x39, 0x85, 0x0c, 0x81, 0xdb, 0xed, 0x95, 0x9c, 0x0a,
+ 0x35, 0x2a, 0xc5, 0xf0, 0xc9, 0xf9, 0x37, 0xa7, 0x76, 0xbe, 0x70, 0xc8, 0xc5, 0xc2, 0x21, 0x97,
+ 0x0b, 0x87, 0x7c, 0x5d, 0x38, 0xe4, 0xd3, 0x95, 0x53, 0xbb, 0xb8, 0x72, 0x6a, 0x97, 0x57, 0x4e,
+ 0xed, 0x43, 0xa3, 0xfc, 0x6b, 0x4c, 0x36, 0xcc, 0x3b, 0x63, 0xf7, 0x47, 0x00, 0x00, 0x00, 0xff,
+ 0xff, 0xdc, 0x91, 0x07, 0x85, 0xd1, 0x04, 0x00, 0x00,
}
func (m *SampledQuery) Marshal() (dAtA []byte, err error) {
@@ -176,6 +239,109 @@ func (m *SampledQuery) MarshalToSizedBuffer(dAtA []byte) (int, error) {
return len(dAtA) - i, nil
}
+func (m *CapturedIndexUsageStats) Marshal() (dAtA []byte, err error) {
+ size := m.Size()
+ dAtA = make([]byte, size)
+ n, err := m.MarshalToSizedBuffer(dAtA[:size])
+ if err != nil {
+ return nil, err
+ }
+ return dAtA[:n], nil
+}
+
+func (m *CapturedIndexUsageStats) MarshalTo(dAtA []byte) (int, error) {
+ size := m.Size()
+ return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *CapturedIndexUsageStats) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+ i := len(dAtA)
+ _ = i
+ var l int
+ _ = l
+ if m.IsInverted {
+ i--
+ if m.IsInverted {
+ dAtA[i] = 1
+ } else {
+ dAtA[i] = 0
+ }
+ i--
+ dAtA[i] = 0x58
+ }
+ if m.IsUnique {
+ i--
+ if m.IsUnique {
+ dAtA[i] = 1
+ } else {
+ dAtA[i] = 0
+ }
+ i--
+ dAtA[i] = 0x50
+ }
+ if len(m.IndexType) > 0 {
+ i -= len(m.IndexType)
+ copy(dAtA[i:], m.IndexType)
+ i = encodeVarintTelemetry(dAtA, i, uint64(len(m.IndexType)))
+ i--
+ dAtA[i] = 0x4a
+ }
+ if len(m.IndexName) > 0 {
+ i -= len(m.IndexName)
+ copy(dAtA[i:], m.IndexName)
+ i = encodeVarintTelemetry(dAtA, i, uint64(len(m.IndexName)))
+ i--
+ dAtA[i] = 0x42
+ }
+ if len(m.TableName) > 0 {
+ i -= len(m.TableName)
+ copy(dAtA[i:], m.TableName)
+ i = encodeVarintTelemetry(dAtA, i, uint64(len(m.TableName)))
+ i--
+ dAtA[i] = 0x3a
+ }
+ if len(m.DatabaseName) > 0 {
+ i -= len(m.DatabaseName)
+ copy(dAtA[i:], m.DatabaseName)
+ i = encodeVarintTelemetry(dAtA, i, uint64(len(m.DatabaseName)))
+ i--
+ dAtA[i] = 0x32
+ }
+ if m.IndexID != 0 {
+ i = encodeVarintTelemetry(dAtA, i, uint64(m.IndexID))
+ i--
+ dAtA[i] = 0x28
+ }
+ if m.TableID != 0 {
+ i = encodeVarintTelemetry(dAtA, i, uint64(m.TableID))
+ i--
+ dAtA[i] = 0x20
+ }
+ if len(m.LastRead) > 0 {
+ i -= len(m.LastRead)
+ copy(dAtA[i:], m.LastRead)
+ i = encodeVarintTelemetry(dAtA, i, uint64(len(m.LastRead)))
+ i--
+ dAtA[i] = 0x1a
+ }
+ if m.TotalReadCount != 0 {
+ i = encodeVarintTelemetry(dAtA, i, uint64(m.TotalReadCount))
+ i--
+ dAtA[i] = 0x10
+ }
+ {
+ size, err := m.CommonEventDetails.MarshalToSizedBuffer(dAtA[:i])
+ if err != nil {
+ return 0, err
+ }
+ i -= size
+ i = encodeVarintTelemetry(dAtA, i, uint64(size))
+ }
+ i--
+ dAtA[i] = 0xa
+ return len(dAtA) - i, nil
+}
+
func encodeVarintTelemetry(dAtA []byte, offset int, v uint64) int {
offset -= sovTelemetry(v)
base := offset
@@ -212,6 +378,52 @@ func (m *SampledQuery) Size() (n int) {
return n
}
+func (m *CapturedIndexUsageStats) Size() (n int) {
+ if m == nil {
+ return 0
+ }
+ var l int
+ _ = l
+ l = m.CommonEventDetails.Size()
+ n += 1 + l + sovTelemetry(uint64(l))
+ if m.TotalReadCount != 0 {
+ n += 1 + sovTelemetry(uint64(m.TotalReadCount))
+ }
+ l = len(m.LastRead)
+ if l > 0 {
+ n += 1 + l + sovTelemetry(uint64(l))
+ }
+ if m.TableID != 0 {
+ n += 1 + sovTelemetry(uint64(m.TableID))
+ }
+ if m.IndexID != 0 {
+ n += 1 + sovTelemetry(uint64(m.IndexID))
+ }
+ l = len(m.DatabaseName)
+ if l > 0 {
+ n += 1 + l + sovTelemetry(uint64(l))
+ }
+ l = len(m.TableName)
+ if l > 0 {
+ n += 1 + l + sovTelemetry(uint64(l))
+ }
+ l = len(m.IndexName)
+ if l > 0 {
+ n += 1 + l + sovTelemetry(uint64(l))
+ }
+ l = len(m.IndexType)
+ if l > 0 {
+ n += 1 + l + sovTelemetry(uint64(l))
+ }
+ if m.IsUnique {
+ n += 2
+ }
+ if m.IsInverted {
+ n += 2
+ }
+ return n
+}
+
func sovTelemetry(x uint64) (n int) {
return (math_bits.Len64(x|1) + 6) / 7
}
@@ -429,6 +641,346 @@ func (m *SampledQuery) Unmarshal(dAtA []byte) error {
}
return nil
}
+func (m *CapturedIndexUsageStats) Unmarshal(dAtA []byte) error {
+ l := len(dAtA)
+ iNdEx := 0
+ for iNdEx < l {
+ preIndex := iNdEx
+ var wire uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowTelemetry
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ wire |= uint64(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ fieldNum := int32(wire >> 3)
+ wireType := int(wire & 0x7)
+ if wireType == 4 {
+ return fmt.Errorf("proto: CapturedIndexUsageStats: wiretype end group for non-group")
+ }
+ if fieldNum <= 0 {
+ return fmt.Errorf("proto: CapturedIndexUsageStats: illegal tag %d (wire type %d)", fieldNum, wire)
+ }
+ switch fieldNum {
+ case 1:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType)
+ }
+ var msglen int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowTelemetry
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ msglen |= int(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ if msglen < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ postIndex := iNdEx + msglen
+ if postIndex < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+ return err
+ }
+ iNdEx = postIndex
+ case 2:
+ if wireType != 0 {
+ return fmt.Errorf("proto: wrong wireType = %d for field TotalReadCount", wireType)
+ }
+ m.TotalReadCount = 0
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowTelemetry
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ m.TotalReadCount |= uint64(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ case 3:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field LastRead", wireType)
+ }
+ var stringLen uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowTelemetry
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ stringLen |= uint64(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ intStringLen := int(stringLen)
+ if intStringLen < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ postIndex := iNdEx + intStringLen
+ if postIndex < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ m.LastRead = string(dAtA[iNdEx:postIndex])
+ iNdEx = postIndex
+ case 4:
+ if wireType != 0 {
+ return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType)
+ }
+ m.TableID = 0
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowTelemetry
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ m.TableID |= uint32(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ case 5:
+ if wireType != 0 {
+ return fmt.Errorf("proto: wrong wireType = %d for field IndexID", wireType)
+ }
+ m.IndexID = 0
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowTelemetry
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ m.IndexID |= uint32(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ case 6:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field DatabaseName", wireType)
+ }
+ var stringLen uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowTelemetry
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ stringLen |= uint64(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ intStringLen := int(stringLen)
+ if intStringLen < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ postIndex := iNdEx + intStringLen
+ if postIndex < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ m.DatabaseName = string(dAtA[iNdEx:postIndex])
+ iNdEx = postIndex
+ case 7:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field TableName", wireType)
+ }
+ var stringLen uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowTelemetry
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ stringLen |= uint64(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ intStringLen := int(stringLen)
+ if intStringLen < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ postIndex := iNdEx + intStringLen
+ if postIndex < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ m.TableName = string(dAtA[iNdEx:postIndex])
+ iNdEx = postIndex
+ case 8:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field IndexName", wireType)
+ }
+ var stringLen uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowTelemetry
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ stringLen |= uint64(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ intStringLen := int(stringLen)
+ if intStringLen < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ postIndex := iNdEx + intStringLen
+ if postIndex < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ m.IndexName = string(dAtA[iNdEx:postIndex])
+ iNdEx = postIndex
+ case 9:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field IndexType", wireType)
+ }
+ var stringLen uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowTelemetry
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ stringLen |= uint64(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ intStringLen := int(stringLen)
+ if intStringLen < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ postIndex := iNdEx + intStringLen
+ if postIndex < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ m.IndexType = string(dAtA[iNdEx:postIndex])
+ iNdEx = postIndex
+ case 10:
+ if wireType != 0 {
+ return fmt.Errorf("proto: wrong wireType = %d for field IsUnique", wireType)
+ }
+ var v int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowTelemetry
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ v |= int(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ m.IsUnique = bool(v != 0)
+ case 11:
+ if wireType != 0 {
+ return fmt.Errorf("proto: wrong wireType = %d for field IsInverted", wireType)
+ }
+ var v int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowTelemetry
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ v |= int(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ m.IsInverted = bool(v != 0)
+ default:
+ iNdEx = preIndex
+ skippy, err := skipTelemetry(dAtA[iNdEx:])
+ if err != nil {
+ return err
+ }
+ if (skippy < 0) || (iNdEx+skippy) < 0 {
+ return ErrInvalidLengthTelemetry
+ }
+ if (iNdEx + skippy) > l {
+ return io.ErrUnexpectedEOF
+ }
+ iNdEx += skippy
+ }
+ }
+
+ if iNdEx > l {
+ return io.ErrUnexpectedEOF
+ }
+ return nil
+}
func skipTelemetry(dAtA []byte) (n int, err error) {
l := len(dAtA)
iNdEx := 0
diff --git a/pkg/util/log/eventpb/telemetry.proto b/pkg/util/log/eventpb/telemetry.proto
index 785b3781c76c..da2fd49700a9 100644
--- a/pkg/util/log/eventpb/telemetry.proto
+++ b/pkg/util/log/eventpb/telemetry.proto
@@ -45,3 +45,29 @@ message SampledQuery {
string distribution = 6 [(gogoproto.jsontag) = ",omitempty", (gogoproto.moretags) = "redact:\"nonsensitive\""];
}
+// CapturedIndexUsageStats
+message CapturedIndexUsageStats {
+ CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true];
+
+ // Couldn't use roachpb.CollectedIndexUsageStatistics due to circular dependency.
+
+ // TotalReadCount is the number of times this index has been read from.
+ uint64 total_read_count = 2;
+
+ // LastRead is the timestamp that this index was last being read from.
+ string last_read = 3 [(gogoproto.jsontag) = ",omitempty"];
+
+ // TableID is the ID of the table this index is created on. This is same as
+ // descpb.TableID and is unique within the cluster.
+ uint32 table_id = 4 [(gogoproto.customname) = "TableID"];
+
+ // IndexID is the ID of the index within the scope of the given table.
+ uint32 index_id = 5 [(gogoproto.customname) = "IndexID"];
+
+ string database_name = 6 [(gogoproto.jsontag) = ",omitempty"];
+ string table_name = 7 [(gogoproto.jsontag) = ",omitempty"];
+ string index_name = 8 [(gogoproto.jsontag) = ",omitempty"];
+ string index_type = 9 [(gogoproto.jsontag) = ",omitempty"];
+ bool is_unique = 10 [(gogoproto.jsontag) = ",omitempty"];
+ bool is_inverted = 11 [(gogoproto.jsontag) = ",omitempty"];
+}