Skip to content

Commit

Permalink
sql: introduce crdb_internal.index_usage_stats virtual table
Browse files Browse the repository at this point in the history
This commit introduce crdb_internal.index_usage_stats virtual
table that is backed by new clusterindexusagestats package. This
new package implements a variant of the indexusagestats interface
and serves the data by issuing cluster RPC fanout.

Addresses cockroachdb#64740

Followup to cockroachdb#66639

Release note (sql change): introduce crdb_internal.index_usage_statistics
virtual table to surface index usage statistics.
sql.metrics.index_usage_stats.enabled cluster setting can be used to
turn on/off the subsystem. It is default to true.
  • Loading branch information
Azhng committed Aug 3, 2021
1 parent 9f15510 commit 52fa667
Show file tree
Hide file tree
Showing 17 changed files with 1,696 additions and 1,572 deletions.
1 change: 1 addition & 0 deletions pkg/cli/testdata/zip/partial1
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null
[cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done
[cluster] retrieving SQL data for crdb_internal.zones... writing output: debug/crdb_internal.zones.txt... done
[cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done
[cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done
[cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done
[cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done
[node 1] node status... converting to JSON... writing binary output: debug/nodes/1/status.json... done
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/testdata/zip/partial1_excluded
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0
[cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done
[cluster] retrieving SQL data for crdb_internal.zones... writing output: debug/crdb_internal.zones.txt... done
[cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done
[cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done
[cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done
[cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done
[node 1] node status... converting to JSON... writing binary output: debug/nodes/1/status.json... done
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/testdata/zip/partial2
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null
[cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done
[cluster] retrieving SQL data for crdb_internal.zones... writing output: debug/crdb_internal.zones.txt... done
[cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done
[cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done
[cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done
[cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done
[node 1] node status... converting to JSON... writing binary output: debug/nodes/1/status.json... done
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/testdata/zip/testzip
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null
[cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done
[cluster] retrieving SQL data for crdb_internal.zones... writing output: debug/crdb_internal.zones.txt... done
[cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done
[cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done
[cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done
[cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done
[cluster] requesting CPU profiles
Expand Down
3 changes: 3 additions & 0 deletions pkg/cli/testdata/zip/testzip_concurrent
Original file line number Diff line number Diff line change
Expand Up @@ -256,6 +256,9 @@ zip
[cluster] retrieving SQL data for crdb_internal.cluster_transactions...
[cluster] retrieving SQL data for crdb_internal.cluster_transactions: done
[cluster] retrieving SQL data for crdb_internal.cluster_transactions: writing output: debug/crdb_internal.cluster_transactions.txt...
[cluster] retrieving SQL data for crdb_internal.index_usage_statistics...
[cluster] retrieving SQL data for crdb_internal.index_usage_statistics: done
[cluster] retrieving SQL data for crdb_internal.index_usage_statistics: writing output: debug/crdb_internal.index_usage_statistics.txt...
[cluster] retrieving SQL data for crdb_internal.invalid_objects...
[cluster] retrieving SQL data for crdb_internal.invalid_objects: done
[cluster] retrieving SQL data for crdb_internal.invalid_objects: writing output: debug/crdb_internal.invalid_objects.txt...
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/zip_cluster_wide.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ var debugZipTablesPerCluster = []string{
"crdb_internal.partitions",
"crdb_internal.zones",
"crdb_internal.invalid_objects",
"crdb_internal.index_usage_statistics",
}

// collectClusterData runs the data collection that only needs to
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/catconstants/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ const (
CrdbInternalGossipLivenessTableID
CrdbInternalGossipNetworkTableID
CrdbInternalIndexColumnsTableID
CrdbInternalIndexUsageStatisticsTableID
CrdbInternalInflightTraceSpanTableID
CrdbInternalJobsTableID
CrdbInternalKVNodeStatusTableID
Expand Down
59 changes: 59 additions & 0 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/idxusage"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
Expand Down Expand Up @@ -110,6 +111,7 @@ var crdbInternal = virtualSchema{
catconstants.CrdbInternalGossipLivenessTableID: crdbInternalGossipLivenessTable,
catconstants.CrdbInternalGossipNetworkTableID: crdbInternalGossipNetworkTable,
catconstants.CrdbInternalIndexColumnsTableID: crdbInternalIndexColumnsTable,
catconstants.CrdbInternalIndexUsageStatisticsTableID: crdbInternalIndexUsageStatistics,
catconstants.CrdbInternalInflightTraceSpanTableID: crdbInternalInflightTraceSpanTable,
catconstants.CrdbInternalJobsTableID: crdbInternalJobsTable,
catconstants.CrdbInternalKVNodeStatusTableID: crdbInternalKVNodeStatusTable,
Expand Down Expand Up @@ -4838,3 +4840,60 @@ CREATE TABLE crdb_internal.lost_descriptors_with_data (
return nil
},
}

var crdbInternalIndexUsageStatistics = virtualSchemaTable{
comment: `cluster-wide index usage statistics (in-memory, not durable).` +
`This is an expensive operation since it creates a cluster-wide RPC fanout.`,
schema: `
CREATE TABLE crdb_internal.index_usage_statistics (
table_id INT NOT NULL,
index_id INT NOT NULL,
total_reads INT NOT NULL,
last_read TIMESTAMPTZ
)
`,
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
// Perform RPC Fanout.
stats, err :=
p.extendedEvalCtx.SQLStatusServer.IndexUsageStatistics(ctx, &serverpb.IndexUsageStatisticsRequest{})
if err != nil {
return nil, nil, err
}
indexStats := idxusage.NewLocalIndexUsageStatsFromExistingStats(&idxusage.Config{}, stats.Statistics)

row := make(tree.Datums, 4 /* number of columns for this virtual table */)
worker := func(pusher rowPusher) error {
return forEachTableDescAll(ctx, p, dbContext, hideVirtual,
func(db catalog.DatabaseDescriptor, _ string, table catalog.TableDescriptor) error {
tableID := table.GetID()
return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(idx catalog.Index) error {
indexID := idx.GetID()
stats := indexStats.Get(roachpb.IndexUsageKey{
TableID: roachpb.TableID(tableID),
IndexID: roachpb.IndexID(indexID),
})

lastScanTs := tree.DNull
if !stats.LastRead.IsZero() {
lastScanTs, err = tree.MakeDTimestampTZ(stats.LastRead, time.Nanosecond)
if err != nil {
return err
}
}

row = row[:0]

row = append(row,
tree.NewDInt(tree.DInt(tableID)), // tableID
tree.NewDInt(tree.DInt(indexID)), // indexID
tree.NewDInt(tree.DInt(stats.TotalReadCount)), // total_reads
lastScanTs, // last_scan
)

return pusher.pushRow(row...)
})
})
}
return setupGenerator(ctx, worker, stopper)
},
}
66 changes: 48 additions & 18 deletions pkg/sql/idxusage/local_idx_usage_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -130,6 +130,17 @@ func NewLocalIndexUsageStats(cfg *Config) *LocalIndexUsageStats {
return is
}

// NewLocalIndexUsageStatsFromExistingStats returns a new instance of
// LocalIndexUsageStats that is populated using given
// []roachpb.CollectedIndexUsageStatistics.
func NewLocalIndexUsageStatsFromExistingStats(
cfg *Config, stats []roachpb.CollectedIndexUsageStatistics,
) *LocalIndexUsageStats {
s := NewLocalIndexUsageStats(cfg)
s.batchInsertUnsafe(stats)
return s
}

// Start starts the background goroutine that is responsible for collecting
// index usage statistics.
func (s *LocalIndexUsageStats) Start(ctx context.Context, stopper *stop.Stopper) {
Expand Down Expand Up @@ -209,7 +220,7 @@ func (s *LocalIndexUsageStats) ForEach(options IteratorOptions, visitor StatsVis
s.mu.RUnlock()

for _, tableID := range tableIDLists {
tableIdxStats := s.getStatsForTableID(tableID, false /* createIfNotExists */)
tableIdxStats := s.getStatsForTableID(tableID, false /* createIfNotExists */, false /* unsafe */)

// This means the data s being cleared before we can fetch it. It's not an
// error, so we simply just skip over it.
Expand All @@ -231,6 +242,20 @@ func (s *LocalIndexUsageStats) ForEach(options IteratorOptions, visitor StatsVis
return nil
}

// batchInsertUnsafe inserts otherStats into s without taking on write lock.
// This should only be called during initialization and we can be sure there's
// not other users of s. This avoids the locking overhead when it's not
// necessary.
func (s *LocalIndexUsageStats) batchInsertUnsafe(
otherStats []roachpb.CollectedIndexUsageStatistics,
) {
for _, newStats := range otherStats {
tableIndexStats := s.getStatsForTableID(newStats.Key.TableID, true /* createIfNotExists */, true /* unsafe */)
stats := tableIndexStats.getStatsForIndexID(newStats.Key.IndexID, true /* createIfNotExists */, true /* unsafe */)
stats.Add(&newStats.Stats)
}
}

func (s *LocalIndexUsageStats) clear() {
s.mu.Lock()
defer s.mu.Unlock()
Expand All @@ -241,8 +266,8 @@ func (s *LocalIndexUsageStats) clear() {
}

func (s *LocalIndexUsageStats) insertIndexUsage(idxUse *indexUse) {
tableStats := s.getStatsForTableID(idxUse.key.TableID, true /* createIfNotExists */)
indexStats := tableStats.getStatsForIndexID(idxUse.key.IndexID, true /* createIfNotExists */)
tableStats := s.getStatsForTableID(idxUse.key.TableID, true /* createIfNotExists */, false /* unsafe */)
indexStats := tableStats.getStatsForIndexID(idxUse.key.IndexID, true /* createIfNotExists */, false /* unsafe */)
indexStats.Lock()
defer indexStats.Unlock()
switch idxUse.usageTyp {
Expand All @@ -260,14 +285,16 @@ func (s *LocalIndexUsageStats) insertIndexUsage(idxUse *indexUse) {
}

func (s *LocalIndexUsageStats) getStatsForTableID(
id roachpb.TableID, createIfNotExists bool,
id roachpb.TableID, createIfNotExists bool, unsafe bool,
) *tableIndexStats {
if createIfNotExists {
s.mu.Lock()
defer s.mu.Unlock()
} else {
s.mu.RLock()
defer s.mu.RUnlock()
if !unsafe {
if createIfNotExists {
s.mu.Lock()
defer s.mu.Unlock()
} else {
s.mu.RLock()
defer s.mu.RUnlock()
}
}

if tableIndexStats, ok := s.mu.usageStats[id]; ok {
Expand All @@ -287,14 +314,17 @@ func (s *LocalIndexUsageStats) getStatsForTableID(
}

func (t *tableIndexStats) getStatsForIndexID(
id roachpb.IndexID, createIfNotExists bool,
id roachpb.IndexID, createIfNotExists bool, unsafe bool,
) *indexStats {
if createIfNotExists {
t.Lock()
defer t.Unlock()
} else {
t.RLock()
defer t.RUnlock()
if !unsafe {
if createIfNotExists {
t.Lock()
defer t.Unlock()
} else {
t.RLock()
defer t.RUnlock()
}

}

if stats, ok := t.stats[id]; ok {
Expand Down Expand Up @@ -329,7 +359,7 @@ func (t *tableIndexStats) iterateIndexStats(
}

for _, indexID := range indexIDs {
indexStats := t.getStatsForIndexID(indexID, false /* createIfNotExists */)
indexStats := t.getStatsForIndexID(indexID, false /* createIfNotExists */, false /* unsafe */)

// This means the data is being cleared before we can fetch it. It's not an
// error, so we simply just skip over it.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/crdb_internal
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ crdb_internal gossip_liveness table NULL NULL NULL
crdb_internal gossip_network table NULL NULL NULL
crdb_internal gossip_nodes table NULL NULL NULL
crdb_internal index_columns table NULL NULL NULL
crdb_internal index_usage_statistics table NULL NULL NULL
crdb_internal interleaved table NULL NULL NULL
crdb_internal invalid_objects table NULL NULL NULL
crdb_internal jobs table NULL NULL NULL
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ crdb_internal gossip_liveness table NULL NULL NULL
crdb_internal gossip_network table NULL NULL NULL
crdb_internal gossip_nodes table NULL NULL NULL
crdb_internal index_columns table NULL NULL NULL
crdb_internal index_usage_statistics table NULL NULL NULL
crdb_internal interleaved table NULL NULL NULL
crdb_internal invalid_objects table NULL NULL NULL
crdb_internal jobs table NULL NULL NULL
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/create_statements
Original file line number Diff line number Diff line change
Expand Up @@ -412,6 +412,17 @@ CREATE TABLE crdb_internal.index_columns (
column_direction STRING NULL,
implicit BOOL NULL
) {} {}
CREATE TABLE crdb_internal.index_usage_statistics (
table_id INT8 NOT NULL,
index_id INT8 NOT NULL,
total_reads INT8 NOT NULL,
last_read TIMESTAMPTZ NULL
) CREATE TABLE crdb_internal.index_usage_statistics (
table_id INT8 NOT NULL,
index_id INT8 NOT NULL,
total_reads INT8 NOT NULL,
last_read TIMESTAMPTZ NULL
) {} {}
CREATE TABLE crdb_internal.interleaved (
database_name STRING NOT NULL,
schema_name STRING NOT NULL,
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/grant_table
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ test crdb_internal gossip_liveness public
test crdb_internal gossip_network public SELECT
test crdb_internal gossip_nodes public SELECT
test crdb_internal index_columns public SELECT
test crdb_internal index_usage_statistics public SELECT
test crdb_internal interleaved public SELECT
test crdb_internal invalid_objects public SELECT
test crdb_internal jobs public SELECT
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -420,6 +420,7 @@ crdb_internal gossip_liveness
crdb_internal gossip_network
crdb_internal gossip_nodes
crdb_internal index_columns
crdb_internal index_usage_statistics
crdb_internal interleaved
crdb_internal invalid_objects
crdb_internal jobs
Expand Down Expand Up @@ -726,6 +727,7 @@ gossip_liveness
gossip_network
gossip_nodes
index_columns
index_usage_statistics
interleaved
invalid_objects
jobs
Expand Down Expand Up @@ -1067,6 +1069,7 @@ system crdb_internal gossip_liveness SYSTEM
system crdb_internal gossip_network SYSTEM VIEW NO 1
system crdb_internal gossip_nodes SYSTEM VIEW NO 1
system crdb_internal index_columns SYSTEM VIEW NO 1
system crdb_internal index_usage_statistics SYSTEM VIEW NO 1
system crdb_internal interleaved SYSTEM VIEW NO 1
system crdb_internal invalid_objects SYSTEM VIEW NO 1
system crdb_internal jobs SYSTEM VIEW NO 1
Expand Down Expand Up @@ -2594,6 +2597,7 @@ NULL public system crdb_internal gossip_liveness
NULL public system crdb_internal gossip_network SELECT NULL YES
NULL public system crdb_internal gossip_nodes SELECT NULL YES
NULL public system crdb_internal index_columns SELECT NULL YES
NULL public system crdb_internal index_usage_statistics SELECT NULL YES
NULL public system crdb_internal interleaved SELECT NULL YES
NULL public system crdb_internal invalid_objects SELECT NULL YES
NULL public system crdb_internal jobs SELECT NULL YES
Expand Down Expand Up @@ -3169,6 +3173,7 @@ NULL public system crdb_internal gossip_liveness
NULL public system crdb_internal gossip_network SELECT NULL YES
NULL public system crdb_internal gossip_nodes SELECT NULL YES
NULL public system crdb_internal index_columns SELECT NULL YES
NULL public system crdb_internal index_usage_statistics SELECT NULL YES
NULL public system crdb_internal interleaved SELECT NULL YES
NULL public system crdb_internal invalid_objects SELECT NULL YES
NULL public system crdb_internal jobs SELECT NULL YES
Expand Down
Loading

0 comments on commit 52fa667

Please sign in to comment.