diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 44f04d5c88bf..6cd660653eb1 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -298,4 +298,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 1000022.1-72 set the active cluster version in the format '.' +version version 1000022.1-74 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index ad47601b090d..09c7d27238a3 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -232,6 +232,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. -versionversion1000022.1-72set the active cluster version in the format '.' +versionversion1000022.1-74set the active cluster version in the format '.' diff --git a/pkg/cloud/amazon/BUILD.bazel b/pkg/cloud/amazon/BUILD.bazel index 0e53f789119d..bb3de63dbd2b 100644 --- a/pkg/cloud/amazon/BUILD.bazel +++ b/pkg/cloud/amazon/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/cloud/externalconn", "//pkg/cloud/externalconn/connectionpb", "//pkg/cloud/externalconn/utils", + "//pkg/clusterversion", "//pkg/security/username", "//pkg/server/telemetry", "//pkg/settings", diff --git a/pkg/cloud/amazon/aws_kms.go b/pkg/cloud/amazon/aws_kms.go index 59ce61fb3ef9..c1233dedf3f0 100644 --- a/pkg/cloud/amazon/aws_kms.go +++ b/pkg/cloud/amazon/aws_kms.go @@ -21,6 +21,7 @@ import ( "github.com/aws/aws-sdk-go/aws/session" "github.com/aws/aws-sdk-go/service/kms" "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/errors" ) @@ -161,6 +162,10 @@ func MakeAWSKMS(ctx context.Context, uri string, env cloud.KMSEnv) (cloud.KMS, e } if kmsURIParams.roleARN != "" { + if !env.ClusterSettings().Version.IsActive(ctx, clusterversion.SupportAssumeRoleAuth) { + return nil, errors.New("cannot authenticate to KMS via assume role until cluster has fully upgraded to 22.2") + } + // If there are delegate roles in the assume-role chain, we create a session // for each role in order for it to fetch the credentials from the next role // in the chain. diff --git a/pkg/cloud/amazon/s3_storage.go b/pkg/cloud/amazon/s3_storage.go index f11674cac78a..e841cc6103e8 100644 --- a/pkg/cloud/amazon/s3_storage.go +++ b/pkg/cloud/amazon/s3_storage.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/cloud/cloudpb" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -470,6 +471,10 @@ func newClient( } if conf.roleARN != "" { + if !settings.Version.IsActive(ctx, clusterversion.SupportAssumeRoleAuth) { + return s3Client{}, "", errors.New("cannot authenticate to cloud storage via assume role until cluster has fully upgraded to 22.2") + } + for _, role := range conf.delegateRoleARNs { intermediateCreds := stscreds.NewCredentials(sess, role) opts.Config.Credentials = intermediateCreds diff --git a/pkg/cloud/gcp/BUILD.bazel b/pkg/cloud/gcp/BUILD.bazel index f48c83f05b5e..726f2ff9ff31 100644 --- a/pkg/cloud/gcp/BUILD.bazel +++ b/pkg/cloud/gcp/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/cloud/externalconn", "//pkg/cloud/externalconn/connectionpb", "//pkg/cloud/externalconn/utils", + "//pkg/clusterversion", "//pkg/security/username", "//pkg/server/telemetry", "//pkg/settings", diff --git a/pkg/cloud/gcp/gcp_kms.go b/pkg/cloud/gcp/gcp_kms.go index 81e60a9f21c7..e411983a047b 100644 --- a/pkg/cloud/gcp/gcp_kms.go +++ b/pkg/cloud/gcp/gcp_kms.go @@ -18,6 +18,7 @@ import ( kms "cloud.google.com/go/kms/apiv1" "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/errors" "google.golang.org/api/option" kmspb "google.golang.org/genproto/googleapis/cloud/kms/v1" @@ -126,6 +127,10 @@ func MakeGCSKMS(ctx context.Context, uri string, env cloud.KMSEnv) (cloud.KMS, e if kmsURIParams.assumeRole == "" { opts = append(opts, credentialsOpt...) } else { + if !env.ClusterSettings().Version.IsActive(ctx, clusterversion.SupportAssumeRoleAuth) { + return nil, errors.New("cannot authenticate to KMS via assume role until cluster has fully upgraded to 22.2") + } + assumeOpt, err := createImpersonateCredentials(ctx, kmsURIParams.assumeRole, kmsURIParams.delegateRoles, kms.DefaultAuthScopes(), credentialsOpt...) if err != nil { return nil, errors.Wrapf(err, "failed to assume role") diff --git a/pkg/cloud/gcp/gcs_storage.go b/pkg/cloud/gcp/gcs_storage.go index 7873df26bd32..1828dd9562ca 100644 --- a/pkg/cloud/gcp/gcs_storage.go +++ b/pkg/cloud/gcp/gcs_storage.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/cloud/cloudpb" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -181,6 +182,10 @@ func makeGCSStorage( if conf.AssumeRole == "" { opts = append(opts, credentialsOpt...) } else { + if !args.Settings.Version.IsActive(ctx, clusterversion.SupportAssumeRoleAuth) { + return nil, errors.New("cannot authenticate to cloud storage via assume role until cluster has fully upgraded to 22.2") + } + assumeOpt, err := createImpersonateCredentials(ctx, conf.AssumeRole, conf.AssumeRoleDelegates, []string{scope}, credentialsOpt...) if err != nil { return nil, errors.Wrapf(err, "failed to assume role") diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 0731d498c150..dd91fa70ab53 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -298,6 +298,9 @@ const ( // leases to nodes that (i) don't expect them for certain keyspans, and (ii) // don't know to upgrade them to efficient epoch-based ones. EnableLeaseUpgrade + // SupportAssumeRoleAuth is the version where assume role authorization is + // supported in cloud storage and KMS. + SupportAssumeRoleAuth // ************************************************* // Step (1): Add new versions here. @@ -481,6 +484,10 @@ var rawVersionsSingleton = keyedVersions{ Key: EnableLeaseUpgrade, Version: roachpb.Version{Major: 22, Minor: 1, Internal: 72}, }, + { + Key: SupportAssumeRoleAuth, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 74}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index d9aa1b8aa342..fe1a7c3e54a1 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -46,11 +46,12 @@ func _() { _ = x[TTLDistSQL-34] _ = x[PrioritizeSnapshots-35] _ = x[EnableLeaseUpgrade-36] + _ = x[SupportAssumeRoleAuth-37] } -const _Key_name = "invalidVersionKeyV22_1Start22_2LocalTimestampsPebbleFormatSplitUserKeysMarkedCompactedEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalitySystemExternalConnectionsTableAlterSystemStatementStatisticsAddIndexRecommendationsRoleIDSequenceAddSystemUserIDColumnSystemUsersIDColumnIsBackfilledSetSystemUsersUserIDColumnNotNullSQLSchemaTelemetryScheduledJobsSchemaChangeSupportsCreateFunctionDeleteRequestReturnKeyPebbleFormatPrePebblev1MarkedRoleOptionsTableHasIDColumnRoleOptionsIDColumnIsBackfilledSetRoleOptionsUserIDColumnNotNullUseDelRangeInGCJobWaitedForDelRangeInGCJobRangefeedUseOneStreamPerNodeNoNonMVCCAddSSTableGCHintInReplicaStateUpdateInvalidColumnIDsInSequenceBackReferencesTTLDistSQLPrioritizeSnapshotsEnableLeaseUpgrade" +const _Key_name = "invalidVersionKeyV22_1Start22_2LocalTimestampsPebbleFormatSplitUserKeysMarkedCompactedEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalitySystemExternalConnectionsTableAlterSystemStatementStatisticsAddIndexRecommendationsRoleIDSequenceAddSystemUserIDColumnSystemUsersIDColumnIsBackfilledSetSystemUsersUserIDColumnNotNullSQLSchemaTelemetryScheduledJobsSchemaChangeSupportsCreateFunctionDeleteRequestReturnKeyPebbleFormatPrePebblev1MarkedRoleOptionsTableHasIDColumnRoleOptionsIDColumnIsBackfilledSetRoleOptionsUserIDColumnNotNullUseDelRangeInGCJobWaitedForDelRangeInGCJobRangefeedUseOneStreamPerNodeNoNonMVCCAddSSTableGCHintInReplicaStateUpdateInvalidColumnIDsInSequenceBackReferencesTTLDistSQLPrioritizeSnapshotsEnableLeaseUpgradeSupportAssumeRoleAuth" -var _Key_index = [...]uint16{0, 17, 22, 31, 46, 86, 120, 154, 176, 196, 215, 248, 267, 287, 308, 343, 377, 407, 460, 474, 495, 526, 559, 590, 624, 646, 675, 702, 733, 766, 784, 808, 836, 855, 875, 921, 931, 950, 968} +var _Key_index = [...]uint16{0, 17, 22, 31, 46, 86, 120, 154, 176, 196, 215, 248, 267, 287, 308, 343, 377, 407, 460, 474, 495, 526, 559, 590, 624, 646, 675, 702, 733, 766, 784, 808, 836, 855, 875, 921, 931, 950, 968, 989} func (i Key) String() string { i -= -1 diff --git a/pkg/kv/kvserver/raft_transport.go b/pkg/kv/kvserver/raft_transport.go index 25fd53b56bd8..33cfb4524bc3 100644 --- a/pkg/kv/kvserver/raft_transport.go +++ b/pkg/kv/kvserver/raft_transport.go @@ -166,8 +166,7 @@ type raftSendQueue struct { reqs chan *kvserverpb.RaftMessageRequest // The number of bytes in flight. Must be updated *atomically* on sending and // receiving from the reqs channel. - // TODO(pavelkalinnikov): replace by atomic.Int64 when CRDB uses Go 1.19. - bytes int64 + bytes atomic.Int64 } // NewDummyRaftTransport returns a dummy raft transport for use in tests which @@ -228,7 +227,7 @@ func (t *RaftTransport) queueMessageCount() int64 { // queueByteSize returns the total bytes size of outgoing messages in the queue. func (t *RaftTransport) queueByteSize() int64 { var size int64 - t.visitQueues(func(q *raftSendQueue) { size += atomic.LoadInt64(&q.bytes) }) + t.visitQueues(func(q *raftSendQueue) { size += q.bytes.Load() }) return size } @@ -441,7 +440,7 @@ func (t *RaftTransport) processQueue( return err case req := <-q.reqs: size := int64(req.Size()) - atomic.AddInt64(&q.bytes, -size) + q.bytes.Add(-size) budget := targetRaftOutgoingBatchSize.Get(&t.st.SV) - size batch.Requests = append(batch.Requests, *req) releaseRaftMessageRequest(req) @@ -450,7 +449,7 @@ func (t *RaftTransport) processQueue( select { case req = <-q.reqs: size := int64(req.Size()) - atomic.AddInt64(&q.bytes, -size) + q.bytes.Add(-size) budget -= size batch.Requests = append(batch.Requests, *req) releaseRaftMessageRequest(req) @@ -532,7 +531,7 @@ func (t *RaftTransport) SendAsync( size := int64(req.Size()) select { case q.reqs <- req: - atomic.AddInt64(&q.bytes, size) + q.bytes.Add(size) return true default: if logRaftSendQueueFullEvery.ShouldLog() { @@ -564,7 +563,7 @@ func (t *RaftTransport) startProcessNewQueue( for { select { case req := <-q.reqs: - atomic.AddInt64(&q.bytes, -int64(req.Size())) + q.bytes.Add(-int64(req.Size())) t.metrics.MessagesDropped.Inc(1) releaseRaftMessageRequest(req) default: diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index 266aede2d050..3e5731f50672 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -925,22 +924,11 @@ func (b *replicaAppBatch) runPreApplyTriggersAfterStagingWriteBatch( func (b *replicaAppBatch) stageTrivialReplicatedEvalResult( ctx context.Context, cmd *replicatedCmd, ) { - raftAppliedIndex := cmd.ent.Index - if raftAppliedIndex == 0 { + if cmd.ent.Index == 0 { log.Fatalf(ctx, "raft entry with index 0") } - b.state.RaftAppliedIndex = raftAppliedIndex - rs := cmd.decodedRaftEntry.replicatedResult().State - // We are post migration or this replicatedCmd is doing the migration. - if b.state.RaftAppliedIndexTerm > 0 || (rs != nil && - rs.RaftAppliedIndexTerm == stateloader.RaftLogTermSignalForAddRaftAppliedIndexTermMigration) { - // Once we populate b.state.RaftAppliedIndexTerm it will flow into the - // persisted RangeAppliedState and into the in-memory representation in - // Replica.mu.state. The latter is used to initialize b.state, so future - // calls to this method will see that the migration has already happened - // and will continue to populate the term. - b.state.RaftAppliedIndexTerm = cmd.ent.Term - } + b.state.RaftAppliedIndex = cmd.ent.Index + b.state.RaftAppliedIndexTerm = cmd.ent.Term if leaseAppliedIndex := cmd.leaseIndex; leaseAppliedIndex != 0 { b.state.LeaseAppliedIndex = leaseAppliedIndex @@ -1001,8 +989,6 @@ func (b *replicaAppBatch) ApplyToStateMachine(ctx context.Context) error { r := b.r r.mu.Lock() r.mu.state.RaftAppliedIndex = b.state.RaftAppliedIndex - // RaftAppliedIndexTerm will be non-zero only when the - // AddRaftAppliedIndexTermMigration has happened. r.mu.state.RaftAppliedIndexTerm = b.state.RaftAppliedIndexTerm r.mu.state.LeaseAppliedIndex = b.state.LeaseAppliedIndex diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 6e7f6c8ae174..a9b01cabf7c4 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -414,15 +414,10 @@ func (r *Replica) GetLeaseAppliedIndex() uint64 { // Snapshot method. func (r *replicaRaftStorage) Snapshot() (raftpb.Snapshot, error) { r.mu.AssertHeld() - appliedIndex := r.mu.state.RaftAppliedIndex - term, err := r.Term(appliedIndex) - if err != nil { - return raftpb.Snapshot{}, err - } return raftpb.Snapshot{ Metadata: raftpb.SnapshotMetadata{ - Index: appliedIndex, - Term: term, + Index: r.mu.state.RaftAppliedIndex, + Term: r.mu.state.RaftAppliedIndexTerm, }, }, nil } @@ -605,17 +600,6 @@ func snapshot( return OutgoingSnapshot{}, err } - term, err := term(ctx, rsl, snap, rangeID, eCache, state.RaftAppliedIndex) - // If we've migrated to populating RaftAppliedIndexTerm, check that the term - // from the two sources are equal. - if state.RaftAppliedIndexTerm != 0 && term != state.RaftAppliedIndexTerm { - return OutgoingSnapshot{}, - errors.AssertionFailedf("unequal terms %d != %d", term, state.RaftAppliedIndexTerm) - } - if err != nil { - return OutgoingSnapshot{}, errors.Wrapf(err, "failed to fetch term of %d", state.RaftAppliedIndex) - } - return OutgoingSnapshot{ RaftEntryCache: eCache, WithSideloaded: withSideloaded, @@ -626,7 +610,7 @@ func snapshot( Data: snapUUID.GetBytes(), Metadata: raftpb.SnapshotMetadata{ Index: state.RaftAppliedIndex, - Term: term, + Term: state.RaftAppliedIndexTerm, // Synthesize our raftpb.ConfState from desc. ConfState: desc.Replicas().ConfState(), }, @@ -979,9 +963,7 @@ func (r *Replica) applySnapshot( log.Fatalf(ctx, "snapshot RaftAppliedIndex %d doesn't match its metadata index %d", state.RaftAppliedIndex, nonemptySnap.Metadata.Index) } - // If we've migrated to populating RaftAppliedIndexTerm, check that the term - // from the two sources are equal. - if state.RaftAppliedIndexTerm != 0 && state.RaftAppliedIndexTerm != nonemptySnap.Metadata.Term { + if state.RaftAppliedIndexTerm != nonemptySnap.Metadata.Term { log.Fatalf(ctx, "snapshot RaftAppliedIndexTerm %d doesn't match its metadata term %d", state.RaftAppliedIndexTerm, nonemptySnap.Metadata.Term) } diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/schemaInsights/index.ts b/pkg/ui/workspaces/cluster-ui/src/insights/schemaInsights/index.ts index 45c5dbc2f62a..5ca0b716c425 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/schemaInsights/index.ts +++ b/pkg/ui/workspaces/cluster-ui/src/insights/schemaInsights/index.ts @@ -11,3 +11,4 @@ export * from "./indexUsageStatsRec"; export * from "./schemaInsightsView"; export * from "./emptySchemaInsightsTablePlaceholder"; +export * from "./schemaInsightsPageConnected"; diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/schemaInsights/schemaInsightsPageConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/schemaInsights/schemaInsightsPageConnected.tsx new file mode 100644 index 000000000000..50962a067826 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/insights/schemaInsights/schemaInsightsPageConnected.tsx @@ -0,0 +1,67 @@ +// 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. + +import { connect } from "react-redux"; +import { RouteComponentProps, withRouter } from "react-router-dom"; +import { + actions, + selectSchemaInsights, + selectSchemaInsightsDatabases, + selectSchemaInsightsError, + selectSchemaInsightsTypes, + selectFilters, + selectSortSetting, +} from "src/store/schemaInsights"; +import { AppState } from "src/store"; +import { + SchemaInsightsView, + SchemaInsightsViewDispatchProps, + SchemaInsightsViewStateProps, +} from "./schemaInsightsView"; +import { SchemaInsightEventFilters } from "../types"; +import { SortSetting } from "src/sortedtable"; +import { actions as localStorageActions } from "../../store/localStorage"; + +const mapStateToProps = ( + state: AppState, + _props: RouteComponentProps, +): SchemaInsightsViewStateProps => ({ + schemaInsights: selectSchemaInsights(state), + schemaInsightsDatabases: selectSchemaInsightsDatabases(state), + schemaInsightsTypes: selectSchemaInsightsTypes(state), + schemaInsightsError: selectSchemaInsightsError(state), + filters: selectFilters(state), + sortSetting: selectSortSetting(state), +}); + +const mapDispatchToProps = { + onFiltersChange: (filters: SchemaInsightEventFilters) => + localStorageActions.update({ + key: "filters/SchemaInsightsPage", + value: filters, + }), + onSortChange: (ss: SortSetting) => + localStorageActions.update({ + key: "sortSetting/SchemaInsightsPage", + value: ss, + }), + refreshSchemaInsights: actions.refresh, +}; + +export const SchemaInsightsPageConnected = withRouter( + connect< + SchemaInsightsViewStateProps, + SchemaInsightsViewDispatchProps, + RouteComponentProps + >( + mapStateToProps, + mapDispatchToProps, + )(SchemaInsightsView), +); diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/schemaInsights/schemaInsightsView.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/schemaInsights/schemaInsightsView.tsx index 5b23dfd16a16..e6a8521ed83b 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/schemaInsights/schemaInsightsView.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/schemaInsights/schemaInsightsView.tsx @@ -202,7 +202,7 @@ export const SchemaInsightsView: React.FC = ({
InsightsError()} @@ -227,7 +227,7 @@ export const SchemaInsightsView: React.FC = ({ renderNoResult={ 0 && filteredSchemaInsights?.length == 0 + search?.length > 0 && filteredSchemaInsights?.length === 0 } /> } diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/utils.ts b/pkg/ui/workspaces/cluster-ui/src/insights/utils.ts index 509129e9fa3f..5141a4ce6c00 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/utils.ts +++ b/pkg/ui/workspaces/cluster-ui/src/insights/utils.ts @@ -90,9 +90,9 @@ export function getTransactionInsightEventDetailsFromState( insightEventDetailsResponse, ); if (insightsForEventDetails.length > 0) { - delete insightEventDetailsResponse.insightName; + const { insightName, ...resp } = insightEventDetailsResponse; insightEventDetails = { - ...insightEventDetailsResponse, + ...resp, insights: insightsForEventDetails, }; } @@ -305,32 +305,36 @@ export function getAppsFromStatementInsights( export function populateStatementInsightsFromProblemAndCauses( statements: StatementInsightEvent[], -): void { +): StatementInsightEvent[] { if (!statements || statements?.length === 0) { return; } - statements.map(x => { + const stmts: StatementInsightEvent[] = []; + statements.forEach(statement => { + const stmt = Object.assign({}, statement); // TODO(ericharmeling,todd): Replace these strings when using the insights protos. - if (x.problem === "SlowExecution") { - if (x.causes?.length === 0) { - x.insights = [ + if (statement.problem === "SlowExecution") { + if (statement.causes?.length === 0) { + stmt.insights = [ getInsightFromProblem( InsightNameEnum.slowExecution, InsightExecEnum.STATEMENT, ), ]; } else { - x.insights = x.causes?.map(x => + stmt.insights = statement.causes?.map(x => getInsightFromProblem(x, InsightExecEnum.STATEMENT), ); } - } else if (x.problem === "FailedExecution") { - x.insights = [ + } else if (statement.problem === "FailedExecution") { + stmt.insights = [ getInsightFromProblem( InsightNameEnum.failedExecution, InsightExecEnum.STATEMENT, ), ]; } + stmts.push(stmt); }); + return stmts; } diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/index.ts b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/index.ts index 3f0d771b189e..839b869cd7ae 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/index.ts +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/index.ts @@ -9,4 +9,6 @@ // licenses/APL.txt. export * from "./transactionInsightDetails"; +export * from "./transactionInsightDetailsConnected"; export * from "./statementInsightDetails"; +export * from "./statementInsightDetailsConnected"; diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/statementInsightDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/statementInsightDetails.tsx index 8250e822c8d3..9b363c61cc48 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/statementInsightDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/statementInsightDetails.tsx @@ -55,11 +55,13 @@ export class StatementInsightDetails extends React.Component this.props.history.goBack(); renderContent = (): React.ReactElement => { - const insightDetailsArr = [this.props.insightEventDetails]; - populateStatementInsightsFromProblemAndCauses(insightDetailsArr); + const insightDetailsArr = populateStatementInsightsFromProblemAndCauses([ + this.props.insightEventDetails, + ]); const insightDetails = insightDetailsArr[0]; const isCockroachCloud = useContext(CockroachCloudContext); const insightsColumns = makeInsightsColumns(isCockroachCloud); @@ -71,7 +73,6 @@ export class StatementInsightDetails extends React.Component { switch (insight.name) { case InsightNameEnum.highContention: diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/statementInsightDetailsConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/statementInsightDetailsConnected.tsx new file mode 100644 index 000000000000..d868ee7bde72 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/statementInsightDetailsConnected.tsx @@ -0,0 +1,38 @@ +// 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. +import { connect } from "react-redux"; +import { RouteComponentProps, withRouter } from "react-router-dom"; +import { + StatementInsightDetails, + StatementInsightDetailsStateProps, +} from "./statementInsightDetails"; +import { AppState } from "src/store"; +import { + selectStatementInsightDetails, + selectStatementInsightsError, +} from "src/store/insights/statementInsights"; + +const mapStateToProps = ( + state: AppState, + props: RouteComponentProps, +): StatementInsightDetailsStateProps => { + const insightStatements = selectStatementInsightDetails(state, props); + const insightError = selectStatementInsightsError(state); + return { + insightEventDetails: insightStatements, + insightError: insightError, + }; +}; + +export const StatementInsightDetailsConnected = withRouter( + connect( + mapStateToProps, + )(StatementInsightDetails), +); diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsConnected.tsx new file mode 100644 index 000000000000..bb5aa21caa91 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsConnected.tsx @@ -0,0 +1,49 @@ +// 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. +import { + TransactionInsightDetails, + TransactionInsightDetailsStateProps, + TransactionInsightDetailsDispatchProps, +} from "./transactionInsightDetails"; +import { connect } from "react-redux"; +import { RouteComponentProps, withRouter } from "react-router-dom"; +import { AppState } from "src/store"; +import { + selectTransactionInsightDetails, + selectTransactionInsightDetailsError, + actions, +} from "src/store/insightDetails/transactionInsightDetails"; + +const mapStateToProps = ( + state: AppState, + _props: RouteComponentProps, +): TransactionInsightDetailsStateProps => { + const insightDetails = selectTransactionInsightDetails(state); + const insightError = selectTransactionInsightDetailsError(state); + return { + insightEventDetails: insightDetails, + insightError: insightError, + }; +}; + +const mapDispatchToProps = { + refreshTransactionInsightDetails: actions.refresh, +}; + +export const TransactionInsightDetailsConnected = withRouter( + connect< + TransactionInsightDetailsStateProps, + TransactionInsightDetailsDispatchProps, + RouteComponentProps + >( + mapStateToProps, + mapDispatchToProps, + )(TransactionInsightDetails), +); diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/index.ts b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/index.ts index cc0bf8c133ad..f4c2823e3518 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/index.ts +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/index.ts @@ -11,3 +11,4 @@ export * from "./transactionInsights"; export * from "./statementInsights"; export * from "./workloadInsightRootControl"; +export * from "./workloadInsightsPageConnected"; diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/statementInsights/statementInsightsView.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/statementInsights/statementInsightsView.tsx index ee61bcc88700..161405f53cfc 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/statementInsights/statementInsightsView.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/statementInsights/statementInsightsView.tsx @@ -202,7 +202,7 @@ export const StatementInsightsView: React.FC = ( const clearFilters = () => onSubmitFilters({ - app: defaultFilters.app, + app: "", }); const apps = getAppsFromStatementInsights( @@ -217,7 +217,8 @@ export const StatementInsightsView: React.FC = ( search, ); - populateStatementInsightsFromProblemAndCauses(filteredStatements); + const statementInsights = + populateStatementInsightsFromProblemAndCauses(filteredStatements); const tableColumns = defaultColumns .filter(c => !c.alwaysShow) .map( @@ -266,21 +267,21 @@ export const StatementInsightsView: React.FC = (
0 && filteredStatements?.length === 0 + search?.length > 0 && statementInsights?.length === 0 } /> } @@ -290,7 +291,7 @@ export const StatementInsightsView: React.FC = ( diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/transactionInsights/transactionInsightsView.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/transactionInsights/transactionInsightsView.tsx index 3cad40750804..7bc1fc06af8e 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/transactionInsights/transactionInsightsView.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/transactionInsights/transactionInsightsView.tsx @@ -172,7 +172,7 @@ export const TransactionInsightsView: React.FC = ( const clearFilters = () => onSubmitFilters({ - app: defaultFilters.app, + app: "", }); const transactionInsights = getInsightsFromState(transactions); diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/workloadInsightsPageConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/workloadInsightsPageConnected.tsx new file mode 100644 index 000000000000..40d07acce4d0 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/workloadInsightsPageConnected.tsx @@ -0,0 +1,143 @@ +// 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. + +import { connect } from "react-redux"; +import { RouteComponentProps, withRouter } from "react-router-dom"; +import { AppState } from "src/store"; +import { actions as localStorageActions } from "src/store/localStorage"; +import { + TransactionInsightsViewDispatchProps, + TransactionInsightsViewStateProps, +} from "./transactionInsights"; +import { + StatementInsightsViewDispatchProps, + StatementInsightsViewStateProps, +} from "./statementInsights"; +import { WorkloadInsightEventFilters } from "../types"; +import { + WorkloadInsightsViewProps, + WorkloadInsightsRootControl, +} from "./workloadInsightRootControl"; +import { SortSetting } from "src/sortedtable"; +import { + actions as statementInsights, + selectColumns, + selectStatementInsights, + selectStatementInsightsError, +} from "src/store/insights/statementInsights"; +import { + actions as transactionInsights, + selectTransactionInsights, + selectTransactionInsightsError, + selectFilters, + selectSortSetting, +} from "src/store/insights/transactionInsights"; +import { bindActionCreators } from "redux"; + +const transactionMapStateToProps = ( + state: AppState, + _props: RouteComponentProps, +): TransactionInsightsViewStateProps => ({ + transactions: selectTransactionInsights(state), + transactionsError: selectTransactionInsightsError(state), + filters: selectFilters(state), + sortSetting: selectSortSetting(state), +}); + +const statementMapStateToProps = ( + state: AppState, + _props: RouteComponentProps, +): StatementInsightsViewStateProps => ({ + statements: selectStatementInsights(state), + statementsError: selectStatementInsightsError(state), + filters: selectFilters(state), + sortSetting: selectSortSetting(state), + selectedColumnNames: selectColumns(state), +}); + +const TransactionDispatchProps = { + onFiltersChange: (filters: WorkloadInsightEventFilters) => + localStorageActions.update({ + key: "filters/InsightsPage", + value: filters, + }), + onSortChange: (ss: SortSetting) => + localStorageActions.update({ + key: "sortSetting/InsightsPage", + value: ss, + }), + refreshTransactionInsights: transactionInsights.refresh, +}; + +const StatementDispatchProps: StatementInsightsViewDispatchProps = { + onFiltersChange: (filters: WorkloadInsightEventFilters) => + localStorageActions.update({ + key: "filters/InsightsPage", + value: filters, + }), + onSortChange: (ss: SortSetting) => + localStorageActions.update({ + key: "sortSetting/InsightsPage", + value: ss, + }), + onColumnsChange: (value: string[]) => + localStorageActions.update({ + key: "showColumns/StatementInsightsPage", + value: value.join(","), + }), + refreshStatementInsights: statementInsights.refresh, +}; + +type StateProps = { + transactionInsightsViewStateProps: TransactionInsightsViewStateProps; + statementInsightsViewStateProps: StatementInsightsViewStateProps; +}; + +type DispatchProps = { + transactionInsightsViewDispatchProps: TransactionInsightsViewDispatchProps; + statementInsightsViewDispatchProps: StatementInsightsViewDispatchProps; +}; + +export const WorkloadInsightsPageConnected = withRouter( + connect< + StateProps, + DispatchProps, + RouteComponentProps, + WorkloadInsightsViewProps + >( + (state: AppState, props: RouteComponentProps) => ({ + transactionInsightsViewStateProps: transactionMapStateToProps( + state, + props, + ), + statementInsightsViewStateProps: statementMapStateToProps(state, props), + }), + dispatch => ({ + transactionInsightsViewDispatchProps: bindActionCreators( + TransactionDispatchProps, + dispatch, + ), + statementInsightsViewDispatchProps: bindActionCreators( + StatementDispatchProps, + dispatch, + ), + }), + (stateProps, dispatchProps) => ({ + transactionInsightsViewProps: { + ...stateProps.transactionInsightsViewStateProps, + ...dispatchProps.transactionInsightsViewDispatchProps, + }, + statementInsightsViewProps: { + ...stateProps.statementInsightsViewStateProps, + ...dispatchProps.statementInsightsViewDispatchProps, + }, + }), + )(WorkloadInsightsRootControl), +); diff --git a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.fixture.ts b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.fixture.ts index ce13d1bb5ecc..4c5f80bcbe98 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.fixture.ts +++ b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.fixture.ts @@ -784,8 +784,6 @@ export const getStatementDetailsPropsFixture = ( key: "Custom", }, statementFingerprintID: "4705782015019656142", - latestQuery: "SELECT * FROM crdb_internal.node_build_info", - latestFormattedQuery: "SELECT * FROM crdb_internal.node_build_info\n", statementDetails: withData ? statementDetailsData : statementDetailsNoData, statementsError: null, nodeNames: { @@ -809,8 +807,6 @@ export const getStatementDetailsPropsFixture = ( dismissStatementDiagnosticsAlertMessage: noop, onTimeScaleChange: noop, createStatementDiagnosticsReport: noop, - onStatementDetailsQueryChange: noop, - onStatementDetailsFormattedQueryChange: noop, uiConfig: { showStatementDiagnosticsLink: true, }, diff --git a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.stories.tsx b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.stories.tsx index 5841b1a425ec..d4f83da68447 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.stories.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.stories.tsx @@ -102,7 +102,5 @@ storiesOf("StatementDetails", module) ) .add("No data for this time frame; no cached statement", () => { const props = getStatementDetailsPropsFixture(false); - props.latestQuery = ""; - props.latestFormattedQuery = ""; return ; }); diff --git a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx index 3012f3a2a92b..1bca56983b65 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx @@ -91,6 +91,20 @@ export type StatementDetailsProps = StatementDetailsOwnProps & export interface StatementDetailsState { currentTab?: string; + + /** + * The latest non-null query text associated with the statement fingerprint in the URL. + * We save this to preserve this data when the time frame changes such that there is no + * longer data for this statement fingerprint in the selected time frame. + */ + query: string; + + /** + * The latest non-null formatted query associated with the statement fingerprint in the URL. + * We save this to preserve data when the time frame changes such that there is no longer + * data for this statement fingerprint in the selected time frame. + */ + formattedQuery: string; } export type NodesSummary = { @@ -127,16 +141,12 @@ export interface StatementDetailsDispatchProps { ascending: boolean, ) => void; onBackToStatementsClick?: () => void; - onStatementDetailsQueryChange: (query: string) => void; - onStatementDetailsFormattedQueryChange: (formattedQuery: string) => void; } export interface StatementDetailsStateProps { statementFingerprintID: string; statementDetails: StatementDetailsResponse; isLoading: boolean; - latestQuery: string; - latestFormattedQuery: string; statementsError: Error | null; timeScale: TimeScale; nodeNames: { [nodeId: string]: string }; @@ -207,11 +217,15 @@ export class StatementDetails extends React.Component< StatementDetailsState > { activateDiagnosticsRef: React.RefObject; + constructor(props: StatementDetailsProps) { super(props); const searchParams = new URLSearchParams(props.history.location.search); this.state = { currentTab: searchParams.get("tab") || "overview", + query: this.props.statementDetails?.statement?.metadata?.query, + formattedQuery: + this.props.statementDetails?.statement?.metadata?.formatted_query, }; this.activateDiagnosticsRef = React.createRef(); @@ -286,30 +300,22 @@ export class StatementDetails extends React.Component< } } - // If new, non-empty-string query text is available (derived from the statement details response), - // cache the query text. + // If a new, non-empty-string query text is available + // (derived from the statement details response), save the query text. + const newQuery = + this.props.statementDetails?.statement?.metadata?.query || + this.state.query; + const newFormattedQuery = + this.props.statementDetails?.statement?.metadata?.formatted_query || + this.state.formattedQuery; if ( - this.props.statementDetails && - this.props.statementDetails.statement.metadata.query != "" && - this.props.latestQuery != - this.props.statementDetails.statement.metadata.query + newQuery !== this.state.query || + newFormattedQuery !== this.state.formattedQuery ) { - this.props.onStatementDetailsQueryChange( - this.props.statementDetails.statement.metadata.query, - ); - } - - // If a new, non-empty-string formatted query text is available (derived from the statement details response), - // cache the query text. - if ( - this.props.statementDetails && - this.props.statementDetails.statement.metadata.formatted_query != "" && - this.props.latestFormattedQuery != - this.props.statementDetails.statement.metadata.formatted_query - ) { - this.props.onStatementDetailsFormattedQueryChange( - this.props.statementDetails.statement.metadata.formatted_query, - ); + this.setState({ + query: newQuery, + formattedQuery: newFormattedQuery, + }); } // If the statementFingerprintID (derived from the URL) changes, invalidate the cached query texts. @@ -317,8 +323,7 @@ export class StatementDetails extends React.Component< // The new query text and the formatted query text would be an empty string, and we need to invalidate the old // query text and formatted query text. if (this.props.statementFingerprintID != prevProps.statementFingerprintID) { - this.props.onStatementDetailsQueryChange(""); - this.props.onStatementDetailsFormattedQueryChange(""); + this.setState({ query: null, formattedQuery: null }); } } @@ -466,11 +471,11 @@ export class StatementDetails extends React.Component<
- {this.props.latestFormattedQuery && ( + {this.state.formattedQuery && ( @@ -614,7 +619,7 @@ export class StatementDetails extends React.Component< @@ -764,7 +769,7 @@ export class StatementDetails extends React.Component< }; renderDiagnosticsTabContent = (hasData: boolean): React.ReactElement => { - if (!hasData && !this.props.latestQuery) { + if (!hasData && !this.state.query) { return this.renderNoDataTabContent(); } @@ -774,7 +779,9 @@ export class StatementDetails extends React.Component< diagnosticsReports={this.props.diagnosticsReports} dismissAlertMessage={this.props.dismissStatementDiagnosticsAlertMessage} hasData={this.hasDiagnosticReports()} - statementFingerprint={this.props.latestQuery} + statementFingerprint={ + this.props.statementDetails?.statement?.metadata?.query + } onDownloadDiagnosticBundleClick={this.props.onDiagnosticBundleDownload} onDiagnosticCancelRequestClick={this.props.onDiagnosticCancelRequest} showDiagnosticsViewLink={ diff --git a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetailsConnected.ts b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetailsConnected.ts index 1fee7390e467..799aed97e559 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetailsConnected.ts +++ b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetailsConnected.ts @@ -61,12 +61,11 @@ const mapStateToProps = (state: AppState, props: RouteComponentProps) => { state, props, ); + const statementFingerprint = statementDetails?.statement.metadata.query; return { statementFingerprintID: getMatchParamByName(props.match, statementAttr), statementDetails, isLoading: isLoading, - latestQuery: state.adminUI.sqlDetailsStats.latestQuery, - latestFormattedQuery: state.adminUI.sqlDetailsStats.latestFormattedQuery, statementsError: lastError, timeScale: selectTimeScale(state), nodeNames: selectIsTenant(state) ? {} : nodeDisplayNameByIDSelector(state), @@ -76,7 +75,7 @@ const mapStateToProps = (state: AppState, props: RouteComponentProps) => { ? [] : selectDiagnosticsReportsByStatementFingerprint( state, - state.adminUI.sqlDetailsStats.latestQuery, + statementFingerprint, ), uiConfig: selectStatementDetailsUiConfig(state), isTenant: selectIsTenant(state), @@ -162,14 +161,6 @@ const mapDispatchToProps = ( }), ); }, - onStatementDetailsQueryChange: (latestQuery: string) => { - dispatch(sqlDetailsStatsActions.setLatestQuery(latestQuery)); - }, - onStatementDetailsFormattedQueryChange: (latestFormattedQuery: string) => { - dispatch( - sqlDetailsStatsActions.setLatestFormattedQuery(latestFormattedQuery), - ); - }, onSortingChange: (tableName, columnName) => dispatch( analyticsActions.track({ diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/insightDetails.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/index.ts similarity index 56% rename from pkg/ui/workspaces/cluster-ui/src/store/insightDetails/insightDetails.selectors.ts rename to pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/index.ts index 946e01ea6b9b..a8c144de2326 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/insightDetails.selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/index.ts @@ -8,13 +8,6 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -import { createSelector } from "reselect"; -import { adminUISelector } from "../utils/selectors"; - -export const selectInsightDetails = createSelector( - adminUISelector, - adminUiState => { - if (!adminUiState.insights) return []; - return adminUiState.insightDetails.data; - }, -); +export * from "./transactionInsightDetails.reducer"; +export * from "./transactionInsightDetails.sagas"; +export * from "./transactionInsightDetails.selectors"; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/insightDetails.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.reducer.ts similarity index 78% rename from pkg/ui/workspaces/cluster-ui/src/store/insightDetails/insightDetails.reducer.ts rename to pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.reducer.ts index a1a4784aa9d4..50c916f3f619 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/insightDetails.reducer.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.reducer.ts @@ -9,29 +9,29 @@ // licenses/APL.txt. import { createSlice, PayloadAction } from "@reduxjs/toolkit"; -import { DOMAIN_NAME } from "../utils"; +import { DOMAIN_NAME } from "src/store/utils"; import moment, { Moment } from "moment"; import { TransactionInsightEventDetailsRequest, TransactionInsightEventDetailsResponse, } from "src/api/insightsApi"; -export type InsightDetailsState = { +export type TransactionInsightDetailsState = { data: TransactionInsightEventDetailsResponse | null; lastUpdated: Moment | null; lastError: Error; valid: boolean; }; -const initialState: InsightDetailsState = { +const initialState: TransactionInsightDetailsState = { data: null, lastUpdated: null, lastError: null, valid: true, }; -const insightDetailsSlice = createSlice({ - name: `${DOMAIN_NAME}/insightDetailsSlice`, +const transactionInsightDetailsSlice = createSlice({ + name: `${DOMAIN_NAME}/transactionInsightDetailsSlice`, initialState, reducers: { received: ( @@ -47,6 +47,9 @@ const insightDetailsSlice = createSlice({ state.valid = false; state.lastError = action.payload; }, + invalidated: state => { + state.valid = false; + }, refresh: ( _, action: PayloadAction, @@ -58,4 +61,4 @@ const insightDetailsSlice = createSlice({ }, }); -export const { reducer, actions } = insightDetailsSlice; +export const { reducer, actions } = transactionInsightDetailsSlice; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/insightDetails.sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.sagas.ts similarity index 76% rename from pkg/ui/workspaces/cluster-ui/src/store/insightDetails/insightDetails.sagas.ts rename to pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.sagas.ts index 9957911917a2..e3fb96491d08 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/insightDetails.sagas.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.sagas.ts @@ -8,9 +8,9 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -import { all, call, put, takeLatest } from "redux-saga/effects"; +import { all, call, delay, put, takeLatest } from "redux-saga/effects"; -import { actions } from "./insightDetails.reducer"; +import { actions } from "./transactionInsightDetails.reducer"; import { getTransactionInsightEventDetailsState, TransactionInsightEventDetailsRequest, @@ -37,8 +37,14 @@ export function* requestTransactionInsightDetailsSaga( } } +export function* receivedTransactionInsightDetailsSaga() { + yield put(actions.invalidated()); +} + export function* transactionInsightDetailsSaga() { yield all([ + takeLatest(actions.refresh, refreshTransactionInsightDetailsSaga), takeLatest(actions.request, requestTransactionInsightDetailsSaga), + takeLatest(actions.received, receivedTransactionInsightDetailsSaga), ]); } diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.selectors.ts new file mode 100644 index 000000000000..8694d199c3ed --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.selectors.ts @@ -0,0 +1,36 @@ +// 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. + +import { createSelector } from "reselect"; +import { adminUISelector } from "src/store/utils/selectors"; + +const selectTransactionInsightDetailsState = createSelector( + adminUISelector, + adminUiState => { + if (!adminUiState.transactionInsightDetails) return null; + return adminUiState.transactionInsightDetails; + }, +); + +export const selectTransactionInsightDetails = createSelector( + selectTransactionInsightDetailsState, + txnInsightDetailsState => { + if (!txnInsightDetailsState) return null; + return txnInsightDetailsState.data; + }, +); + +export const selectTransactionInsightDetailsError = createSelector( + selectTransactionInsightDetailsState, + txnInsightDetailsState => { + if (!txnInsightDetailsState) return null; + return txnInsightDetailsState.lastError; + }, +); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/index.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/index.ts similarity index 73% rename from pkg/ui/workspaces/cluster-ui/src/store/insightDetails/index.ts rename to pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/index.ts index 129097315975..488275ec89ff 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/index.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/index.ts @@ -8,6 +8,6 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -export * from "./insightDetails.reducer"; -export * from "./insightDetails.sagas"; -export * from "./insightDetails.selectors"; +export * from "./statementInsights.reducer"; +export * from "./statementInsights.sagas"; +export * from "./statementInsights.selectors"; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.reducer.ts new file mode 100644 index 000000000000..ca67df47b8a2 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.reducer.ts @@ -0,0 +1,53 @@ +// 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. + +import { createSlice, PayloadAction } from "@reduxjs/toolkit"; +import { DOMAIN_NAME, noopReducer } from "../../utils"; +import moment, { Moment } from "moment"; +import { StatementInsights } from "src/api/insightsApi"; + +export type StatementInsightsState = { + data: StatementInsights; + lastUpdated: Moment; + lastError: Error; + valid: boolean; +}; + +const initialState: StatementInsightsState = { + data: null, + lastUpdated: null, + lastError: null, + valid: true, +}; + +const statementInsightsSlice = createSlice({ + name: `${DOMAIN_NAME}/statementInsightsSlice`, + initialState, + reducers: { + received: (state, action: PayloadAction) => { + state.data = action.payload; + state.valid = true; + state.lastError = null; + state.lastUpdated = moment.utc(); + }, + failed: (state, action: PayloadAction) => { + state.valid = false; + state.lastError = action.payload; + }, + invalidated: state => { + state.valid = false; + }, + // Define actions that don't change state. + refresh: noopReducer, + request: noopReducer, + }, +}); + +export const { reducer, actions } = statementInsightsSlice; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.sagas.ts new file mode 100644 index 000000000000..19b061e29482 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.sagas.ts @@ -0,0 +1,53 @@ +// 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. + +import { all, call, delay, put, takeLatest } from "redux-saga/effects"; + +import { actions } from "./statementInsights.reducer"; +import { getStatementInsightsApi } from "src/api/insightsApi"; +import { throttleWithReset } from "../../utils"; +import { rootActions } from "../../reducers"; + +export function* refreshStatementInsightsSaga() { + yield put(actions.request()); +} + +export function* requestStatementInsightsSaga(): any { + try { + const result = yield call(getStatementInsightsApi); + yield put(actions.received(result)); + } catch (e) { + yield put(actions.failed(e)); + } +} + +export function* receivedStatementInsightsSaga(delayMs: number) { + yield delay(delayMs); + yield put(actions.invalidated()); +} + +export function* statementInsightsSaga( + cacheInvalidationPeriod: number = 10 * 1000, +) { + yield all([ + throttleWithReset( + cacheInvalidationPeriod, + actions.refresh, + [actions.invalidated, rootActions.resetState], + refreshStatementInsightsSaga, + ), + takeLatest(actions.request, requestStatementInsightsSaga), + takeLatest( + actions.received, + receivedStatementInsightsSaga, + cacheInvalidationPeriod, + ), + ]); +} diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.selectors.ts new file mode 100644 index 000000000000..4694e1fb5e0b --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.selectors.ts @@ -0,0 +1,64 @@ +// 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. + +import { createSelector } from "reselect"; +import { + adminUISelector, + localStorageSelector, +} from "src/store/utils/selectors"; +import { getMatchParamByName } from "../../../util"; +import { RouteComponentProps } from "react-router"; +import { StatementInsightEvent } from "../../../insights"; +import { AppState } from "../../reducers"; + +const selectStatementInsightsState = createSelector( + adminUISelector, + adminUiState => { + if (!adminUiState.statementInsights) return null; + return adminUiState.statementInsights; + }, +); + +export const selectStatementInsights = createSelector( + selectStatementInsightsState, + stmtInsightsState => { + if (!stmtInsightsState) return []; + return stmtInsightsState.data; + }, +); + +export const selectStatementInsightsError = createSelector( + selectStatementInsightsState, + stmtInsightsState => { + if (!stmtInsightsState) return null; + return stmtInsightsState.lastError; + }, +); + +export const selectStatementInsightDetails = createSelector( + selectStatementInsightsState, + (_state: AppState, props: RouteComponentProps) => props, + (statementInsights, props): StatementInsightEvent => { + if (!statementInsights) return null; + + const insightId = getMatchParamByName(props.match, "id"); + return statementInsights.data?.find( + statementInsight => statementInsight.statementID === insightId, + ); + }, +); + +export const selectColumns = createSelector( + localStorageSelector, + localStorage => + localStorage["showColumns/StatementInsightsPage"] + ? localStorage["showColumns/StatementInsightsPage"].split(",") + : null, +); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights.selectors.ts deleted file mode 100644 index de77aad385b0..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights.selectors.ts +++ /dev/null @@ -1,17 +0,0 @@ -// Copyright 2020 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. - -import { createSelector } from "reselect"; -import { adminUISelector } from "../utils/selectors"; - -export const selectInsights = createSelector(adminUISelector, adminUiState => { - if (!adminUiState.insights) return []; - return adminUiState.insights.data; -}); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/index.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/index.ts similarity index 100% rename from pkg/ui/workspaces/cluster-ui/src/store/insights/index.ts rename to pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/index.ts diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.reducer.ts similarity index 80% rename from pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights.reducer.ts rename to pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.reducer.ts index d62171350e6f..44c4d2005af9 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights.reducer.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.reducer.ts @@ -9,26 +9,26 @@ // licenses/APL.txt. import { createSlice, PayloadAction } from "@reduxjs/toolkit"; -import { DOMAIN_NAME, noopReducer } from "../utils"; +import { DOMAIN_NAME, noopReducer } from "src/store/utils"; import moment, { Moment } from "moment"; import { TransactionInsightEventsResponse } from "src/api/insightsApi"; -export type InsightsState = { +export type TransactionInsightsState = { data: TransactionInsightEventsResponse; lastUpdated: Moment; lastError: Error; valid: boolean; }; -const initialState: InsightsState = { +const initialState: TransactionInsightsState = { data: null, lastUpdated: null, lastError: null, valid: true, }; -const insightsSlice = createSlice({ - name: `${DOMAIN_NAME}/insightsSlice`, +const transactionInsightsSlice = createSlice({ + name: `${DOMAIN_NAME}/transactionInsightsSlice`, initialState, reducers: { received: ( @@ -53,4 +53,4 @@ const insightsSlice = createSlice({ }, }); -export const { reducer, actions } = insightsSlice; +export const { reducer, actions } = transactionInsightsSlice; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights.sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.sagas.ts similarity index 93% rename from pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights.sagas.ts rename to pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.sagas.ts index 29518bd608cc..ae93cc3fec93 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights.sagas.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.sagas.ts @@ -12,8 +12,8 @@ import { all, call, delay, put, takeLatest } from "redux-saga/effects"; import { actions } from "./transactionInsights.reducer"; import { getTransactionInsightEventState } from "src/api/insightsApi"; -import { throttleWithReset } from "../utils"; -import { rootActions } from "../reducers"; +import { throttleWithReset } from "src/store/utils"; +import { rootActions } from "src/store/reducers"; export function* refreshTransactionInsightsSaga() { yield put(actions.request()); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.selectors.ts new file mode 100644 index 000000000000..2c9ddebc8b1a --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.selectors.ts @@ -0,0 +1,49 @@ +// 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. + +import { createSelector } from "reselect"; +import { + adminUISelector, + localStorageSelector, +} from "src/store/utils/selectors"; + +const selectTransactionInsightsState = createSelector( + adminUISelector, + adminUiState => { + if (!adminUiState.transactionInsights) return null; + return adminUiState.transactionInsights; + }, +); + +export const selectTransactionInsights = createSelector( + selectTransactionInsightsState, + txnInsightsState => { + if (!txnInsightsState) return []; + return txnInsightsState.data; + }, +); + +export const selectTransactionInsightsError = createSelector( + selectTransactionInsightsState, + txnInsightsState => { + if (!txnInsightsState) return null; + return txnInsightsState.lastError; + }, +); + +export const selectSortSetting = createSelector( + localStorageSelector, + localStorage => localStorage["sortSetting/InsightsPage"], +); + +export const selectFilters = createSelector( + localStorageSelector, + localStorage => localStorage["filters/InsightsPage"], +); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/localStorage/localStorage.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/localStorage/localStorage.reducer.ts index 5e2ac26a815a..a7304d4baf71 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/localStorage/localStorage.reducer.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/localStorage/localStorage.reducer.ts @@ -10,8 +10,9 @@ import { createSlice, PayloadAction } from "@reduxjs/toolkit"; import { DOMAIN_NAME } from "../utils"; -import { defaultFilters, Filters } from "../../queryFilter"; +import { defaultFilters, Filters } from "src/queryFilter/"; import { TimeScale, defaultTimeScaleSelected } from "../../timeScaleDropdown"; +import { WorkloadInsightEventFilters } from "src/insights"; type SortSetting = { ascending: boolean; @@ -25,6 +26,7 @@ export type LocalStorageState = { "showColumns/StatementsPage": string; "showColumns/TransactionPage": string; "showColumns/SessionsPage": string; + "showColumns/StatementInsightsPage": string; "timeScale/SQLActivity": TimeScale; "sortSetting/ActiveStatementsPage": SortSetting; "sortSetting/ActiveTransactionsPage": SortSetting; @@ -39,7 +41,7 @@ export type LocalStorageState = { "filters/StatementsPage": Filters; "filters/TransactionsPage": Filters; "filters/SessionsPage": Filters; - "filters/InsightsPage": Filters; + "filters/InsightsPage": WorkloadInsightEventFilters; "filters/SchemaInsightsPage": Filters; "search/StatementsPage": string; "search/TransactionsPage": string; @@ -74,16 +76,16 @@ const defaultSortSettingSchemaInsights: SortSetting = { }; const defaultFiltersActiveExecutions = { - app: defaultFilters.app, + app: "", }; const defaultFiltersInsights = { - app: defaultFilters.app, + app: "", }; const defaultFiltersSchemaInsights = { - database: defaultFilters.database, - schemaInsightType: defaultFilters.schemaInsightType, + database: "", + schemaInsightType: "", }; const defaultSessionsSortSetting: SortSetting = { @@ -119,6 +121,9 @@ const initialState: LocalStorageState = { JSON.parse(localStorage.getItem("showColumns/TransactionPage")) || null, "showColumns/SessionsPage": JSON.parse(localStorage.getItem("showColumns/SessionsPage")) || null, + "showColumns/StatementInsightsPage": + JSON.parse(localStorage.getItem("showColumns/StatementInsightsPage")) || + null, "showSetting/JobsPage": JSON.parse(localStorage.getItem("showSetting/JobsPage")) || defaultJobShowSetting, diff --git a/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts b/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts index fd639758947e..6fef2628cb10 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts @@ -39,15 +39,22 @@ import { ClusterLocksReqState, reducer as clusterLocks, } from "./clusterLocks/clusterLocks.reducer"; -import { InsightsState, reducer as insights } from "./insights"; import { - InsightDetailsState, - reducer as insightDetails, -} from "./insightDetails"; + TransactionInsightsState, + reducer as transactionInsights, +} from "./insights/transactionInsights"; +import { + StatementInsightsState, + reducer as statementInsights, +} from "./insights/statementInsights"; import { SchemaInsightsState, reducer as schemaInsights, } from "./schemaInsights"; +import { + TransactionInsightDetailsState, + reducer as transactionInsightDetails, +} from "./insightDetails/transactionInsightDetails"; export type AdminUiState = { statementDiagnostics: StatementDiagnosticsState; @@ -63,8 +70,9 @@ export type AdminUiState = { jobs: JobsState; job: JobState; clusterLocks: ClusterLocksReqState; - insights: InsightsState; - insightDetails: InsightDetailsState; + transactionInsights: TransactionInsightsState; + transactionInsightDetails: TransactionInsightDetailsState; + statementInsights: StatementInsightsState; schemaInsights: SchemaInsightsState; }; @@ -78,8 +86,9 @@ export const reducers = combineReducers({ nodes, liveness, sessions, - insights, - insightDetails, + transactionInsights, + transactionInsightDetails, + statementInsights, terminateQuery, uiConfig, sqlStats, diff --git a/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts index 8de25ef346ee..7a0ad282634a 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts @@ -24,8 +24,9 @@ import { sqlStatsSaga } from "./sqlStats"; import { sqlDetailsStatsSaga } from "./statementDetails"; import { indexStatsSaga } from "./indexStats"; import { clusterLocksSaga } from "./clusterLocks/clusterLocks.saga"; -import { transactionInsightsSaga } from "./insights"; -import { transactionInsightDetailsSaga } from "./insightDetails"; +import { transactionInsightsSaga } from "./insights/transactionInsights"; +import { transactionInsightDetailsSaga } from "./insightDetails/transactionInsightDetails"; +import { statementInsightsSaga } from "./insights/statementInsights"; import { schemaInsightsSaga } from "./schemaInsights"; export function* sagas(cacheInvalidationPeriod?: number): SagaIterator { @@ -36,6 +37,7 @@ export function* sagas(cacheInvalidationPeriod?: number): SagaIterator { fork(livenessSaga, cacheInvalidationPeriod), fork(transactionInsightsSaga), fork(transactionInsightDetailsSaga), + fork(statementInsightsSaga), fork(jobsSaga), fork(jobSaga), fork(sessionsSaga), diff --git a/pkg/ui/workspaces/cluster-ui/src/store/schemaInsights/schemaInsights.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/schemaInsights/schemaInsights.selectors.ts index 8266fc36a761..e4462deea97b 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/schemaInsights/schemaInsights.selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/schemaInsights/schemaInsights.selectors.ts @@ -9,14 +9,30 @@ // licenses/APL.txt. import { createSelector } from "reselect"; -import { adminUISelector } from "../utils/selectors"; +import { adminUISelector, localStorageSelector } from "../utils/selectors"; import { insightType } from "../../insights"; -export const selectSchemaInsights = createSelector( +const selectSchemaInsightState = createSelector( adminUISelector, adminUiState => { - if (!adminUiState.schemaInsights) return []; - return adminUiState.schemaInsights.data; + if (!adminUiState.schemaInsights) return null; + return adminUiState.schemaInsights; + }, +); + +export const selectSchemaInsights = createSelector( + selectSchemaInsightState, + schemaInsightState => { + if (!schemaInsightState.data) return null; + return schemaInsightState.data; + }, +); + +export const selectSchemaInsightsError = createSelector( + selectSchemaInsightState, + schemaInsightState => { + if (!schemaInsightState) return null; + return schemaInsightState.lastError; }, ); @@ -41,3 +57,13 @@ export const selectSchemaInsightsTypes = createSelector( ).sort(); }, ); + +export const selectSortSetting = createSelector( + localStorageSelector, + localStorage => localStorage["sortSetting/SchemaInsightsPage"], +); + +export const selectFilters = createSelector( + localStorageSelector, + localStorage => localStorage["filters/SchemaInsightsPage"], +); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/statementDetails/statementDetails.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/statementDetails/statementDetails.reducer.ts index 2b09150d4383..400b79576d35 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/statementDetails/statementDetails.reducer.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/statementDetails/statementDetails.reducer.ts @@ -29,14 +29,10 @@ export type SQLDetailsStatsReducerState = { cachedData: { [id: string]: SQLDetailsStatsState; }; - latestQuery: string; - latestFormattedQuery: string; }; const initialState: SQLDetailsStatsReducerState = { cachedData: {}, - latestQuery: "", - latestFormattedQuery: "", }; const sqlDetailsStatsSlice = createSlice({ @@ -103,12 +99,6 @@ const sqlDetailsStatsSlice = createSlice({ inFlight: true, }; }, - setLatestQuery: (state, action: PayloadAction) => { - state.latestQuery = action.payload; - }, - setLatestFormattedQuery: (state, action: PayloadAction) => { - state.latestFormattedQuery = action.payload; - }, }, }); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/statementDetails/statementDetails.sagas.spec.ts b/pkg/ui/workspaces/cluster-ui/src/store/statementDetails/statementDetails.sagas.spec.ts index f2af2843d0a2..0f12e1fb7eba 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/statementDetails/statementDetails.sagas.spec.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/statementDetails/statementDetails.sagas.spec.ts @@ -667,8 +667,6 @@ describe("SQLDetailsStats sagas", () => { inFlight: false, }, }, - latestQuery: "", - latestFormattedQuery: "", }) .run(); }); @@ -694,8 +692,6 @@ describe("SQLDetailsStats sagas", () => { inFlight: false, }, }, - latestQuery: "", - latestFormattedQuery: "", }) .run(); }); diff --git a/pkg/ui/workspaces/db-console/src/app.spec.tsx b/pkg/ui/workspaces/db-console/src/app.spec.tsx index 8a9793d8abce..efb5d1b2f3d4 100644 --- a/pkg/ui/workspaces/db-console/src/app.spec.tsx +++ b/pkg/ui/workspaces/db-console/src/app.spec.tsx @@ -31,22 +31,16 @@ stubComponentInModule( "src/views/transactions/activeTransactionDetailsConnected", "default", ); +stubComponentInModule("src/views/insights/workloadInsightsPage", "default"); stubComponentInModule( - "src/views/insights/workloadInsightsPageConnected", + "src/views/insights/transactionInsightDetailsPage", "default", ); stubComponentInModule( - "src/views/insights/transactionInsightDetailsPageConnected", - "default", -); -stubComponentInModule( - "src/views/insights/statementInsightDetailsPageConnected", - "default", -); -stubComponentInModule( - "src/views/insights/schemaInsightsPageConnected", + "src/views/insights/statementInsightDetailsPage", "default", ); +stubComponentInModule("src/views/insights/schemaInsightsPage", "default"); stubComponentInModule("src/views/schedules/schedulesPage", "default"); stubComponentInModule("src/views/schedules/scheduleDetails", "default"); @@ -451,23 +445,23 @@ describe("Routing to", () => { describe("'/insights' path", () => { test("routes to component - workload insights page", () => { navigateToPath("/insights"); - screen.getByTestId("workloadInsightsPageConnected"); + screen.getByTestId("workloadInsightsPage"); }); test("routes to component - schema insights page", () => { navigateToPath("/insights?tab=Schema+Insights"); - screen.getByTestId("schemaInsightsPageConnected"); + screen.getByTestId("schemaInsightsPage"); }); }); describe("'/insights/transaction/insightID' path", () => { - test("routes to component", () => { + test("routes to component", () => { navigateToPath("/insights/transaction/insightID"); - screen.getByTestId("transactionInsightDetailsPageConnected"); + screen.getByTestId("transactionInsightDetailsPage"); }); }); describe("'/insights/statement/insightID' path", () => { - test("routes to component", () => { + test("routes to component", () => { navigateToPath("/insights/statement/insightID"); - screen.getByTestId("statementInsightDetailsPageConnected"); + screen.getByTestId("statementInsightDetailsPage"); }); }); { diff --git a/pkg/ui/workspaces/db-console/src/app.tsx b/pkg/ui/workspaces/db-console/src/app.tsx index 3cd8310128fb..aaa89e44cb3b 100644 --- a/pkg/ui/workspaces/db-console/src/app.tsx +++ b/pkg/ui/workspaces/db-console/src/app.tsx @@ -80,9 +80,9 @@ import ActiveStatementDetails from "./views/statements/activeStatementDetailsCon import ActiveTransactionDetails from "./views/transactions/activeTransactionDetailsConnected"; import "styl/app.styl"; import { Tracez } from "src/views/tracez/tracez"; -import InsightsOverviewPage from "src/views/insights/insightsOverview"; -import TransactionInsightDetailsPageConnected from "src/views/insights/transactionInsightDetailsPageConnected"; -import StatementInsightDetailsPageConnected from "src/views/insights/statementInsightDetailsPageConnected"; +import InsightsOverviewPage from "./views/insights/insightsOverview"; +import TransactionInsightDetailsPage from "./views/insights/transactionInsightDetailsPage"; +import StatementInsightDetailsPage from "./views/insights/statementInsightDetailsPage"; import { CockroachCloudContext } from "@cockroachlabs/cluster-ui"; // NOTE: If you are adding a new path to the router, and that path contains any @@ -312,11 +312,11 @@ export const App: React.FC = (props: AppProps) => { /> {/* debug pages */} diff --git a/pkg/ui/workspaces/db-console/src/redux/sqlActivity.ts b/pkg/ui/workspaces/db-console/src/redux/sqlActivity.ts deleted file mode 100644 index 2fbc7939998d..000000000000 --- a/pkg/ui/workspaces/db-console/src/redux/sqlActivity.ts +++ /dev/null @@ -1,78 +0,0 @@ -// 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. -import { Action } from "redux"; -import _ from "lodash"; -import { PayloadAction } from "src/interfaces/action"; - -/** - * SqlActivityState maintains a MetricQuerySet collection, along with some - * metadata relevant to server queries. - */ -export class SqlActivityState { - /** - * Caches the latest query text associated with the statement fingerprint in the URL, to preserve this data even if - * the time frame changes such that there is no longer data for this statement fingerprint in the selected time frame. - */ - statementDetailsLatestQuery: string; - /** - * Caches the latest formatted query associated with the statement fingerprint in the URL, to preserve this data even if - * the time frame changes such that there is no longer data for this statement fingerprint in the selected time frame. - */ - statementDetailsLatestFormattedQuery: string; -} - -const SET_STATEMENT_DETAILS_LATEST_QUERY = - "cockroachui/sqlActivity/SET_STATEMENT_DETAILS_LATEST_QUERY"; -const SET_STATEMENT_DETAILS_LATEST_FORMATTED_QUERY = - "cockroachui/sqlActivity/SET_STATEMENT_DETAILS_LATEST_FORMATTED_QUERY"; - -export function statementDetailsLatestQueryAction( - query: string, -): PayloadAction { - return { - type: SET_STATEMENT_DETAILS_LATEST_QUERY, - payload: query, - }; -} -export function statementDetailsLatestFormattedQueryAction( - formattedQuery: string, -): PayloadAction { - return { - type: SET_STATEMENT_DETAILS_LATEST_FORMATTED_QUERY, - payload: formattedQuery, - }; -} - -/** - * The metrics reducer accepts events for individual MetricQuery objects, - * dispatching them based on ID. It also accepts actions which indicate the - * state of the connection to the server. - */ -export function sqlActivityReducer( - state: SqlActivityState = new SqlActivityState(), - action: Action, -): SqlActivityState { - switch (action.type) { - case SET_STATEMENT_DETAILS_LATEST_QUERY: { - const { payload: query } = action as PayloadAction; - state = _.clone(state); - state.statementDetailsLatestQuery = query; - return state; - } - case SET_STATEMENT_DETAILS_LATEST_FORMATTED_QUERY: { - const { payload: formattedQuery } = action as PayloadAction; - state = _.clone(state); - state.statementDetailsLatestFormattedQuery = formattedQuery; - return state; - } - default: - return state; - } -} diff --git a/pkg/ui/workspaces/db-console/src/redux/state.ts b/pkg/ui/workspaces/db-console/src/redux/state.ts index 661eff51d8fa..9e716ffa0000 100644 --- a/pkg/ui/workspaces/db-console/src/redux/state.ts +++ b/pkg/ui/workspaces/db-console/src/redux/state.ts @@ -31,7 +31,6 @@ import { apiReducersReducer, APIReducersState } from "./apiReducers"; import { hoverReducer, HoverState } from "./hover"; import { localSettingsReducer, LocalSettingsState } from "./localsettings"; import { metricsReducer, MetricsState } from "./metrics"; -import { sqlActivityReducer, SqlActivityState } from "src/redux/sqlActivity"; import { queryManagerReducer, QueryManagerState } from "./queryManager/reducer"; import { timeScaleReducer, TimeScaleState } from "./timeScale"; import { uiDataReducer, UIDataState } from "./uiData"; @@ -44,7 +43,6 @@ export interface AdminUIState { hover: HoverState; localSettings: LocalSettingsState; metrics: MetricsState; - sqlActivity: SqlActivityState; queryManager: QueryManagerState; router: RouterState; @@ -65,7 +63,6 @@ export function createAdminUIStore(historyInst: History) { hover: hoverReducer, localSettings: localSettingsReducer, metrics: metricsReducer, - sqlActivity: sqlActivityReducer, queryManager: queryManagerReducer, router: routerReducer, diff --git a/pkg/ui/workspaces/db-console/src/views/insights/insightsOverview.tsx b/pkg/ui/workspaces/db-console/src/views/insights/insightsOverview.tsx index fc401f659a3b..19845bf762f7 100644 --- a/pkg/ui/workspaces/db-console/src/views/insights/insightsOverview.tsx +++ b/pkg/ui/workspaces/db-console/src/views/insights/insightsOverview.tsx @@ -18,8 +18,8 @@ import "antd/lib/tabs/style"; import { commonStyles, util } from "@cockroachlabs/cluster-ui"; import { RouteComponentProps } from "react-router-dom"; import { tabAttr, viewAttr } from "src/util/constants"; -import WorkloadInsightsPageConnected from "src/views/insights/workloadInsightsPageConnected"; -import SchemaInsightsPageConnected from "src/views/insights/schemaInsightsPageConnected"; +import WorkloadInsightsPage from "./workloadInsightsPage"; +import SchemaInsightsPage from "./schemaInsightsPage"; const { TabPane } = Tabs; @@ -63,10 +63,10 @@ const InsightsOverviewPage = (props: RouteComponentProps) => { destroyInactiveTabPane > - + - + diff --git a/pkg/ui/workspaces/db-console/src/views/insights/insightsSelectors.ts b/pkg/ui/workspaces/db-console/src/views/insights/insightsSelectors.ts index 64cd53723c60..a40c717b9e88 100644 --- a/pkg/ui/workspaces/db-console/src/views/insights/insightsSelectors.ts +++ b/pkg/ui/workspaces/db-console/src/views/insights/insightsSelectors.ts @@ -28,7 +28,7 @@ export const filtersLocalSetting = new LocalSetting< AdminUIState, WorkloadInsightEventFilters >("filters/InsightsPage", (state: AdminUIState) => state.localSettings, { - app: defaultFilters.app, + app: "", }); export const sortSettingLocalSetting = new LocalSetting< diff --git a/pkg/ui/workspaces/db-console/src/views/insights/schemaInsightsPageConnected.tsx b/pkg/ui/workspaces/db-console/src/views/insights/schemaInsightsPage.tsx similarity index 95% rename from pkg/ui/workspaces/db-console/src/views/insights/schemaInsightsPageConnected.tsx rename to pkg/ui/workspaces/db-console/src/views/insights/schemaInsightsPage.tsx index 1131c72d5a14..d057e5556d26 100644 --- a/pkg/ui/workspaces/db-console/src/views/insights/schemaInsightsPageConnected.tsx +++ b/pkg/ui/workspaces/db-console/src/views/insights/schemaInsightsPage.tsx @@ -46,7 +46,7 @@ const mapDispatchToProps = { refreshSchemaInsights: refreshSchemaInsights, }; -const SchemaInsightsPageConnected = withRouter( +const SchemaInsightsPage = withRouter( connect< SchemaInsightsViewStateProps, SchemaInsightsViewDispatchProps, @@ -57,4 +57,4 @@ const SchemaInsightsPageConnected = withRouter( )(SchemaInsightsView), ); -export default SchemaInsightsPageConnected; +export default SchemaInsightsPage; diff --git a/pkg/ui/workspaces/db-console/src/views/insights/statementInsightDetailsPageConnected.tsx b/pkg/ui/workspaces/db-console/src/views/insights/statementInsightDetailsPage.tsx similarity index 91% rename from pkg/ui/workspaces/db-console/src/views/insights/statementInsightDetailsPageConnected.tsx rename to pkg/ui/workspaces/db-console/src/views/insights/statementInsightDetailsPage.tsx index 864a8b550c63..86d16c15d642 100644 --- a/pkg/ui/workspaces/db-console/src/views/insights/statementInsightDetailsPageConnected.tsx +++ b/pkg/ui/workspaces/db-console/src/views/insights/statementInsightDetailsPage.tsx @@ -28,10 +28,10 @@ const mapStateToProps = ( }; }; -const StatementInsightDetailsPageConnected = withRouter( +const StatementInsightDetailsPage = withRouter( connect( mapStateToProps, )(StatementInsightDetails), ); -export default StatementInsightDetailsPageConnected; +export default StatementInsightDetailsPage; diff --git a/pkg/ui/workspaces/db-console/src/views/insights/transactionInsightDetailsPageConnected.tsx b/pkg/ui/workspaces/db-console/src/views/insights/transactionInsightDetailsPage.tsx similarity index 93% rename from pkg/ui/workspaces/db-console/src/views/insights/transactionInsightDetailsPageConnected.tsx rename to pkg/ui/workspaces/db-console/src/views/insights/transactionInsightDetailsPage.tsx index f197c43d3920..12401e70f5eb 100644 --- a/pkg/ui/workspaces/db-console/src/views/insights/transactionInsightDetailsPageConnected.tsx +++ b/pkg/ui/workspaces/db-console/src/views/insights/transactionInsightDetailsPage.tsx @@ -37,7 +37,7 @@ const mapDispatchToProps = { refreshTransactionInsightDetails: refreshTransactionInsightDetails, }; -const TransactionInsightDetailsPageConnected = withRouter( +const TransactionInsightDetailsPage = withRouter( connect< TransactionInsightDetailsStateProps, TransactionInsightDetailsDispatchProps, @@ -48,4 +48,4 @@ const TransactionInsightDetailsPageConnected = withRouter( )(TransactionInsightDetails), ); -export default TransactionInsightDetailsPageConnected; +export default TransactionInsightDetailsPage; diff --git a/pkg/ui/workspaces/db-console/src/views/insights/workloadInsightsPageConnected.tsx b/pkg/ui/workspaces/db-console/src/views/insights/workloadInsightsPage.tsx similarity index 97% rename from pkg/ui/workspaces/db-console/src/views/insights/workloadInsightsPageConnected.tsx rename to pkg/ui/workspaces/db-console/src/views/insights/workloadInsightsPage.tsx index e12a92efa928..45ff91ba834b 100644 --- a/pkg/ui/workspaces/db-console/src/views/insights/workloadInsightsPageConnected.tsx +++ b/pkg/ui/workspaces/db-console/src/views/insights/workloadInsightsPage.tsx @@ -91,7 +91,7 @@ type DispatchProps = { statementInsightsViewDispatchProps: StatementInsightsViewDispatchProps; }; -const WorkloadInsightsPageConnected = withRouter( +const WorkloadInsightsPage = withRouter( connect< StateProps, DispatchProps, @@ -128,4 +128,4 @@ const WorkloadInsightsPageConnected = withRouter( )(WorkloadInsightsRootControl), ); -export default WorkloadInsightsPageConnected; +export default WorkloadInsightsPage; diff --git a/pkg/ui/workspaces/db-console/src/views/statements/statementDetails.tsx b/pkg/ui/workspaces/db-console/src/views/statements/statementDetails.tsx index 96504356345f..a7c66a0ce973 100644 --- a/pkg/ui/workspaces/db-console/src/views/statements/statementDetails.tsx +++ b/pkg/ui/workspaces/db-console/src/views/statements/statementDetails.tsx @@ -50,10 +50,6 @@ import { StatementDetailsResponseMessage } from "src/util/api"; import { getMatchParamByName, queryByName } from "src/util/query"; import { appNamesAttr, statementAttr } from "src/util/constants"; -import { - statementDetailsLatestQueryAction, - statementDetailsLatestFormattedQueryAction, -} from "src/redux/sqlActivity"; import { selectTimeScale } from "src/redux/timeScale"; type IStatementDiagnosticsReport = @@ -109,20 +105,18 @@ const mapStateToProps = ( state, props, ); + const statementFingerprint = statementDetails?.statement.metadata.query; return { statementFingerprintID: getMatchParamByName(props.match, statementAttr), statementDetails, isLoading: isLoading, - latestQuery: state.sqlActivity.statementDetailsLatestQuery, - latestFormattedQuery: - state.sqlActivity.statementDetailsLatestFormattedQuery, statementsError: lastError, timeScale: selectTimeScale(state), nodeNames: nodeDisplayNameByIDSelector(state), nodeRegions: nodeRegionsByIDSelector(state), diagnosticsReports: selectDiagnosticsReportsByStatementFingerprint( state, - state.sqlActivity.statementDetailsLatestQuery, + statementFingerprint, ), hasViewActivityRedactedRole: selectHasViewActivityRedactedRole(state), }; @@ -145,9 +139,6 @@ const mapDispatchToProps: StatementDetailsDispatchProps = { ); }; }, - onStatementDetailsQueryChange: statementDetailsLatestQueryAction, - onStatementDetailsFormattedQueryChange: - statementDetailsLatestFormattedQueryAction, refreshNodes: refreshNodes, refreshNodesLiveness: refreshLiveness, refreshUserSQLRoles: refreshUserSQLRoles,