Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
77642: sql: backend changes to surface unused index recommendations r=THardy98 a=THardy98

This change introduces backend changes to surface drop index
recommendations for unused indices. As a first iteration, this change
implements a naive approach where we compute index recommendations
ad-hoc on request (for the database details, table details, and index
usage stats requests), and offers no persistence.

Release note (api change): Added logic to support dropping unused index
recommendations.

78034: [CRDB-13198] ui, server: update cluster settings to include last update time r=Santamaura a=Santamaura

Resolves #76626

Previously it was difficult to determine from the cluster
settings page which settings have been altered.
This patch updates the cluster settings endpoint to include a
last updated field, indicating when the setting was last altered,
which allows the settings page to include that column in the
table and is pre-sorted by this value.

Release note: None

Release justification: Fairly minor changes for QoL upgrade

![Screen Shot 2022-03-21 at 10 42 27 AM](https://user-images.githubusercontent.com/17861665/159289222-28970bbe-cac0-4003-853b-35e619908f0f.png)

78532: clusterversion: introduce version key for `crdb_internal.cluster_locks` r=AlexTalks a=AlexTalks

This change adds a new cluster version to enable usage of the virtual
table `crdb_internal.cluster_locks`, so that we can disable usage of new
KV `QueryLocksRequest` API on cluster versions prior to 22.1.

Release note: None

Release justification: introduces a new cluster version without any
logic changes

78631: lease: add a defensive check for nil-ness r=Xiang-Gu a=Xiang-Gu

Previously, return from mananger.findDescriptorState is used directly
without checking nil-ness. It seems the original author assumed the
that return will never be nil. It will be safer to check nil-ness.

Release note: None

Co-authored-by: Thomas Hardy <[email protected]>
Co-authored-by: Santamaura <[email protected]>
Co-authored-by: Alex Sarkesian <[email protected]>
Co-authored-by: Xiang Gu <[email protected]>
  • Loading branch information
5 people committed Mar 28, 2022
5 parents e5793a0 + 3d6ebc5 + 2ded655 + 75bca29 + f1c5335 commit d5afa87
Show file tree
Hide file tree
Showing 25 changed files with 734 additions and 50 deletions.
4 changes: 4 additions & 0 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -4672,6 +4672,7 @@ Response object returned by TableIndexStatsResponse.
| ----- | ---- | ----- | ----------- | -------------- |
| statistics | [TableIndexStatsResponse.ExtendedCollectedIndexUsageStatistics](#cockroach.server.serverpb.TableIndexStatsResponse-cockroach.server.serverpb.TableIndexStatsResponse.ExtendedCollectedIndexUsageStatistics) | repeated | | [reserved](#support-status) |
| last_reset | [google.protobuf.Timestamp](#cockroach.server.serverpb.TableIndexStatsResponse-google.protobuf.Timestamp) | | Timestamp of the last index usage stats reset. | [reserved](#support-status) |
| index_recommendations | [cockroach.sql.IndexRecommendation](#cockroach.server.serverpb.TableIndexStatsResponse-cockroach.sql.IndexRecommendation) | repeated | | [reserved](#support-status) |



Expand Down Expand Up @@ -5096,6 +5097,7 @@ zone configuration, and size statistics for a database.
| range_count | [int64](#cockroach.server.serverpb.DatabaseDetailsResponse-int64) | | The number of ranges, as determined from a query of range meta keys, across all tables. | [reserved](#support-status) |
| approximate_disk_bytes | [uint64](#cockroach.server.serverpb.DatabaseDetailsResponse-uint64) | | An approximation of the disk space (in bytes) used for all replicas of all tables across the cluster. | [reserved](#support-status) |
| node_ids | [int32](#cockroach.server.serverpb.DatabaseDetailsResponse-int32) | repeated | node_ids is the ordered list of node ids on which data is stored. | [reserved](#support-status) |
| num_index_recommendations | [int32](#cockroach.server.serverpb.DatabaseDetailsResponse-int32) | | | [reserved](#support-status) |



Expand Down Expand Up @@ -5165,6 +5167,7 @@ a table.
| descriptor_id | [int64](#cockroach.server.serverpb.TableDetailsResponse-int64) | | descriptor_id is an identifier used to uniquely identify this table. It can be used to find events pertaining to this table by filtering on the 'target_id' field of events. | [reserved](#support-status) |
| configure_zone_statement | [string](#cockroach.server.serverpb.TableDetailsResponse-string) | | configure_zone_statement is the output of "SHOW ZONE CONFIGURATION FOR TABLE" for this table. It is a SQL statement that would re-configure the table's current zone if executed. | [reserved](#support-status) |
| stats_last_created_at | [google.protobuf.Timestamp](#cockroach.server.serverpb.TableDetailsResponse-google.protobuf.Timestamp) | | stats_last_created_at is the time at which statistics were last created. | [reserved](#support-status) |
| has_index_recommendations | [bool](#cockroach.server.serverpb.TableDetailsResponse-bool) | | has_index_recommendations notifies if the there are index recommendations on this table. | [reserved](#support-status) |



Expand Down Expand Up @@ -5724,6 +5727,7 @@ SettingsResponse is the response to SettingsRequest.
| type | [string](#cockroach.server.serverpb.SettingsResponse-string) | | | [reserved](#support-status) |
| description | [string](#cockroach.server.serverpb.SettingsResponse-string) | | | [reserved](#support-status) |
| public | [bool](#cockroach.server.serverpb.SettingsResponse-bool) | | | [reserved](#support-status) |
| last_updated | [google.protobuf.Timestamp](#cockroach.server.serverpb.SettingsResponse-google.protobuf.Timestamp) | | | [reserved](#support-status) |



Expand Down
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -193,4 +193,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using
trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.
trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https://<ui>/#/debug/tracez
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.
version version 21.2-96 set the active cluster version in the format '<major>.<minor>'
version version 21.2-98 set the active cluster version in the format '<major>.<minor>'
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -208,6 +208,6 @@
<tr><td><code>trace.opentelemetry.collector</code></td><td>string</td><td><code></code></td><td>address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.</td></tr>
<tr><td><code>trace.span_registry.enabled</code></td><td>boolean</td><td><code>true</code></td><td>if set, ongoing traces can be seen at https://<ui>/#/debug/tracez</td></tr>
<tr><td><code>trace.zipkin.collector</code></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.2-96</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.2-98</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
5 changes: 5 additions & 0 deletions docs/generated/swagger/spec.json
Original file line number Diff line number Diff line change
Expand Up @@ -1276,6 +1276,11 @@
},
"x-go-name": "Grants"
},
"has_index_recommendations": {
"description": "has_index_recommendations notifies if the there are index recommendations\non this table.",
"type": "boolean",
"x-go-name": "HasIndexRecommendations"
},
"indexes": {
"type": "array",
"items": {
Expand Down
1 change: 1 addition & 0 deletions pkg/base/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,4 +49,5 @@ type TestingKnobs struct {
ProtectedTS ModuleTestingKnobs
CapturedIndexUsageStatsKnobs ModuleTestingKnobs
AdmissionControl ModuleTestingKnobs
UnusedIndexRecommendKnobs ModuleTestingKnobs
}
7 changes: 7 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -324,6 +324,9 @@ const (
// EnableNewStoreRebalancer enables the new store rebalancer introduced in
// 22.1.
EnableNewStoreRebalancer
// ClusterLocksVirtualTable enables querying the crdb_internal.cluster_locks
// virtual table, which sends a QueryLocksRequest RPC to all cluster ranges.
ClusterLocksVirtualTable

// *************************************************
// Step (1): Add new versions here.
Expand Down Expand Up @@ -543,6 +546,10 @@ var versionsSingleton = keyedVersions{
Key: EnableNewStoreRebalancer,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 96},
},
{
Key: ClusterLocksVirtualTable,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 98},
},

// *************************************************
// Step (2): Add new versions here.
Expand Down
5 changes: 3 additions & 2 deletions pkg/clusterversion/key_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 2 additions & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ go_library(
srcs = [
"addjoin.go",
"admin.go",
"admin_test_utils.go",
"api_v2.go",
"api_v2_auth.go",
"api_v2_error.go",
Expand Down Expand Up @@ -388,6 +389,7 @@ go_test(
"//pkg/sql/catalog/dbdesc",
"//pkg/sql/catalog/descpb",
"//pkg/sql/execinfrapb",
"//pkg/sql/idxusage",
"//pkg/sql/roleoption",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
Expand Down
64 changes: 63 additions & 1 deletion pkg/server/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -510,6 +510,10 @@ func (s *adminServer) databaseDetailsHelper(
if err != nil {
return nil, err
}
resp.Stats.NumIndexRecommendations, err = s.getNumDatabaseIndexRecommendations(ctx, req.Database, resp.TableNames)
if err != nil {
return nil, err
}
}

return &resp, nil
Expand Down Expand Up @@ -607,6 +611,25 @@ func (s *adminServer) getDatabaseStats(
return &stats, nil
}

func (s *adminServer) getNumDatabaseIndexRecommendations(
ctx context.Context, databaseName string, tableNames []string,
) (int32, error) {
var numDatabaseIndexRecommendations int
idxUsageStatsProvider := s.server.sqlServer.pgServer.SQLServer.GetLocalIndexStatistics()
for _, tableName := range tableNames {
tableIndexStatsRequest := &serverpb.TableIndexStatsRequest{
Database: databaseName,
Table: tableName,
}
tableIndexStatsResponse, err := getTableIndexUsageStats(ctx, tableIndexStatsRequest, idxUsageStatsProvider, s.ie, s.server.st, s.server.sqlServer.execCfg)
if err != nil {
return 0, err
}
numDatabaseIndexRecommendations += len(tableIndexStatsResponse.IndexRecommendations)
}
return int32(numDatabaseIndexRecommendations), nil
}

// getFullyQualifiedTableName, given a database name and a tableName that either
// is a unqualified name or a schema-qualified name, returns a maximally
// qualified name: either database.table if the input wasn't schema qualified,
Expand Down Expand Up @@ -960,6 +983,16 @@ func (s *adminServer) tableDetailsHelper(
resp.RangeCount = rangeCount
}

idxUsageStatsProvider := s.server.sqlServer.pgServer.SQLServer.GetLocalIndexStatistics()
tableIndexStatsRequest := &serverpb.TableIndexStatsRequest{
Database: req.Database,
Table: req.Table,
}
tableIndexStatsResponse, err := getTableIndexUsageStats(ctx, tableIndexStatsRequest, idxUsageStatsProvider, s.ie, s.server.st, s.server.sqlServer.execCfg)
if err != nil {
return nil, err
}
resp.HasIndexRecommendations = len(tableIndexStatsResponse.IndexRecommendations) > 0
return &resp, nil
}

Expand Down Expand Up @@ -1750,21 +1783,50 @@ func (s *adminServer) Settings(
}
}

// Read the system.settings table to determine the settings for which we have
// explicitly set values -- the in-memory SV has the set and default values
// flattened for quick reads, but we'd only need the non-defaults for comparison.
alteredSettings := make(map[string]*time.Time)
if it, err := s.server.sqlServer.internalExecutor.QueryIteratorEx(
ctx, "read-setting", nil, /* txn */
sessiondata.InternalExecutorOverride{User: security.RootUserName()},
`SELECT name, "lastUpdated" FROM system.settings`,
); err != nil {
log.Warningf(ctx, "failed to read settings: %s", err)
} else {
var ok bool
for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) {
row := it.Cur()
name := string(tree.MustBeDString(row[0]))
lastUpdated := row[1].(*tree.DTimestamp)
alteredSettings[name] = &lastUpdated.Time
}
if err != nil {
// No need to clear AlteredSettings map since we only make best
// effort to populate it.
log.Warningf(ctx, "failed to read settings: %s", err)
}
}

resp := serverpb.SettingsResponse{KeyValues: make(map[string]serverpb.SettingsResponse_Value)}
for _, k := range keys {
v, ok := settings.Lookup(k, lookupPurpose, settings.ForSystemTenant)
if !ok {
continue
}
var altered *time.Time
if val, ok := alteredSettings[k]; ok {
altered = val
}
resp.KeyValues[k] = serverpb.SettingsResponse_Value{
Type: v.Typ(),
// Note: v.String() redacts the values if the purpose is not "LocalAccess".
Value: v.String(&s.server.st.SV),
Description: v.Description(),
Public: v.Visibility() == settings.Public,
LastUpdated: altered,
}
}

return &resp, nil
}

Expand Down
Loading

0 comments on commit d5afa87

Please sign in to comment.