diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index 12305be2aa1a..92814c5a0327 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -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) | @@ -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) | @@ -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) | @@ -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) | diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 64eaf86df224..a45f0470272d 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -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 :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 21.2-96 set the active cluster version in the format '.' +version version 21.2-98 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 63f03b4f1514..e8b482753e60 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -208,6 +208,6 @@ trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabledbooleantrueif set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion21.2-96set the active cluster version in the format '.' +versionversion21.2-98set the active cluster version in the format '.' diff --git a/docs/generated/swagger/spec.json b/docs/generated/swagger/spec.json index 6e5daf8aa9ea..4fe16b70a3c9 100644 --- a/docs/generated/swagger/spec.json +++ b/docs/generated/swagger/spec.json @@ -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": { diff --git a/pkg/base/testing_knobs.go b/pkg/base/testing_knobs.go index 15abf6afe8d2..22d37ea9b34e 100644 --- a/pkg/base/testing_knobs.go +++ b/pkg/base/testing_knobs.go @@ -49,4 +49,5 @@ type TestingKnobs struct { ProtectedTS ModuleTestingKnobs CapturedIndexUsageStatsKnobs ModuleTestingKnobs AdmissionControl ModuleTestingKnobs + UnusedIndexRecommendKnobs ModuleTestingKnobs } diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 2da3bc3a4c24..1c0dcb09e2c7 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -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. @@ -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. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 9da0cbfc19de..607a7f06cd67 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -54,11 +54,12 @@ func _() { _ = x[IncrementalBackupSubdir-43] _ = x[DateStyleIntervalStyleCastRewrite-44] _ = x[EnableNewStoreRebalancer-45] + _ = x[ClusterLocksVirtualTable-46] } -const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancer" +const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTable" -var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 175, 189, 230, 256, 275, 309, 321, 352, 376, 397, 425, 455, 483, 504, 517, 536, 570, 608, 642, 674, 710, 742, 778, 820, 839, 879, 911, 930, 954, 975, 1006, 1024, 1065, 1095, 1106, 1137, 1160, 1193, 1217} +var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 175, 189, 230, 256, 275, 309, 321, 352, 376, 397, 425, 455, 483, 504, 517, 536, 570, 608, 642, 674, 710, 742, 778, 820, 839, 879, 911, 930, 954, 975, 1006, 1024, 1065, 1095, 1106, 1137, 1160, 1193, 1217, 1241} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 8971584cac62..47dd9900e594 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -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", @@ -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", diff --git a/pkg/server/admin.go b/pkg/server/admin.go index d4d153c7cbd5..946d7812c27b 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -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 @@ -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, @@ -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 } @@ -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 } diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index 03901dacac1a..f38e9cbb213a 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -46,6 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/idxusage" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -1087,7 +1088,7 @@ func TestAdminAPISettings(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - s, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + s, conn, _ := serverutils.StartServer(t, base.TestServerArgs{}) defer s.Stopper().Stop(context.Background()) // Any bool that defaults to true will work here. @@ -1122,6 +1123,20 @@ func TestAdminAPISettings(t *testing.T) { if typ != v.Type { t.Errorf("%s: expected type %s, got %s", k, typ, v.Type) } + if v.LastUpdated != nil { + db := sqlutils.MakeSQLRunner(conn) + q := makeSQLQuery() + q.Append(`SELECT name, "lastUpdated" FROM system.settings WHERE name=$`, k) + rows := db.Query( + t, + q.String(), + q.QueryArguments()..., + ) + defer rows.Close() + if rows.Next() == false { + t.Errorf("missing sql row for %s", k) + } + } } t.Run("all", func(t *testing.T) { @@ -2602,3 +2617,155 @@ func TestAdminPrivilegeChecker(t *testing.T) { } } } + +func TestDatabaseAndTableIndexRecommendations(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + stubTime := stubUnusedIndexTime{} + stubDropUnusedDuration := time.Hour + + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + UnusedIndexRecommendKnobs: &idxusage.UnusedIndexRecommendationTestingKnobs{ + GetCreatedAt: stubTime.getCreatedAt, + GetLastRead: stubTime.getLastRead, + GetCurrentTime: stubTime.getCurrent, + }, + }, + }) + idxusage.DropUnusedIndexDuration.Override(context.Background(), &s.ClusterSettings().SV, stubDropUnusedDuration) + defer s.Stopper().Stop(context.Background()) + + db := sqlutils.MakeSQLRunner(sqlDB) + db.Exec(t, "CREATE DATABASE test") + db.Exec(t, "USE test") + // Create a table, the statistics on its primary index will be fetched. + db.Exec(t, "CREATE TABLE test.test_table (num INT PRIMARY KEY, letter char)") + + // Test when last read does not exist and there is no creation time. Expect + // an index recommendation (index never used). + stubTime.setLastRead(time.Time{}) + stubTime.setCreatedAt(nil) + + // Test database details endpoint. + var dbDetails serverpb.DatabaseDetailsResponse + if err := getAdminJSONProto( + s, + "databases/test?include_stats=true", + &dbDetails, + ); err != nil { + t.Fatal(err) + } + require.Equal(t, int32(1), dbDetails.Stats.NumIndexRecommendations) + + // Test table details endpoint. + var tableDetails serverpb.TableDetailsResponse + if err := getAdminJSONProto(s, "databases/test/tables/test_table", &tableDetails); err != nil { + t.Fatal(err) + } + require.Equal(t, true, tableDetails.HasIndexRecommendations) + + // Test when last read does not exist and there is a creation time, and the + // unused index duration has been exceeded. Expect an index recommendation. + currentTime := timeutil.Now() + createdTime := currentTime.Add(-stubDropUnusedDuration) + stubTime.setCurrent(currentTime) + stubTime.setLastRead(time.Time{}) + stubTime.setCreatedAt(&createdTime) + + // Test database details endpoint. + dbDetails = serverpb.DatabaseDetailsResponse{} + if err := getAdminJSONProto( + s, + "databases/test?include_stats=true", + &dbDetails, + ); err != nil { + t.Fatal(err) + } + require.Equal(t, int32(1), dbDetails.Stats.NumIndexRecommendations) + + // Test table details endpoint. + tableDetails = serverpb.TableDetailsResponse{} + if err := getAdminJSONProto(s, "databases/test/tables/test_table", &tableDetails); err != nil { + t.Fatal(err) + } + require.Equal(t, true, tableDetails.HasIndexRecommendations) + + // Test when last read does not exist and there is a creation time, and the + // unused index duration has not been exceeded. Expect no index + // recommendation. + currentTime = timeutil.Now() + stubTime.setCurrent(currentTime) + stubTime.setLastRead(time.Time{}) + stubTime.setCreatedAt(¤tTime) + + // Test database details endpoint. + dbDetails = serverpb.DatabaseDetailsResponse{} + if err := getAdminJSONProto( + s, + "databases/test?include_stats=true", + &dbDetails, + ); err != nil { + t.Fatal(err) + } + require.Equal(t, int32(0), dbDetails.Stats.NumIndexRecommendations) + + // Test table details endpoint. + tableDetails = serverpb.TableDetailsResponse{} + if err := getAdminJSONProto(s, "databases/test/tables/test_table", &tableDetails); err != nil { + t.Fatal(err) + } + require.Equal(t, false, tableDetails.HasIndexRecommendations) + + // Test when last read exists and the unused index duration has been + // exceeded. Expect an index recommendation. + currentTime = timeutil.Now() + lastRead := currentTime.Add(-stubDropUnusedDuration) + stubTime.setCurrent(currentTime) + stubTime.setLastRead(lastRead) + stubTime.setCreatedAt(nil) + + // Test database details endpoint. + dbDetails = serverpb.DatabaseDetailsResponse{} + if err := getAdminJSONProto( + s, + "databases/test?include_stats=true", + &dbDetails, + ); err != nil { + t.Fatal(err) + } + require.Equal(t, int32(1), dbDetails.Stats.NumIndexRecommendations) + + // Test table details endpoint. + tableDetails = serverpb.TableDetailsResponse{} + if err := getAdminJSONProto(s, "databases/test/tables/test_table", &tableDetails); err != nil { + t.Fatal(err) + } + require.Equal(t, true, tableDetails.HasIndexRecommendations) + + // Test when last read exists and the unused index duration has not been + // exceeded. Expect no index recommendation. + currentTime = timeutil.Now() + stubTime.setCurrent(currentTime) + stubTime.setLastRead(currentTime) + stubTime.setCreatedAt(nil) + + // Test database details endpoint. + dbDetails = serverpb.DatabaseDetailsResponse{} + if err := getAdminJSONProto( + s, + "databases/test?include_stats=true", + &dbDetails, + ); err != nil { + t.Fatal(err) + } + require.Equal(t, int32(0), dbDetails.Stats.NumIndexRecommendations) + + // Test table details endpoint. + tableDetails = serverpb.TableDetailsResponse{} + if err := getAdminJSONProto(s, "databases/test/tables/test_table", &tableDetails); err != nil { + t.Fatal(err) + } + require.Equal(t, false, tableDetails.HasIndexRecommendations) +} diff --git a/pkg/server/admin_test_utils.go b/pkg/server/admin_test_utils.go new file mode 100644 index 000000000000..99a17af78f49 --- /dev/null +++ b/pkg/server/admin_test_utils.go @@ -0,0 +1,60 @@ +// 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 server + +import ( + "time" + + "github.com/cockroachdb/cockroach/pkg/util/syncutil" +) + +type stubUnusedIndexTime struct { + syncutil.RWMutex + current time.Time + lastRead time.Time + createdAt *time.Time +} + +func (s *stubUnusedIndexTime) setCurrent(t time.Time) { + s.RWMutex.Lock() + defer s.RWMutex.Unlock() + s.current = t +} + +func (s *stubUnusedIndexTime) setLastRead(t time.Time) { + s.RWMutex.Lock() + defer s.RWMutex.Unlock() + s.lastRead = t +} + +func (s *stubUnusedIndexTime) setCreatedAt(t *time.Time) { + s.RWMutex.Lock() + defer s.RWMutex.Unlock() + s.createdAt = t +} + +func (s *stubUnusedIndexTime) getCurrent() time.Time { + s.RWMutex.RLock() + defer s.RWMutex.RUnlock() + return s.current +} + +func (s *stubUnusedIndexTime) getLastRead() time.Time { + s.RWMutex.RLock() + defer s.RWMutex.RUnlock() + return s.lastRead +} + +func (s *stubUnusedIndexTime) getCreatedAt() *time.Time { + s.RWMutex.RLock() + defer s.RWMutex.RUnlock() + return s.createdAt +} diff --git a/pkg/server/index_usage_stats.go b/pkg/server/index_usage_stats.go index 14698fb51849..f647e3e04691 100644 --- a/pkg/server/index_usage_stats.go +++ b/pkg/server/index_usage_stats.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/idxusage" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -197,7 +198,7 @@ func (s *statusServer) TableIndexStats( return nil, err } return getTableIndexUsageStats(ctx, req, s.sqlServer.pgServer.SQLServer.GetLocalIndexStatistics(), - s.sqlServer.internalExecutor) + s.sqlServer.internalExecutor, s.st, s.sqlServer.execCfg) } // getTableIndexUsageStats is a helper function that reads the indexes @@ -208,6 +209,8 @@ func getTableIndexUsageStats( req *serverpb.TableIndexStatsRequest, idxUsageStatsProvider *idxusage.LocalIndexUsageStats, ie *sql.InternalExecutor, + st *cluster.Settings, + execConfig *sql.ExecutorConfig, ) (*serverpb.TableIndexStatsResponse, error) { userName, err := userFromContext(ctx) if err != nil { @@ -254,6 +257,7 @@ func getTableIndexUsageStats( } var idxUsageStats []*serverpb.TableIndexStatsResponse_ExtendedCollectedIndexUsageStatistics + var idxRecommendations []*serverpb.IndexRecommendation var ok bool // We have to make sure to close the iterator since we might return from the @@ -306,14 +310,21 @@ func getTableIndexUsageStats( CreatedAt: createdAt, } + statsRow := idxusage.IndexStatsRow{ + Row: idxStatsRow, + UnusedIndexKnobs: execConfig.UnusedIndexRecommendationsKnobs, + } + recommendations := statsRow.GetRecommendationsFromIndexStats(st) + idxRecommendations = append(idxRecommendations, recommendations...) idxUsageStats = append(idxUsageStats, idxStatsRow) } lastReset := idxUsageStatsProvider.GetLastReset() resp := &serverpb.TableIndexStatsResponse{ - Statistics: idxUsageStats, - LastReset: &lastReset, + Statistics: idxUsageStats, + LastReset: &lastReset, + IndexRecommendations: idxRecommendations, } return resp, nil @@ -335,7 +346,10 @@ func getTableIDFromDatabaseAndTableName( return 0, err } names := strings.Split(fqtName, ".") - + // Strip quotations marks from db and table names. + for idx := range names { + names[idx] = strings.Trim(names[idx], "\"") + } q := makeSQLQuery() q.Append(`SELECT table_id FROM crdb_internal.tables WHERE database_name = $ `, names[0]) diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 2eb01bfe0ef7..48d91ac62488 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -69,6 +69,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" "github.com/cockroachdb/cockroach/pkg/sql/gcjob/gcjobnotifier" + "github.com/cockroachdb/cockroach/pkg/sql/idxusage" "github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness" "github.com/cockroachdb/cockroach/pkg/sql/pgwire" "github.com/cockroachdb/cockroach/pkg/sql/querycache" @@ -808,6 +809,10 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { execCfg.CaptureIndexUsageStatsKnobs = capturedIndexUsageStatsKnobs.(*scheduledlogging.CaptureIndexUsageStatsTestingKnobs) } + if unusedIndexRecommendationsKnobs := cfg.TestingKnobs.UnusedIndexRecommendKnobs; unusedIndexRecommendationsKnobs != nil { + execCfg.UnusedIndexRecommendationsKnobs = unusedIndexRecommendationsKnobs.(*idxusage.UnusedIndexRecommendationTestingKnobs) + } + statsRefresher := stats.MakeRefresher( cfg.AmbientCtx, cfg.Settings, diff --git a/pkg/server/serverpb/BUILD.bazel b/pkg/server/serverpb/BUILD.bazel index 38a660c79f50..d5373c78f6a0 100644 --- a/pkg/server/serverpb/BUILD.bazel +++ b/pkg/server/serverpb/BUILD.bazel @@ -9,6 +9,7 @@ proto_library( srcs = [ "admin.proto", "authentication.proto", + "index_recommendations.proto", "init.proto", "migration.proto", "status.proto", diff --git a/pkg/server/serverpb/admin.proto b/pkg/server/serverpb/admin.proto index 197c942b58bc..50a48a9f543d 100644 --- a/pkg/server/serverpb/admin.proto +++ b/pkg/server/serverpb/admin.proto @@ -93,6 +93,8 @@ message DatabaseDetailsResponse { // node_ids is the ordered list of node ids on which data is stored. repeated int32 node_ids = 4 [(gogoproto.customname) = "NodeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; + + int32 num_index_recommendations = 5; } // grants are the results of SHOW GRANTS for this database. @@ -213,6 +215,9 @@ message TableDetailsResponse { string configure_zone_statement = 9; // stats_last_created_at is the time at which statistics were last created. google.protobuf.Timestamp stats_last_created_at = 10 [(gogoproto.stdtime) = true]; + // has_index_recommendations notifies if the there are index recommendations + // on this table. + bool has_index_recommendations = 11; } // TableStatsRequest is a request for detailed, computationally expensive @@ -539,6 +544,7 @@ message SettingsResponse { string type = 2; string description = 3; bool public = 4; + google.protobuf.Timestamp last_updated = 5 [(gogoproto.nullable) = true, (gogoproto.stdtime) = true]; } map key_values = 1 [(gogoproto.nullable) = false]; } diff --git a/pkg/server/serverpb/index_recommendations.proto b/pkg/server/serverpb/index_recommendations.proto new file mode 100644 index 000000000000..de786776d7cd --- /dev/null +++ b/pkg/server/serverpb/index_recommendations.proto @@ -0,0 +1,33 @@ +// 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. + +syntax = "proto2"; +package cockroach.sql; +option go_package = "serverpb"; + +import "gogoproto/gogo.proto"; + +message IndexRecommendation { + enum RecommendationType { + DROP_UNUSED = 0; + } + // TableID is the ID of the table this index is created on. This is same as + // descpb.TableID and is unique within the cluster. + optional uint32 table_id = 1 [(gogoproto.nullable) = false, + (gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TableID"]; + // IndexID is the ID of the index within the scope of the given table. + optional uint32 index_id = 2 [(gogoproto.nullable) = false, + (gogoproto.customname) = "IndexID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.IndexID" ]; + + // Type of recommendation for the index. + optional RecommendationType type = 3 [(gogoproto.nullable) = false]; + // Reason for our recommendation type. + optional string reason = 4 [(gogoproto.nullable) = false]; +} diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index 39cf95b28260..836b5ae6843b 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -20,8 +20,7 @@ import "roachpb/data.proto"; import "roachpb/index_usage_stats.proto"; import "roachpb/metadata.proto"; import "server/diagnostics/diagnosticspb/diagnostics.proto"; -import "util/tracing/tracingpb/recorded_span.proto"; -import "util/tracing/tracingpb/tracing.proto"; +import "server/serverpb/index_recommendations.proto"; import "server/status/statuspb/status.proto"; import "sql/contentionpb/contention.proto"; import "sql/execinfrapb/api.proto"; @@ -34,6 +33,8 @@ import "kv/kvserver/liveness/livenesspb/liveness.proto"; import "util/log/logpb/log.proto"; import "util/unresolved_addr.proto"; import "util/hlc/timestamp.proto"; +import "util/tracing/tracingpb/recorded_span.proto"; +import "util/tracing/tracingpb/tracing.proto"; import "etcd/raft/v3/raftpb/raft.proto"; @@ -1682,6 +1683,7 @@ message TableIndexStatsResponse { repeated ExtendedCollectedIndexUsageStatistics statistics = 1; // Timestamp of the last index usage stats reset. google.protobuf.Timestamp last_reset = 2 [(gogoproto.stdtime) = true]; + repeated cockroach.sql.IndexRecommendation index_recommendations = 3; } // Request object for issuing a index usage stats reset request. diff --git a/pkg/server/tenant_status.go b/pkg/server/tenant_status.go index 4a8a2764cbef..e5d4e2093282 100644 --- a/pkg/server/tenant_status.go +++ b/pkg/server/tenant_status.go @@ -982,7 +982,7 @@ func (t *tenantStatusServer) TableIndexStats( } return getTableIndexUsageStats(ctx, req, t.sqlServer.pgServer.SQLServer.GetLocalIndexStatistics(), - t.sqlServer.internalExecutor) + t.sqlServer.internalExecutor, t.st, t.sqlServer.execCfg) } // Details returns information for a given instance ID such as diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index a95021818b11..eaa8df15401f 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -722,6 +722,9 @@ func NameMatchesDescriptor( // findNewest returns the newest descriptor version state for the ID. func (m *Manager) findNewest(id descpb.ID) *descriptorVersionState { t := m.findDescriptorState(id, false /* create */) + if t == nil { + return nil + } t.mu.Lock() defer t.mu.Unlock() return t.mu.active.findNewest() diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 4aade071c22f..f6fad24c48eb 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -64,6 +64,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/gcjob/gcjobnotifier" + "github.com/cockroachdb/cockroach/pkg/sql/idxusage" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness" @@ -1201,6 +1202,8 @@ type ExecutorConfig struct { TelemetryLoggingTestingKnobs *TelemetryLoggingTestingKnobs SpanConfigTestingKnobs *spanconfig.TestingKnobs CaptureIndexUsageStatsKnobs *scheduledlogging.CaptureIndexUsageStatsTestingKnobs + UnusedIndexRecommendationsKnobs *idxusage.UnusedIndexRecommendationTestingKnobs + // HistogramWindowInterval is (server.Config).HistogramWindowInterval. HistogramWindowInterval time.Duration diff --git a/pkg/sql/idxusage/BUILD.bazel b/pkg/sql/idxusage/BUILD.bazel index 117c5a068b3c..9f46a8f6de65 100644 --- a/pkg/sql/idxusage/BUILD.bazel +++ b/pkg/sql/idxusage/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "cluster_settings.go", "index_usage_stats_controller.go", + "index_usage_stats_rec.go", "local_idx_usage_stats.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/idxusage", @@ -22,10 +23,14 @@ go_library( go_test( name = "idxusage_test", - srcs = ["local_index_usage_stats_test.go"], + srcs = [ + "index_usage_stats_rec_test.go", + "local_index_usage_stats_test.go", + ], embed = [":idxusage"], deps = [ "//pkg/roachpb", + "//pkg/server/serverpb", "//pkg/settings/cluster", "//pkg/util/leaktest", "//pkg/util/log", diff --git a/pkg/sql/idxusage/index_usage_stats_rec.go b/pkg/sql/idxusage/index_usage_stats_rec.go new file mode 100644 index 000000000000..19e1acac7b78 --- /dev/null +++ b/pkg/sql/idxusage/index_usage_stats_rec.go @@ -0,0 +1,129 @@ +// 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 idxusage + +import ( + "fmt" + "time" + + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" +) + +// IndexStatsRow is a wrapper type around +// serverpb.TableIndexStatsResponse_ExtendedCollectedIndexUsageStatistics that +// implements additional methods to support unused index recommendations and +// hold testing knobs. +type IndexStatsRow struct { + Row *serverpb.TableIndexStatsResponse_ExtendedCollectedIndexUsageStatistics + UnusedIndexKnobs *UnusedIndexRecommendationTestingKnobs +} + +// defaultUnusedIndexDuration is a week. +const defaultUnusedIndexDuration = 7 * 24 * time.Hour + +// DropUnusedIndexDuration registers the index unuse duration at which we +// begin to recommend dropping the index. +var DropUnusedIndexDuration = settings.RegisterDurationSetting( + settings.TenantWritable, + "sql.index_recommendation.drop_unused_duration", + "the index unuse duration at which we begin to recommend dropping the index", + defaultUnusedIndexDuration, + settings.NonNegativeDuration, +) + +const indexExceedUsageDurationReasonPlaceholder = "This index has not been used in over %s and can be removed for better write performance." +const indexNeverUsedReason = "This index has not been used and can be removed for better write performance." + +// UnusedIndexRecommendationTestingKnobs provides hooks and knobs for unit tests. +type UnusedIndexRecommendationTestingKnobs struct { + // GetCreatedAt allows tests to override the creation time of the index. + GetCreatedAt func() *time.Time + // GetLastRead allows tests to override the time the index was last read. + GetLastRead func() time.Time + // GetCurrentTime allows tests to override the current time. + GetCurrentTime func() time.Time +} + +// ModuleTestingKnobs implements base.ModuleTestingKnobs interface. +func (*UnusedIndexRecommendationTestingKnobs) ModuleTestingKnobs() {} + +// GetRecommendationsFromIndexStats gets index recommendations from the given index +// if applicable. +func (i IndexStatsRow) GetRecommendationsFromIndexStats( + st *cluster.Settings, +) []*serverpb.IndexRecommendation { + var recommendations []*serverpb.IndexRecommendation + rec := i.maybeAddUnusedIndexRecommendation(DropUnusedIndexDuration.Get(&st.SV)) + if rec != nil { + recommendations = append(recommendations, rec) + } + return recommendations +} + +func (i IndexStatsRow) maybeAddUnusedIndexRecommendation( + unusedIndexDuration time.Duration, +) *serverpb.IndexRecommendation { + var rec *serverpb.IndexRecommendation + + if i.UnusedIndexKnobs == nil { + rec = i.recommendDropUnusedIndex(timeutil.Now(), i.Row.CreatedAt, + i.Row.Statistics.Stats.LastRead, unusedIndexDuration) + } else { + rec = i.recommendDropUnusedIndex(i.UnusedIndexKnobs.GetCurrentTime(), + i.UnusedIndexKnobs.GetCreatedAt(), i.UnusedIndexKnobs.GetLastRead(), unusedIndexDuration) + } + return rec +} + +// recommendDropUnusedIndex checks whether the last usage of an index +// qualifies the index as unused, if so returns an index recommendation. +func (i IndexStatsRow) recommendDropUnusedIndex( + currentTime time.Time, + createdAt *time.Time, + lastRead time.Time, + unusedIndexDuration time.Duration, +) *serverpb.IndexRecommendation { + lastActive := lastRead + if lastActive.Equal(time.Time{}) && createdAt != nil { + lastActive = *createdAt + } + // If we do not have the creation time and index has never been read. Recommend + // dropping with a "never used" reason. + if lastActive.Equal(time.Time{}) { + return &serverpb.IndexRecommendation{ + TableID: i.Row.Statistics.Key.TableID, + IndexID: i.Row.Statistics.Key.IndexID, + Type: serverpb.IndexRecommendation_DROP_UNUSED, + Reason: indexNeverUsedReason, + } + } + // Last usage of the index exceeds the unused index duration. + if currentTime.Sub(lastActive) >= unusedIndexDuration { + return &serverpb.IndexRecommendation{ + TableID: i.Row.Statistics.Key.TableID, + IndexID: i.Row.Statistics.Key.IndexID, + Type: serverpb.IndexRecommendation_DROP_UNUSED, + Reason: fmt.Sprintf(indexExceedUsageDurationReasonPlaceholder, formatDuration(unusedIndexDuration)), + } + } + return nil +} + +func formatDuration(d time.Duration) string { + days := d / (24 * time.Hour) + hours := d % (24 * time.Hour) + minutes := hours % time.Hour + + return fmt.Sprintf("%dd%dh%dm", days, hours/time.Hour, minutes) +} diff --git a/pkg/sql/idxusage/index_usage_stats_rec_test.go b/pkg/sql/idxusage/index_usage_stats_rec_test.go new file mode 100644 index 000000000000..942d676ae4b5 --- /dev/null +++ b/pkg/sql/idxusage/index_usage_stats_rec_test.go @@ -0,0 +1,120 @@ +// 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 idxusage + +import ( + "fmt" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/stretchr/testify/require" +) + +func TestRecommendDropUnusedIndex(t *testing.T) { + + var currentTime = timeutil.Now() + var anHourBefore = currentTime.Add(-time.Hour) + var aMinuteBefore = currentTime.Add(-time.Minute) + + type expectedReturn struct { + recommendation *serverpb.IndexRecommendation + } + + stubIndexStatsRow := &serverpb.TableIndexStatsResponse_ExtendedCollectedIndexUsageStatistics{ + Statistics: &roachpb.CollectedIndexUsageStatistics{ + Key: roachpb.IndexUsageKey{ + TableID: 1, + IndexID: 1, + }, + }, + } + + indexStatsRow := IndexStatsRow{ + Row: stubIndexStatsRow, + } + + testData := []struct { + currentTime time.Time + createdAt *time.Time + lastRead time.Time + unusedIndexDuration time.Duration + expectedReturn expectedReturn + }{ + { + currentTime: currentTime, + createdAt: nil, + lastRead: anHourBefore, + unusedIndexDuration: time.Hour, + expectedReturn: expectedReturn{ + recommendation: &serverpb.IndexRecommendation{ + TableID: 1, + IndexID: 1, + Type: serverpb.IndexRecommendation_DROP_UNUSED, + Reason: fmt.Sprintf(indexExceedUsageDurationReasonPlaceholder, formatDuration(time.Hour)), + }, + }, + }, + { + currentTime: currentTime, + createdAt: nil, + lastRead: aMinuteBefore, + unusedIndexDuration: time.Hour, + expectedReturn: expectedReturn{ + recommendation: nil, + }, + }, + { + currentTime: currentTime, + createdAt: nil, + lastRead: time.Time{}, + unusedIndexDuration: time.Hour, + expectedReturn: expectedReturn{ + recommendation: &serverpb.IndexRecommendation{ + TableID: 1, + IndexID: 1, + Type: serverpb.IndexRecommendation_DROP_UNUSED, + Reason: indexNeverUsedReason, + }, + }, + }, + { + currentTime: currentTime, + createdAt: &anHourBefore, + lastRead: time.Time{}, + unusedIndexDuration: time.Hour, + expectedReturn: expectedReturn{ + recommendation: &serverpb.IndexRecommendation{ + TableID: 1, + IndexID: 1, + Type: serverpb.IndexRecommendation_DROP_UNUSED, + Reason: fmt.Sprintf(indexExceedUsageDurationReasonPlaceholder, formatDuration(time.Hour)), + }, + }, + }, + { + currentTime: currentTime, + createdAt: &aMinuteBefore, + lastRead: time.Time{}, + unusedIndexDuration: time.Hour, + expectedReturn: expectedReturn{ + recommendation: nil, + }, + }, + } + + for _, tc := range testData { + actualRecommendation := indexStatsRow.recommendDropUnusedIndex(tc.currentTime, tc.createdAt, tc.lastRead, tc.unusedIndexDuration) + require.Equal(t, tc.expectedReturn.recommendation, actualRecommendation) + } +} diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index a65b2dd63f9a..08c475880701 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -123,7 +123,7 @@ func (ef *execFactory) ConstructScan( scan.lockingWaitPolicy = descpb.ToScanLockingWaitPolicy(params.Locking.WaitPolicy) } scan.localityOptimized = params.LocalityOptimized - if !ef.isExplain { + if !ef.isExplain && !ef.planner.isInternalPlanner { idxUsageKey := roachpb.IndexUsageKey{ TableID: roachpb.TableID(tabDesc.GetID()), IndexID: roachpb.IndexID(idx.GetID()), diff --git a/pkg/ui/workspaces/db-console/src/views/reports/containers/settings/index.tsx b/pkg/ui/workspaces/db-console/src/views/reports/containers/settings/index.tsx index c6506c7bba4f..6ffdad42cb37 100644 --- a/pkg/ui/workspaces/db-console/src/views/reports/containers/settings/index.tsx +++ b/pkg/ui/workspaces/db-console/src/views/reports/containers/settings/index.tsx @@ -17,7 +17,14 @@ import { RouteComponentProps, withRouter } from "react-router-dom"; import * as protos from "src/js/protos"; import { refreshSettings } from "src/redux/apiReducers"; import { AdminUIState } from "src/redux/state"; -import { Loading } from "@cockroachlabs/cluster-ui"; +import { DATE_FORMAT_24_UTC } from "src/util/format"; +import { + Loading, + ColumnDescriptor, + SortedTable, + SortSetting, + util, +} from "@cockroachlabs/cluster-ui"; import "./index.styl"; import { CachedDataReducerState } from "src/redux/cachedDataReducer"; @@ -28,12 +35,37 @@ interface SettingsOwnProps { refreshSettings: typeof refreshSettings; } +interface IterableSetting { + key: string; + description?: string; + type?: string; + value?: string; + public?: boolean; + last_updated?: moment.Moment; +} + +interface SettingsState { + sortSetting: { + ascending: boolean; + columnTitle: string; + }; +} + type SettingsProps = SettingsOwnProps & RouteComponentProps; /** * Renders the Cluster Settings Report page. */ -export class Settings extends React.Component { +export class Settings extends React.Component { + constructor(props: SettingsProps) { + super(props); + this.state = { + sortSetting: { ascending: true, columnTitle: "lastUpdated" }, + }; + } + + sortSetting: { ascending: boolean; columnTitle: string | null }; + refresh(props = this.props) { props.refreshSettings( new protos.cockroach.server.serverpb.SettingsRequest(), @@ -51,41 +83,63 @@ export class Settings extends React.Component { } const { key_values } = this.props.settings.data; - const data: any = _.keys(key_values); + const dataArray: IterableSetting[] = Object.keys(key_values) + .map(key => ({ + key, + ...key_values[key], + })) + .map(obj => { + return { + ...obj, + last_updated: obj.last_updated + ? util.TimestampToMoment(obj.last_updated) + : null, + }; + }); + const columns: ColumnDescriptor[] = [ + { + name: "name", + title: "Setting", + cell: (setting: IterableSetting) => setting.key, + sort: (setting: IterableSetting) => setting.key, + }, + { + name: "value", + title: "Value", + cell: (setting: IterableSetting) => setting.value, + }, + { + name: "lastUpdated", + title: "Last Updated", + cell: (setting: IterableSetting) => + setting.last_updated + ? setting.last_updated.format(DATE_FORMAT_24_UTC) + : "No overrides", + sort: (setting: IterableSetting) => setting.last_updated?.valueOf(), + }, + { + name: "description", + title: "Description", + cell: (setting: IterableSetting) => setting.description, + }, + ]; return ( - - - - - - - - - - {_.chain(data) - .filter(key => key_values[key].public === wantPublic) - .sort() - .map((key: number) => ( - - - - - - )) - .value()} - -
- Setting - - Value - - Description -
{key} - {key_values[key].value} - - {key_values[key].description} -
+ + wantPublic ? obj.public : obj.public === undefined, + )} + columns={columns} + sortSetting={this.state.sortSetting} + onChangeSortSetting={(ss: SortSetting) => + this.setState({ + sortSetting: { + ascending: ss.ascending, + columnTitle: ss.columnTitle, + }, + }) + } + /> ); }