diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index 215dbcc4f884..60ef6ca5260e 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -3932,6 +3932,8 @@ tenant pods. | last_reset | [google.protobuf.Timestamp](#cockroach.server.serverpb.StatementsResponse-google.protobuf.Timestamp) | | Timestamp of the last stats reset. | [reserved](#support-status) | | internal_app_name_prefix | [string](#cockroach.server.serverpb.StatementsResponse-string) | | If set and non-empty, indicates the prefix to application_name used for statements/queries issued internally by CockroachDB. | [reserved](#support-status) | | transactions | [StatementsResponse.ExtendedCollectedTransactionStatistics](#cockroach.server.serverpb.StatementsResponse-cockroach.server.serverpb.StatementsResponse.ExtendedCollectedTransactionStatistics) | repeated | Transactions is transaction-level statistics for the collection of statements in this response. | [reserved](#support-status) | +| stmts_total_runtime_secs | [float](#cockroach.server.serverpb.StatementsResponse-float) | | | [reserved](#support-status) | +| txns_total_runtime_secs | [float](#cockroach.server.serverpb.StatementsResponse-float) | | | [reserved](#support-status) | @@ -4004,12 +4006,28 @@ Support status: [reserved](#support-status) | ----- | ---- | ----- | ----------- | -------------- | | start | [int64](#cockroach.server.serverpb.CombinedStatementsStatsRequest-int64) | | Unix time range for aggregated statements. | [reserved](#support-status) | | end | [int64](#cockroach.server.serverpb.CombinedStatementsStatsRequest-int64) | | | [reserved](#support-status) | +| fetch_mode | [CombinedStatementsStatsRequest.FetchMode](#cockroach.server.serverpb.CombinedStatementsStatsRequest-cockroach.server.serverpb.CombinedStatementsStatsRequest.FetchMode) | | Note that if fetch_mode is set to transactions only, we will also include the statement statistics for the stmts in the transactions response. This is more of a hack-y method to get the complete stats for txns, because in the client we need to fill in some txn stats info from its stmt stats, such as the query string.

We prefer this hackier method right now to reduce surface area for backporting these changes, but in the future we will introduce more endpoints to properly organize these differing requests. TODO (xinhaoz) - Split this API into stmts and txns properly instead of using this param. | [reserved](#support-status) | +| limit | [int64](#cockroach.server.serverpb.CombinedStatementsStatsRequest-int64) | | | [reserved](#support-status) | + +#### CombinedStatementsStatsRequest.FetchMode + + + +| Field | Type | Label | Description | Support status | +| ----- | ---- | ----- | ----------- | -------------- | +| stats_type | [CombinedStatementsStatsRequest.StatsType](#cockroach.server.serverpb.CombinedStatementsStatsRequest-cockroach.server.serverpb.CombinedStatementsStatsRequest.StatsType) | | | [reserved](#support-status) | +| sort | [StatsSortOptions](#cockroach.server.serverpb.CombinedStatementsStatsRequest-cockroach.server.serverpb.StatsSortOptions) | | | [reserved](#support-status) | + + + + + #### Response Parameters @@ -4025,6 +4043,8 @@ Support status: [reserved](#support-status) | last_reset | [google.protobuf.Timestamp](#cockroach.server.serverpb.StatementsResponse-google.protobuf.Timestamp) | | Timestamp of the last stats reset. | [reserved](#support-status) | | internal_app_name_prefix | [string](#cockroach.server.serverpb.StatementsResponse-string) | | If set and non-empty, indicates the prefix to application_name used for statements/queries issued internally by CockroachDB. | [reserved](#support-status) | | transactions | [StatementsResponse.ExtendedCollectedTransactionStatistics](#cockroach.server.serverpb.StatementsResponse-cockroach.server.serverpb.StatementsResponse.ExtendedCollectedTransactionStatistics) | repeated | Transactions is transaction-level statistics for the collection of statements in this response. | [reserved](#support-status) | +| stmts_total_runtime_secs | [float](#cockroach.server.serverpb.StatementsResponse-float) | | | [reserved](#support-status) | +| txns_total_runtime_secs | [float](#cockroach.server.serverpb.StatementsResponse-float) | | | [reserved](#support-status) | diff --git a/pkg/ccl/serverccl/statusccl/tenant_status_test.go b/pkg/ccl/serverccl/statusccl/tenant_status_test.go index 67e2264c8739..0eef15cd0bfb 100644 --- a/pkg/ccl/serverccl/statusccl/tenant_status_test.go +++ b/pkg/ccl/serverccl/statusccl/tenant_status_test.go @@ -187,11 +187,9 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) { require.NoError(t, err) request := &serverpb.StatementsRequest{} - combinedStatsRequest := &serverpb.CombinedStatementsStatsRequest{} var tenantStats *serverpb.StatementsResponse - var tenantCombinedStats *serverpb.StatementsResponse - // Populate `tenantStats` and `tenantCombinedStats`. The tenant server + // Populate `tenantStats`. The tenant server // `Statements` and `CombinedStatements` methods are backed by the // sqlinstance system which uses a cache populated through rangefeed // for keeping track of SQL pod data. We use `SucceedsSoon` to eliminate @@ -206,10 +204,6 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) { return errors.New("tenant statements are unexpectedly empty") } - tenantCombinedStats, err = tenantStatusServer.CombinedStatementStats(ctx, combinedStatsRequest) - if tenantCombinedStats == nil || len(tenantCombinedStats.Statements) == 0 { - return errors.New("tenant combined statements are unexpectedly empty") - } return nil }) @@ -218,11 +212,6 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) { err = serverutils.GetJSONProto(nonTenant, path, &nonTenantStats) require.NoError(t, err) - path = "/_status/combinedstmts" - var nonTenantCombinedStats serverpb.StatementsResponse - err = serverutils.GetJSONProto(nonTenant, path, &nonTenantCombinedStats) - require.NoError(t, err) - checkStatements := func(t *testing.T, tc []testCase, actual *serverpb.StatementsResponse) { t.Helper() var expectedStatements []string @@ -258,13 +247,11 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) { // First we verify that we have expected stats from tenants. t.Run("tenant-stats", func(t *testing.T) { checkStatements(t, testCaseTenant, tenantStats) - checkStatements(t, testCaseTenant, tenantCombinedStats) }) // Now we verify the non tenant stats are what we expected. t.Run("non-tenant-stats", func(t *testing.T) { checkStatements(t, testCaseNonTenant, &nonTenantStats) - checkStatements(t, testCaseNonTenant, &nonTenantCombinedStats) }) // Now we verify that tenant and non-tenant have no visibility into each other's stats. @@ -281,17 +268,6 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) { } } - for _, tenantStmt := range tenantCombinedStats.Statements { - for _, nonTenantStmt := range nonTenantCombinedStats.Statements { - require.NotEqual(t, tenantStmt, nonTenantStmt, "expected tenant to have no visibility to non-tenant's statement stats, but found:", nonTenantStmt) - } - } - - for _, tenantTxn := range tenantCombinedStats.Transactions { - for _, nonTenantTxn := range nonTenantCombinedStats.Transactions { - require.NotEqual(t, tenantTxn, nonTenantTxn, "expected tenant to have no visibility to non-tenant's transaction stats, but found:", nonTenantTxn) - } - } }) } @@ -307,43 +283,46 @@ func testResetSQLStatsRPCForTenant( testCluster := testHelper.TestCluster() controlCluster := testHelper.ControlCluster() - // Disable automatic flush to ensure tests are deterministic. + // Set automatic flush to some long duration we'll never hit to + // ensure tests are deterministic. testCluster.TenantConn(0 /* idx */). - Exec(t, "SET CLUSTER SETTING sql.stats.flush.enabled = false") + Exec(t, "SET CLUSTER SETTING sql.stats.flush.interval = '24h'") controlCluster.TenantConn(0 /* idx */). - Exec(t, "SET CLUSTER SETTING sql.stats.flush.enabled = false") + Exec(t, "SET CLUSTER SETTING sql.stats.flush.interval = '24h'") defer func() { // Cleanup testCluster.TenantConn(0 /* idx */). - Exec(t, "SET CLUSTER SETTING sql.stats.flush.enabled = true") + Exec(t, "SET CLUSTER SETTING sql.stats.flush.interval = '10m'") controlCluster.TenantConn(0 /* idx */). - Exec(t, "SET CLUSTER SETTING sql.stats.flush.enabled = true") + Exec(t, "SET CLUSTER SETTING sql.stats.flush.interval = '10m'") }() for _, flushed := range []bool{false, true} { + testTenant := testCluster.Tenant(serverccl.RandomServer) + testTenantConn := testTenant.GetTenantConn() t.Run(fmt.Sprintf("flushed=%t", flushed), func(t *testing.T) { // Clears the SQL Stats at the end of each test via builtin. defer func() { - testCluster.TenantConn(serverccl.RandomServer).Exec(t, "SELECT crdb_internal.reset_sql_stats()") + testTenantConn.Exec(t, "SELECT crdb_internal.reset_sql_stats()") controlCluster.TenantConn(serverccl.RandomServer).Exec(t, "SELECT crdb_internal.reset_sql_stats()") }() for _, stmt := range stmts { - testCluster.TenantConn(serverccl.RandomServer).Exec(t, stmt) + testTenantConn.Exec(t, stmt) controlCluster.TenantConn(serverccl.RandomServer).Exec(t, stmt) } if flushed { - testCluster.TenantSQLStats(serverccl.RandomServer).Flush(ctx) + testTenant.TenantSQLStats().Flush(ctx) controlCluster.TenantSQLStats(serverccl.RandomServer).Flush(ctx) } - status := testCluster.TenantStatusSrv(serverccl.RandomServer) + status := testTenant.TenantStatusSrv() statsPreReset, err := status.Statements(ctx, &serverpb.StatementsRequest{ - Combined: true, + Combined: flushed, }) require.NoError(t, err) @@ -357,7 +336,7 @@ func testResetSQLStatsRPCForTenant( require.NoError(t, err) statsPostReset, err := status.Statements(ctx, &serverpb.StatementsRequest{ - Combined: true, + Combined: flushed, }) require.NoError(t, err) @@ -382,7 +361,7 @@ func testResetSQLStatsRPCForTenant( // Ensures that sql stats reset is isolated by tenant boundary. statsFromControlCluster, err := controlCluster.TenantStatusSrv(serverccl.RandomServer).Statements(ctx, &serverpb.StatementsRequest{ - Combined: true, + Combined: flushed, }) require.NoError(t, err) diff --git a/pkg/server/combined_statement_stats.go b/pkg/server/combined_statement_stats.go index 3d4c4b1284ac..4e4663285a41 100644 --- a/pkg/server/combined_statement_stats.go +++ b/pkg/server/combined_statement_stats.go @@ -67,18 +67,36 @@ func getCombinedStatementStats( settings *cluster.Settings, testingKnobs *sqlstats.TestingKnobs, ) (*serverpb.StatementsResponse, error) { - startTime := getTimeFromSeconds(req.Start) - endTime := getTimeFromSeconds(req.End) - limit := SQLStatsResponseMax.Get(&settings.SV) showInternal := SQLStatsShowInternal.Get(&settings.SV) whereClause, orderAndLimit, args := getCombinedStatementsQueryClausesAndArgs( - startTime, endTime, limit, testingKnobs, showInternal) - statements, err := collectCombinedStatements(ctx, ie, whereClause, args, orderAndLimit) + req, testingKnobs, showInternal, settings) + + var statements []serverpb.StatementsResponse_CollectedStatementStatistics + var transactions []serverpb.StatementsResponse_ExtendedCollectedTransactionStatistics + var err error + + if req.FetchMode == nil || req.FetchMode.StatsType == serverpb.CombinedStatementsStatsRequest_TxnStatsOnly { + transactions, err = collectCombinedTransactions(ctx, ie, whereClause, args, orderAndLimit, testingKnobs) + if err != nil { + return nil, serverError(ctx, err) + } + } + + if req.FetchMode != nil && req.FetchMode.StatsType == serverpb.CombinedStatementsStatsRequest_TxnStatsOnly { + // Change the whereClause for the statements to those matching the txn_fingerprint_ids in the + // transactions response that are within the desired interval. We also don't need the order and + // limit anymore. + orderAndLimit = "" + whereClause, args = buildWhereClauseForStmtsByTxn(req, transactions, testingKnobs) + } + + statements, err = collectCombinedStatements(ctx, ie, whereClause, args, orderAndLimit, testingKnobs) if err != nil { return nil, serverError(ctx, err) } - transactions, err := collectCombinedTransactions(ctx, ie, whereClause, args, orderAndLimit) + stmtsRunTime, txnsRunTime, err := getTotalRuntimeSecs(ctx, req, ie, testingKnobs) + if err != nil { return nil, serverError(ctx, err) } @@ -88,11 +106,181 @@ func getCombinedStatementStats( Transactions: transactions, LastReset: statsProvider.GetLastReset(), InternalAppNamePrefix: catconstants.InternalAppNamePrefix, + StmtsTotalRuntimeSecs: stmtsRunTime, + TxnsTotalRuntimeSecs: txnsRunTime, } return response, nil } +func getTotalRuntimeSecs( + ctx context.Context, + req *serverpb.CombinedStatementsStatsRequest, + ie *sql.InternalExecutor, + testingKnobs *sqlstats.TestingKnobs, +) (stmtsRuntime float32, txnsRuntime float32, err error) { + var buffer strings.Builder + buffer.WriteString(testingKnobs.GetAOSTClause()) + var args []interface{} + startTime := getTimeFromSeconds(req.Start) + endTime := getTimeFromSeconds(req.End) + + buffer.WriteString(" WHERE true") + + if startTime != nil { + args = append(args, *startTime) + buffer.WriteString(fmt.Sprintf(" AND aggregated_ts >= $%d", len(args))) + } + + if endTime != nil { + args = append(args, *endTime) + buffer.WriteString(fmt.Sprintf(" AND aggregated_ts <= $%d", len(args))) + } + + whereClause := buffer.String() + + queryWithPlaceholders := ` +SELECT +COALESCE( + sum( + (statistics -> 'statistics' -> 'svcLat' ->> 'mean')::FLOAT * + (statistics-> 'statistics' ->> 'cnt')::FLOAT + ) +, 0) +FROM crdb_internal.%s_statistics_persisted +%s +` + + getRuntime := func(table string) (float32, error) { + it, err := ie.QueryIteratorEx( + ctx, + fmt.Sprintf(`%s-total-runtime`, table), + nil, + sessiondata.NodeUserSessionDataOverride, + fmt.Sprintf(queryWithPlaceholders, table, whereClause), + args...) + + if err != nil { + return 0, err + } + + defer func() { + closeErr := it.Close() + if closeErr != nil { + err = errors.CombineErrors(err, closeErr) + } + }() + + ok, err := it.Next(ctx) + if err != nil { + return 0, err + } + + if !ok { + return 0, errors.New("expected one row but got none") + } + + var row tree.Datums + if row = it.Cur(); row == nil { + return 0, errors.New("unexpected null row") + } + + return float32(tree.MustBeDFloat(row[0])), nil + + } + + if req.FetchMode == nil || req.FetchMode.StatsType != serverpb.CombinedStatementsStatsRequest_TxnStatsOnly { + stmtsRuntime, err = getRuntime("statement") + if err != nil { + return 0, 0, err + } + } + + if req.FetchMode == nil || req.FetchMode.StatsType != serverpb.CombinedStatementsStatsRequest_StmtStatsOnly { + txnsRuntime, err = getRuntime("transaction") + if err != nil { + return 0, 0, err + } + } + + return stmtsRuntime, txnsRuntime, err +} + +// Common stmt and txn columns to sort on. +const ( + sortSvcLatDesc = `(statistics -> 'statistics' -> 'svcLat' ->> 'mean')::FLOAT DESC` + sortExecCountDesc = `(statistics -> 'statistics' ->> 'cnt')::INT DESC` + sortContentionTimeDesc = `(statistics -> 'execution_statistics' -> 'contentionTime' ->> 'mean')::FLOAT DESC` + sortPCTRuntimeDesc = `((statistics -> 'statistics' -> 'svcLat' ->> 'mean')::FLOAT * + (statistics -> 'statistics' ->> 'cnt')::FLOAT) DESC` +) + +func getStmtColumnFromSortOption(sort serverpb.StatsSortOptions) string { + switch sort { + case serverpb.StatsSortOptions_SERVICE_LAT: + return sortSvcLatDesc + case serverpb.StatsSortOptions_EXECUTION_COUNT: + return sortExecCountDesc + case serverpb.StatsSortOptions_CONTENTION_TIME: + return sortContentionTimeDesc + default: + return sortSvcLatDesc + } +} + +func getTxnColumnFromSortOption(sort serverpb.StatsSortOptions) string { + switch sort { + case serverpb.StatsSortOptions_SERVICE_LAT: + return sortSvcLatDesc + case serverpb.StatsSortOptions_EXECUTION_COUNT: + return sortExecCountDesc + case serverpb.StatsSortOptions_CONTENTION_TIME: + return sortContentionTimeDesc + case serverpb.StatsSortOptions_PCT_RUNTIME: + return sortPCTRuntimeDesc + default: + return sortSvcLatDesc + } +} + +// buildWhereClauseForStmtsByTxn builds the where clause to get the statement +// stats based on a list of transactions. The list of transactions provided must +// contain no duplicate transaction fingerprint ids. +func buildWhereClauseForStmtsByTxn( + req *serverpb.CombinedStatementsStatsRequest, + transactions []serverpb.StatementsResponse_ExtendedCollectedTransactionStatistics, + testingKnobs *sqlstats.TestingKnobs, +) (whereClause string, args []interface{}) { + var buffer strings.Builder + buffer.WriteString(testingKnobs.GetAOSTClause()) + + buffer.WriteString(" WHERE true") + + // Add start and end filters from request. + startTime := getTimeFromSeconds(req.Start) + endTime := getTimeFromSeconds(req.End) + if startTime != nil { + args = append(args, *startTime) + buffer.WriteString(fmt.Sprintf(" AND aggregated_ts >= $%d", len(args))) + } + + if endTime != nil { + args = append(args, *endTime) + buffer.WriteString(fmt.Sprintf(" AND aggregated_ts <= $%d", len(args))) + } + + txnFingerprints := make([]string, 0, len(transactions)) + for i := range transactions { + fingerprint := uint64(transactions[i].StatsData.TransactionFingerprintID) + txnFingerprints = append(txnFingerprints, fmt.Sprintf("\\x%016x", fingerprint)) + } + + args = append(args, txnFingerprints) + buffer.WriteString(fmt.Sprintf(" AND transaction_fingerprint_id = any $%d", len(args))) + + return buffer.String(), args +} + // getCombinedStatementsQueryClausesAndArgs returns: // - where clause (filtering by name and aggregates_ts when defined) // - order and limit clause @@ -100,7 +288,10 @@ func getCombinedStatementStats( // The whereClause will be in the format `WHERE A = $1 AND B = $2` and // args will return the list of arguments in order that will replace the actual values. func getCombinedStatementsQueryClausesAndArgs( - start, end *time.Time, limit int64, testingKnobs *sqlstats.TestingKnobs, showInternal bool, + req *serverpb.CombinedStatementsStatsRequest, + testingKnobs *sqlstats.TestingKnobs, + showInternal bool, + settings *cluster.Settings, ) (whereClause string, orderAndLimitClause string, args []interface{}) { var buffer strings.Builder buffer.WriteString(testingKnobs.GetAOSTClause()) @@ -112,17 +303,37 @@ func getCombinedStatementsQueryClausesAndArgs( buffer.WriteString(fmt.Sprintf(" WHERE app_name NOT LIKE '%s%%'", catconstants.InternalAppNamePrefix)) } - if start != nil { + // Add start and end filters from request. + startTime := getTimeFromSeconds(req.Start) + endTime := getTimeFromSeconds(req.End) + if startTime != nil { buffer.WriteString(" AND aggregated_ts >= $1") - args = append(args, *start) + args = append(args, *startTime) } - if end != nil { - args = append(args, *end) + if endTime != nil { + args = append(args, *endTime) buffer.WriteString(fmt.Sprintf(" AND aggregated_ts <= $%d", len(args))) } + + // Add LIMIT from request. + limit := req.Limit + if limit == 0 { + limit = SQLStatsResponseMax.Get(&settings.SV) + } args = append(args, limit) - orderAndLimitClause = fmt.Sprintf(` ORDER BY aggregated_ts DESC LIMIT $%d`, len(args)) + + // Determine sort column. + var col string + if req.FetchMode == nil { + col = "fingerprint_id" + } else if req.FetchMode.StatsType == serverpb.CombinedStatementsStatsRequest_StmtStatsOnly { + col = getStmtColumnFromSortOption(req.FetchMode.Sort) + } else if req.FetchMode.StatsType == serverpb.CombinedStatementsStatsRequest_TxnStatsOnly { + col = getTxnColumnFromSortOption(req.FetchMode.Sort) + } + + orderAndLimitClause = fmt.Sprintf(` ORDER BY %s LIMIT $%d`, col, len(args)) return buffer.String(), orderAndLimitClause, args } @@ -133,28 +344,28 @@ func collectCombinedStatements( whereClause string, args []interface{}, orderAndLimit string, + testingKnobs *sqlstats.TestingKnobs, ) ([]serverpb.StatementsResponse_CollectedStatementStatistics, error) { + aostClause := testingKnobs.GetAOSTClause() + query := fmt.Sprintf(` +SELECT * FROM ( +SELECT + fingerprint_id, + transaction_fingerprint_id, + app_name, + max(aggregated_ts) as aggregated_ts, + metadata, + crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics +FROM crdb_internal.statement_statistics_persisted %s +GROUP BY + fingerprint_id, + transaction_fingerprint_id, + app_name, + metadata +) %s +%s`, whereClause, aostClause, orderAndLimit) - query := fmt.Sprintf( - `SELECT - fingerprint_id, - transaction_fingerprint_id, - app_name, - max(aggregated_ts) as aggregated_ts, - metadata, - crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics, - max(sampled_plan) AS sampled_plan, - aggregation_interval - FROM crdb_internal.statement_statistics %s - GROUP BY - fingerprint_id, - transaction_fingerprint_id, - app_name, - metadata, - aggregation_interval - %s`, whereClause, orderAndLimit) - - const expectedNumDatums = 8 + const expectedNumDatums = 6 it, err := ie.QueryIteratorEx(ctx, "combined-stmts-by-interval", nil, sessiondata.InternalExecutorOverride{ @@ -212,20 +423,10 @@ func collectCombinedStatements( return nil, serverError(ctx, err) } - planJSON := tree.MustBeDJSON(row[6]).JSON - plan, err := sqlstatsutil.JSONToExplainTreePlanNode(planJSON) - if err != nil { - return nil, serverError(ctx, err) - } - metadata.Stats.SensitiveInfo.MostRecentPlanDescription = *plan - - aggInterval := tree.MustBeDInterval(row[7]).Duration - stmt := serverpb.StatementsResponse_CollectedStatementStatistics{ Key: serverpb.StatementsResponse_ExtendedStatementStatisticsKey{ - KeyData: metadata.Key, - AggregatedTs: aggregatedTs, - AggregationInterval: time.Duration(aggInterval.Nanos()), + KeyData: metadata.Key, + AggregatedTs: aggregatedTs, }, ID: roachpb.StmtFingerprintID(statementFingerprintID), Stats: metadata.Stats, @@ -248,25 +449,27 @@ func collectCombinedTransactions( whereClause string, args []interface{}, orderAndLimit string, + testingKnobs *sqlstats.TestingKnobs, ) ([]serverpb.StatementsResponse_ExtendedCollectedTransactionStatistics, error) { + aostClause := testingKnobs.GetAOSTClause() + + query := fmt.Sprintf(` +SELECT * FROM ( +SELECT + app_name, + max(aggregated_ts) as aggregated_ts, + fingerprint_id, + metadata, + crdb_internal.merge_transaction_stats(array_agg(statistics)) AS statistics +FROM crdb_internal.transaction_statistics_persisted %s +GROUP BY + app_name, + fingerprint_id, + metadata +) %s +%s`, whereClause, aostClause, orderAndLimit) - query := fmt.Sprintf( - `SELECT - app_name, - max(aggregated_ts) as aggregated_ts, - fingerprint_id, - metadata, - crdb_internal.merge_transaction_stats(array_agg(statistics)) AS statistics, - aggregation_interval - FROM crdb_internal.transaction_statistics %s - GROUP BY - app_name, - fingerprint_id, - metadata, - aggregation_interval - %s`, whereClause, orderAndLimit) - - const expectedNumDatums = 6 + const expectedNumDatums = 5 it, err := ie.QueryIteratorEx(ctx, "combined-txns-by-interval", nil, sessiondata.InternalExecutorOverride{ @@ -314,15 +517,12 @@ func collectCombinedTransactions( return nil, serverError(ctx, err) } - aggInterval := tree.MustBeDInterval(row[5]).Duration - txnStats := serverpb.StatementsResponse_ExtendedCollectedTransactionStatistics{ StatsData: roachpb.CollectedTransactionStatistics{ StatementFingerprintIDs: metadata.StatementFingerprintIDs, App: app, Stats: metadata.Stats, AggregatedTs: aggregatedTs, - AggregationInterval: time.Duration(aggInterval.Nanos()), TransactionFingerprintID: roachpb.TransactionFingerprintID(fingerprintID), }, } @@ -482,18 +682,14 @@ func getTotalStatementDetails( query := fmt.Sprintf( `SELECT crdb_internal.merge_stats_metadata(array_agg(metadata)) AS metadata, - aggregation_interval, array_agg(app_name) as app_names, - crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics, - max(sampled_plan) as sampled_plan, - encode(fingerprint_id, 'hex') as fingerprint_id - FROM crdb_internal.statement_statistics %s + crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics + FROM crdb_internal.statement_statistics_persisted %s GROUP BY - aggregation_interval, fingerprint_id LIMIT 1`, whereClause) - const expectedNumDatums = 6 + const expectedNumDatums = 3 var statement serverpb.StatementDetailsResponse_CollectedStatementSummary row, err := ie.QueryRowEx(ctx, "combined-stmts-details-total", nil, @@ -519,34 +715,23 @@ func getTotalStatementDetails( return statement, serverError(ctx, err) } - aggInterval := tree.MustBeDInterval(row[1]).Duration - - apps := tree.MustBeDArray(row[2]) + apps := tree.MustBeDArray(row[1]) var appNames []string for _, s := range apps.Array { appNames = util.CombineUniqueString(appNames, []string{string(tree.MustBeDString(s))}) } aggregatedMetadata.AppNames = appNames - statsJSON := tree.MustBeDJSON(row[3]).JSON + statsJSON := tree.MustBeDJSON(row[2]).JSON if err = sqlstatsutil.DecodeStmtStatsStatisticsJSON(statsJSON, &statistics.Stats); err != nil { return statement, serverError(ctx, err) } - planJSON := tree.MustBeDJSON(row[4]).JSON - plan, err := sqlstatsutil.JSONToExplainTreePlanNode(planJSON) - if err != nil { - return statement, serverError(ctx, err) - } - statistics.Stats.SensitiveInfo.MostRecentPlanDescription = *plan - aggregatedMetadata.FormattedQuery = aggregatedMetadata.Query - aggregatedMetadata.FingerprintID = string(tree.MustBeDString(row[5])) statement = serverpb.StatementDetailsResponse_CollectedStatementSummary{ - Metadata: aggregatedMetadata, - AggregationInterval: time.Duration(aggInterval.Nanos()), - Stats: statistics.Stats, + Metadata: aggregatedMetadata, + Stats: statistics.Stats, } return statement, nil @@ -566,18 +751,15 @@ func getStatementDetailsPerAggregatedTs( `SELECT aggregated_ts, crdb_internal.merge_stats_metadata(array_agg(metadata)) AS metadata, - crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics, - max(sampled_plan) as sampled_plan, - aggregation_interval - FROM crdb_internal.statement_statistics %s + crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics + FROM crdb_internal.statement_statistics_persisted %s GROUP BY - aggregated_ts, - aggregation_interval + aggregated_ts ORDER BY aggregated_ts ASC LIMIT $%d`, whereClause, len(args)+1) args = append(args, limit) - const expectedNumDatums = 5 + const expectedNumDatums = 3 it, err := ie.QueryIteratorEx(ctx, "combined-stmts-details-by-aggregated-timestamp", nil, sessiondata.InternalExecutorOverride{ @@ -621,20 +803,10 @@ func getStatementDetailsPerAggregatedTs( return nil, serverError(ctx, err) } - planJSON := tree.MustBeDJSON(row[3]).JSON - plan, err := sqlstatsutil.JSONToExplainTreePlanNode(planJSON) - if err != nil { - return nil, serverError(ctx, err) - } - metadata.Stats.SensitiveInfo.MostRecentPlanDescription = *plan - - aggInterval := tree.MustBeDInterval(row[4]).Duration - stmt := serverpb.StatementDetailsResponse_CollectedStatementGroupedByAggregatedTs{ - AggregatedTs: aggregatedTs, - AggregationInterval: time.Duration(aggInterval.Nanos()), - Stats: metadata.Stats, - Metadata: aggregatedMetadata, + AggregatedTs: aggregatedTs, + Stats: metadata.Stats, + Metadata: aggregatedMetadata, } statements = append(statements, stmt) @@ -695,18 +867,15 @@ func getStatementDetailsPerPlanHash( plan_hash, (statistics -> 'statistics' -> 'planGists'->>0) as plan_gist, crdb_internal.merge_stats_metadata(array_agg(metadata)) AS metadata, - crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics, - max(sampled_plan) as sampled_plan, - aggregation_interval - FROM crdb_internal.statement_statistics %s + crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics + FROM crdb_internal.statement_statistics_persisted %s GROUP BY plan_hash, - plan_gist, - aggregation_interval + plan_gist LIMIT $%d`, whereClause, len(args)+1) args = append(args, limit) - const expectedNumDatums = 6 + const expectedNumDatums = 4 it, err := ie.QueryIteratorEx(ctx, "combined-stmts-details-by-plan-hash", nil, sessiondata.InternalExecutorOverride{ @@ -758,14 +927,6 @@ func getStatementDetailsPerPlanHash( return nil, serverError(ctx, err) } - planJSON := tree.MustBeDJSON(row[4]).JSON - plan, err := sqlstatsutil.JSONToExplainTreePlanNode(planJSON) - if err != nil { - return nil, serverError(ctx, err) - } - metadata.Stats.SensitiveInfo.MostRecentPlanDescription = *plan - aggInterval := tree.MustBeDInterval(row[5]).Duration - // A metadata is unique for each plan, meaning if any of the counts are greater than zero, // we can update the value of each count with the execution count of this plan hash to // have the correct count of each metric. @@ -784,11 +945,10 @@ func getStatementDetailsPerPlanHash( aggregatedMetadata.TotalCount = metadata.Stats.Count stmt := serverpb.StatementDetailsResponse_CollectedStatementGroupedByPlanHash{ - AggregationInterval: time.Duration(aggInterval.Nanos()), - ExplainPlan: explainPlan, - PlanHash: planHash, - Stats: metadata.Stats, - Metadata: aggregatedMetadata, + ExplainPlan: explainPlan, + PlanHash: planHash, + Stats: metadata.Stats, + Metadata: aggregatedMetadata, } statements = append(statements, stmt) diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index 43d341477bfe..685463de2254 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -1452,12 +1452,50 @@ message StatementsResponse { // Transactions is transaction-level statistics for the collection of // statements in this response. repeated ExtendedCollectedTransactionStatistics transactions = 5 [(gogoproto.nullable) = false]; + + float stmts_total_runtime_secs = 6; + + float txns_total_runtime_secs = 7; +} + +enum StatsSortOptions { + SERVICE_LAT = 0; + reserved 1; // This is for CPU Time in 23.1 + EXECUTION_COUNT = 2; + reserved 3; // This is for P99 in 23.1 + CONTENTION_TIME = 4; + PCT_RUNTIME = 5; } message CombinedStatementsStatsRequest { + enum StatsType { + StmtStatsOnly = 0; + TxnStatsOnly = 1; + } + + message FetchMode { + StatsType stats_type = 1; + StatsSortOptions sort = 2; + } + // Unix time range for aggregated statements. int64 start = 1 [(gogoproto.nullable) = true]; int64 end = 2 [(gogoproto.nullable) = true]; + + // Note that if fetch_mode is set to transactions only, we will also + // include the statement statistics for the stmts in the transactions + // response. This is more of a hack-y method to get the complete stats + // for txns, because in the client we need to fill in some txn stats info + // from its stmt stats, such as the query string. + // + // We prefer this hackier method right now to reduce surface area for backporting + // these changes, but in the future we will introduce more endpoints to properly + // organize these differing requests. + // TODO (xinhaoz) - Split this API into stmts and txns properly instead of using + // this param. + FetchMode fetch_mode = 5 [(gogoproto.nullable) = true]; + + int64 limit = 6; } // StatementDetailsRequest requests the details of a Statement, based on its keys. @@ -2051,6 +2089,7 @@ service Status { get: "/_status/combinedstmts" }; } + rpc StatementDetails(StatementDetailsRequest) returns (StatementDetailsResponse) { option (google.api.http) = { get: "/_status/stmtdetails/{fingerprint_id}" diff --git a/pkg/server/stats_test.go b/pkg/server/stats_test.go index bedca43947fb..03858e8ba579 100644 --- a/pkg/server/stats_test.go +++ b/pkg/server/stats_test.go @@ -325,7 +325,7 @@ func TestClusterResetSQLStats(t *testing.T) { } statsPreReset, err := status.Statements(ctx, &serverpb.StatementsRequest{ - Combined: true, + Combined: flushed, }) require.NoError(t, err) @@ -339,7 +339,7 @@ func TestClusterResetSQLStats(t *testing.T) { require.NoError(t, err) statsPostReset, err := status.Statements(ctx, &serverpb.StatementsRequest{ - Combined: true, + Combined: flushed, }) require.NoError(t, err) diff --git a/pkg/server/status_test.go b/pkg/server/status_test.go index d5a6e820e844..068b122b7cbe 100644 --- a/pkg/server/status_test.go +++ b/pkg/server/status_test.go @@ -49,6 +49,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" + "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -1617,6 +1618,9 @@ func TestStatusAPICombinedTransactions(t *testing.T) { } } + // Flush stats, as combinedstmts reads only from system. + thirdServer.SQLServer().(*sql.Server).GetSQLStatsProvider().(*persistedsqlstats.PersistedSQLStats).Flush(ctx) + // Hit query endpoint. var resp serverpb.StatementsResponse if err := getStatusJSONProto(firstServerProto, "combinedstmts", &resp); err != nil { @@ -1989,6 +1993,8 @@ func TestStatusAPICombinedStatements(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + ctx := context.Background() + // Aug 30 2021 19:50:00 GMT+0000 aggregatedTs := int64(1630353000) testCluster := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{ @@ -2027,6 +2033,8 @@ func TestStatusAPICombinedStatements(t *testing.T) { thirdServerSQL.Exec(t, stmt.stmt) } + testCluster.Server(2).SQLServer().(*sql.Server).GetSQLStatsProvider().(*persistedsqlstats.PersistedSQLStats).Flush(ctx) + var resp serverpb.StatementsResponse // Test that non-admin without VIEWACTIVITY privileges cannot access. err := getStatusJSONProtoWithAdminOption(firstServerProto, "combinedstmts", &resp, false) @@ -2034,7 +2042,7 @@ func TestStatusAPICombinedStatements(t *testing.T) { t.Fatalf("expected privilege error, got %v", err) } - testPath := func(path string, expectedStmts []string) { + verifyStmts := func(path string, expectedStmts []string, hasTxns bool, t *testing.T) { // Hit query endpoint. if err := getStatusJSONProtoWithAdminOption(firstServerProto, path, &resp, false); err != nil { t.Fatal(err) @@ -2042,6 +2050,7 @@ func TestStatusAPICombinedStatements(t *testing.T) { // See if the statements returned are what we executed. var statementsInResponse []string + expectedTxnFingerprints := map[roachpb.TransactionFingerprintID]struct{}{} for _, respStatement := range resp.Statements { if respStatement.Key.KeyData.Failed { // We ignore failed statements here as the INSERT statement can fail and @@ -2058,14 +2067,24 @@ func TestStatusAPICombinedStatements(t *testing.T) { } statementsInResponse = append(statementsInResponse, respStatement.Key.KeyData.Query) + expectedTxnFingerprints[respStatement.Key.KeyData.TransactionFingerprintID] = struct{}{} + } + + for _, respTxn := range resp.Transactions { + delete(expectedTxnFingerprints, respTxn.StatsData.TransactionFingerprintID) } sort.Strings(expectedStmts) sort.Strings(statementsInResponse) if !reflect.DeepEqual(expectedStmts, statementsInResponse) { - t.Fatalf("expected queries\n\n%v\n\ngot queries\n\n%v\n%s", - expectedStmts, statementsInResponse, pretty.Sprint(resp)) + t.Fatalf("expected queries\n\n%v\n\ngot queries\n\n%v\n%s\n path: %s", + expectedStmts, statementsInResponse, pretty.Sprint(resp), path) + } + if hasTxns { + assert.Empty(t, expectedTxnFingerprints) + } else { + assert.Empty(t, resp.Transactions) } } @@ -2078,33 +2097,65 @@ func TestStatusAPICombinedStatements(t *testing.T) { expectedStatements = append(expectedStatements, expectedStmt) } - // Grant VIEWACTIVITY. - thirdServerSQL.Exec(t, fmt.Sprintf("ALTER USER %s VIEWACTIVITY", authenticatedUserNameNoAdmin().Normalized())) + oneMinAfterAggregatedTs := aggregatedTs + 60 - // Test with no query params. - testPath("combinedstmts", expectedStatements) + t.Run("fetch_mode=combined, VIEWACTIVITY", func(t *testing.T) { + // Grant VIEWACTIVITY. + thirdServerSQL.Exec(t, fmt.Sprintf("ALTER USER %s VIEWACTIVITY", authenticatedUserNameNoAdmin().Normalized())) + + // Test with no query params. + verifyStmts("combinedstmts", expectedStatements, true, t) + // Test with end = 1 min after aggregatedTs; should give the same results as get all. + verifyStmts(fmt.Sprintf("combinedstmts?end=%d", oneMinAfterAggregatedTs), expectedStatements, true, t) + // Test with start = 1 hour before aggregatedTs end = 1 min after aggregatedTs; should give same results as get all. + verifyStmts(fmt.Sprintf("combinedstmts?start=%d&end=%d", aggregatedTs-3600, oneMinAfterAggregatedTs), + expectedStatements, true, t) + // Test with start = 1 min after aggregatedTs; should give no results + verifyStmts(fmt.Sprintf("combinedstmts?start=%d", oneMinAfterAggregatedTs), nil, true, t) + }) - oneMinAfterAggregatedTs := aggregatedTs + 60 - // Test with end = 1 min after aggregatedTs; should give the same results as get all. - testPath(fmt.Sprintf("combinedstmts?end=%d", oneMinAfterAggregatedTs), expectedStatements) - // Test with start = 1 hour before aggregatedTs end = 1 min after aggregatedTs; should give same results as get all. - testPath(fmt.Sprintf("combinedstmts?start=%d&end=%d", aggregatedTs-3600, oneMinAfterAggregatedTs), expectedStatements) - // Test with start = 1 min after aggregatedTs; should give no results - testPath(fmt.Sprintf("combinedstmts?start=%d", oneMinAfterAggregatedTs), nil) + t.Run("fetch_mode=combined, VIEWACTIVITYREDACTED", func(t *testing.T) { + // Remove VIEWACTIVITY so we can test with just the VIEWACTIVITYREDACTED role. + thirdServerSQL.Exec(t, fmt.Sprintf("ALTER USER %s NOVIEWACTIVITY", authenticatedUserNameNoAdmin().Normalized())) + // Grant VIEWACTIVITYREDACTED. + thirdServerSQL.Exec(t, fmt.Sprintf("ALTER USER %s VIEWACTIVITYREDACTED", authenticatedUserNameNoAdmin().Normalized())) + + // Test with no query params. + verifyStmts("combinedstmts", expectedStatements, true, t) + // Test with end = 1 min after aggregatedTs; should give the same results as get all. + verifyStmts(fmt.Sprintf("combinedstmts?end=%d", oneMinAfterAggregatedTs), expectedStatements, true, t) + // Test with start = 1 hour before aggregatedTs end = 1 min after aggregatedTs; should give same results as get all. + verifyStmts(fmt.Sprintf("combinedstmts?start=%d&end=%d", aggregatedTs-3600, oneMinAfterAggregatedTs), expectedStatements, true, t) + // Test with start = 1 min after aggregatedTs; should give no results + verifyStmts(fmt.Sprintf("combinedstmts?start=%d", oneMinAfterAggregatedTs), nil, true, t) + }) - // Remove VIEWACTIVITY so we can test with just the VIEWACTIVITYREDACTED role. - thirdServerSQL.Exec(t, fmt.Sprintf("ALTER USER %s NOVIEWACTIVITY", authenticatedUserNameNoAdmin().Normalized())) - // Grant VIEWACTIVITYREDACTED. - thirdServerSQL.Exec(t, fmt.Sprintf("ALTER USER %s VIEWACTIVITYREDACTED", authenticatedUserNameNoAdmin().Normalized())) + t.Run("fetch_mode=StmtsOnly", func(t *testing.T) { + verifyStmts("combinedstmts?fetch_mode.stats_type=0", expectedStatements, false, t) + }) - // Test with no query params. - testPath("combinedstmts", expectedStatements) - // Test with end = 1 min after aggregatedTs; should give the same results as get all. - testPath(fmt.Sprintf("combinedstmts?end=%d", oneMinAfterAggregatedTs), expectedStatements) - // Test with start = 1 hour before aggregatedTs end = 1 min after aggregatedTs; should give same results as get all. - testPath(fmt.Sprintf("combinedstmts?start=%d&end=%d", aggregatedTs-3600, oneMinAfterAggregatedTs), expectedStatements) - // Test with start = 1 min after aggregatedTs; should give no results - testPath(fmt.Sprintf("combinedstmts?start=%d", oneMinAfterAggregatedTs), nil) + t.Run("fetch_mode=TxnsOnly with limit", func(t *testing.T) { + // Verify that we only return stmts for the txns in the response. + // We'll add a limit in a later commit to help verify this behaviour. + if err := getStatusJSONProtoWithAdminOption(firstServerProto, "combinedstmts?fetch_mode.stats_type=1&limit=2", + &resp, false); err != nil { + t.Fatal(err) + } + + assert.Equal(t, 2, len(resp.Transactions)) + stmtFingerprintIDs := map[roachpb.StmtFingerprintID]struct{}{} + for _, txn := range resp.Transactions { + for _, stmtFingerprint := range txn.StatsData.StatementFingerprintIDs { + stmtFingerprintIDs[stmtFingerprint] = struct{}{} + } + } + + for _, stmt := range resp.Statements { + if _, ok := stmtFingerprintIDs[stmt.ID]; !ok { + t.Fatalf("unexpected stmt; stmt unrelated to a txn int he response: %s", stmt.Key.KeyData.Query) + } + } + }) } func TestStatusAPIStatementDetails(t *testing.T) { @@ -2113,6 +2164,8 @@ func TestStatusAPIStatementDetails(t *testing.T) { // The liveness session might expire before the stress race can finish. skip.UnderStressRace(t, "expensive tests") + ctx := context.Background() + // Aug 30 2021 19:50:00 GMT+0000 aggregatedTs := int64(1630353000) testCluster := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{ @@ -2147,6 +2200,7 @@ func TestStatusAPIStatementDetails(t *testing.T) { for _, stmt := range statements { thirdServerSQL.Exec(t, stmt) } + query := `INSERT INTO posts VALUES (_, '_')` fingerprintID := roachpb.ConstructStatementFingerprintID(query, false, true, `roachblog`) @@ -2170,6 +2224,9 @@ func TestStatusAPIStatementDetails(t *testing.T) { } testPath := func(path string, expected resultValues) { + // Need to flush since this EP reads only flushed data. + testCluster.Server(2).SQLServer().(*sql.Server).GetSQLStatsProvider().(*persistedsqlstats.PersistedSQLStats).Flush(ctx) + err := getStatusJSONProtoWithAdminOption(firstServerProto, path, &resp, false) require.NoError(t, err) require.Equal(t, int64(expected.totalCount), resp.Statement.Stats.Count) diff --git a/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.spec.ts b/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.spec.ts new file mode 100644 index 000000000000..8814e6cc2bde --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.spec.ts @@ -0,0 +1,433 @@ +// Copyright 2023 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 Long from "long"; +import { + createCombinedStmtsRequest, + getCombinedStatements, + getFlushedTxnStatsApi, + SqlStatsSortOptions, + SqlStatsSortType, +} from "./statementsApi"; +import { mockStmtStats, mockTxnStats } from "./testUtils"; +import * as fetchData from "./fetchData"; +import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; +import { shuffle } from "lodash"; + +type Stmt = cockroach.server.serverpb.StatementsResponse.ICollectedStatementStatistics; +type Txn = cockroach.server.serverpb.StatementsResponse.IExtendedCollectedTransactionStatistics; + +const mockReturnVal = (stmts: Stmt[], txns: Txn[]) => { + jest.spyOn(fetchData, "fetchData").mockReturnValue( + Promise.resolve( + new cockroach.server.serverpb.StatementsResponse({ + statements: stmts, + transactions: txns, + }), + ), + ); +}; + +type GeneratorFn = (length: number, i: number) => Partial; + +function createStmtsOverLoop( + length: number, + stmtGenerator: GeneratorFn, +): Stmt[] { + return Array.from(new Array(length)).map((_, i) => + mockStmtStats(stmtGenerator(length, i)), + ); +} + +function createTxnsOverLoop( + length: number, + generator: GeneratorFn, +): Txn[] { + return Array.from(new Array(length)).map((_, i) => + mockTxnStats(generator(length, i)), + ); +} + +describe("getCombinedStatements", () => { + afterAll(() => { + jest.resetModules(); + }); + + it("truncate response when the payload does not adhere to the limit from the request", async () => { + const tests = [ + { limit: 50, respSize: 100 }, + { limit: 10, respSize: 10 }, // No truncation occurs. + { limit: 10, respSize: 11 }, + { limit: 100, respSize: 11 }, // No truncation occurs. + ]; + + for (const tc of tests) { + const req = createCombinedStmtsRequest({ + limit: tc.limit, + sort: SqlStatsSortOptions.EXECUTION_COUNT, + start: null, + end: null, + }); + + const stmts: Stmt[] = []; + const txns: Txn[] = []; + + for (let i = 1; i <= tc.respSize; ++i) { + stmts.push(mockStmtStats({ id: Long.fromInt(i) })); + txns.push( + mockTxnStats({ + stats_data: { transaction_fingerprint_id: Long.fromInt(i) }, + }), + ); + } + + mockReturnVal(stmts, txns); + + const res = await getCombinedStatements(req); + + const expectedLen = tc.limit > tc.respSize ? tc.respSize : tc.limit; + expect(res?.statements?.length).toBe(expectedLen); + + // // Transactions half should have been discarded, regardless of whether we have truncated. + expect(res?.transactions?.length).toBe(0); + } + }); + + // Each test case will take a list of statements ordered by the provided + // sort value. The test will shuffle the given array to use as the mocked + // return value, and verify that the truncated list is ordered by the sort. + it.each([ + [ + "EXECUTION_COUNT", + createStmtsOverLoop(100, (length, i) => ({ + id: Long.fromInt(i), + stats: { count: Long.fromInt(length - i) }, + })), + SqlStatsSortOptions.EXECUTION_COUNT, + ], + [ + "CONTENTION_TIME", + createStmtsOverLoop(100, (length, i) => ({ + id: Long.fromInt(i), + stats: { + count: Long.fromInt(i), + exec_stats: { + contention_time: { + mean: length - i, + squared_diffs: 0, + }, + }, + }, + })), + SqlStatsSortOptions.CONTENTION_TIME, + ], + [ + "SVC_LAT", + createStmtsOverLoop(100, (length, i) => ({ + id: Long.fromInt(i), + stats: { + count: Long.fromInt(i), + service_lat: { + mean: length - i, + squared_diffs: 0, + }, + }, + })), + SqlStatsSortOptions.SERVICE_LAT, + ], + [ + "PCT_RUNTIME", + createStmtsOverLoop(100, (length, i) => ({ + id: Long.fromInt(i), + stats: { + count: Long.fromInt(5), + service_lat: { mean: length - i, squared_diffs: 0 }, + }, + })), + SqlStatsSortOptions.PCT_RUNTIME, + ], + ])( + "sorts data by requested option before truncating > %s", + async (_name: string, stmtsOrdered: Stmt[], sortBy: SqlStatsSortType) => { + const shuffledStmts = shuffle(stmtsOrdered); + + mockReturnVal(shuffledStmts, null); + + const limit = Math.floor(stmtsOrdered.length / 2); + const req = createCombinedStmtsRequest({ + limit, + sort: sortBy, + start: null, + end: null, + }); + + const res = await getCombinedStatements(req); + + expect(res.statements.length).toEqual(limit); + + res.statements.forEach((stmt, i) => + expect(stmt.id.toInt()).toEqual(stmtsOrdered[i].id.toInt()), + ); + }, + ); +}); + +describe("getFlushedTxnStatsApi", () => { + afterAll(() => { + jest.resetModules(); + }); + + it("truncates response when the payload does not adhere to the limit from the request", async () => { + const tests = [ + { limit: 50, respSize: 100, txnIDForStmts: 20 }, + { limit: 20, respSize: 56, txnIDForStmts: 57 }, + { limit: 10, respSize: 10, txnIDForStmts: 1 }, // No truncation occurs. + { limit: 10, respSize: 11, txnIDForStmts: 11 }, + { limit: 100, respSize: 11, txnIDForStmts: 23 }, // No truncation occurs. + ]; + + for (const tc of tests) { + const stmts: Stmt[] = []; + const txns: Txn[] = []; + + for (let i = 1; i <= tc.respSize; ++i) { + stmts.push( + mockStmtStats({ + id: Long.fromInt(i), + key: { + key_data: { + transaction_fingerprint_id: Long.fromInt(tc.txnIDForStmts), + }, + }, + }), + ); + txns.push( + mockTxnStats({ + stats_data: { + transaction_fingerprint_id: Long.fromInt(i), + stats: { + count: Long.fromInt(tc.respSize - i), + }, + }, + }), + ); + } + + mockReturnVal(stmts, txns); + + const req = createCombinedStmtsRequest({ + limit: tc.limit, + sort: SqlStatsSortOptions.EXECUTION_COUNT, + start: null, + end: null, + }); + const res = await getFlushedTxnStatsApi(req); + + const expectedLen = tc.limit > tc.respSize ? tc.respSize : tc.limit; + expect(res?.transactions?.length).toBe(expectedLen); + + if (tc.txnIDForStmts > tc.limit && tc.respSize > tc.limit) { + // No txn in the resp will have this ID and so we'll expect nothing in the + // transformed stmts response. + expect(res?.statements?.length).toBe(0); + } else { + // For this test we assign the txn id for all stmts to be either in the + // response or not, so we either get the entire stmts resp or none. + expect(res?.statements?.length).toBe(tc.respSize); + } + } + }); + + it("should filter out stmts that don't match any txn id if truncation occurs", async () => { + const tests = [ + { + stmts: [ + { stmtID: 1, txnID: 1 }, + { stmtID: 2, txnID: 3 }, + { stmtID: 3, txnID: 5 }, + { stmtID: 4, txnID: 7 }, + { stmtID: 5, txnID: 9 }, + { stmtID: 8, txnID: 11 }, + ], + txnIDs: [3, 9, 7, 8], + expectedStmts: [2, 4, 5], + }, + { + stmts: [ + { stmtID: 1, txnID: 8 }, + { stmtID: 2, txnID: 8 }, + { stmtID: 3, txnID: 8 }, + { stmtID: 4, txnID: 8 }, + { stmtID: 5, txnID: 8 }, + { stmtID: 8, txnID: 8 }, + ], + txnIDs: [3, 9, 7, 5], + expectedStmts: [], + }, + { + stmts: [ + { stmtID: 1, txnID: 1 }, + { stmtID: 2, txnID: 1 }, + { stmtID: 3, txnID: 1 }, + { stmtID: 4, txnID: 2 }, + { stmtID: 5, txnID: 3 }, + { stmtID: 6, txnID: 4 }, + ], + txnIDs: [1, 2, 3, 4], + expectedStmts: [1, 2, 3, 4, 5, 6], + }, + ]; + + for (const tc of tests) { + const stmts: Stmt[] = []; + const txns: Txn[] = []; + + for (let i = 0; i < tc.stmts.length; ++i) { + stmts.push( + mockStmtStats({ + id: Long.fromInt(tc.stmts[i].stmtID), + key: { + key_data: { + transaction_fingerprint_id: Long.fromInt(tc.stmts[i].txnID), + }, + }, + }), + ); + } + for (let i = 0; i <= tc.txnIDs.length; ++i) { + let txnID: number; + let count: number; + if (i === tc.txnIDs.length) { + // This additional txn will trigger truncation. + txnID = 10000000000; + count = -1; + } else { + txnID = tc.txnIDs[i]; + count = tc.txnIDs.length - i; + } + txns.push( + mockTxnStats({ + stats_data: { + transaction_fingerprint_id: Long.fromInt(txnID), + stats: { + count: Long.fromInt(count), + }, + }, + }), + ); + } + + mockReturnVal(stmts, txns); + + const req = createCombinedStmtsRequest({ + limit: tc.txnIDs.length, + sort: SqlStatsSortOptions.EXECUTION_COUNT, + start: null, + end: null, + }); + const res = await getFlushedTxnStatsApi(req); + + res.statements.sort((stmtA, stmtB) => stmtA.id.comp(stmtB.id)); + expect(res.statements.map(stmt => stmt.id.toInt())).toEqual( + tc.expectedStmts, + ); + } + }); + + // Each test case will take a list of transactions ordered by the provided + // sort value. The test will shuffle the given array to use as the mocked + // return value, and verify that the truncated list is ordered by the sort. + it.each([ + [ + "EXECUTION_COUNT", + createTxnsOverLoop(100, (length, i) => ({ + stats_data: { + transaction_fingerprint_id: Long.fromInt(i), + stats: { count: Long.fromInt(length - i) }, + }, + })), + SqlStatsSortOptions.EXECUTION_COUNT, + ], + [ + "CONTENTION_TIME", + createTxnsOverLoop(100, (length, i) => ({ + stats_data: { + transaction_fingerprint_id: Long.fromInt(i), + stats: { + count: Long.fromInt(i), + exec_stats: { + contention_time: { + mean: length - i, + squared_diffs: 0, + }, + }, + }, + }, + })), + SqlStatsSortOptions.CONTENTION_TIME, + ], + [ + "SVC_LAT", + createTxnsOverLoop(100, (length, i) => ({ + stats_data: { + transaction_fingerprint_id: Long.fromInt(i), + stats: { + count: Long.fromInt(i), + service_lat: { + mean: length - i, + squared_diffs: 0, + }, + }, + }, + })), + SqlStatsSortOptions.SERVICE_LAT, + ], + [ + "PCT_RUNTIME", + createTxnsOverLoop(100, (length, i) => ({ + stats_data: { + transaction_fingerprint_id: Long.fromInt(i), + stats: { + count: Long.fromInt(2), + service_lat: { + mean: length - i, + squared_diffs: 0, + }, + }, + }, + })), + SqlStatsSortOptions.PCT_RUNTIME, + ], + ])( + "sorts data by requested option before truncating > %s", + async (_name: string, txnsOrdered: Txn[], sortBy: SqlStatsSortType) => { + const shuffledTxns = shuffle(txnsOrdered); + + mockReturnVal(null, shuffledTxns); + + const limit = Math.floor(txnsOrdered.length / 2); + const req = createCombinedStmtsRequest({ + limit, + sort: sortBy, + start: null, + end: null, + }); + + const res = await getFlushedTxnStatsApi(req); + + expect(res.transactions.length).toEqual(limit); + + res.transactions.forEach((txn, i) => + expect(txn.stats_data.transaction_fingerprint_id.toInt()).toEqual( + txnsOrdered[i].stats_data.transaction_fingerprint_id.toInt(), + ), + ); + }, + ); +}); diff --git a/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts index f21983553927..23a901d1a873 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts @@ -9,8 +9,10 @@ // licenses/APL.txt. import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; -import { fetchData } from "src/api"; +import { fetchData } from "src/api/fetchData"; import { propsToQueryString } from "src/util"; +import Long from "long"; +import moment from "moment"; const STATEMENTS_PATH = "/_status/combinedstmts"; const STATEMENT_DETAILS_PATH = "/_status/stmtdetails"; @@ -24,25 +26,224 @@ export type StatementDetailsResponseWithKey = { key: string; }; +export type SqlStatsResponse = cockroach.server.serverpb.StatementsResponse; +export const SqlStatsSortOptions = cockroach.server.serverpb.StatsSortOptions; +export type SqlStatsSortType = cockroach.server.serverpb.StatsSortOptions; + +type Stmt = cockroach.server.serverpb.StatementsResponse.ICollectedStatementStatistics; +type Txn = cockroach.server.serverpb.StatementsResponse.IExtendedCollectedTransactionStatistics; + +const FetchStatsMode = + cockroach.server.serverpb.CombinedStatementsStatsRequest.StatsType; + export type ErrorWithKey = { err: Error; key: string; }; +export const DEFAULT_STATS_REQ_OPTIONS = { + limit: 100, + sort: SqlStatsSortOptions.SERVICE_LAT, +}; + +// The required fields to create a stmts request. +type StmtReqFields = { + limit: number; + sort: SqlStatsSortType; + start: moment.Moment | null; + end: moment.Moment | null; +}; + +export function createCombinedStmtsRequest({ + limit, + sort, + start, + end, +}: StmtReqFields): StatementsRequest { + return new cockroach.server.serverpb.CombinedStatementsStatsRequest({ + start: start != null ? Long.fromNumber(start.unix()) : null, + end: end != null ? Long.fromNumber(end.unix()) : null, + limit: Long.fromNumber(limit ?? DEFAULT_STATS_REQ_OPTIONS.limit), + fetch_mode: new cockroach.server.serverpb.CombinedStatementsStatsRequest.FetchMode( + { + sort: sort, + }, + ), + }); +} + +// Mutates the sqlstats response to conform to the provided sort and limit params. +export function sortAndTruncateStmtsResponse( + res: SqlStatsResponse, + sort: SqlStatsSortType, + limit: number, +): void { + // Discard txn half of the response. This is a little wasteful but the + // cleanest and least complex way of handling this scenario. + res.transactions = []; + + switch (sort) { + case SqlStatsSortOptions.SERVICE_LAT: + res.statements?.sort((stmtA: Stmt, stmtB: Stmt): number => { + return stmtB.stats.service_lat.mean - stmtA.stats.service_lat.mean; + }); + break; + case SqlStatsSortOptions.CONTENTION_TIME: + res.statements?.sort((stmtA: Stmt, stmtB: Stmt): number => { + return ( + stmtB.stats.exec_stats.contention_time.mean - + stmtA.stats.exec_stats.contention_time.mean + ); + }); + break; + case SqlStatsSortOptions.EXECUTION_COUNT: + res.statements?.sort((stmtA: Stmt, stmtB: Stmt): number => { + return stmtB.stats.count.toInt() - stmtA.stats.count.toInt(); + }); + break; + case SqlStatsSortOptions.PCT_RUNTIME: + default: + res.statements?.sort((stmtA: Stmt, stmtB: Stmt): number => { + return ( + stmtB.stats.service_lat.mean * stmtB.stats.count.toInt() - + stmtA.stats.service_lat.mean * stmtA.stats.count.toInt() + ); + }); + } + + // Finally, truncate the response not fitting into limit. + res.statements.splice(limit); +} + export const getCombinedStatements = ( req: StatementsRequest, -): Promise => { +): Promise => { + const limit = req.limit?.toInt() ?? DEFAULT_STATS_REQ_OPTIONS.limit; + const queryStr = propsToQueryString({ start: req.start.toInt(), end: req.end.toInt(), + "fetch_mode.stats_type": FetchStatsMode.StmtStatsOnly, + "fetch_mode.sort": req.fetch_mode?.sort, + limit, }); + return fetchData( cockroach.server.serverpb.StatementsResponse, `${STATEMENTS_PATH}?${queryStr}`, null, null, - "30M", + "10M", + ).then(res => { + // We may fall into the scenario of a newer UI version talking to an older server + // version that does not support the fetch_mode and limit request params. In that + // case We will have to manually sort and truncate the data to align the UI with + // the data returned. + + const isOldServer = + res?.transactions?.length || res?.statements?.length > limit; + + if (isOldServer) { + sortAndTruncateStmtsResponse(res, req?.fetch_mode?.sort, limit); + } + + return res; + }); +}; + +// Mutates the sqlstats txns response to conform to the provided sort and limit params. +function sortAndTruncateTxnsResponse( + res: SqlStatsResponse, + sort: SqlStatsSortType, + limit: number, +): void { + switch (sort) { + case SqlStatsSortOptions.SERVICE_LAT: + res.transactions?.sort((txnA: Txn, txnB: Txn): number => { + return ( + txnB.stats_data.stats.service_lat.mean - + txnA.stats_data.stats.service_lat.mean + ); + }); + break; + case SqlStatsSortOptions.CONTENTION_TIME: + res.transactions?.sort((txnA: Txn, txnB: Txn): number => { + return ( + txnB.stats_data.stats.exec_stats.contention_time.mean - + txnA.stats_data.stats.exec_stats.contention_time.mean + ); + }); + break; + case SqlStatsSortOptions.EXECUTION_COUNT: + res.transactions?.sort((txnA: Txn, txnB: Txn): number => { + return ( + txnB.stats_data.stats.count.toInt() - + txnA.stats_data.stats.count.toInt() + ); + }); + break; + case SqlStatsSortOptions.PCT_RUNTIME: + default: + res.transactions?.sort((txnA: Txn, txnB: Txn): number => { + return ( + txnB.stats_data.stats.service_lat.mean * + txnB.stats_data.stats.count.toInt() - + txnA.stats_data.stats.service_lat.mean * + txnA.stats_data.stats.count.toInt() + ); + }); + } + + // Finally, truncate the response not fitting into limit. + res.transactions.splice(limit); + + const txnFingerprintsIDs = new Set( + res.transactions.map(txn => + txn.stats_data.transaction_fingerprint_id?.toInt(), + ), + ); + + // Filter out stmts not belonging to txns response. + res.statements = res.statements.filter(stmt => + txnFingerprintsIDs.has( + stmt.key.key_data.transaction_fingerprint_id?.toInt(), + ), ); +} + +export const getFlushedTxnStatsApi = ( + req: StatementsRequest, +): Promise => { + const limit = req.limit?.toInt() ?? DEFAULT_STATS_REQ_OPTIONS.limit; + + const queryStr = propsToQueryString({ + start: req.start?.toInt(), + end: req.end?.toInt(), + "fetch_mode.stats_type": FetchStatsMode.TxnStatsOnly, + "fetch_mode.sort": req.fetch_mode?.sort, + limit, + }); + + return fetchData( + cockroach.server.serverpb.StatementsResponse, + `${STATEMENTS_PATH}?${queryStr}`, + null, + null, + "10M", + ).then(res => { + // We may fall into the scenario of a newer UI version talking to an older server + // version that does not support the fetch_mode and limit request params. In that + // case We will have to manually sort and truncate the data to align the UI with + // the data returned. + + const isOldServer = res?.transactions?.length > limit; + + if (isOldServer) { + sortAndTruncateTxnsResponse(res, req.fetch_mode?.sort, limit); + } + + return res; + }); }; export const getStatementDetails = ( diff --git a/pkg/ui/workspaces/cluster-ui/src/api/testUtils.ts b/pkg/ui/workspaces/cluster-ui/src/api/testUtils.ts new file mode 100644 index 000000000000..282d8f77734a --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/api/testUtils.ts @@ -0,0 +1,207 @@ +// Copyright 2023 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 { cockroach, google } from "@cockroachlabs/crdb-protobuf-client"; +import Long from "long"; + +type Stmt = cockroach.server.serverpb.StatementsResponse.ICollectedStatementStatistics; +type Txn = cockroach.server.serverpb.StatementsResponse.IExtendedCollectedTransactionStatistics; + +const baseStmt: Partial = { + id: Long.fromInt(11871906682067483964), + key: { + key_data: { + query: "SELECT node_id FROM system.statement_statistics", + app: "$ cockroach sql", + distSQL: true, + failed: false, + implicit_txn: true, + vec: true, + full_scan: true, + database: "defaultdb", + query_summary: "SELECT node_id FROM system.statement_statistics", + transaction_fingerprint_id: Long.fromInt(1), + }, + node_id: 0, + }, + stats: { + count: Long.fromInt(1), + first_attempt_count: Long.fromInt(1), + max_retries: Long.fromInt(0), + num_rows: { + mean: 1576, + squared_diffs: 0, + }, + parse_lat: { + mean: 0.000044584, + squared_diffs: 0, + }, + plan_lat: { + mean: 0.037206708, + squared_diffs: 0, + }, + run_lat: { + mean: 0.003240459, + squared_diffs: 0, + }, + service_lat: { + mean: 0.040506917, + squared_diffs: 0, + }, + overhead_lat: { + mean: 0.000015166000000003954, + squared_diffs: 0, + }, + sensitive_info: { + last_err: "", + most_recent_plan_description: { + name: "", + attrs: [], + children: [], + }, + most_recent_plan_timestamp: new google.protobuf.Timestamp(), + }, + bytes_read: { + mean: 162109, + squared_diffs: 0, + }, + rows_read: { + mean: 1576, + squared_diffs: 0, + }, + rows_written: { + mean: 0, + squared_diffs: 0, + }, + exec_stats: { + count: Long.fromInt(1), + network_bytes: { + mean: 0, + squared_diffs: 0, + }, + max_mem_usage: { + mean: 184320, + squared_diffs: 0, + }, + contention_time: { + mean: 0, + squared_diffs: 0, + }, + network_messages: { + mean: 0, + squared_diffs: 0, + }, + max_disk_usage: { + mean: 0, + squared_diffs: 0, + }, + }, + sql_type: "TypeDML", + last_exec_timestamp: new google.protobuf.Timestamp(), + plan_gists: ["AgFUBAAgAAAABgI="], + }, +}; + +const baseTxn: Partial = { + stats_data: { + statement_fingerprint_ids: [Long.fromInt(18262870370352730905)], + app: "$ cockroach sql", + stats: { + count: Long.fromInt(8), + max_retries: Long.fromInt(0), + num_rows: { + mean: 0, + squared_diffs: 0, + }, + service_lat: { + mean: 0.00013457312500000002, + squared_diffs: 5.992246806875002e-9, + }, + retry_lat: { + mean: 0, + squared_diffs: 0, + }, + commit_lat: { + mean: 0.0000031143749999999997, + squared_diffs: 1.1728737874999997e-11, + }, + bytes_read: { + mean: 0, + squared_diffs: 0, + }, + rows_read: { + mean: 0, + squared_diffs: 0, + }, + exec_stats: { + count: Long.fromInt(8), + network_bytes: { + mean: 0, + squared_diffs: 0, + }, + max_mem_usage: { + mean: 10240, + squared_diffs: 0, + }, + contention_time: { + mean: 0, + squared_diffs: 0, + }, + network_messages: { + mean: 0, + squared_diffs: 0, + }, + max_disk_usage: { + mean: 0, + squared_diffs: 0, + }, + }, + rows_written: { + mean: 0, + squared_diffs: 0, + }, + }, + aggregated_ts: new google.protobuf.Timestamp(), + transaction_fingerprint_id: Long.fromInt(5913510653911377094), + }, + node_id: 0, +}; + +const assignObjectPropsIfExists = ( + baseObj: T, + overrides: Partial, +): T => { + const copiedObj: T = { ...baseObj }; + for (const prop in baseObj) { + if (overrides[prop] === undefined) { + continue; + } + + const val = copiedObj[prop]; + if (typeof val === "object") { + copiedObj[prop] = assignObjectPropsIfExists( + val as Record, + overrides[prop] as Record, + ) as typeof val; + } else { + copiedObj[prop] = overrides[prop]; + } + } + + return copiedObj; +}; + +export const mockStmtStats = (partialStmt: Partial = {}): Stmt => { + return assignObjectPropsIfExists(baseStmt, partialStmt); +}; + +export const mockTxnStats = (partialTxn: Partial = {}): Txn => { + return assignObjectPropsIfExists(baseTxn, partialTxn); +}; diff --git a/pkg/ui/workspaces/cluster-ui/src/barCharts/barCharts.tsx b/pkg/ui/workspaces/cluster-ui/src/barCharts/barCharts.tsx index e3494301336e..6dd593baf721 100644 --- a/pkg/ui/workspaces/cluster-ui/src/barCharts/barCharts.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/barCharts/barCharts.tsx @@ -132,10 +132,11 @@ export function workloadPctBarChart( return barChartFactory( "grey", [ - bar( - "pct-workload", - (d: StatementStatistics) => - (d.stats.service_lat.mean * longToInt(d.stats.count)) / totalWorkload, + bar("pct-workload", (d: StatementStatistics) => + totalWorkload !== 0 + ? (d.stats.service_lat.mean * longToInt(d.stats.count)) / + totalWorkload + : 0, ), ], v => Percentage(v, 1, 1), diff --git a/pkg/ui/workspaces/cluster-ui/src/columnsSelector/columnsSelector.module.scss b/pkg/ui/workspaces/cluster-ui/src/columnsSelector/columnsSelector.module.scss index 02279ebb21b1..45cb2f667790 100644 --- a/pkg/ui/workspaces/cluster-ui/src/columnsSelector/columnsSelector.module.scss +++ b/pkg/ui/workspaces/cluster-ui/src/columnsSelector/columnsSelector.module.scss @@ -6,7 +6,7 @@ } &__btn { - height: $line-height--large; + height: $line-height--larger; width: 67px; font-size: $font-size--small; } @@ -36,9 +36,10 @@ } } -.float { - float: left; - margin-right: 7px; +.btn-area { + float: left; + margin-right: 7px; + font-size: $font-size--medium; } .label { diff --git a/pkg/ui/workspaces/cluster-ui/src/columnsSelector/columnsSelector.tsx b/pkg/ui/workspaces/cluster-ui/src/columnsSelector/columnsSelector.tsx index c9a53c43de46..3821f137daff 100644 --- a/pkg/ui/workspaces/cluster-ui/src/columnsSelector/columnsSelector.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/columnsSelector/columnsSelector.tsx @@ -36,6 +36,7 @@ export interface ColumnsSelectorProps { // options provides the list of available columns and their initial selection state options: SelectOption[]; onSubmitColumns: (selectedColumns: string[]) => void; + size?: "default" | "small"; } export interface ColumnsSelectorState { @@ -222,6 +223,7 @@ export default class ColumnsSelector extends React.Component< render() { const { hide } = this.state; + const { size = "default" } = this.props; const dropdownArea = hide ? hidden : dropdown; const options = this.getOptions(); const columnsSelected = options.filter(o => o.isSelected); @@ -230,10 +232,11 @@ export default class ColumnsSelector extends React.Component<
-
diff --git a/pkg/ui/workspaces/cluster-ui/src/common/index.tsx b/pkg/ui/workspaces/cluster-ui/src/common/index.tsx index 96ff5b297bb8..b4c1f2eddd0d 100644 --- a/pkg/ui/workspaces/cluster-ui/src/common/index.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/common/index.tsx @@ -12,3 +12,31 @@ import classNames from "classnames/bind"; import styles from "./styles.module.scss"; export const commonStyles = classNames.bind(styles); + +export const selectCustomStyles = { + container: (provided: any) => ({ + ...provided, + border: "none", + }), + option: (provided: any, state: any) => ({ + ...provided, + backgroundColor: state.isSelected ? "#DEEBFF" : provided.backgroundColor, + color: "#394455", + }), + control: (provided: any) => ({ + ...provided, + width: "100%", + borderColor: "#C0C6D9", + }), + dropdownIndicator: (provided: any) => ({ + ...provided, + color: "#C0C6D9", + }), + singleValue: (provided: any) => ({ + ...provided, + color: "#475872", + }), + indicatorSeparator: (provided: any) => ({ + ...provided, + }), +}; diff --git a/pkg/ui/workspaces/cluster-ui/src/common/styles.module.scss b/pkg/ui/workspaces/cluster-ui/src/common/styles.module.scss index 906bc98eb35d..fb40013eeff6 100644 --- a/pkg/ui/workspaces/cluster-ui/src/common/styles.module.scss +++ b/pkg/ui/workspaces/cluster-ui/src/common/styles.module.scss @@ -46,7 +46,16 @@ h3.base-heading { font-weight: 600; font-style: normal; font-stretch: normal; - font-size: 20px; + font-size: $font-size--large; + padding-bottom: 12px; +} + +h5.base-heading { + color: $colors--neutral-7; + font-family: $font-family--semi-bold; + font-style: normal; + font-stretch: normal; + font-size: $font-size--tall; padding-bottom: 12px; } diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx index b0cc6e0bf69c..2a29e5b4c067 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx @@ -41,7 +41,6 @@ import { commonStyles } from "src/common"; import { baseHeadingClasses } from "src/transactionsPage/transactionsPageClasses"; import moment, { Moment } from "moment"; import { Search as IndexIcon } from "@cockroachlabs/icons"; -import { formatDate } from "antd/es/date-picker/utils"; import { Link } from "react-router-dom"; import classnames from "classnames/bind"; import booleanSettingStyles from "../settings/booleanSetting.module.scss"; @@ -313,17 +312,14 @@ export class DatabaseTablePage extends React.Component< const lastReset = this.props.indexStats.lastReset; switch (indexStat.lastUsedType) { case "read": - return formatDate( - indexStat.lastUsed, - "[Last read:] MMM DD, YYYY [at] H:mm", - ); + return indexStat.lastUsed.format("[Last read:] MMM DD, YYYY [at] H:mm"); case "reset": default: // TODO(lindseyjin): replace default case with create time after it's added to table_indexes if (lastReset.isSame(this.minDate)) { return "Never"; } else { - return formatDate(lastReset, "[Last reset:] MMM DD, YYYY [at] H:mm"); + return lastReset.format("[Last reset:] MMM DD, YYYY [at] H:mm"); } } } @@ -457,8 +453,7 @@ export class DatabaseTablePage extends React.Component< {this.props.details.statsLastUpdated && ( diff --git a/pkg/ui/workspaces/cluster-ui/src/pageConfig/pageConfig.tsx b/pkg/ui/workspaces/cluster-ui/src/pageConfig/pageConfig.tsx index 086913ccb553..7b176fb3e4fe 100644 --- a/pkg/ui/workspaces/cluster-ui/src/pageConfig/pageConfig.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/pageConfig/pageConfig.tsx @@ -16,6 +16,7 @@ import { CockroachCloudContext } from "../contexts"; export interface PageConfigProps { layout?: "list" | "spread"; children?: React.ReactNode; + className?: string; } const cx = classnames.bind(styles); @@ -30,9 +31,9 @@ export function PageConfig(props: PageConfigProps): React.ReactElement { return (
    {props.children}
diff --git a/pkg/ui/workspaces/cluster-ui/src/queryFilter/filter.module.scss b/pkg/ui/workspaces/cluster-ui/src/queryFilter/filter.module.scss index 2c36d4454242..d87bfc5b4903 100644 --- a/pkg/ui/workspaces/cluster-ui/src/queryFilter/filter.module.scss +++ b/pkg/ui/workspaces/cluster-ui/src/queryFilter/filter.module.scss @@ -6,13 +6,13 @@ $dropdown-hover-color: darken($colors--background, 2.5%); font-family: $font-family--semi-bold; padding: 8px 0 8px 17px; vertical-align: middle; - border: 1px solid $colors--neutral-5; + border: 1px solid $colors--neutral-4; border-radius: 3px; white-space: nowrap; color: $colors--neutral-7; cursor: pointer; position: relative; - background: $colors--neutral-1; + background: white; display: flex; align-items: center; height: 40px; diff --git a/pkg/ui/workspaces/cluster-ui/src/queryFilter/filter.tsx b/pkg/ui/workspaces/cluster-ui/src/queryFilter/filter.tsx index 42c6d55acf56..c9fadf9960ad 100644 --- a/pkg/ui/workspaces/cluster-ui/src/queryFilter/filter.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/queryFilter/filter.tsx @@ -28,6 +28,7 @@ import { } from "./filterClasses"; import { MultiSelectCheckbox } from "../multiSelectCheckbox/multiSelectCheckbox"; import { syncHistory } from "../util"; +import { selectCustomStyles } from "../common"; interface QueryFilter { onSubmitFilters: (filters: Filters) => void; @@ -366,33 +367,7 @@ export class Filter extends React.Component { timeLabel, } = this.props; const dropdownArea = hide ? hidden : dropdown; - const customStyles = { - container: (provided: any) => ({ - ...provided, - border: "none", - }), - option: (provided: any, state: any) => ({ - ...provided, - backgroundColor: state.isSelected - ? "#DEEBFF" - : provided.backgroundColor, - color: "#394455", - }), - control: (provided: any) => ({ - ...provided, - width: "100%", - borderColor: "#C0C6D9", - }), - dropdownIndicator: (provided: any) => ({ - ...provided, - color: "#C0C6D9", - }), - singleValue: (provided: any) => ({ - ...provided, - color: "#475872", - }), - }; - const customStylesSmall = { ...customStyles }; + const customStylesSmall = { ...selectCustomStyles }; customStylesSmall.container = (provided: any) => ({ ...provided, width: "141px", diff --git a/pkg/ui/workspaces/cluster-ui/src/searchCriteria/index.ts b/pkg/ui/workspaces/cluster-ui/src/searchCriteria/index.ts new file mode 100644 index 000000000000..b45162ad446a --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/searchCriteria/index.ts @@ -0,0 +1,11 @@ +// Copyright 2023 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. + +export * from "./searchCriteria"; diff --git a/pkg/ui/workspaces/cluster-ui/src/searchCriteria/searchCriteria.module.scss b/pkg/ui/workspaces/cluster-ui/src/searchCriteria/searchCriteria.module.scss new file mode 100644 index 000000000000..d5b7211c9169 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/searchCriteria/searchCriteria.module.scss @@ -0,0 +1,29 @@ +@import "../core/index.module"; + +.search-area { + border: 1px solid $colors--neutral-3; + border-radius: 3px; + margin-bottom: 10px; + margin-right: 10px; + padding: 20px; + background-color: white; +} + +.top-area { + z-index: 5; + background-color: white; +} + +.timescale-small { + width: 455px; +} + +.label { + color: $colors--neutral-6; + font-family: $font-family--semi-bold; + font-size: $font-size--medium; +} + +.margin-top-btn { + margin-top: 22px; +} diff --git a/pkg/ui/workspaces/cluster-ui/src/searchCriteria/searchCriteria.tsx b/pkg/ui/workspaces/cluster-ui/src/searchCriteria/searchCriteria.tsx new file mode 100644 index 000000000000..74df0a20eb5d --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/searchCriteria/searchCriteria.tsx @@ -0,0 +1,126 @@ +// Copyright 2023 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 React from "react"; +import classNames from "classnames/bind"; +import styles from "./searchCriteria.module.scss"; +import { + Button, + commonStyles, + PageConfig, + PageConfigItem, + selectCustomStyles, + TimeScale, + timeScale1hMinOptions, + TimeScaleDropdown, +} from "src"; +import { applyBtn } from "../queryFilter/filterClasses"; +import Select from "react-select"; +import { limitOptions } from "../util/sqlActivityConstants"; +import { SqlStatsSortType } from "src/api/statementsApi"; +const cx = classNames.bind(styles); + +type SortOption = { + label: string; + value: SqlStatsSortType; +}; +export interface SearchCriteriaProps { + sortOptions: SortOption[]; + currentScale: TimeScale; + topValue: number; + byValue: SqlStatsSortType; + onChangeTimeScale: (ts: TimeScale) => void; + onChangeTop: (top: number) => void; + onChangeBy: (by: SqlStatsSortType) => void; + onApply: () => void; +} + +export function SearchCriteria(props: SearchCriteriaProps): React.ReactElement { + const { + topValue, + byValue, + currentScale, + onChangeTop, + onChangeBy, + onChangeTimeScale, + sortOptions, + } = props; + const customStyles = { ...selectCustomStyles }; + customStyles.indicatorSeparator = (provided: any) => ({ + ...provided, + display: "none", + }); + + const customStylesTop = { ...customStyles }; + customStylesTop.container = (provided: any) => ({ + ...provided, + width: "80px", + border: "none", + }); + + const customStylesBy = { ...customStyles }; + customStylesBy.container = (provided: any) => ({ + ...provided, + width: "170px", + border: "none", + }); + + return ( +
+
Search Criteria
+ + + + + + + + + + + +
+ ); +} diff --git a/pkg/ui/workspaces/cluster-ui/src/sessions/sessionsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/sessions/sessionsPage.tsx index 3db69b29fb68..ecf79b2485bf 100644 --- a/pkg/ui/workspaces/cluster-ui/src/sessions/sessionsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/sessions/sessionsPage.tsx @@ -367,6 +367,7 @@ export class SessionsPage extends React.Component< { return ( <> - reset SQL stats + Reset SQL Stats void; onColumnsChange?: (selectedColumns: string[]) => void; onTimeScaleChange: (ts: TimeScale) => void; + onChangeLimit: (limit: number) => void; + onChangeReqSort: (sort: SqlStatsSortType) => void; } export interface StatementsPageStateProps { @@ -128,6 +132,8 @@ export interface StatementsPageStateProps { isReqInFlight: boolean; lastUpdated: moment.Moment | null; timeScale: TimeScale; + limit: number; + reqSortSetting: SqlStatsSortType; statementsError: Error | null; apps: string[]; databases: string[]; @@ -141,23 +147,34 @@ export interface StatementsPageStateProps { isTenant?: UIConfigState["isTenant"]; hasViewActivityRedactedRole?: UIConfigState["hasViewActivityRedactedRole"]; hasAdminRole?: UIConfigState["hasAdminRole"]; + stmtsTotalRuntimeSecs: number; } export interface StatementsPageState { pagination: ISortedTablePagination; filters?: Filters; activeFilters?: number; + timeScale: TimeScale; + limit: number; + reqSortSetting: SqlStatsSortType; } export type StatementsPageProps = StatementsPageDispatchProps & StatementsPageStateProps & RouteComponentProps; -function stmtsRequestFromTimeScale(ts: TimeScale): StatementsRequest { - const [start, end] = toRoundedDateRange(ts); - return new cockroach.server.serverpb.CombinedStatementsStatsRequest({ - start: Long.fromNumber(start.unix()), - end: Long.fromNumber(end.unix()), +type RequestParams = Pick< + StatementsPageState, + "limit" | "reqSortSetting" | "timeScale" +>; + +function stmtsRequestFromParams(params: RequestParams): StatementsRequest { + const [start, end] = toRoundedDateRange(params.timeScale); + return createCombinedStmtsRequest({ + start, + end, + limit: params.limit, + sort: params.reqSortSetting, }); } @@ -194,9 +211,12 @@ export class StatementsPage extends React.Component< super(props); const defaultState = { pagination: { - pageSize: 20, + pageSize: 50, current: 1, }, + limit: this.props.limit, + timeScale: this.props.timeScale, + reqSortSetting: this.props.reqSortSetting, }; const stateFromHistory = this.getStateFromHistory(); this.state = merge(defaultState, stateFromHistory); @@ -256,9 +276,17 @@ export class StatementsPage extends React.Component< }; changeTimeScale = (ts: TimeScale): void => { - if (this.props.onTimeScaleChange) { - this.props.onTimeScaleChange(ts); - } + this.setState(prevState => ({ + ...prevState, + timeScale: ts, + })); + }; + + updateRequestParams = (): void => { + this.props.onChangeLimit(this.state.limit); + this.props.onChangeReqSort(this.state.reqSortSetting); + this.props.onTimeScaleChange(this.state.timeScale); + this.refreshStatements(); }; resetPagination = (): void => { @@ -273,7 +301,7 @@ export class StatementsPage extends React.Component< }; refreshStatements = (): void => { - const req = stmtsRequestFromTimeScale(this.props.timeScale); + const req = stmtsRequestFromParams(this.state); this.props.refreshStatements(req); }; @@ -337,7 +365,7 @@ export class StatementsPage extends React.Component< ); } - componentDidUpdate = (prevProps: StatementsPageProps): void => { + componentDidUpdate = (): void => { this.updateQueryParams(); if (!this.props.isTenant) { this.props.refreshNodes(); @@ -345,13 +373,6 @@ export class StatementsPage extends React.Component< this.props.refreshStatementDiagnosticsRequests(); } } - - if ( - prevProps.timeScale !== this.props.timeScale || - (prevProps.isDataValid && !this.props.isDataValid) - ) { - this.refreshStatements(); - } }; componentWillUnmount(): void { @@ -360,7 +381,10 @@ export class StatementsPage extends React.Component< onChangePage = (current: number): void => { const { pagination } = this.state; - this.setState({ pagination: { ...pagination, current } }); + this.setState(prevState => ({ + ...prevState, + pagination: { ...pagination, current }, + })); this.props.onPageChanged != null && this.props.onPageChanged(current); }; @@ -522,7 +546,15 @@ export class StatementsPage extends React.Component< ); }; - renderStatements = (regions: string[]): React.ReactElement => { + onChangeLimit = (newLimit: number): void => { + this.setState(prevState => ({ ...prevState, limit: newLimit })); + }; + + onChangeReqSort = (newSort: SqlStatsSortType): void => { + this.setState(prevState => ({ ...prevState, reqSortSetting: newSort })); + }; + + renderStatements = (): React.ReactElement => { const { pagination, filters, activeFilters } = this.state; const { onSelectDiagnosticsReportDropdownOption, @@ -534,24 +566,33 @@ export class StatementsPage extends React.Component< hasViewActivityRedactedRole, sortSetting, search, + apps, + databases, + hasAdminRole, } = this.props; const data = this.filteredStatementsData(); - const totalWorkload = calculateTotalWorkload(data); const statements = this.props.statements ?? []; - - const totalCount = data.length; const isEmptySearchResults = statements?.length > 0 && search?.length > 0; // If the cluster is a tenant cluster we don't show info // about nodes/regions. populateRegionNodeForStatements(statements, nodeRegions, isTenant); + const nodes = isTenant + ? [] + : Object.keys(nodeRegions) + .map(n => Number(n)) + .sort(); + const regions = isTenant + ? [] + : unique(nodes.map(node => nodeRegions[node.toString()])).sort(); + // Creates a list of all possible columns, // hiding nodeRegions if is not multi-region and // hiding columns that won't be displayed for tenants. const columns = makeStatementsColumns( statements, filters.app.split(","), - totalWorkload, + this.props.stmtsTotalRuntimeSecs, nodeRegions, "statement", isTenant, @@ -590,25 +631,78 @@ export class StatementsPage extends React.Component< const displayColumns = columns.filter(c => isColumnSelected(c)); const period = timeScaleToString(this.props.timeScale); + const clearFilter = activeFilters ? ( + + + + ) : ( + <> + ); + + const sortSettingLabel = getSortLabel(this.props.reqSortSetting); return ( -
+ <> +
+ {`Results - Top ${this.props.limit} Statement Fingerprints by ${sortSettingLabel}`} +
+
+ + + + + + "n" + n)} + activeFilters={activeFilters} + filters={filters} + showDB={true} + showSqlType={true} + showScan={true} + showRegions={regions.length > 1} + showNodes={!isTenant && nodes.length > 1} + /> + + + + + {clearFilter} + + + +

+ Showing aggregated stats from{" "} + {period} +

+
+ {hasAdminRole && ( + + + + )} +
+
-
- - -
-
+ ); }; @@ -638,24 +732,8 @@ export class StatementsPage extends React.Component< refreshStatementDiagnosticsRequests, onActivateStatementDiagnostics, onDiagnosticsModalOpen, - apps, - databases, - search, - isTenant, - nodeRegions, - hasAdminRole, } = this.props; - const nodes = isTenant - ? [] - : Object.keys(nodeRegions) - .map(n => Number(n)) - .sort(); - const regions = isTenant - ? [] - : unique(nodes.map(node => nodeRegions[node.toString()])).sort(); - const { filters, activeFilters } = this.state; - const longLoadingMessage = ( - - - - - - "n" + n)} - activeFilters={activeFilters} - filters={filters} - showDB={true} - showSqlType={true} - showScan={true} - showRegions={regions.length > 1} - showNodes={nodes.length > 1} - /> - - - - - {hasAdminRole && ( - - - - )} - - this.renderStatements(regions)} - renderError={() => - LoadingError({ - statsType: "statements", - timeout: this.props.statementsError?.name - ?.toLowerCase() - .includes("timeout"), - }) - } - /> - {this.props.isReqInFlight && - getValidErrorsList(this.props.statementsError) == null && - longLoadingMessage} - + +
+ this.renderStatements()} + renderError={() => + LoadingError({ + statsType: "statements", + timeout: this.props.statementsError?.name + ?.toLowerCase() + .includes("timeout"), + }) + } + /> + {this.props.isReqInFlight && + getValidErrorsList(this.props.statementsError) == null && + longLoadingMessage} + +
{" "}
); } diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx index ba8d029b274e..b4e84900f604 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx @@ -15,7 +15,11 @@ import { Dispatch } from "redux"; import { AppState, uiConfigActions } from "src/store"; import { actions as statementDiagnosticsActions } from "src/store/statementDiagnostics"; import { actions as analyticsActions } from "src/store/analytics"; -import { actions as localStorageActions } from "src/store/localStorage"; +import { + actions as localStorageActions, + updateStmtsPageLimitAction, + updateStmsPageReqSortAction, +} from "src/store/localStorage"; import { actions as sqlStatsActions } from "src/store/sqlStats"; import { actions as nodesActions } from "../store/nodes"; import { @@ -39,6 +43,10 @@ import { selectSearch, selectStatementsLastUpdated, } from "./statementsPage.selectors"; +import { + selectStmtsPageLimit, + selectStmtsPageReqSort, +} from "../store/utils/selectors"; import { selectIsTenant, selectHasViewActivityRedactedRole, @@ -48,6 +56,7 @@ import { nodeRegionsByIDSelector } from "../store/nodes"; import { StatementsRequest } from "src/api/statementsApi"; import { TimeScale } from "../timeScaleDropdown"; import { cockroach, google } from "@cockroachlabs/crdb-protobuf-client"; +import { SqlStatsSortType } from "../api"; type IStatementDiagnosticsReport = cockroach.server.serverpb.IStatementDiagnosticsReport; type IDuration = google.protobuf.IDuration; @@ -83,6 +92,10 @@ export const ConnectedStatementsPage = withRouter( lastUpdated: selectStatementsLastUpdated(state), statementsError: selectStatementsLastError(state), totalFingerprints: selectTotalFingerprints(state), + limit: selectStmtsPageLimit(state), + reqSortSetting: selectStmtsPageReqSort(state), + stmtsTotalRuntimeSecs: + state.adminUI?.statements?.data?.stmts_total_runtime_secs ?? 0, }), (dispatch: Dispatch) => ({ refreshStatements: (req: StatementsRequest) => @@ -226,6 +239,10 @@ export const ConnectedStatementsPage = withRouter( selectedColumns.length === 0 ? " " : selectedColumns.join(","), }), ), + onChangeLimit: (limit: number) => + dispatch(updateStmtsPageLimitAction(limit)), + onChangeReqSort: (sort: SqlStatsSortType) => + dispatch(updateStmsPageReqSortAction(sort)), }), )(StatementsPage), ); 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 044ea5581bce..003867ad6bb2 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 @@ -12,6 +12,10 @@ import { createSlice, PayloadAction } from "@reduxjs/toolkit"; import { DOMAIN_NAME } from "../utils"; import { defaultFilters, Filters } from "../../queryFilter"; import { TimeScale, defaultTimeScaleSelected } from "../../timeScaleDropdown"; +import { + SqlStatsSortType, + DEFAULT_STATS_REQ_OPTIONS, +} from "src/api/statementsApi"; type SortSetting = { ascending: boolean; @@ -20,6 +24,10 @@ type SortSetting = { export enum LocalStorageKeys { GLOBAL_TIME_SCALE = "timeScale/SQLActivity", + STMT_FINGERPRINTS_LIMIT = "limit/StatementsPage", + STMT_FINGERPRINTS_SORT = "sort/StatementsPage", + TXN_FINGERPRINTS_LIMIT = "limit/TransactionsPage", + TXN_FINGERPRINTS_SORT = "sort/TransactionsPage", } export type LocalStorageState = { @@ -28,6 +36,10 @@ export type LocalStorageState = { "showColumns/TransactionPage": string; "showColumns/SessionsPage": string; [LocalStorageKeys.GLOBAL_TIME_SCALE]: TimeScale; + [LocalStorageKeys.STMT_FINGERPRINTS_LIMIT]: number; + [LocalStorageKeys.STMT_FINGERPRINTS_SORT]: SqlStatsSortType; + [LocalStorageKeys.TXN_FINGERPRINTS_LIMIT]: number; + [LocalStorageKeys.TXN_FINGERPRINTS_SORT]: SqlStatsSortType; "sortSetting/StatementsPage": SortSetting; "sortSetting/TransactionsPage": SortSetting; "sortSetting/SessionsPage": SortSetting; @@ -62,10 +74,23 @@ const initialState: LocalStorageState = { "adminUi/showDiagnosticsModal": Boolean(JSON.parse(localStorage.getItem("adminUi/showDiagnosticsModal"))) || false, + [LocalStorageKeys.STMT_FINGERPRINTS_LIMIT]: + JSON.parse( + localStorage.getItem(LocalStorageKeys.STMT_FINGERPRINTS_LIMIT), + ) || DEFAULT_STATS_REQ_OPTIONS.limit, + [LocalStorageKeys.STMT_FINGERPRINTS_SORT]: + JSON.parse(localStorage.getItem(LocalStorageKeys.STMT_FINGERPRINTS_SORT)) || + DEFAULT_STATS_REQ_OPTIONS.sort, "showColumns/StatementsPage": JSON.parse(localStorage.getItem("showColumns/StatementsPage")) || null, "showColumns/TransactionPage": JSON.parse(localStorage.getItem("showColumns/TransactionPage")) || null, + [LocalStorageKeys.TXN_FINGERPRINTS_LIMIT]: + JSON.parse(localStorage.getItem(LocalStorageKeys.TXN_FINGERPRINTS_LIMIT)) || + DEFAULT_STATS_REQ_OPTIONS.limit, + [LocalStorageKeys.TXN_FINGERPRINTS_SORT]: + JSON.parse(localStorage.getItem(LocalStorageKeys.TXN_FINGERPRINTS_SORT)) || + DEFAULT_STATS_REQ_OPTIONS.sort, "showColumns/SessionsPage": JSON.parse(localStorage.getItem("showColumns/SessionsPage")) || null, [LocalStorageKeys.GLOBAL_TIME_SCALE]: @@ -111,3 +136,35 @@ const localStorageSlice = createSlice({ }); export const { actions, reducer } = localStorageSlice; + +export const updateStmtsPageLimitAction = ( + limit: number, +): PayloadAction => + localStorageSlice.actions.update({ + key: LocalStorageKeys.STMT_FINGERPRINTS_LIMIT, + value: limit, + }); + +export const updateStmsPageReqSortAction = ( + sort: SqlStatsSortType, +): PayloadAction => + localStorageSlice.actions.update({ + key: LocalStorageKeys.STMT_FINGERPRINTS_SORT, + value: sort, + }); + +export const updateTxnsPageLimitAction = ( + limit: number, +): PayloadAction => + localStorageSlice.actions.update({ + key: LocalStorageKeys.TXN_FINGERPRINTS_LIMIT, + value: limit, + }); + +export const updateTxnsPageReqSortAction = ( + sort: SqlStatsSortType, +): PayloadAction => + localStorageSlice.actions.update({ + key: LocalStorageKeys.TXN_FINGERPRINTS_SORT, + value: sort, + }); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/localStorage/localStorage.saga.ts b/pkg/ui/workspaces/cluster-ui/src/store/localStorage/localStorage.saga.ts index 0604c38e4f7e..a60a09aabba3 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/localStorage/localStorage.saga.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/localStorage/localStorage.saga.ts @@ -10,11 +10,14 @@ import { AnyAction } from "redux"; import { all, call, takeEvery, takeLatest, put } from "redux-saga/effects"; -import { actions, LocalStorageKeys } from "./localStorage.reducer"; +import { + actions, + LocalStorageKeys, + TypedPayload, +} from "./localStorage.reducer"; import { actions as sqlStatsActions } from "src/store/sqlStats"; import { PayloadAction } from "@reduxjs/toolkit"; -import { TypedPayload } from "./localStorage.reducer"; -import { TimeScale } from "../../timeScaleDropdown"; +import { TimeScale } from "src/timeScaleDropdown"; export function* updateLocalStorageItemSaga(action: AnyAction) { const { key, value } = action.payload; @@ -28,12 +31,11 @@ export function* updateLocalStorageItemSaga(action: AnyAction) { export function* updateTimeScale( action: PayloadAction>, ) { - yield all([put(sqlStatsActions.invalidated())]); - const { value } = action.payload; + yield put(sqlStatsActions.invalidated()); yield call( { context: localStorage, fn: localStorage.setItem }, LocalStorageKeys.GLOBAL_TIME_SCALE, - JSON.stringify(value), + JSON.stringify(action.payload?.value), ); } diff --git a/pkg/ui/workspaces/cluster-ui/src/store/reducers.spec.ts b/pkg/ui/workspaces/cluster-ui/src/store/reducers.spec.ts index 1f37059671eb..77c4a8be0a81 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/reducers.spec.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/reducers.spec.ts @@ -25,8 +25,8 @@ describe("rootReducer", () => { assert.deepEqual(initState, resetState); assert.notDeepEqual( - resetState.sqlStats.lastError, - changedState.sqlStats.lastError, + resetState.statements.lastError, + changedState.statements.lastError, ); }); }); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts b/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts index ed7750f5626c..275db06fe377 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts @@ -29,6 +29,7 @@ import { SQLDetailsStatsReducerState, reducer as sqlDetailsStats, } from "./statementDetails"; +import { reducer as txnStats, TxnStatsState } from "./transactionStats"; export type AdminUiState = { statementDiagnostics: StatementDiagnosticsState; @@ -38,7 +39,8 @@ export type AdminUiState = { sessions: SessionsState; terminateQuery: TerminateQueryState; uiConfig: UIConfigState; - sqlStats: SQLStatsState; + statements: SQLStatsState; + transactions: TxnStatsState; sqlDetailsStats: SQLDetailsStatsReducerState; }; @@ -54,7 +56,8 @@ export const reducers = combineReducers({ sessions, terminateQuery, uiConfig, - sqlStats, + statements: sqlStats, + transactions: txnStats, sqlDetailsStats, }); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts index 1adf716c14dd..c21f4b12adcf 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts @@ -21,6 +21,7 @@ import { notifificationsSaga } from "./notifications"; import { sqlStatsSaga } from "./sqlStats"; import { sqlDetailsStatsSaga } from "./statementDetails"; import { uiConfigSaga } from "./uiConfig"; +import { txnStatsSaga } from "./transactionStats"; export function* sagas(cacheInvalidationPeriod?: number): SagaIterator { yield all([ @@ -34,5 +35,6 @@ export function* sagas(cacheInvalidationPeriod?: number): SagaIterator { fork(sqlStatsSaga), fork(sqlDetailsStatsSaga), fork(uiConfigSaga, cacheInvalidationPeriod), + fork(txnStatsSaga), ]); } diff --git a/pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.reducer.ts index 970e7aad0079..9c5a6115fa7c 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.reducer.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.reducer.ts @@ -37,11 +37,14 @@ export type UpdateTimeScalePayload = { ts: TimeScale; }; +// This is actually statements only, despite the SQLStatsState name. +// We can rename this in the future. Leaving it now to reduce backport surface area. const sqlStatsSlice = createSlice({ name: `${DOMAIN_NAME}/sqlstats`, initialState, reducers: { received: (state, action: PayloadAction) => { + state.inFlight = false; state.data = action.payload; state.valid = true; state.lastError = null; @@ -55,6 +58,7 @@ const sqlStatsSlice = createSlice({ state.inFlight = false; }, invalidated: state => { + state.inFlight = false; state.valid = false; }, refresh: (state, _action: PayloadAction) => { diff --git a/pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.sagas.ts index d6d6ae21911c..fb938a0e2ba4 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.sagas.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.sagas.ts @@ -20,6 +20,7 @@ import { actions as sqlStatsActions, UpdateTimeScalePayload, } from "./sqlStats.reducer"; +import { actions as txnStatsActions } from "../transactionStats"; import { actions as sqlDetailsStatsActions } from "../statementDetails/statementDetails.reducer"; export function* refreshSQLStatsSaga(action: PayloadAction) { @@ -54,6 +55,7 @@ export function* resetSQLStatsSaga() { yield all([ put(sqlDetailsStatsActions.invalidateAll()), put(sqlStatsActions.invalidated()), + put(txnStatsActions.invalidated()), ]); } catch (e) { yield put(sqlStatsActions.failed(e)); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.selector.ts b/pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.selector.ts index 0d16f292b2f8..255680324379 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.selector.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/sqlStats/sqlStats.selector.ts @@ -18,5 +18,5 @@ const adminUISelector = createSelector( export const sqlStatsSelector = createSelector( adminUISelector, - adminUiState => adminUiState?.sqlStats, + adminUiState => adminUiState?.statements, ); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/statementDetails/statementDetails.sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/statementDetails/statementDetails.sagas.ts index e5d8bb6612f6..37303e123867 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/statementDetails/statementDetails.sagas.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/statementDetails/statementDetails.sagas.ts @@ -18,7 +18,7 @@ import { } from "src/api/statementsApi"; import { actions as sqlDetailsStatsActions } from "./statementDetails.reducer"; import { CACHE_INVALIDATION_PERIOD } from "src/store/utils"; -import { generateStmtDetailsToID } from "../../util"; +import { generateStmtDetailsToID } from "src/util/appStats"; export function* refreshSQLDetailsStatsSaga( action: PayloadAction, diff --git a/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/index.ts b/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/index.ts new file mode 100644 index 000000000000..954b02cff226 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/index.ts @@ -0,0 +1,12 @@ +// Copyright 2023 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. + +export * from "./txnStats.reducer"; +export * from "./txnStats.sagas"; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/txnStats.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/txnStats.reducer.ts new file mode 100644 index 000000000000..71731100874c --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/txnStats.reducer.ts @@ -0,0 +1,66 @@ +// Copyright 2023 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 } from "../utils"; +import { StatementsRequest } from "src/api/statementsApi"; +import { TimeScale } from "../../timeScaleDropdown"; +import moment from "moment"; +import { StatementsResponse } from "../sqlStats"; + +export type TxnStatsState = { + // Note that we request transactions from the + // statements api, hence the StatementsResponse type here. + data: StatementsResponse; + inFlight: boolean; + lastError: Error; + valid: boolean; + lastUpdated: moment.Moment | null; +}; + +const initialState: TxnStatsState = { + data: null, + inFlight: false, + lastError: null, + valid: false, + lastUpdated: null, +}; + +const txnStatsSlice = createSlice({ + name: `${DOMAIN_NAME}/txnStats`, + initialState, + reducers: { + received: (state, action: PayloadAction) => { + state.inFlight = false; + state.data = action.payload; + state.valid = true; + state.lastError = null; + state.lastUpdated = moment.utc(); + }, + failed: (state, action: PayloadAction) => { + state.inFlight = false; + state.valid = false; + state.lastError = action.payload; + state.lastUpdated = moment.utc(); + }, + invalidated: state => { + state.inFlight = false; + state.valid = false; + }, + refresh: (state, _: PayloadAction) => { + state.inFlight = true; + }, + request: (state, _: PayloadAction) => { + state.inFlight = true; + }, + }, +}); + +export const { reducer, actions } = txnStatsSlice; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/txnStats.sagas.spec.ts b/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/txnStats.sagas.spec.ts new file mode 100644 index 000000000000..7f88306fa492 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/txnStats.sagas.spec.ts @@ -0,0 +1,101 @@ +// Copyright 2023 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 { expectSaga } from "redux-saga-test-plan"; +import { + EffectProviders, + StaticProvider, + throwError, +} from "redux-saga-test-plan/providers"; +import * as matchers from "redux-saga-test-plan/matchers"; +import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; + +import { getFlushedTxnStatsApi } from "src/api/statementsApi"; +import { refreshTxnStatsSaga, requestTxnStatsSaga } from "./txnStats.sagas"; +import { actions, reducer, TxnStatsState } from "./txnStats.reducer"; +import Long from "long"; +import moment from "moment"; + +const lastUpdated = moment(); + +describe("txnStats sagas", () => { + let spy: jest.SpyInstance; + beforeAll(() => { + spy = jest.spyOn(moment, "utc").mockImplementation(() => lastUpdated); + }); + + afterAll(() => { + spy.mockRestore(); + }); + + const payload = new cockroach.server.serverpb.CombinedStatementsStatsRequest({ + start: Long.fromNumber(1596816675), + end: Long.fromNumber(1596820675), + limit: Long.fromNumber(100), + }); + + const txnStatsResponse = new cockroach.server.serverpb.StatementsResponse({ + transactions: [ + { + stats_data: { transaction_fingerprint_id: new Long(1) }, + }, + { + stats_data: { transaction_fingerprint_id: new Long(2) }, + }, + ], + last_reset: null, + }); + + const txnStatsAPIProvider: (EffectProviders | StaticProvider)[] = [ + [matchers.call.fn(getFlushedTxnStatsApi), txnStatsResponse], + ]; + + describe("refreshTxnStatsSaga", () => { + it("dispatches request txnStats action", () => { + return expectSaga(refreshTxnStatsSaga, actions.request(payload)) + .provide(txnStatsAPIProvider) + .put(actions.request(payload)) + .run(); + }); + }); + + describe("requestTxnStatsSaga", () => { + it("successfully requests statements list", () => { + return expectSaga(requestTxnStatsSaga, actions.request(payload)) + .provide(txnStatsAPIProvider) + .put(actions.received(txnStatsResponse)) + .withReducer(reducer) + .hasFinalState({ + inFlight: false, + data: txnStatsResponse, + lastError: null, + valid: true, + lastUpdated, + }) + .run(); + }); + + it("returns error on failed request", () => { + const error = new Error("Failed request"); + return expectSaga(requestTxnStatsSaga, actions.request(payload)) + .provide([[matchers.call.fn(getFlushedTxnStatsApi), throwError(error)]]) + .put(actions.failed(error)) + .withReducer(reducer) + .hasFinalState({ + inFlight: false, + data: null, + lastError: error, + valid: false, + lastUpdated, + }) + .run(); + }); + }); +}); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/txnStats.sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/txnStats.sagas.ts new file mode 100644 index 000000000000..86553376c4f6 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/txnStats.sagas.ts @@ -0,0 +1,41 @@ +// Copyright 2023 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 { PayloadAction } from "@reduxjs/toolkit"; +import { all, call, put, takeLatest } from "redux-saga/effects"; +import { + getFlushedTxnStatsApi, + StatementsRequest, +} from "src/api/statementsApi"; +import { actions as txnStatsActions } from "./txnStats.reducer"; + +export function* refreshTxnStatsSaga( + action: PayloadAction, +): any { + yield put(txnStatsActions.request(action.payload)); +} + +export function* requestTxnStatsSaga( + action: PayloadAction, +): any { + try { + const result = yield call(getFlushedTxnStatsApi, action.payload); + yield put(txnStatsActions.received(result)); + } catch (e) { + yield put(txnStatsActions.failed(e)); + } +} + +export function* txnStatsSaga(): any { + yield all([ + takeLatest(txnStatsActions.refresh, refreshTxnStatsSaga), + takeLatest(txnStatsActions.request, requestTxnStatsSaga), + ]); +} diff --git a/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/txnStats.selector.ts b/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/txnStats.selector.ts new file mode 100644 index 000000000000..b232cdd18ce4 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/transactionStats/txnStats.selector.ts @@ -0,0 +1,17 @@ +// Copyright 2023 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 txnStatsSelector = createSelector( + adminUISelector, + adminUiState => adminUiState?.transactions, +); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/utils/selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/utils/selectors.ts index 9b986ae61c7d..e74072454513 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/utils/selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/utils/selectors.ts @@ -26,3 +26,23 @@ export const selectTimeScale = createSelector( localStorageSelector, localStorage => localStorage[LocalStorageKeys.GLOBAL_TIME_SCALE], ); + +export const selectStmtsPageLimit = createSelector( + localStorageSelector, + localStorage => localStorage[LocalStorageKeys.STMT_FINGERPRINTS_LIMIT], +); + +export const selectStmtsPageReqSort = createSelector( + localStorageSelector, + localStorage => localStorage[LocalStorageKeys.STMT_FINGERPRINTS_SORT], +); + +export const selectTxnsPageLimit = createSelector( + localStorageSelector, + localStorage => localStorage[LocalStorageKeys.TXN_FINGERPRINTS_LIMIT], +); + +export const selectTxnsPageReqSort = createSelector( + localStorageSelector, + localStorage => localStorage[LocalStorageKeys.TXN_FINGERPRINTS_SORT], +); diff --git a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/rangeSelector.module.scss b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/rangeSelector.module.scss index 419b27f59a74..ba2e88e86552 100644 --- a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/rangeSelector.module.scss +++ b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/rangeSelector.module.scss @@ -32,7 +32,7 @@ width: 100%; .trigger-button { - width: 423px; + min-width: 285px; height: fit-content; } } diff --git a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeScaleDropdown.tsx b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeScaleDropdown.tsx index 2f4b78ffec51..bac3ff96ad12 100644 --- a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeScaleDropdown.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeScaleDropdown.tsx @@ -39,6 +39,7 @@ export interface TimeScaleDropdownProps { curTimeScale: TimeScale, timeWindow: TimeWindow, ) => TimeScale; + className?: string; } export const getTimeLabel = ( @@ -127,6 +128,7 @@ export const TimeScaleDropdown: React.FC = ({ options = defaultTimeScaleOptions, setTimeScale, adjustTimeScaleOnChange, + className, }): React.ReactElement => { const end = currentScale.fixedWindowEnd ? moment.utc(currentScale.fixedWindowEnd) @@ -245,7 +247,7 @@ export const TimeScaleDropdown: React.FC = ({ }; return ( -
+
)) .add("with loading indicator", () => ( @@ -68,6 +71,8 @@ storiesOf("Transactions Details", module) refreshNodes={noop} lastUpdated={lastUpdated} isDataValid={true} + limit={100} + reqSortSetting={SqlStatsSortOptions.EXECUTION_COUNT} /> )) .add("with error alert", () => ( @@ -88,6 +93,8 @@ storiesOf("Transactions Details", module) refreshNodes={noop} lastUpdated={lastUpdated} isDataValid={false} + limit={100} + reqSortSetting={SqlStatsSortOptions.EXECUTION_COUNT} /> )) .add("No data for this time frame; no cached transaction text", () => { @@ -108,6 +115,8 @@ storiesOf("Transactions Details", module) refreshNodes={noop} lastUpdated={lastUpdated} isDataValid={true} + limit={100} + reqSortSetting={SqlStatsSortOptions.EXECUTION_COUNT} /> ); }); diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetails.tsx index 6d8660c6f5dd..14a095bb380f 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetails.tsx @@ -58,7 +58,7 @@ import { } from "src/statementsTable/statementsTable"; import { Transaction } from "src/transactionsTable"; import Long from "long"; -import { StatementsRequest } from "../api"; +import { createCombinedStmtsRequest, StatementsRequest } from "../api"; import { getValidOption, TimeScale, @@ -68,7 +68,7 @@ import { toRoundedDateRange, } from "../timeScaleDropdown"; import timeScaleStyles from "../timeScaleDropdown/timeScale.module.scss"; - +import { SqlStatsSortType } from "src/api/statementsApi"; const { containerClass } = tableClasses; const cx = classNames.bind(statementsStyles); const timeScaleStylesCx = classNames.bind(timeScaleStyles); @@ -80,6 +80,8 @@ const transactionDetailsStylesCx = classNames.bind(transactionDetailsStyles); export interface TransactionDetailsStateProps { timeScale: TimeScale; + limit: number; + reqSortSetting: SqlStatsSortType; error?: Error | null; isTenant: UIConfigState["isTenant"]; hasViewActivityRedactedRole?: UIConfigState["hasViewActivityRedactedRole"]; @@ -113,9 +115,11 @@ function statementsRequestFromProps( props: TransactionDetailsProps, ): protos.cockroach.server.serverpb.CombinedStatementsStatsRequest { const [start, end] = toRoundedDateRange(props.timeScale); - return new protos.cockroach.server.serverpb.CombinedStatementsStatsRequest({ - start: Long.fromNumber(start.unix()), - end: Long.fromNumber(end.unix()), + return createCombinedStmtsRequest({ + start, + end, + limit: props.limit, + sort: props.reqSortSetting, }); } diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetailsConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetailsConnected.tsx index 6e12e0ae6861..40e60b8ca752 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetailsConnected.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetailsConnected.tsx @@ -16,6 +16,7 @@ import { Dispatch } from "redux"; import { AppState, uiConfigActions } from "src/store"; import { actions as nodesActions } from "../store/nodes"; import { actions as sqlStatsActions } from "src/store/sqlStats"; +import { actions as txnStatsActions } from "src/store/transactionStats"; import { TransactionDetails, TransactionDetailsDispatchProps, @@ -31,13 +32,17 @@ import { selectHasViewActivityRedactedRole, } from "../store/uiConfig"; import { nodeRegionsByIDSelector } from "../store/nodes"; -import { selectTimeScale } from "src/statementsPage/statementsPage.selectors"; +import { + selectTimeScale, + selectTxnsPageLimit, + selectTxnsPageReqSort, +} from "../store/utils/selectors"; import { StatementsRequest } from "src/api/statementsApi"; import { txnFingerprintIdAttr, getMatchParamByName } from "../util"; import { TimeScale } from "../timeScaleDropdown"; export const selectTransaction = createSelector( - (state: AppState) => state.adminUI?.sqlStats, + (state: AppState) => state.adminUI?.transactions, (_state: AppState, props: RouteComponentProps) => props, (transactionState, props) => { const transactions = transactionState.data?.transactions; @@ -92,6 +97,8 @@ const mapStateToProps = ( hasViewActivityRedactedRole: selectHasViewActivityRedactedRole(state), isDataValid: isValid, lastUpdated, + limit: selectTxnsPageLimit(state), + reqSortSetting: selectTxnsPageReqSort(state), }; }; @@ -99,7 +106,7 @@ const mapDispatchToProps = ( dispatch: Dispatch, ): TransactionDetailsDispatchProps => ({ refreshData: (req?: StatementsRequest) => - dispatch(sqlStatsActions.refresh(req)), + dispatch(txnStatsActions.refresh(req)), refreshNodes: () => dispatch(nodesActions.refresh()), refreshUserSQLRoles: () => dispatch(uiConfigActions.refreshUserSQLRoles()), onTimeScaleChange: (ts: TimeScale) => { diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.selectors.ts index bef5b4e521c1..3a9fd6b740e4 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.selectors.ts @@ -10,18 +10,31 @@ import { createSelector } from "reselect"; -import { localStorageSelector } from "../statementsPage/statementsPage.selectors"; -import { sqlStatsSelector } from "../store/sqlStats/sqlStats.selector"; +import { localStorageSelector } from "../store/utils/selectors"; +import { txnStatsSelector } from "../store/transactionStats/txnStats.selector"; export const selectTransactionsData = createSelector( - sqlStatsSelector, - transactionsState => - // The state is valid if we have successfully fetched data, and it has not yet been invalidated. - transactionsState.valid ? transactionsState.data : null, + txnStatsSelector, + transactionsState => transactionsState?.data, +); + +export const selectTransactionsDataValid = createSelector( + txnStatsSelector, + state => state?.valid, +); + +export const selectTransactionsDataInFlight = createSelector( + txnStatsSelector, + state => state?.inFlight, +); + +export const selectTransactionsLastUpdated = createSelector( + txnStatsSelector, + state => state.lastUpdated, ); export const selectTransactionsLastError = createSelector( - sqlStatsSelector, + txnStatsSelector, state => state.lastError, ); diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.stories.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.stories.tsx index 37003704d5cb..a62648ad9969 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.stories.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.stories.tsx @@ -25,12 +25,20 @@ import { import { TransactionsPage } from "."; import { RequestError } from "../util"; import moment from "moment"; +import { SqlStatsSortOptions } from "../api"; const getEmptyData = () => extend({}, data, { transactions: [], statements: [] }); const lastUpdated = moment.utc(); +const defaultReqParaProps = { + limit: 100, + reqSortSetting: SqlStatsSortOptions.PCT_RUNTIME, + onChangeLimit: noop, + onChangeReqSort: noop, +}; + storiesOf("Transactions Page", module) .addDecorator(storyFn => {storyFn()}) .addDecorator(storyFn => ( @@ -56,6 +64,7 @@ storiesOf("Transactions Page", module) lastUpdated={lastUpdated} isDataValid={true} isReqInFlight={false} + {...defaultReqParaProps} /> )) .add("without data", () => { @@ -79,6 +88,7 @@ storiesOf("Transactions Page", module) lastUpdated={lastUpdated} isDataValid={true} isReqInFlight={false} + {...defaultReqParaProps} /> ); }) @@ -110,6 +120,7 @@ storiesOf("Transactions Page", module) lastUpdated={lastUpdated} isDataValid={true} isReqInFlight={false} + {...defaultReqParaProps} /> ); }) @@ -134,6 +145,7 @@ storiesOf("Transactions Page", module) lastUpdated={lastUpdated} isDataValid={true} isReqInFlight={true} + {...defaultReqParaProps} /> ); }) @@ -165,6 +177,7 @@ storiesOf("Transactions Page", module) lastUpdated={lastUpdated} isDataValid={false} isReqInFlight={false} + {...defaultReqParaProps} /> ); }); diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.tsx index f9324cb8e19b..12c3c06fdce5 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.tsx @@ -26,7 +26,6 @@ import { updateSortSettingQueryParamsOnTab, } from "../sortedtable"; import { Pagination } from "../pagination"; -import { TableStatistics } from "../tableStatistics"; import { statisticsClasses } from "./transactionsPageClasses"; import { aggregateAcrossNodeIDs, @@ -35,7 +34,6 @@ import { searchTransactionsData, filterTransactions, } from "./utils"; -import Long from "long"; import { merge } from "lodash"; import { unique, syncHistory } from "src/util"; import { EmptyTransactionsPlaceholder } from "./emptyTransactionsPlaceholder"; @@ -51,7 +49,11 @@ import { updateFiltersQueryParamsOnTab, } from "../queryFilter"; import { UIConfigState } from "../store"; -import { StatementsRequest } from "src/api/statementsApi"; +import { + SqlStatsSortType, + createCombinedStmtsRequest, + StatementsRequest, +} from "src/api/statementsApi"; import ColumnsSelector from "../columnsSelector/columnsSelector"; import { SelectOption } from "../multiSelectCheckbox/multiSelectCheckbox"; import { @@ -62,7 +64,6 @@ import ClearStats from "../sqlActivity/clearStats"; import LoadingError from "../sqlActivity/errorComponent"; import { commonStyles } from "../common"; import { - TimeScaleDropdown, TimeScale, timeScaleToString, timeScale1hMinOptions, @@ -71,15 +72,26 @@ import { } from "../timeScaleDropdown"; import { InlineAlert } from "@cockroachlabs/ui-components"; import moment from "moment"; -import { STATS_LONG_LOADING_DURATION } from "../util/constants"; +import { + STATS_LONG_LOADING_DURATION, + txnRequestSortOptions, + getSortLabel, +} from "src/util/sqlActivityConstants"; +import { Button } from "src/button"; +import { SearchCriteria } from "src/searchCriteria/searchCriteria"; +import timeScaleStyles from "../timeScaleDropdown/timeScale.module.scss"; type IStatementsResponse = protos.cockroach.server.serverpb.IStatementsResponse; const cx = classNames.bind(styles); +const timeScaleStylesCx = classNames.bind(timeScaleStyles); interface TState { filters?: Filters; pagination: ISortedTablePagination; + timeScale: TimeScale; + limit: number; + reqSortSetting: SqlStatsSortType; } export interface TransactionsPageStateProps { @@ -89,11 +101,12 @@ export interface TransactionsPageStateProps { isReqInFlight: boolean; lastUpdated: moment.Moment | null; timeScale: TimeScale; + limit: number; + reqSortSetting: SqlStatsSortType; error?: Error | null; filters: Filters; isTenant?: UIConfigState["isTenant"]; nodeRegions: { [nodeId: string]: string }; - pageSize?: number; search: string; sortSetting: SortSetting; hasAdminRole?: UIConfigState["hasAdminRole"]; @@ -105,6 +118,8 @@ export interface TransactionsPageDispatchProps { refreshUserSQLRoles: () => void; resetSQLStats: () => void; onTimeScaleChange?: (ts: TimeScale) => void; + onChangeLimit: (limit: number) => void; + onChangeReqSort: (sort: SqlStatsSortType) => void; onColumnsChange?: (selectedColumns: string[]) => void; onFilterChange?: (value: Filters) => void; onSearchComplete?: (query: string) => void; @@ -119,11 +134,15 @@ export type TransactionsPageProps = TransactionsPageStateProps & TransactionsPageDispatchProps & RouteComponentProps; -function stmtsRequestFromTimeScale(ts: TimeScale): StatementsRequest { - const [start, end] = toRoundedDateRange(ts); - return new protos.cockroach.server.serverpb.CombinedStatementsStatsRequest({ - start: Long.fromNumber(start.unix()), - end: Long.fromNumber(end.unix()), +type RequestParams = Pick; + +function stmtsRequestFromParams(params: RequestParams): StatementsRequest { + const [start, end] = toRoundedDateRange(params.timeScale); + return createCombinedStmtsRequest({ + start, + end, + limit: params.limit, + sort: params.reqSortSetting, }); } export class TransactionsPage extends React.Component< @@ -132,9 +151,13 @@ export class TransactionsPage extends React.Component< > { constructor(props: TransactionsPageProps) { super(props); + this.state = { + limit: this.props.limit, + timeScale: this.props.timeScale, + reqSortSetting: this.props.reqSortSetting, pagination: { - pageSize: this.props.pageSize || 20, + pageSize: 50, current: 1, }, }; @@ -181,7 +204,7 @@ export class TransactionsPage extends React.Component< }; refreshData = (): void => { - const req = stmtsRequestFromTimeScale(this.props.timeScale); + const req = stmtsRequestFromParams(this.state); this.props.refreshData(req); }; @@ -243,18 +266,11 @@ export class TransactionsPage extends React.Component< ); } - componentDidUpdate(prevProps: TransactionsPageProps): void { + componentDidUpdate(): void { this.updateQueryParams(); if (!this.props.isTenant) { this.props.refreshNodes(); } - - if ( - prevProps.timeScale !== this.props.timeScale || - (prevProps.isDataValid && !this.props.isDataValid) - ) { - this.refreshData(); - } } onChangeSortSetting = (ss: SortSetting): void => { @@ -360,12 +376,25 @@ export class TransactionsPage extends React.Component< }; changeTimeScale = (ts: TimeScale): void => { - if (this.props.onTimeScaleChange) { - this.props.onTimeScaleChange(ts); - } + this.setState(prevState => ({ ...prevState, timeScale: ts })); }; - render(): React.ReactElement { + onChangeLimit = (newLimit: number): void => { + this.setState(prevState => ({ ...prevState, limit: newLimit })); + }; + + onChangeReqSort = (newSort: SqlStatsSortType): void => { + this.setState(prevState => ({ ...prevState, reqSortSetting: newSort })); + }; + + updateRequestParams = (): void => { + this.props.onChangeLimit(this.state.limit); + this.props.onChangeReqSort(this.state.reqSortSetting); + this.props.onTimeScaleChange(this.state.timeScale); + this.refreshData(); + }; + + renderTransactions(): React.ReactElement { const { data, nodeRegions, @@ -376,9 +405,9 @@ export class TransactionsPage extends React.Component< search, hasAdminRole, } = this.props; + const { pagination, filters } = this.state; const internal_app_name_prefix = data?.internal_app_name_prefix || ""; const statements = data?.statements || []; - const { filters } = this.state; // If the cluster is a tenant cluster we don't show info // about nodes/regions. @@ -390,6 +419,7 @@ export class TransactionsPage extends React.Component< const regions = isTenant ? [] : unique(nodes.map(node => nodeRegions[node.toString()])).sort(); + // We apply the search filters and app name filters prior to aggregating across Node IDs // in order to match what's done on the Statements Page. // @@ -412,160 +442,179 @@ export class TransactionsPage extends React.Component< internal_app_name_prefix, ); + const transactionsToDisplay: TransactionInfo[] = aggregateAcrossNodeIDs( + filteredTransactions, + statements, + ).map(t => ({ + stats_data: t.stats_data, + node_id: t.node_id, + regionNodes: isTenant + ? [] + : generateRegionNode(t, statements, nodeRegions), + })); + const { current, pageSize } = pagination; + const hasData = data?.transactions?.length > 0; + const isUsedFilter = search?.length > 0; + + // Creates a list of all possible columns, + // hiding nodeRegions if is not multi-region and + // hiding columns that won't be displayed for tenants. + const columns = makeTransactionsColumns( + transactionsToDisplay, + statements, + isTenant, + search, + ) + .filter(c => !(c.name === "regionNodes" && regions.length < 2)) + .filter(c => !(isTenant && c.hideIfTenant)); + + const isColumnSelected = (c: ColumnDescriptor) => { + return ( + ((userSelectedColumnsToShow === null || + userSelectedColumnsToShow === undefined) && + c.showByDefault !== false) || // show column if list of visible was never defined and can be show by default. + (userSelectedColumnsToShow !== null && + userSelectedColumnsToShow.includes(c.name)) || // show column if user changed its visibility. + c.alwaysShow === true // show column if alwaysShow option is set explicitly. + ); + }; + + // Iterate over all available columns and create list of SelectOptions with initial selection + // values based on stored user selections in local storage and default column configs. + // Columns that are set to alwaysShow are filtered from the list. + const tableColumns = columns + .filter(c => !c.alwaysShow) + .map( + (c): SelectOption => ({ + label: getLabel(c.name as StatisticTableColumnKeys, "transaction"), + value: c.name, + isSelected: isColumnSelected(c), + }), + ); + // List of all columns that will be displayed based on the column selection. + const displayColumns = columns.filter(c => isColumnSelected(c)); + + const period = timeScaleToString(this.props.timeScale); + + const clearFilter = activeFilters ? ( + + + + ) : ( + <> + ); + + const sortSettingLabel = getSortLabel(this.props.reqSortSetting); + return ( + <> +
+ {`Results - Top ${this.props.limit} Transaction Fingerprints by ${sortSettingLabel}`} +
+
+ + + + + + "n" + n)} + activeFilters={activeFilters} + filters={filters} + showRegions={regions.length > 1} + showNodes={!isTenant && nodes.length > 1} + /> + + + + + {clearFilter} + + + +

+ Showing aggregated stats from{" "} + {period} +

+
+ {hasAdminRole && ( + + + + )} +
+
+
+ + } + /> +
+ + + ); + } + + render(): React.ReactElement { const longLoadingMessage = ( ); return ( <> - - - - - - "n" + n)} - activeFilters={activeFilters} - filters={filters} - showRegions={regions.length > 1} - showNodes={nodes.length > 1} - /> - - - - - {hasAdminRole && ( - - - - )} - +
{ - const { pagination } = this.state; - const transactionsToDisplay: TransactionInfo[] = aggregateAcrossNodeIDs( - filteredTransactions, - statements, - ).map(t => ({ - stats_data: t.stats_data, - node_id: t.node_id, - regionNodes: isTenant - ? [] - : generateRegionNode(t, statements, nodeRegions), - })); - const { current, pageSize } = pagination; - const hasData = data?.transactions?.length > 0; - const isUsedFilter = search?.length > 0; - - // Creates a list of all possible columns, - // hiding nodeRegions if is not multi-region and - // hiding columns that won't be displayed for tenants. - const columns = makeTransactionsColumns( - transactionsToDisplay, - statements, - isTenant, - search, - ) - .filter(c => !(c.name === "regionNodes" && regions.length < 2)) - .filter(c => !(isTenant && c.hideIfTenant)); - - const isColumnSelected = ( - c: ColumnDescriptor, - ) => { - return ( - ((userSelectedColumnsToShow === null || - userSelectedColumnsToShow === undefined) && - c.showByDefault !== false) || // show column if list of visible was never defined and can be show by default. - (userSelectedColumnsToShow !== null && - userSelectedColumnsToShow.includes(c.name)) || // show column if user changed its visibility. - c.alwaysShow === true // show column if alwaysShow option is set explicitly. - ); - }; - - // Iterate over all available columns and create list of SelectOptions with initial selection - // values based on stored user selections in local storage and default column configs. - // Columns that are set to alwaysShow are filtered from the list. - const tableColumns = columns - .filter(c => !c.alwaysShow) - .map( - (c): SelectOption => ({ - label: getLabel( - c.name as StatisticTableColumnKeys, - "transaction", - ), - value: c.name, - isSelected: isColumnSelected(c), - }), - ); - - // List of all columns that will be displayed based on the column selection. - const displayColumns = columns.filter(c => isColumnSelected(c)); - - const period = timeScaleToString(this.props.timeScale); - - return ( - <> -
- - - - } - /> -
- - - ); - }} + render={() => this.renderTransactions()} renderError={() => LoadingError({ statsType: "transactions", diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPageConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPageConnected.tsx index 6b763671a9b5..ab94b5b22f10 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPageConnected.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPageConnected.tsx @@ -16,6 +16,7 @@ import { AppState, uiConfigActions } from "src/store"; import { actions as nodesActions } from "src/store/nodes"; import { actions as sqlStatsActions } from "src/store/sqlStats"; import { TransactionsPage } from "./transactionsPage"; +import { actions as txnStatsActions } from "src/store/transactionStats"; import { TransactionsPageStateProps, TransactionsPageDispatchProps, @@ -27,17 +28,23 @@ import { selectSortSetting, selectFilters, selectSearch, + selectTransactionsDataValid, + selectTransactionsLastUpdated, + selectTransactionsDataInFlight, } from "./transactionsPage.selectors"; import { selectHasAdminRole, selectIsTenant } from "../store/uiConfig"; import { nodeRegionsByIDSelector } from "../store/nodes"; import { - selectStatementsLastUpdated, - selectStatementsDataValid, - selectStatementsDataInFlight, -} from "src/statementsPage/statementsPage.selectors"; -import { selectTimeScale } from "../store/utils/selectors"; -import { StatementsRequest } from "src/api/statementsApi"; -import { actions as localStorageActions } from "../store/localStorage"; + selectTxnsPageLimit, + selectTxnsPageReqSort, + selectTimeScale, +} from "../store/utils/selectors"; +import { SqlStatsSortType, StatementsRequest } from "src/api/statementsApi"; +import { + actions as localStorageActions, + updateTxnsPageLimitAction, + updateTxnsPageReqSortAction, +} from "../store/localStorage"; import { Filters } from "../queryFilter"; import { actions as analyticsActions } from "../store/analytics"; import { TimeScale } from "../timeScaleDropdown"; @@ -51,9 +58,9 @@ export const TransactionsPageConnected = withRouter( (state: AppState) => ({ columns: selectTxnColumns(state), data: selectTransactionsData(state), - isDataValid: selectStatementsDataValid(state), - isReqInFlight: selectStatementsDataInFlight(state), - lastUpdated: selectStatementsLastUpdated(state), + isDataValid: selectTransactionsDataValid(state), + isReqInFlight: selectTransactionsDataInFlight(state), + lastUpdated: selectTransactionsLastUpdated(state), timeScale: selectTimeScale(state), error: selectTransactionsLastError(state), filters: selectFilters(state), @@ -62,10 +69,12 @@ export const TransactionsPageConnected = withRouter( search: selectSearch(state), sortSetting: selectSortSetting(state), hasAdminRole: selectHasAdminRole(state), + limit: selectTxnsPageLimit(state), + reqSortSetting: selectTxnsPageReqSort(state), }), (dispatch: Dispatch) => ({ refreshData: (req: StatementsRequest) => - dispatch(sqlStatsActions.refresh(req)), + dispatch(txnStatsActions.refresh(req)), refreshNodes: () => dispatch(nodesActions.refresh()), refreshUserSQLRoles: () => dispatch(uiConfigActions.refreshUserSQLRoles()), @@ -131,6 +140,10 @@ export const TransactionsPageConnected = withRouter( }), ); }, + onChangeLimit: (limit: number) => + dispatch(updateTxnsPageLimitAction(limit)), + onChangeReqSort: (sort: SqlStatsSortType) => + dispatch(updateTxnsPageReqSortAction(sort)), }), )(TransactionsPage), ); diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx index d1305a882889..0e185b08fd6e 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx @@ -128,14 +128,16 @@ export function makeTransactionsColumns( title: statisticsTableTitles.transactions(statType), cell: (item: TransactionInfo) => transactionLink({ - transactionText: statementFingerprintIdsToText( - item.stats_data.statement_fingerprint_ids, - statements, - ), - transactionSummary: statementFingerprintIdsToSummarizedText( - item.stats_data.statement_fingerprint_ids, - statements, - ), + transactionText: + statementFingerprintIdsToText( + item.stats_data.statement_fingerprint_ids, + statements, + ) || "Transaction query unavailable.", + transactionSummary: + statementFingerprintIdsToSummarizedText( + item.stats_data.statement_fingerprint_ids, + statements, + ) || "Transaction query unavailable.", aggregatedTs: TimestampToString(item.stats_data.aggregated_ts), transactionFingerprintId: item.stats_data.transaction_fingerprint_id.toString(), search, diff --git a/pkg/ui/workspaces/cluster-ui/src/util/constants.ts b/pkg/ui/workspaces/cluster-ui/src/util/constants.ts index 43df28f9a999..66e2ca61d8be 100644 --- a/pkg/ui/workspaces/cluster-ui/src/util/constants.ts +++ b/pkg/ui/workspaces/cluster-ui/src/util/constants.ts @@ -41,5 +41,3 @@ export const serverToClientErrorMessageMap = new Map([ REMOTE_DEBUGGING_ERROR_TEXT, ], ]); - -export const STATS_LONG_LOADING_DURATION = duration(2, "s"); diff --git a/pkg/ui/workspaces/cluster-ui/src/util/sqlActivityConstants.ts b/pkg/ui/workspaces/cluster-ui/src/util/sqlActivityConstants.ts new file mode 100644 index 000000000000..f5430e746e44 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/util/sqlActivityConstants.ts @@ -0,0 +1,47 @@ +// Copyright 2023 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 { duration } from "moment"; +import { SqlStatsSortOptions, SqlStatsSortType } from "src/api/statementsApi"; + +export const limitOptions = [ + { value: 25, label: "25" }, + { value: 50, label: "50" }, + { value: 100, label: "100" }, + { value: 500, label: "500" }, +]; + +export function getSortLabel(sort: SqlStatsSortType): string { + switch (sort) { + case SqlStatsSortOptions.SERVICE_LAT: + return "Service Latency"; + case SqlStatsSortOptions.EXECUTION_COUNT: + return "Execution Count"; + case SqlStatsSortOptions.CONTENTION_TIME: + return "Contention Time"; + case SqlStatsSortOptions.PCT_RUNTIME: + return "% Of All Run Time"; + default: + return ""; + } +} + +export const stmtRequestSortOptions = Object.values(SqlStatsSortOptions).map( + sortVal => ({ + value: sortVal as SqlStatsSortType, + label: getSortLabel(sortVal as SqlStatsSortType), + }), +); + +export const txnRequestSortOptions = stmtRequestSortOptions.filter( + option => option.value !== SqlStatsSortOptions.PCT_RUNTIME, +); + +export const STATS_LONG_LOADING_DURATION = duration(2, "s"); diff --git a/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts b/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts index e5cb56351fb7..6107a4816e0c 100644 --- a/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts +++ b/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts @@ -26,6 +26,7 @@ import * as api from "src/util/api"; import { VersionList } from "src/interfaces/cockroachlabs"; import { versionCheck } from "src/util/cockroachlabsAPI"; import { INodeStatus, RollupStoreMetrics } from "src/util/proto"; +import { api as clusterUiApi } from "@cockroachlabs/cluster-ui"; import * as protos from "src/js/protos"; // The primary export of this file are the "refresh" functions of the various @@ -317,15 +318,25 @@ const storesReducerObj = new KeyedCachedDataReducer( export const refreshStores = storesReducerObj.refresh; const queriesReducerObj = new CachedDataReducer( - api.getCombinedStatements, + clusterUiApi.getCombinedStatements, "statements", - moment.duration(5, "m"), - moment.duration(30, "m"), - true, // Allow new requests to replace in flight ones. + null, + moment.duration(10, "m"), + true, ); export const invalidateStatements = queriesReducerObj.invalidateData; export const refreshStatements = queriesReducerObj.refresh; +const txnFingerprintStatsReducerObj = new CachedDataReducer( + clusterUiApi.getFlushedTxnStatsApi, + "transactions", + null, + moment.duration(30, "m"), + true, +); +export const invalidateTxns = txnFingerprintStatsReducerObj.invalidateData; +export const refreshTxns = txnFingerprintStatsReducerObj.refresh; + export const statementDetailsRequestToID = ( req: api.StatementDetailsRequestMessage, ): string => @@ -416,6 +427,7 @@ export interface APIReducersState { settings: CachedDataReducerState; stores: KeyedCachedDataReducerState; statements: CachedDataReducerState; + transactions: CachedDataReducerState; statementDetails: KeyedCachedDataReducerState< api.StatementDetailsResponseMessage >; @@ -457,6 +469,8 @@ export const apiReducersReducer = combineReducers({ [sessionsReducerObj.actionNamespace]: sessionsReducerObj.reducer, [storesReducerObj.actionNamespace]: storesReducerObj.reducer, [queriesReducerObj.actionNamespace]: queriesReducerObj.reducer, + [txnFingerprintStatsReducerObj.actionNamespace]: + txnFingerprintStatsReducerObj.reducer, [statementDetailsReducerObj.actionNamespace]: statementDetailsReducerObj.reducer, [dataDistributionReducerObj.actionNamespace]: diff --git a/pkg/ui/workspaces/db-console/src/redux/timeScale.ts b/pkg/ui/workspaces/db-console/src/redux/timeScale.ts index 03bf87b55d41..e372ca0bfc71 100644 --- a/pkg/ui/workspaces/db-console/src/redux/timeScale.ts +++ b/pkg/ui/workspaces/db-console/src/redux/timeScale.ts @@ -14,7 +14,7 @@ */ import { Action } from "redux"; -import { put, takeEvery } from "redux-saga/effects"; +import { put, takeEvery, all } from "redux-saga/effects"; import { PayloadAction } from "src/interfaces/action"; import _ from "lodash"; import { defaultTimeScaleOptions } from "@cockroachlabs/cluster-ui"; @@ -25,7 +25,7 @@ import { getValueFromSessionStorage, setLocalSetting, } from "src/redux/localsettings"; -import { invalidateStatements } from "./apiReducers"; +import { invalidateStatements, invalidateTxns } from "./apiReducers"; export const SET_SCALE = "cockroachui/timewindow/SET_SCALE"; export const SET_METRICS_MOVING_WINDOW = @@ -258,6 +258,6 @@ export const adjustTimeScale = ( export function* timeScaleSaga() { yield takeEvery(SET_SCALE, function*({ payload }: PayloadAction) { yield put(setLocalSetting(TIME_SCALE_SESSION_STORAGE_KEY, payload)); - yield put(invalidateStatements()); + yield all([put(invalidateStatements()), put(invalidateTxns())]); }); } diff --git a/pkg/ui/workspaces/db-console/src/util/api.ts b/pkg/ui/workspaces/db-console/src/util/api.ts index 806690dcdd57..fe994cd53c44 100644 --- a/pkg/ui/workspaces/db-console/src/util/api.ts +++ b/pkg/ui/workspaces/db-console/src/util/api.ts @@ -724,23 +724,6 @@ export function getStores( ); } -// getCombinedStatements returns statements the cluster has recently executed, and some stats about them. -export function getCombinedStatements( - req: StatementsRequestMessage, - timeout?: moment.Duration, -): Promise { - const queryStr = propsToQueryString({ - start: req.start.toInt(), - end: req.end.toInt(), - }); - return timeoutFetch( - serverpb.StatementsResponse, - `${STATUS_PREFIX}/combinedstmts?${queryStr}`, - null, - timeout, - ); -} - // getStatementDetails returns the statistics about the selected statement. export function getStatementDetails( req: StatementDetailsRequestMessage, diff --git a/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx b/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx index d42278fc87ca..4742784fd4eb 100644 --- a/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx +++ b/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx @@ -40,6 +40,7 @@ import { Filters, defaultFilters, util, + api, } from "@cockroachlabs/cluster-ui"; import { cancelStatementDiagnosticsReportAction, @@ -260,7 +261,7 @@ export const statementColumnsLocalSetting = new LocalSetting( ); export const sortSettingLocalSetting = new LocalSetting( - "sortSetting/StatementsPage", + "tableSortSetting/StatementsPage", (state: AdminUIState) => state.localSettings, { ascending: false, columnTitle: "executionCount" }, ); @@ -277,6 +278,18 @@ export const searchLocalSetting = new LocalSetting( null, ); +export const reqSortSetting = new LocalSetting( + "reqSortSetting/StatementsPage", + (state: AdminUIState) => state.localSettings, + api.DEFAULT_STATS_REQ_OPTIONS.sort, +); + +export const limitSetting = new LocalSetting( + "reqLimitSetting/StatementsPage", + (state: AdminUIState) => state.localSettings, + api.DEFAULT_STATS_REQ_OPTIONS.limit, +); + export default withRouter( connect( (state: AdminUIState, props: RouteComponentProps) => ({ @@ -297,6 +310,10 @@ export default withRouter( totalFingerprints: selectTotalFingerprints(state), hasViewActivityRedactedRole: selectHasViewActivityRedactedRole(state), hasAdminRole: selectHasAdminRole(state), + limit: limitSetting.selector(state), + reqSortSetting: reqSortSetting.selector(state), + stmtsTotalRuntimeSecs: + state.cachedData?.statements?.data?.stmts_total_runtime_secs ?? 0, }), { refreshStatements: refreshStatements, @@ -353,6 +370,8 @@ export default withRouter( statementColumnsLocalSetting.set( value.length === 0 ? " " : value.join(","), ), + onChangeLimit: (newLimit: number) => limitSetting.set(newLimit), + onChangeReqSort: (sort: api.SqlStatsSortType) => reqSortSetting.set(sort), }, )(StatementsPage), ); diff --git a/pkg/ui/workspaces/db-console/src/views/transactions/transactionDetails.tsx b/pkg/ui/workspaces/db-console/src/views/transactions/transactionDetails.tsx index 2497ae96901b..f47d83ace752 100644 --- a/pkg/ui/workspaces/db-console/src/views/transactions/transactionDetails.tsx +++ b/pkg/ui/workspaces/db-console/src/views/transactions/transactionDetails.tsx @@ -13,7 +13,7 @@ import { createSelector } from "reselect"; import { RouteComponentProps, withRouter } from "react-router-dom"; import { refreshNodes, - refreshStatements, + refreshTxns, refreshUserSQLRoles, } from "src/redux/apiReducers"; import { AdminUIState } from "src/redux/state"; @@ -21,8 +21,10 @@ import { txnFingerprintIdAttr } from "src/util/constants"; import { getMatchParamByName } from "src/util/query"; import { nodeRegionsByIDSelector } from "src/redux/nodes"; import { + reqSortSetting, selectData, selectLastError, + limitSetting, } from "src/views/transactions/transactionsPage"; import { TransactionDetailsStateProps, @@ -34,7 +36,7 @@ import { setGlobalTimeScaleAction } from "src/redux/statements"; import { selectTimeScale } from "src/redux/timeScale"; export const selectTransaction = createSelector( - (state: AdminUIState) => state.cachedData.statements, + (state: AdminUIState) => state.cachedData.transactions, (_state: AdminUIState, props: RouteComponentProps) => props, (transactionState, props) => { const transactions = transactionState.data?.transactions; @@ -92,10 +94,12 @@ export default withRouter( isLoading: isLoading, lastUpdated: lastUpdated, isDataValid: isValid, + limit: limitSetting.selector(state), + reqSortSetting: reqSortSetting.selector(state), }; }, { - refreshData: refreshStatements, + refreshData: refreshTxns, refreshNodes, refreshUserSQLRoles, onTimeScaleChange: setGlobalTimeScaleAction, diff --git a/pkg/ui/workspaces/db-console/src/views/transactions/transactionsPage.tsx b/pkg/ui/workspaces/db-console/src/views/transactions/transactionsPage.tsx index f97693b8bcb5..62d68ed64ce2 100644 --- a/pkg/ui/workspaces/db-console/src/views/transactions/transactionsPage.tsx +++ b/pkg/ui/workspaces/db-console/src/views/transactions/transactionsPage.tsx @@ -13,7 +13,7 @@ import { createSelector } from "reselect"; import { withRouter } from "react-router-dom"; import { refreshNodes, - refreshStatements, + refreshTxns, refreshUserSQLRoles, } from "src/redux/apiReducers"; import { resetSQLStatsAction } from "src/redux/sqlStats"; @@ -29,16 +29,26 @@ import { Filters, defaultFilters, util, + api, } from "@cockroachlabs/cluster-ui"; import { nodeRegionsByIDSelector } from "src/redux/nodes"; import { setGlobalTimeScaleAction } from "src/redux/statements"; import { LocalSetting } from "src/redux/localsettings"; import { selectTimeScale } from "src/redux/timeScale"; -// selectStatements returns the array of AggregateStatistics to show on the +export const selectTxnsLastUpdated = (state: AdminUIState) => + state.cachedData.transactions?.setAt?.utc(); + +export const selectTxnsDataValid = (state: AdminUIState) => + state.cachedData.transactions?.valid; + +export const selectTxnsDataInFlight = (state: AdminUIState) => + state.cachedData.transactions?.inFlight; + +// selectData returns the array of AggregateStatistics to show on the // TransactionsPage, based on if the appAttr route parameter is set. export const selectData = createSelector( - (state: AdminUIState) => state.cachedData.statements, + (state: AdminUIState) => state.cachedData.transactions, (state: CachedDataReducerState) => { if (!state.data || state.inFlight || !state.valid) return null; return state.data; @@ -48,7 +58,7 @@ export const selectData = createSelector( // selectLastReset returns a string displaying the last time the statement // statistics were reset. export const selectLastReset = createSelector( - (state: AdminUIState) => state.cachedData.statements, + (state: AdminUIState) => state.cachedData.transactions, (state: CachedDataReducerState) => { if (!state.data) { return "unknown"; @@ -59,7 +69,7 @@ export const selectLastReset = createSelector( ); export const selectLastError = createSelector( - (state: AdminUIState) => state.cachedData.statements, + (state: AdminUIState) => state.cachedData.transactions, (state: CachedDataReducerState) => state.lastError, ); @@ -87,14 +97,26 @@ export const transactionColumnsLocalSetting = new LocalSetting( null, ); +export const reqSortSetting = new LocalSetting( + "reqSortSetting/TransactionsPage", + (state: AdminUIState) => state.localSettings, + api.DEFAULT_STATS_REQ_OPTIONS.sort, +); + +export const limitSetting = new LocalSetting( + "reqLimitSetting/TransactionsPage", + (state: AdminUIState) => state.localSettings, + api.DEFAULT_STATS_REQ_OPTIONS.limit, +); + const TransactionsPageConnected = withRouter( connect( (state: AdminUIState) => ({ columns: transactionColumnsLocalSetting.selectorToArray(state), data: selectData(state), - isDataValid: state?.cachedData?.statements?.valid ?? false, - isReqInFlight: state?.cachedData?.statements?.inFlight ?? false, - lastUpdated: state?.cachedData?.statements?.setAt, + isDataValid: selectTxnsDataValid(state), + isReqInFlight: selectTxnsDataInFlight(state), + lastUpdated: selectTxnsLastUpdated(state), timeScale: selectTimeScale(state), error: selectLastError(state), filters: filtersLocalSetting.selector(state), @@ -102,11 +124,13 @@ const TransactionsPageConnected = withRouter( nodeRegions: nodeRegionsByIDSelector(state), search: searchLocalSetting.selector(state), sortSetting: sortSettingLocalSetting.selector(state), - statementsError: state.cachedData.statements.lastError, + statementsError: state.cachedData.transactions.lastError, hasAdminRole: selectHasAdminRole(state), + limit: limitSetting.selector(state), + reqSortSetting: reqSortSetting.selector(state), }), { - refreshData: refreshStatements, + refreshData: refreshTxns, refreshNodes, refreshUserSQLRoles, resetSQLStats: resetSQLStatsAction, @@ -130,6 +154,8 @@ const TransactionsPageConnected = withRouter( }), onFilterChange: (filters: Filters) => filtersLocalSetting.set(filters), onSearchComplete: (query: string) => searchLocalSetting.set(query), + onChangeLimit: (newLimit: number) => limitSetting.set(newLimit), + onChangeReqSort: (sort: api.SqlStatsSortType) => reqSortSetting.set(sort), }, )(TransactionsPage), );