From d45991457cea2be98e9b8d1590472c378ce5bca9 Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Tue, 28 Mar 2023 15:09:08 -0400 Subject: [PATCH 1/5] server: stop grouping stmts by transaction_fingerprint_id Previously, the query for the stmt fingerprints overview page grouped stmts on transaction_fingerprint_id. We should remove this grouping criteria since we do not care about the txn at this page level, and including it means we can't return as many unique stmt fingerprint ids on the page. To view txn specific stats / stmts for a txn, users can use the txn fingerprints page. We add the distinct txn_fingerprint_ids for the grouped stmts as a new field for testing purposes. Note that the `transaction_fingerprint_id` in the key_data field for stmts returned by the CombinedStatements api will now be blank. New proto fields: - txn_fingerprint_ids (for StatementsResponse_CollectedStatementStatistics) Part of: #99708 Release note: None --- docs/generated/http/full.md | 2 + pkg/server/combined_statement_stats.go | 159 +++++++++++++++++++++---- pkg/server/serverpb/status.proto | 7 +- pkg/server/status_test.go | 6 +- 4 files changed, 150 insertions(+), 24 deletions(-) diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index d149793676aa..8f71510ef067 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -4000,6 +4000,7 @@ tenant pods. | key | [StatementsResponse.ExtendedStatementStatisticsKey](#cockroach.server.serverpb.StatementsResponse-cockroach.server.serverpb.StatementsResponse.ExtendedStatementStatisticsKey) | | | [reserved](#support-status) | | id | [uint64](#cockroach.server.serverpb.StatementsResponse-uint64) | | | [reserved](#support-status) | | stats | [cockroach.sql.StatementStatistics](#cockroach.server.serverpb.StatementsResponse-cockroach.sql.StatementStatistics) | | | [reserved](#support-status) | +| txn_fingerprint_ids | [uint64](#cockroach.server.serverpb.StatementsResponse-uint64) | repeated | In 23.1 we expect the response to only group on fingerprint_id and app_name in the overview page. We now return the aggregated list of unique txn fingerprint ids, leaving the txn_fingerprint_id field in the key empty. | [reserved](#support-status) | @@ -4111,6 +4112,7 @@ Support status: [reserved](#support-status) | key | [StatementsResponse.ExtendedStatementStatisticsKey](#cockroach.server.serverpb.StatementsResponse-cockroach.server.serverpb.StatementsResponse.ExtendedStatementStatisticsKey) | | | [reserved](#support-status) | | id | [uint64](#cockroach.server.serverpb.StatementsResponse-uint64) | | | [reserved](#support-status) | | stats | [cockroach.sql.StatementStatistics](#cockroach.server.serverpb.StatementsResponse-cockroach.sql.StatementStatistics) | | | [reserved](#support-status) | +| txn_fingerprint_ids | [uint64](#cockroach.server.serverpb.StatementsResponse-uint64) | repeated | In 23.1 we expect the response to only group on fingerprint_id and app_name in the overview page. We now return the aggregated list of unique txn fingerprint ids, leaving the txn_fingerprint_id field in the key empty. | [reserved](#support-status) | diff --git a/pkg/server/combined_statement_stats.go b/pkg/server/combined_statement_stats.go index e43743327435..a82ec06234b3 100644 --- a/pkg/server/combined_statement_stats.go +++ b/pkg/server/combined_statement_stats.go @@ -96,15 +96,13 @@ func getCombinedStatementStats( } 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) + // If we're fetching for txns, the client still expects statement stats for + // stmts in the txns response. + statements, err = collectStmtsForTxns(ctx, ie, req, transactions, testingKnobs) + } else { + statements, err = collectCombinedStatements(ctx, ie, whereClause, args, orderAndLimit, settings, testingKnobs) } - statements, err = collectCombinedStatements(ctx, ie, whereClause, args, orderAndLimit, settings, testingKnobs) - if err != nil { return nil, serverError(ctx, err) } @@ -401,17 +399,15 @@ func collectCombinedStatements( SELECT * FROM ( SELECT fingerprint_id, - transaction_fingerprint_id, + array_agg(distinct transaction_fingerprint_id), app_name, max(aggregated_ts) as aggregated_ts, - metadata, + crdb_internal.merge_stats_metadata(array_agg(metadata)) as metadata, crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics FROM %s %s GROUP BY fingerprint_id, - transaction_fingerprint_id, - app_name, - metadata + app_name ) %s %s` @@ -469,12 +465,18 @@ GROUP BY return nil, serverError(ctx, err) } - var transactionFingerprintID uint64 - if transactionFingerprintID, err = sqlstatsutil.DatumToUint64(row[1]); err != nil { - return nil, serverError(ctx, err) + var txnFingerprintID uint64 + txnFingerprintDatums := tree.MustBeDArray(row[1]) + txnFingerprintIDs := make([]roachpb.TransactionFingerprintID, 0, txnFingerprintDatums.Array.Len()) + for _, idDatum := range txnFingerprintDatums.Array { + if txnFingerprintID, err = sqlstatsutil.DatumToUint64(idDatum); err != nil { + return nil, serverError(ctx, err) + } + txnFingerprintIDs = append(txnFingerprintIDs, roachpb.TransactionFingerprintID(txnFingerprintID)) } app := string(tree.MustBeDString(row[2])) + aggregatedTs := tree.MustBeDTimestampTZ(row[3]).Time var metadata roachpb.CollectedStatementStatistics @@ -484,8 +486,6 @@ GROUP BY } metadata.Key.App = app - metadata.Key.TransactionFingerprintID = - roachpb.TransactionFingerprintID(transactionFingerprintID) statsJSON := tree.MustBeDJSON(row[5]).JSON if err = sqlstatsutil.DecodeStmtStatsStatisticsJSON(statsJSON, &metadata.Stats); err != nil { @@ -497,8 +497,9 @@ GROUP BY KeyData: metadata.Key, AggregatedTs: aggregatedTs, }, - ID: roachpb.StmtFingerprintID(statementFingerprintID), - Stats: metadata.Stats, + ID: roachpb.StmtFingerprintID(statementFingerprintID), + Stats: metadata.Stats, + TxnFingerprintIDs: txnFingerprintIDs, } statements = append(statements, stmt) @@ -528,13 +529,12 @@ SELECT app_name, max(aggregated_ts) as aggregated_ts, fingerprint_id, - metadata, + max(metadata), crdb_internal.merge_transaction_stats(array_agg(statistics)) AS statistics FROM %s %s GROUP BY app_name, - fingerprint_id, - metadata + fingerprint_id ) %s %s` @@ -589,6 +589,7 @@ GROUP BY } app := string(tree.MustBeDString(row[0])) + aggregatedTs := tree.MustBeDTimestampTZ(row[1]).Time fingerprintID, err := sqlstatsutil.DatumToUint64(row[2]) if err != nil { @@ -626,6 +627,120 @@ GROUP BY return transactions, nil } +func collectStmtsForTxns( + ctx context.Context, + ie *sql.InternalExecutor, + req *serverpb.CombinedStatementsStatsRequest, + transactions []serverpb.StatementsResponse_ExtendedCollectedTransactionStatistics, + testingKnobs *sqlstats.TestingKnobs, +) ([]serverpb.StatementsResponse_CollectedStatementStatistics, error) { + + whereClause, args := buildWhereClauseForStmtsByTxn(req, transactions, testingKnobs) + + queryFormat := ` +SELECT + fingerprint_id, + transaction_fingerprint_id, + crdb_internal.merge_stats_metadata(array_agg(metadata)) AS metadata, + crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics, + app_name +FROM %s %s +GROUP BY + fingerprint_id, + transaction_fingerprint_id, + app_name +` + + const expectedNumDatums = 5 + + query := fmt.Sprintf(queryFormat, "crdb_internal.statement_statistics_persisted", whereClause) + + it, err := ie.QueryIteratorEx(ctx, "stmts-for-txn", nil, + sessiondata.NodeUserSessionDataOverride, query, args...) + + if err != nil { + return nil, serverError(ctx, err) + } + + // If there are no results from the persisted table, retrieve the data from the combined view + // with data in-memory. + if !it.HasResults() { + err = closeIterator(it, err) + + query = fmt.Sprintf(queryFormat, `crdb_internal.statement_statistics`, whereClause) + + it, err = ie.QueryIteratorEx(ctx, "stmts-for-txn-with-memory", nil, + sessiondata.NodeUserSessionDataOverride, query, args...) + + if err != nil { + return nil, serverError(ctx, err) + } + } + + defer func() { + closeErr := it.Close() + if closeErr != nil { + err = errors.CombineErrors(err, closeErr) + } + }() + + var statements []serverpb.StatementsResponse_CollectedStatementStatistics + var ok bool + for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { + var row tree.Datums + if row = it.Cur(); row == nil { + return nil, errors.New("unexpected null row on collectStmtsForTxns") + } + + if row.Len() != expectedNumDatums { + return nil, errors.Newf("expected %d columns, received %d on collectStmtsForTxns", expectedNumDatums) + } + + var statementFingerprintID uint64 + if statementFingerprintID, err = sqlstatsutil.DatumToUint64(row[0]); err != nil { + return nil, serverError(ctx, err) + } + + var txnFingerprintID uint64 + if txnFingerprintID, err = sqlstatsutil.DatumToUint64(row[1]); err != nil { + return nil, serverError(ctx, err) + } + + var metadata roachpb.CollectedStatementStatistics + metadataJSON := tree.MustBeDJSON(row[2]).JSON + if err = sqlstatsutil.DecodeStmtStatsMetadataJSON(metadataJSON, &metadata); err != nil { + return nil, serverError(ctx, err) + } + + metadata.Key.TransactionFingerprintID = roachpb.TransactionFingerprintID(txnFingerprintID) + + statsJSON := tree.MustBeDJSON(row[3]).JSON + if err = sqlstatsutil.DecodeStmtStatsStatisticsJSON(statsJSON, &metadata.Stats); err != nil { + return nil, serverError(ctx, err) + } + + app := string(tree.MustBeDString(row[4])) + metadata.Key.App = app + + stmt := serverpb.StatementsResponse_CollectedStatementStatistics{ + Key: serverpb.StatementsResponse_ExtendedStatementStatisticsKey{ + KeyData: metadata.Key, + }, + ID: roachpb.StmtFingerprintID(statementFingerprintID), + Stats: metadata.Stats, + } + + statements = append(statements, stmt) + + } + + if err != nil { + return nil, serverError(ctx, err) + } + + return statements, nil +} + func (s *statusServer) StatementDetails( ctx context.Context, req *serverpb.StatementDetailsRequest, ) (*serverpb.StatementDetailsResponse, error) { diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index ee878a444087..a2dd8e81572d 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -1,4 +1,4 @@ -// Copyright 2016 The Cockroach Authors. + // Copyright 2016 The Cockroach Authors. // // Use of this software is governed by the Business Source License // included in the file licenses/BSL.txt. @@ -1542,6 +1542,11 @@ message StatementsResponse { uint64 id = 3 [(gogoproto.customname) = "ID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StmtFingerprintID"]; cockroach.sql.StatementStatistics stats = 2 [(gogoproto.nullable) = false]; + // In 23.1 we expect the response to only group on fingerprint_id and app_name + // in the overview page. We now return the aggregated list of unique txn fingerprint ids, + // leaving the txn_fingerprint_id field in the key empty. + repeated uint64 txn_fingerprint_ids = 4 [(gogoproto.customname) = "TxnFingerprintIDs", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TransactionFingerprintID"]; } repeated CollectedStatementStatistics statements = 1 [(gogoproto.nullable) = false]; diff --git a/pkg/server/status_test.go b/pkg/server/status_test.go index 6a83c5d1bb98..e681f1257df5 100644 --- a/pkg/server/status_test.go +++ b/pkg/server/status_test.go @@ -1932,7 +1932,9 @@ func TestStatusAPICombinedStatements(t *testing.T) { } statementsInResponse = append(statementsInResponse, respStatement.Key.KeyData.Query) - expectedTxnFingerprints[respStatement.Key.KeyData.TransactionFingerprintID] = struct{}{} + for _, txnFingerprintID := range respStatement.TxnFingerprintIDs { + expectedTxnFingerprints[txnFingerprintID] = struct{}{} + } } for _, respTxn := range resp.Transactions { @@ -1947,6 +1949,8 @@ func TestStatusAPICombinedStatements(t *testing.T) { expectedStmts, statementsInResponse, pretty.Sprint(resp), path) } if hasTxns { + // We expect that expectedTxnFingerprints is now empty since + // we should have removed them all. assert.Empty(t, expectedTxnFingerprints) } else { assert.Empty(t, resp.Transactions) From 47fc8e8c22d29a5cb8eb9879f555f7d990ecb488 Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Mon, 27 Mar 2023 14:41:54 -0400 Subject: [PATCH 2/5] ui: stop aggregating fingerprints response on UI In the combined stats EP, we group entries on both fingerprint_id and app_name for the overview page queries. The client has been grouping only on fingerprint_id for both overview pages. With the introduction of the limit param, this mismatch in aggregation criteria can lead to fewer than the requested rows returned from the server. This commit ensures that the UI does no additional aggregation with the payload from the server. In the next commit, we will provide the app name of the selected row when routing to statement fingerprint details from the overview page, so that we only req stats for the app of the selected row. As a part of this change, we create a common combiner body for the stmts selector in both db-console and cluster-ui to reduce duplicated code. Relevant aggregation code on the client has also been removed. Part of: #99708 Fixes: #99390 Release note (ui change): By default, we now show the `Application Name` column in the fingerprints overview pages. Stmts and txns fingerprints will be displayed per application on the overview pages rather than grouped into a single fingerprint id. --- .../cluster-ui/src/api/statementsApi.ts | 61 ++++++- .../cluster-ui/src/selectors/common.ts | 100 +++++++++++- .../statementsPage.selectors.ts | 150 +---------------- .../statementsPageConnected.tsx | 4 +- .../src/statementsTable/statementsTable.tsx | 16 +- .../statementDiagnostics.selectors.ts | 2 +- .../src/transactionsPage/transactionsPage.tsx | 21 +-- .../cluster-ui/src/transactionsPage/utils.ts | 124 +------------- .../transactionsTable/transactionsTable.tsx | 6 +- .../appStats/aggregateStatementStats.spec.ts | 22 --- .../cluster-ui/src/util/appStats/appStats.ts | 43 +---- .../src/views/statements/statements.spec.tsx | 140 +++++----------- .../src/views/statements/statementsPage.tsx | 153 +----------------- 13 files changed, 238 insertions(+), 604 deletions(-) delete mode 100644 pkg/ui/workspaces/cluster-ui/src/util/appStats/aggregateStatementStats.spec.ts diff --git a/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts index c69647ab48d1..b0a57a6dcfb9 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts @@ -10,7 +10,11 @@ import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; import { fetchData } from "src/api/fetchData"; -import { propsToQueryString } from "src/util"; +import { + combineStatementStats, + StatementStatistics, +} from "src/util/appStats/appStats"; +import { propsToQueryString } from "src/util/query"; import Long from "long"; import moment from "moment"; @@ -76,6 +80,46 @@ export function createCombinedStmtsRequest({ }); } +type StmtWithMultiStats = Stmt & { + combinedStats: StatementStatistics[]; +}; + +export function aggregateOnStmtFingerprintAndAppName( + stmts: Stmt[] | null, +): Stmt[] { + if (!stmts?.length) { + return []; + } + + const groupedStmts: Record = {}; + + stmts.forEach(stmt => { + const key = stmt.id?.toString() + stmt.key?.key_data?.app; + + if (groupedStmts[key] == null) { + groupedStmts[key] = { + ...stmt, + combinedStats: [], + txn_fingerprint_ids: [], + }; + } + + groupedStmts[key].combinedStats.push(stmt.stats); + groupedStmts[key].txn_fingerprint_ids.push( + stmt.key.key_data?.transaction_fingerprint_id, + ); + }); + + return Object.values(groupedStmts).map(stmt => { + const { combinedStats, ...stmtFields } = stmt; + + return { + ...stmtFields, + stats: combineStatementStats(combinedStats), + }; + }); +} + // Mutates the sqlstats response to conform to the provided sort and limit params. export function sortAndTruncateStmtsResponse( res: SqlStatsResponse, @@ -86,14 +130,21 @@ export function sortAndTruncateStmtsResponse( // cleanest and least complex way of handling this scenario. res.transactions = []; + if (!res.statements?.length) { + return; + } + + // Aggregate on only stmt fingerprint id and app name. + res.statements = aggregateOnStmtFingerprintAndAppName(res.statements); + switch (sort) { case SqlStatsSortOptions.SERVICE_LAT: - res.statements?.sort((stmtA: Stmt, stmtB: Stmt): number => { + 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 => { + res.statements.sort((stmtA: Stmt, stmtB: Stmt): number => { return ( stmtB.stats.exec_stats.contention_time.mean - stmtA.stats.exec_stats.contention_time.mean @@ -101,13 +152,13 @@ export function sortAndTruncateStmtsResponse( }); break; case SqlStatsSortOptions.EXECUTION_COUNT: - res.statements?.sort((stmtA: Stmt, stmtB: Stmt): number => { + 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 => { + 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() diff --git a/pkg/ui/workspaces/cluster-ui/src/selectors/common.ts b/pkg/ui/workspaces/cluster-ui/src/selectors/common.ts index c3516317a481..81e6a8d3de25 100644 --- a/pkg/ui/workspaces/cluster-ui/src/selectors/common.ts +++ b/pkg/ui/workspaces/cluster-ui/src/selectors/common.ts @@ -9,7 +9,23 @@ // licenses/APL.txt. import { RouteComponentProps } from "react-router"; -import { getMatchParamByName, executionIdAttr, idAttr } from "src/util"; +import { + getMatchParamByName, + executionIdAttr, + idAttr, + statementAttr, + txnFingerprintIdAttr, + unset, + ExecutionStatistics, + queryByName, + appAttr, + flattenStatementStats, + FixFingerprintHexValue, +} from "src/util"; +import { createSelector } from "@reduxjs/toolkit"; +import { SqlStatsResponse } from "../api"; +import { AggregateStatistics } from "src/statementsTable"; +import { StatementDiagnosticsDictionary } from "src/store/statementDiagnostics"; // The functions in this file are agnostic to the different shape of each // state in db-console and cluster-ui. This file contains selector functions @@ -18,15 +34,93 @@ import { getMatchParamByName, executionIdAttr, idAttr } from "src/util"; // between db-console and cluster-ui. export const selectExecutionID = ( - _state: any, + _state: unknown, props: RouteComponentProps, ): string | null => { return getMatchParamByName(props.match, executionIdAttr); }; export const selectID = ( - _state: any, + _state: unknown, props: RouteComponentProps, ): string | null => { return getMatchParamByName(props.match, idAttr); }; + +// selectStmtsAllApps returns the array of all unique apps within the data. +export const selectStmtsAllApps = createSelector( + (data: SqlStatsResponse) => data, + data => { + if (!data) { + return []; + } + + const apps = new Set(); + data.statements?.forEach(statement => { + const app = statement.key.key_data.app; + if ( + data.internal_app_name_prefix && + app.startsWith(data.internal_app_name_prefix) + ) { + apps.add(data.internal_app_name_prefix); + return; + } + apps.add(app ? app : unset); + }); + + return Array.from(apps).sort(); + }, +); + +export const selectStmtsCombiner = ( + statsResp: SqlStatsResponse, + props: RouteComponentProps, + diagnosticsReportsPerStatement: StatementDiagnosticsDictionary, +): AggregateStatistics[] => { + // State is valid if we successfully fetched data, and the data has not yet been invalidated. + if (!statsResp) { + return null; + } + let statements = flattenStatementStats(statsResp.statements); + const app = queryByName(props.location, appAttr); + + const isInternal = (statement: ExecutionStatistics) => + statement.app?.startsWith(statsResp.internal_app_name_prefix); + + if (app && app !== "All") { + const criteria = decodeURIComponent(app).split(","); + let showInternal = false; + if (criteria.includes(statsResp.internal_app_name_prefix)) { + showInternal = true; + } + if (criteria.includes(unset)) { + criteria.push(""); + } + statements = statements.filter( + (statement: ExecutionStatistics) => + (showInternal && isInternal(statement)) || + criteria.includes(statement.app), + ); + } else { + // We don't want to show statements that only come from internal apps. + statements = statements.filter( + (statement: ExecutionStatistics) => !isInternal(statement), + ); + } + + return statements.map(stmt => ({ + aggregatedFingerprintID: stmt.statement_fingerprint_id?.toString(), + aggregatedFingerprintHexID: FixFingerprintHexValue( + stmt.statement_fingerprint_id?.toString(16), + ), + label: stmt.statement, + summary: stmt.statement_summary, + aggregatedTs: stmt.aggregated_ts, + implicitTxn: stmt.implicit_txn, + fullScan: stmt.full_scan, + database: stmt.database, + applicationName: stmt.app, + stats: stmt.stats, + diagnosticsReports: diagnosticsReportsPerStatement[stmt.statement], + })); +}; diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts index 087b31877669..440a62e6dbcb 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts @@ -9,82 +9,21 @@ // licenses/APL.txt. import { createSelector } from "reselect"; -import { - aggregateStatementStats, - appAttr, - FixFingerprintHexValue, - combineStatementStats, - ExecutionStatistics, - flattenStatementStats, - formatDate, - queryByName, - statementKey, - StatementStatistics, - TimestampToMoment, - unset, -} from "src/util"; -import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; +import { formatDate, TimestampToMoment, unset } from "src/util"; import { RouteComponentProps } from "react-router-dom"; import { AppState } from "src/store"; import { selectDiagnosticsReportsPerStatement } from "../store/statementDiagnostics"; -import { AggregateStatistics } from "../statementsTable"; import { sqlStatsSelector } from "../store/sqlStats/sqlStats.selector"; import { SQLStatsState } from "../store/sqlStats"; import { localStorageSelector } from "../store/utils/selectors"; - -type ICollectedStatementStatistics = - cockroach.server.serverpb.StatementsResponse.ICollectedStatementStatistics; -export interface StatementsSummaryData { - statementFingerprintID: string; - statementFingerprintHexID: string; - statement: string; - statementSummary: string; - aggregatedTs: number; - implicitTxn: boolean; - fullScan: boolean; - database: string; - applicationName: string; - stats: StatementStatistics[]; -} +import { selectStmtsCombiner } from "src/selectors/common"; export const selectStatementsLastUpdated = createSelector( sqlStatsSelector, sqlStats => sqlStats.lastUpdated, ); -// selectApps returns the array of all apps with statement statistics present -// in the data. -export const selectApps = createSelector(sqlStatsSelector, sqlStatsState => { - if (!sqlStatsState?.data || !sqlStatsState?.valid) { - return []; - } - - let sawBlank = false; - let sawInternal = false; - const apps: { [app: string]: boolean } = {}; - sqlStatsState.data.statements.forEach( - (statement: ICollectedStatementStatistics) => { - if ( - sqlStatsState.data.internal_app_name_prefix && - statement.key.key_data.app.startsWith( - sqlStatsState.data.internal_app_name_prefix, - ) - ) { - sawInternal = true; - } else if (statement.key.key_data.app) { - apps[statement.key.key_data.app] = true; - } else { - sawBlank = true; - } - }, - ); - return [] - .concat(sawInternal ? [sqlStatsState.data.internal_app_name_prefix] : []) - .concat(sawBlank ? [unset] : []) - .concat(Object.keys(apps).sort()); -}); - // selectDatabases returns the array of all databases with statement statistics present // in the data. export const selectDatabases = createSelector( @@ -114,8 +53,7 @@ export const selectTotalFingerprints = createSelector( if (!state?.data) { return 0; } - const aggregated = aggregateStatementStats(state.data.statements); - return aggregated.length; + return state.data.statements?.length ?? 0; }, ); @@ -144,88 +82,10 @@ export const selectStatementsDataInFlight = createSelector( ); export const selectStatements = createSelector( - sqlStatsSelector, + (state: AppState) => state.adminUI?.statements?.data, (_: AppState, props: RouteComponentProps) => props, selectDiagnosticsReportsPerStatement, - ( - state: SQLStatsState, - props: RouteComponentProps, - diagnosticsReportsPerStatement, - ): AggregateStatistics[] => { - // State is valid if we successfully fetched data, and the data has not yet been invalidated. - if (!state?.data || !state?.valid) { - return null; - } - let statements = flattenStatementStats(state.data.statements); - const app = queryByName(props.location, appAttr); - const isInternal = (statement: ExecutionStatistics) => - statement.app.startsWith(state.data.internal_app_name_prefix); - - if (app && app !== "All") { - const criteria = decodeURIComponent(app).split(","); - let showInternal = false; - if (criteria.includes(state.data.internal_app_name_prefix)) { - showInternal = true; - } - if (criteria.includes(unset)) { - criteria.push(""); - } - - statements = statements.filter( - (statement: ExecutionStatistics) => - (showInternal && isInternal(statement)) || - criteria.includes(statement.app), - ); - } else { - // We don't want to show internal statements by default. - statements = statements.filter( - (statement: ExecutionStatistics) => !isInternal(statement), - ); - } - - const statsByStatementKey: { - [statement: string]: StatementsSummaryData; - } = {}; - statements.forEach(stmt => { - const key = statementKey(stmt); - if (!(key in statsByStatementKey)) { - statsByStatementKey[key] = { - statementFingerprintID: stmt.statement_fingerprint_id?.toString(), - statementFingerprintHexID: FixFingerprintHexValue( - stmt.statement_fingerprint_id?.toString(16), - ), - statement: stmt.statement, - statementSummary: stmt.statement_summary, - aggregatedTs: stmt.aggregated_ts, - implicitTxn: stmt.implicit_txn, - fullScan: stmt.full_scan, - database: stmt.database, - applicationName: stmt.app, - stats: [], - }; - } - statsByStatementKey[key].stats.push(stmt.stats); - }); - - return Object.keys(statsByStatementKey).map(key => { - const stmt = statsByStatementKey[key]; - return { - aggregatedFingerprintID: stmt.statementFingerprintID, - aggregatedFingerprintHexID: FixFingerprintHexValue( - stmt.statementFingerprintHexID, - ), - label: stmt.statement, - summary: stmt.statementSummary, - aggregatedTs: stmt.aggregatedTs, - implicitTxn: stmt.implicitTxn, - fullScan: stmt.fullScan, - database: stmt.database, - applicationName: stmt.applicationName, - stats: combineStatementStats(stmt.stats), - diagnosticsReports: diagnosticsReportsPerStatement[stmt.statement], - }; - }); - }, + selectStmtsCombiner, ); export const selectStatementsLastError = createSelector( diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx index b544c2b7847b..a5c1c37ede31 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx @@ -27,7 +27,6 @@ import { StatementsPageStateProps, } from "./statementsPage"; import { - selectApps, selectDatabases, selectLastReset, selectStatements, @@ -67,6 +66,7 @@ import { mapDispatchToActiveStatementsPageProps, mapStateToActiveStatementsPageProps, } from "./activeStatementsPage.selectors"; +import { selectStmtsAllApps } from "../selectors"; type IStatementDiagnosticsReport = cockroach.server.serverpb.IStatementDiagnosticsReport; @@ -98,7 +98,7 @@ export const ConnectedStatementsPage = withRouter( (state: AppState, props: RouteComponentProps) => ({ fingerprintsPageProps: { ...props, - apps: selectApps(state), + apps: selectStmtsAllApps(state.adminUI?.statements?.data), columns: selectColumns(state), databases: selectDatabases(state), timeScale: selectTimeScale(state), diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsTable/statementsTable.tsx b/pkg/ui/workspaces/cluster-ui/src/statementsTable/statementsTable.tsx index 765d99be2858..2ac95cd61634 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsTable/statementsTable.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementsTable/statementsTable.tsx @@ -46,6 +46,7 @@ import { NodeNames, StatisticType, } from "../statsTableUtil/statsTableUtil"; +import { BarChartOptions } from "src/barCharts/barChartFactory"; type IStatementDiagnosticsReport = cockroach.server.serverpb.IStatementDiagnosticsReport; @@ -66,12 +67,14 @@ function makeCommonColumns( label: cx("statements-table__col--bar-chart__label"), }, }; - const sampledExecStatsBarChartOptions = { - classes: defaultBarChartOptions.classes, - displayNoSamples: (d: ICollectedStatementStatistics) => { - return longToInt(d.stats.exec_stats?.count) == 0; - }, - }; + + const sampledExecStatsBarChartOptions: BarChartOptions = + { + classes: defaultBarChartOptions.classes, + displayNoSamples: (d: ICollectedStatementStatistics) => { + return longToInt(d.stats.exec_stats?.count) == 0; + }, + }; const countBar = countBarChart(statements, defaultBarChartOptions); const bytesReadBar = bytesReadBarChart(statements, defaultBarChartOptions); @@ -113,7 +116,6 @@ function makeCommonColumns( cell: (stmt: AggregateStatistics) => stmt.applicationName?.length > 0 ? stmt.applicationName : unset, sort: (stmt: AggregateStatistics) => stmt.applicationName, - showByDefault: false, }, { name: "rowsProcessed", diff --git a/pkg/ui/workspaces/cluster-ui/src/store/statementDiagnostics/statementDiagnostics.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/statementDiagnostics/statementDiagnostics.selectors.ts index 08f1935ff55e..65e23724299d 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/statementDiagnostics/statementDiagnostics.selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/statementDiagnostics/statementDiagnostics.selectors.ts @@ -26,7 +26,7 @@ export const selectStatementDiagnosticsReports = createSelector( state => state.data?.reports, ); -type StatementDiagnosticsDictionary = { +export type StatementDiagnosticsDictionary = { [statementFingerprint: string]: IStatementDiagnosticsReport[]; }; diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.tsx index 239a194c424f..a1c649ed4e75 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.tsx @@ -27,7 +27,6 @@ import { import { Pagination, ResultsPerPageLabel } from "../pagination"; import { statisticsClasses } from "./transactionsPageClasses"; import { - aggregateAcrossNodeIDs, generateRegionNode, getTrxAppFilterOptions, searchTransactionsData, @@ -448,11 +447,6 @@ export class TransactionsPage extends React.Component< const internal_app_name_prefix = data?.internal_app_name_prefix || ""; const statements = data?.statements || []; - // 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. - // - // TODO(davidh): Once the redux layer for TransactionsPage is added to this repo, - // extract this work into the selector const { transactions: filteredTransactions, activeFilters } = filterTransactions( searchTransactionsData(search, data?.transactions || [], statements), @@ -468,14 +462,13 @@ 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: generateRegionNode(t, statements, nodeRegions), - })); + const transactionsToDisplay: TransactionInfo[] = filteredTransactions.map( + t => ({ + stats_data: t.stats_data, + node_id: t.node_id, + regionNodes: generateRegionNode(t, statements, nodeRegions), + }), + ); const { current, pageSize } = pagination; const hasData = data?.transactions?.length > 0; const isUsedFilter = search?.length > 0; diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/utils.ts b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/utils.ts index efe9c04fd16e..18a3740a1293 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/utils.ts +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/utils.ts @@ -16,16 +16,10 @@ import { } from "../queryFilter"; import { AggregateStatistics } from "../statementsTable"; import Long from "long"; -import _ from "lodash"; import { - addExecStats, - aggregateNumericStats, - FixLong, longToInt, - TimestampToNumber, addStatementStats, flattenStatementStats, - DurationToNumber, computeOrUseStmtSummary, transactionScopedStatementKey, unset, @@ -33,7 +27,6 @@ import { type Statement = protos.cockroach.server.serverpb.StatementsResponse.ICollectedStatementStatistics; -type TransactionStats = protos.cockroach.sql.ITransactionStatistics; type Transaction = protos.cockroach.server.serverpb.StatementsResponse.IExtendedCollectedTransactionStatistics; @@ -53,7 +46,6 @@ export const getTrxAppFilterOptions = ( return Array.from(uniqueAppNames).sort(); }; - export const collectStatementsText = (statements: Statement[]): string => statements.map(s => s.key.key_data.query).join("\n"); @@ -181,8 +173,8 @@ export const filterTransactions = ( // Current filters: app, service latency, nodes and regions. const filteredTransactions = data .filter((t: Transaction) => { - const isInternal = (t: Transaction) => - t.stats_data.app.startsWith(internalAppNamePrefix); + const app = t.stats_data.app; + const isInternal = app.startsWith(internalAppNamePrefix); if (filters.app && filters.app != "All") { const apps = filters.app.split(","); @@ -195,13 +187,13 @@ export const filterTransactions = ( } return ( - (showInternal && isInternal(t)) || - t.stats_data.app === filters.app || - apps.includes(t.stats_data.app) + (showInternal && isInternal) || + app === filters.app || + apps.includes(app) ); } else { // We don't want to show internal transactions by default. - return !isInternal(t); + return !isInternal; } }) .filter( @@ -294,107 +286,3 @@ export const generateRegionNode = ( }); return regionNodes; }; - -type TransactionWithFingerprint = Transaction & { fingerprint: string }; - -// withFingerprint adds the concatenated statement fingerprints to the Transaction object since it -// only comes with statement_fingerprint_ids -const withFingerprint = function ( - t: Transaction, - stmts: Statement[], -): TransactionWithFingerprint { - return { - ...t, - fingerprint: statementFingerprintIdsToText( - t.stats_data.statement_fingerprint_ids, - stmts, - ), - }; -}; - -// addTransactionStats adds together two stat objects into one using their counts to compute a new -// average for the numeric statistics. It's modeled after the similar `addStatementStats` function -function addTransactionStats( - a: TransactionStats, - b: TransactionStats, -): Required { - const countA = FixLong(a.count).toInt(); - const countB = FixLong(b.count).toInt(); - return { - count: a.count.add(b.count), - max_retries: a.max_retries.greaterThan(b.max_retries) - ? a.max_retries - : b.max_retries, - num_rows: aggregateNumericStats(a.num_rows, b.num_rows, countA, countB), - service_lat: aggregateNumericStats( - a.service_lat, - b.service_lat, - countA, - countB, - ), - retry_lat: aggregateNumericStats(a.retry_lat, b.retry_lat, countA, countB), - commit_lat: aggregateNumericStats( - a.commit_lat, - b.commit_lat, - countA, - countB, - ), - rows_read: aggregateNumericStats(a.rows_read, b.rows_read, countA, countB), - rows_written: aggregateNumericStats( - a.rows_written, - b.rows_written, - countA, - countB, - ), - bytes_read: aggregateNumericStats( - a.bytes_read, - b.bytes_read, - countA, - countB, - ), - exec_stats: addExecStats(a.exec_stats, b.exec_stats), - }; -} - -function combineTransactionStats( - txnStats: TransactionStats[], -): TransactionStats { - return _.reduce(txnStats, addTransactionStats); -} - -// mergeTransactionStats takes a list of transactions (assuming they're all for the same fingerprint -// and returns a copy of the first element with its `stats_data.stats` object replaced with a -// merged stats object that aggregates statistics from every copy of the fingerprint in the list -// provided -// This function SHOULD NOT mutate any objects in the provided txns array. -const mergeTransactionStats = function (txns: Transaction[]): Transaction { - if (txns.length === 0) { - return null; - } - const txn = _.cloneDeep(txns[0]); - txn.stats_data.stats = combineTransactionStats( - txns.map(t => t.stats_data.stats), - ); - return txn; -}; - -// aggregateAcrossNodeIDs takes a list of transactions and a list of statements that those -// transactions reference and returns a list of transactions that have been grouped by their -// fingerprints and had their statistics aggregated across copies of the transaction. This is used -// to deduplicate identical copies of the transaction that are run on different nodes. CRDB returns -// different objects to represent those transactions. -// -// The function uses the fingerprint and the `app` that ran the transaction as the key to group the -// transactions when deduping. -// -export const aggregateAcrossNodeIDs = function ( - t: Transaction[], - stmts: Statement[], -): Transaction[] { - return _.chain(t) - .map(t => withFingerprint(t, stmts)) - .groupBy(t => t.fingerprint) - .mapValues(mergeTransactionStats) - .values() - .value(); -}; diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx index 6a77d3642255..cfd147954202 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx @@ -45,6 +45,7 @@ import { } from "../transactionsPage/utils"; import classNames from "classnames/bind"; import statsTablePageStyles from "src/statementsTable/statementsTableContent.module.scss"; +import { BarChartOptions } from "../barCharts/barChartFactory"; export type Transaction = protos.cockroach.server.serverpb.StatementsResponse.IExtendedCollectedTransactionStatistics; @@ -93,7 +94,7 @@ export function makeTransactionsColumns( label: cx("statements-table__col--bar-chart__label"), }, }; - const sampledExecStatsBarChartOptions = { + const sampledExecStatsBarChartOptions: BarChartOptions = { classes: defaultBarChartOptions.classes, displayNoSamples: (d: TransactionInfo) => { return longToInt(d.stats_data.stats.exec_stats?.count) == 0; @@ -166,9 +167,8 @@ export function makeTransactionsColumns( title: statisticsTableTitles.applicationName(statType), className: cx("statements-table__col-app-name"), cell: (item: TransactionInfo) => - item.stats_data?.app?.length > 0 ? item.stats_data?.app : unset, + item.stats_data?.app?.length ? item.stats_data.app : unset, sort: (item: TransactionInfo) => item.stats_data?.app, - showByDefault: false, }, { name: "rowsProcessed", diff --git a/pkg/ui/workspaces/cluster-ui/src/util/appStats/aggregateStatementStats.spec.ts b/pkg/ui/workspaces/cluster-ui/src/util/appStats/aggregateStatementStats.spec.ts deleted file mode 100644 index 0a1c0ebc880b..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/util/appStats/aggregateStatementStats.spec.ts +++ /dev/null @@ -1,22 +0,0 @@ -// Copyright 2021 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 { assert } from "chai"; -import { aggregateStatementStats } from "./appStats"; -import { statementsWithSameIdButDifferentNodeId } from "./appStats.fixture"; - -describe("aggregateStatementStats", () => { - it("groups duplicate statements by node id", () => { - const aggregated = aggregateStatementStats( - statementsWithSameIdButDifferentNodeId, - ); - assert.equal(aggregated.length, 1); - }); -}); diff --git a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts index 82e00353b05c..2a22f5f39ee0 100644 --- a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts +++ b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts @@ -8,7 +8,6 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -import _ from "lodash"; import * as protos from "@cockroachlabs/crdb-protobuf-client"; import { FixLong, @@ -200,30 +199,6 @@ export function addStatementStats( }; } -export function aggregateStatementStats( - statementStats: CollectedStatementStatistics[], -): CollectedStatementStatistics[] { - const statementsMap: { - [statement: string]: CollectedStatementStatistics[]; - } = {}; - statementStats.forEach((statement: CollectedStatementStatistics) => { - const matches = - statementsMap[statement.key.key_data.query] || - (statementsMap[statement.key.key_data.query] = []); - matches.push(statement); - }); - - return _.values(statementsMap).map(statements => - _.reduce( - statements, - (a: CollectedStatementStatistics, b: CollectedStatementStatistics) => ({ - key: a.key, - stats: addStatementStats(a.stats, b.stats), - }), - ), - ); -} - export interface ExecutionStatistics { statement_fingerprint_id: Long; statement: string; @@ -238,7 +213,7 @@ export interface ExecutionStatistics { full_scan: boolean; failed: boolean; node_id: number; - transaction_fingerprint_id: Long; + txn_fingerprint_ids: Long[]; stats: StatementStatistics; } @@ -259,7 +234,7 @@ export function flattenStatementStats( full_scan: stmt.key.key_data.full_scan, failed: stmt.key.key_data.failed, node_id: stmt.key.node_id, - transaction_fingerprint_id: stmt.key.key_data.transaction_fingerprint_id, + txn_fingerprint_ids: stmt.txn_fingerprint_ids, stats: stmt.stats, })); } @@ -267,22 +242,16 @@ export function flattenStatementStats( export function combineStatementStats( statementStats: StatementStatistics[], ): StatementStatistics { - return _.reduce(statementStats, addStatementStats); + return statementStats.reduce(addStatementStats); } -export const getSearchParams = (searchParams: string) => { - const sp = new URLSearchParams(searchParams); - return (key: string, defaultValue?: string | boolean | number) => - sp.get(key) || defaultValue; -}; - // This function returns a key based on all parameters // that should be used to group statements. -// Currently, using only statement_fingerprint_id +// Currently, using only statement_fingerprint_id and app anme. // (created by ConstructStatementFingerprintID using: // query, implicit_txn, database, failed). export function statementKey(stmt: ExecutionStatistics): string { - return stmt.statement_fingerprint_id?.toString(); + return stmt.statement_fingerprint_id?.toString() + stmt.app; } // transactionScopedStatementKey is similar to statementKey, except that @@ -290,7 +259,7 @@ export function statementKey(stmt: ExecutionStatistics): string { export function transactionScopedStatementKey( stmt: ExecutionStatistics, ): string { - return statementKey(stmt) + stmt.transaction_fingerprint_id.toString(); + return statementKey(stmt) + stmt.txn_fingerprint_ids?.toString(); } export const generateStmtDetailsToID = ( diff --git a/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx b/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx index 2220c5c2fc16..fac47a7a9de9 100644 --- a/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx +++ b/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx @@ -24,14 +24,19 @@ import { } from "src/util/constants"; import { selectStatements, - selectApps, selectTotalFingerprints, selectLastReset, } from "./statementsPage"; import { selectStatementDetails } from "./statementDetails"; import ISensitiveInfo = protos.cockroach.sql.ISensitiveInfo; import { AdminUIState, createAdminUIStore } from "src/redux/state"; -import { TimeScale, toRoundedDateRange, util } from "@cockroachlabs/cluster-ui"; +import { + TimeScale, + toRoundedDateRange, + util, + selectStmtsAllApps as selectApps, +} from "@cockroachlabs/cluster-ui"; +import { allSchedulesFixture } from "@cockroachlabs/cluster-ui/dist/types/schedules/schedulesPage/schedulesPage.fixture"; const { generateStmtDetailsToID, longToInt } = util; @@ -84,70 +89,42 @@ describe("selectStatements", () => { expect(actualFingerprints).toEqual(expectedFingerprints); }); - it("returns the statements without Internal for default ALL filter", () => { + it("returns the statements currently without grouping on fingerprint_id", () => { const stmtA = makeFingerprint(1); - const stmtB = makeFingerprint(2, INTERNAL_STATEMENT_PREFIX); - const stmtC = makeFingerprint(3, INTERNAL_STATEMENT_PREFIX); - const stmtD = makeFingerprint(3, "another"); - const state = makeStateWithStatements( - [stmtA, stmtB, stmtC, stmtD], - timeScale, + const stmtB = makeFingerprint(2, "foobar"); + const stmtC = makeFingerprint(3, "another"); + const stmtsWithSameFingerprintAsA = Array.from(new Array(5)).map(() => + makeFingerprint(1, "duplicate_fingerprints"), ); - const props = makeEmptyRouteProps(); - - const result = selectStatements(state, props); - - expect(result.length).toBe(2); - }); - - it("coalesces statements from different apps", () => { - const stmtA = makeFingerprint(1); - const stmtB = makeFingerprint(1, "foobar"); - const stmtC = makeFingerprint(1, "another"); - const sumCount = stmtA.stats.count - .add(stmtB.stats.count.add(stmtC.stats.count)) - .toNumber(); - const state = makeStateWithStatements([stmtA, stmtB, stmtC], timeScale); - const props = makeEmptyRouteProps(); - const result = selectStatements(state, props); - - expect(result.length).toBe(1); - expect(result[0].label).toEqual(stmtA.key.key_data.query); - expect(result[0].stats.count.toNumber()).toEqual(sumCount); - }); - - it("coalesces statements with differing node ids", () => { - const state = makeStateWithStatements( - [ - makeFingerprint(1, "", 1), - makeFingerprint(1, "", 2), - makeFingerprint(1, "", 3), - ], - timeScale, - ); + const stmts = [stmtA, stmtB, stmtC, ...stmtsWithSameFingerprintAsA]; + const state = makeStateWithStatements(stmts, timeScale); const props = makeEmptyRouteProps(); const result = selectStatements(state, props); + expect(result.length).toBe(8); - expect(result.length).toBe(1); + const expectedFingerprints = stmts.map(stmt => stmt.key.key_data.query); + expectedFingerprints.sort(); + const actualFingerprints = result.map((stmt: any) => stmt.label); + actualFingerprints.sort(); + expect(actualFingerprints).toEqual(expectedFingerprints); }); - it("coalesces statements with differing distSQL and failed values", () => { + it("returns the statements without Internal for default ALL filter", () => { + const stmtA = makeFingerprint(1); + const stmtB = makeFingerprint(2, INTERNAL_STATEMENT_PREFIX); + const stmtC = makeFingerprint(3, INTERNAL_STATEMENT_PREFIX); + const stmtD = makeFingerprint(3, "another"); const state = makeStateWithStatements( - [ - makeFingerprint(1, "", 1, false, false), - makeFingerprint(1, "", 1, false, true), - makeFingerprint(1, "", 1, true, false), - makeFingerprint(1, "", 1, true, true), - ], + [stmtA, stmtB, stmtC, stmtD], timeScale, ); const props = makeEmptyRouteProps(); const result = selectStatements(state, props); - expect(result.length).toBe(1); + expect(result.length).toBe(2); }); it("filters out statements when app param is set", () => { @@ -201,7 +178,7 @@ describe("selectApps", () => { it("returns an empty array if the statements data is invalid", () => { const state = makeInvalidState(); - const result = selectApps(state); + const result = selectApps(state?.cachedData?.statements?.data); expect(result).toEqual([]); }); @@ -210,6 +187,8 @@ describe("selectApps", () => { const state = makeStateWithStatements( [ makeFingerprint(1), + makeFingerprint(1, "hello"), + makeFingerprint(1, "world"), makeFingerprint(1, "foobar"), makeFingerprint(2, "foobar"), makeFingerprint(3, "cockroach sql"), @@ -217,9 +196,15 @@ describe("selectApps", () => { timeScale, ); - const result = selectApps(state); + const result = selectApps(state?.cachedData?.statements?.data); - expect(result).toEqual([unset, "cockroach sql", "foobar"]); + expect(result).toEqual([ + unset, + "cockroach sql", + "foobar", + "hello", + "world", + ]); }); }); @@ -242,52 +227,6 @@ describe("selectTotalFingerprints", () => { expect(result).toBe(3); }); - - it("coalesces statements from different apps", () => { - const state = makeStateWithStatements( - [ - makeFingerprint(1), - makeFingerprint(1, "foobar"), - makeFingerprint(1, "another"), - ], - timeScale, - ); - - const result = selectTotalFingerprints(state); - - expect(result).toBe(1); - }); - - it("coalesces statements with differing node ids", () => { - const state = makeStateWithStatements( - [ - makeFingerprint(1, "", 1), - makeFingerprint(1, "", 2), - makeFingerprint(1, "", 3), - ], - timeScale, - ); - - const result = selectTotalFingerprints(state); - - expect(result).toBe(1); - }); - - it("coalesces statements with differing distSQL and failed keys", () => { - const state = makeStateWithStatements( - [ - makeFingerprint(1, "", 1, false, false), - makeFingerprint(1, "", 1, false, true), - makeFingerprint(1, "", 1, true, false), - makeFingerprint(1, "", 1, true, true), - ], - timeScale, - ); - - const result = selectTotalFingerprints(state); - - expect(result).toBe(1); - }); }); describe("selectLastReset", () => { @@ -314,9 +253,8 @@ describe("selectStatement", () => { const state = makeInvalidState(); const props = makeEmptyRouteProps(); const { statementDetails } = selectStatementDetails(state, props); - const result = statementDetails; - expect(result).toBeNull(); + expect(statementDetails).toBeNull(); }); it("returns the statement currently loaded", () => { 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 543261b87197..0beaa0566b83 100644 --- a/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx +++ b/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx @@ -22,7 +22,7 @@ import { import { CachedDataReducerState } from "src/redux/cachedDataReducer"; import { AdminUIState, AppDispatch } from "src/redux/state"; import { StatementsResponseMessage } from "src/util/api"; -import { appAttr, unset } from "src/util/constants"; +import { unset } from "src/util/constants"; import { PrintTime } from "src/views/reports/containers/range/print"; import { selectDiagnosticsReportsPerStatement } from "src/redux/statements/statementsSelectors"; import { @@ -33,10 +33,8 @@ import { selectHasViewActivityRedactedRole, selectHasAdminRole, } from "src/redux/user"; -import { queryByName } from "src/util/query"; import { - AggregateStatistics, Filters, defaultFilters, util, @@ -47,6 +45,8 @@ import { StatementsPageDispatchProps, StatementsPageRootProps, api, + selectStmtsAllApps, + selectStmtsCombiner, } from "@cockroachlabs/cluster-ui"; import { cancelStatementDiagnosticsReportAction, @@ -74,154 +74,16 @@ import { selectStatementsDataInFlight, } from "src/selectors/executionFingerprintsSelectors"; -type ICollectedStatementStatistics = - protos.cockroach.server.serverpb.StatementsResponse.ICollectedStatementStatistics; type IStatementDiagnosticsReport = protos.cockroach.server.serverpb.IStatementDiagnosticsReport; -const { - aggregateStatementStats, - combineStatementStats, - flattenStatementStats, - statementKey, -} = util; - -type ExecutionStatistics = util.ExecutionStatistics; -type StatementStatistics = util.StatementStatistics; - -interface StatementsSummaryData { - statementFingerprintID: string; - statementFingerprintHexID: string; - statement: string; - statementSummary: string; - aggregatedTs: number; - implicitTxn: boolean; - fullScan: boolean; - database: string; - applicationName: string; - stats: StatementStatistics[]; -} - // selectStatements returns the array of AggregateStatistics to show on the // StatementsPage, based on if the appAttr route parameter is set. export const selectStatements = createSelector( - (state: AdminUIState) => state.cachedData.statements, + (state: AdminUIState) => state.cachedData.statements?.data, (_state: AdminUIState, props: RouteComponentProps) => props, selectDiagnosticsReportsPerStatement, - ( - state: CachedDataReducerState, - props: RouteComponentProps, - diagnosticsReportsPerStatement, - ): AggregateStatistics[] => { - if (!state?.data || !state?.valid) { - return null; - } - let statements = flattenStatementStats(state.data.statements); - const app = queryByName(props.location, appAttr); - const isInternal = (statement: ExecutionStatistics) => - statement.app.startsWith(state.data.internal_app_name_prefix); - - if (app && app !== "All") { - const criteria = decodeURIComponent(app).split(","); - let showInternal = false; - if (criteria.includes(state.data.internal_app_name_prefix)) { - showInternal = true; - } - if (criteria.includes(unset)) { - criteria.push(""); - } - - statements = statements.filter( - (statement: ExecutionStatistics) => - (showInternal && isInternal(statement)) || - criteria.includes(statement.app), - ); - } else { - // We don't want to show internal statements by default. - statements = statements.filter( - (statement: ExecutionStatistics) => !isInternal(statement), - ); - } - - const statsByStatementKey: { - [statement: string]: StatementsSummaryData; - } = {}; - statements.forEach(stmt => { - const key = statementKey(stmt); - if (!(key in statsByStatementKey)) { - statsByStatementKey[key] = { - statementFingerprintID: stmt.statement_fingerprint_id?.toString(), - statementFingerprintHexID: util.FixFingerprintHexValue( - stmt.statement_fingerprint_id?.toString(16), - ), - statement: stmt.statement, - statementSummary: stmt.statement_summary, - aggregatedTs: stmt.aggregated_ts, - implicitTxn: stmt.implicit_txn, - fullScan: stmt.full_scan, - database: stmt.database, - applicationName: stmt.app, - stats: [], - }; - } - statsByStatementKey[key].stats.push(stmt.stats); - }); - - return Object.keys(statsByStatementKey).map(key => { - const stmt = statsByStatementKey[key]; - return { - aggregatedFingerprintID: stmt.statementFingerprintID, - aggregatedFingerprintHexID: util.FixFingerprintHexValue( - stmt.statementFingerprintHexID, - ), - label: stmt.statement, - summary: stmt.statementSummary, - aggregatedTs: stmt.aggregatedTs, - implicitTxn: stmt.implicitTxn, - fullScan: stmt.fullScan, - database: stmt.database, - applicationName: stmt.applicationName, - stats: combineStatementStats(stmt.stats), - diagnosticsReports: diagnosticsReportsPerStatement[stmt.statement], - }; - }); - }, -); - -// selectApps returns the array of all apps with statement statistics present -// in the data. -export const selectApps = createSelector( - (state: AdminUIState) => state.cachedData.statements, - (state: CachedDataReducerState) => { - if (!state?.data) { - return []; - } - - let sawBlank = false; - let sawInternal = false; - const apps: { [app: string]: boolean } = {}; - state.data.statements.forEach( - (statement: ICollectedStatementStatistics) => { - if ( - state.data.internal_app_name_prefix && - statement.key.key_data.app.startsWith( - state.data.internal_app_name_prefix, - ) - ) { - sawInternal = true; - } else if (statement.key.key_data.app) { - apps[statement.key.key_data.app] = true; - } else { - sawBlank = true; - } - }, - ); - return [] - .concat(sawInternal ? [state.data.internal_app_name_prefix] : []) - .concat(sawBlank ? [unset] : []) - .concat(Object.keys(apps)) - .sort(); - }, + selectStmtsCombiner, ); // selectDatabases returns the array of all databases with statement statistics present @@ -252,8 +114,7 @@ export const selectTotalFingerprints = createSelector( if (!state?.data) { return 0; } - const aggregated = aggregateStatementStats(state.data.statements); - return aggregated.length; + return state.data.statements?.length ?? 0; }, ); @@ -383,7 +244,7 @@ export default withRouter( (state: AdminUIState, props: RouteComponentProps) => ({ fingerprintsPageProps: { ...props, - apps: selectApps(state), + apps: selectStmtsAllApps(state.cachedData.statements?.data), columns: statementColumnsLocalSetting.selectorToArray(state), databases: selectDatabases(state), timeScale: selectTimeScale(state), From 1b25c16a45356fa973414b5c97a51985f58b1a31 Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Tue, 28 Mar 2023 15:46:49 -0400 Subject: [PATCH 3/5] ui: use app name of selected row when fetching fingerprint details Previously, the fingerprint details pages would show the details aggregated by the selected app names (from the filter, or aggregated across all apps if none were selected). Since we no longer perform any aggregation in the overview pages we now always display each fingerprint attached to a single app for each row. When going to the details page via clicking a row from the overview page, we should fetch details only for the selected app. Note that the txn fingerprint details page simply looks for a corresponding entry from the overview response, thus we change that search criteria to incude the app name of the selected row. The app name has also been included in the link to stmt details from insights details. Release note (ui change): When going from the fingerprint overview pages or the insight details pages to the fingerprint details page, the details page will fetch data for the stmt with the provided application name. For overview pages, this is the app name of the selected row. For insight details, it is the app of the execution that generated the insight. Part of: #99708 --- .../cluster-ui/src/insights/utils.ts | 4 +--- .../insightDetailsTables.tsx | 6 +---- .../statementInsightDetailsOverviewTab.tsx | 3 +-- .../transactionInsightDetails.tsx | 3 +-- .../transactionInsightsTable.tsx | 16 ++++--------- .../workloadInsights/util/detailsLinks.tsx | 19 +++++++-------- .../src/statementsTable/statementsTable.tsx | 19 --------------- .../statementsTableContent.tsx | 13 +++++++---- .../transactionDetailsConnected.tsx | 23 +++++++++++++++---- .../transactionsCells/transactionsCells.tsx | 8 +++---- .../transactionsTable/transactionsTable.tsx | 13 +++++++---- .../views/transactions/transactionDetails.tsx | 19 ++++++++++----- 12 files changed, 69 insertions(+), 77 deletions(-) diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/utils.ts b/pkg/ui/workspaces/cluster-ui/src/insights/utils.ts index a8b7f48d4b36..d7085c220e00 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/utils.ts +++ b/pkg/ui/workspaces/cluster-ui/src/insights/utils.ts @@ -327,7 +327,7 @@ export function populateStatementInsightsFromProblemAndCauses( return []; } - const stmtsWithInsights: StatementInsights = statements.map(statement => { + return statements.map(statement => { // TODO(ericharmeling,todd): Replace these strings when using the insights protos. const insights: Insight[] = []; switch (statement.problem) { @@ -362,6 +362,4 @@ export function populateStatementInsightsFromProblemAndCauses( return { ...statement, insights }; }); - - return stmtsWithInsights; } diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/insightDetailsTables.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/insightDetailsTables.tsx index a3358636d67a..34202bb343ee 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/insightDetailsTables.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/insightDetailsTables.tsx @@ -40,11 +40,7 @@ export function makeInsightDetailsColumns( name: "fingerprintID", title: insightsTableTitles.fingerprintID(execType), cell: (item: EventExecution) => - TransactionDetailsLink( - item.fingerprintID, - item.startTime, - setTimeScale, - ), + TransactionDetailsLink(item.fingerprintID), sort: (item: EventExecution) => item.fingerprintID, }, { diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/statementInsightDetailsOverviewTab.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/statementInsightDetailsOverviewTab.tsx index c948d0dd6956..2fb28e156ad0 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/statementInsightDetailsOverviewTab.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/statementInsightDetailsOverviewTab.tsx @@ -195,8 +195,7 @@ export const StatementInsightDetailsOverviewTab: React.FC< label="Transaction Fingerprint ID" value={TransactionDetailsLink( insightDetails.transactionFingerprintID, - insightDetails.startTime, - setTimeScale, + insightDetails.application, )} /> diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/transactionInsights/transactionInsightsTable.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/transactionInsights/transactionInsightsTable.tsx index 40163a5073bb..d918c7601412 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/transactionInsights/transactionInsightsTable.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/transactionInsights/transactionInsightsTable.tsx @@ -35,9 +35,7 @@ interface TransactionInsightsTable { setTimeScale: (ts: TimeScale) => void; } -export function makeTransactionInsightsColumns( - setTimeScale: (ts: TimeScale) => void, -): ColumnDescriptor[] { +export function makeTransactionInsightsColumns(): ColumnDescriptor[] { const execType = InsightExecEnum.TRANSACTION; return [ { @@ -53,13 +51,9 @@ export function makeTransactionInsightsColumns( { name: "fingerprintID", title: insightsTableTitles.fingerprintID(execType), - cell: (item: TransactionInsightEvent) => - TransactionDetailsLink( - item.fingerprintID, - item.startTime, - setTimeScale, - ), - sort: (item: TransactionInsightEvent) => item.fingerprintID, + cell: item => + TransactionDetailsLink(item.fingerprintID, item.application), + sort: item => item.fingerprintID, }, { name: "query", @@ -104,7 +98,7 @@ export function makeTransactionInsightsColumns( export const TransactionInsightsTable: React.FC< TransactionInsightsTable > = props => { - const columns = makeTransactionInsightsColumns(props.setTimeScale); + const columns = makeTransactionInsightsColumns(); return ( ); diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/util/detailsLinks.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/util/detailsLinks.tsx index 049a60d66cb4..f0a3e22c6850 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/util/detailsLinks.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/util/detailsLinks.tsx @@ -15,23 +15,20 @@ import { Link } from "react-router-dom"; import { StatementLinkTarget } from "../../../statementsTable"; import moment from "moment/moment"; import { TimeScale } from "../../../timeScaleDropdown"; -import { Moment } from "moment"; +import { TransactionLinkTarget } from "../../../transactionsTable"; export function TransactionDetailsLink( transactionFingerprintID: string, - startTime: Moment, - setTimeScale: (tw: TimeScale) => void, + application?: string, ): React.ReactElement { const txnID = HexStringToInt64String(transactionFingerprintID); - const path = `/transaction/${txnID}`; - const timeScale: TimeScale = { - windowSize: moment.duration(65, "minutes"), - fixedWindowEnd: moment(startTime).add(1, "hour"), - sampleSize: moment.duration(1, "hour"), - key: "Custom", - }; return ( - setTimeScale(timeScale)}> +
{String(transactionFingerprintID)}
); diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsTable/statementsTable.tsx b/pkg/ui/workspaces/cluster-ui/src/statementsTable/statementsTable.tsx index 2ac95cd61634..dc14da9332d8 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsTable/statementsTable.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementsTable/statementsTable.tsx @@ -329,25 +329,6 @@ export function makeStatementsColumns( return columns; } -export function makeNodesColumns( - statements: AggregateStatistics[], - nodeNames: NodeNames, - totalWorkload: number, - nodeRegions: { [nodeId: string]: string }, -): ColumnDescriptor[] { - const original: ColumnDescriptor[] = [ - { - name: "nodes", - title: null, - cell: StatementTableCell.nodeLink(nodeNames), - }, - ]; - - return original.concat( - makeCommonColumns(statements, totalWorkload, nodeRegions, "statement"), - ); -} - /** * For each statement, generate the list of regions and nodes it was * executed on. Each node is assigned to only one region and a region can diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsTable/statementsTableContent.tsx b/pkg/ui/workspaces/cluster-ui/src/statementsTable/statementsTableContent.tsx index 0d81db849987..5d0074eebe8e 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsTable/statementsTableContent.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementsTable/statementsTableContent.tsx @@ -27,6 +27,7 @@ import { TimestampToMoment, computeOrUseStmtSummary, appNamesAttr, + unset, } from "src/util"; import styles from "./statementsTableContent.module.scss"; import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; @@ -52,7 +53,13 @@ export const StatementTableCell = { statement={stmt.label} statementSummary={stmt.summary} aggregatedTs={stmt.aggregatedTs} - appNames={selectedApp} + appNames={[ + stmt.applicationName != null + ? stmt.applicationName + ? stmt.applicationName + : unset + : null, + ]} implicitTxn={stmt.implicitTxn} search={search} onClick={onStatementClick} @@ -158,10 +165,6 @@ export const StatementTableCell = { ); }, - nodeLink: - (nodeNames: NodeNames) => - (stmt: AggregateStatistics): React.ReactElement => - , }; type StatementLinkTargetProps = { diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetailsConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetailsConnected.tsx index 0dda0f2d850d..b3b363ece6f9 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetailsConnected.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetailsConnected.tsx @@ -38,7 +38,13 @@ import { selectTxnsPageReqSort, } from "../store/utils/selectors"; import { StatementsRequest } from "src/api/statementsApi"; -import { txnFingerprintIdAttr, getMatchParamByName } from "../util"; +import { + txnFingerprintIdAttr, + getMatchParamByName, + queryByName, + appNamesAttr, + unset, +} from "../util"; import { TimeScale } from "../timeScaleDropdown"; import { actions as analyticsActions } from "../store/analytics"; @@ -54,16 +60,23 @@ export const selectTransaction = createSelector( isValid: transactionState.valid, }; } + + const apps = queryByName(props.location, appNamesAttr) + ?.split(",") + .map(s => s.trim()); + const txnFingerprintId = getMatchParamByName( props.match, txnFingerprintIdAttr, ); - const transaction = transactions.filter( + const transaction = transactions.find( txn => - txn.stats_data.transaction_fingerprint_id.toString() == - txnFingerprintId, - )[0]; + txn.stats_data.transaction_fingerprint_id.toString() === + txnFingerprintId && + (apps?.length ? apps.includes(txn.stats_data.app ?? unset) : true), + ); + return { isLoading: transactionState.inFlight, transaction: transaction, diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsCells/transactionsCells.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsCells/transactionsCells.tsx index beee56090fc3..c2380da2d0cd 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsCells/transactionsCells.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsCells/transactionsCells.tsx @@ -12,7 +12,7 @@ import React from "react"; import { Link } from "react-router-dom"; import { getHighlightedText } from "src/highlightedText"; import { Tooltip } from "@cockroachlabs/ui-components"; -import { limitText } from "src/util"; +import { limitText, unset } from "src/util"; import classNames from "classnames/bind"; import statementsStyles from "../../statementsTable/statementsTableContent.module.scss"; import transactionsCellsStyles from "./transactionsCells.module.scss"; @@ -25,22 +25,22 @@ const textWrapper = ownCellStyles("text-wrapper"); const hoverAreaClassName = ownCellStyles("hover-area"); interface TextCellProps { + appName: string; transactionText: string; transactionSummary: string; - aggregatedTs: string; transactionFingerprintId: string; search: string; } export const transactionLink = ({ + appName, transactionText, transactionSummary, - aggregatedTs, transactionFingerprintId, search, }: TextCellProps): React.ReactElement => { const linkProps = { - aggregatedTs, + application: appName, transactionFingerprintId, }; diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx index cfd147954202..36a0854649c2 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsTable/transactionsTable.tsx @@ -33,8 +33,9 @@ import { Count, FixLong, longToInt, - TimestampToString, unset, + appNamesAttr, + propsToQueryString, } from "src/util"; import { SortSetting } from "../sortedtable"; import { @@ -70,8 +71,8 @@ const { latencyClasses } = tableClasses; const cx = classNames.bind(statsTablePageStyles); interface TransactionLinkTargetProps { - aggregatedTs: string; transactionFingerprintId: string; + application?: string; } // TransactionLinkTarget returns the link to the relevant transaction page, given @@ -79,7 +80,11 @@ interface TransactionLinkTargetProps { export const TransactionLinkTarget = ( props: TransactionLinkTargetProps, ): string => { - return `/transaction/${props.transactionFingerprintId}`; + const searchParams = propsToQueryString({ + [appNamesAttr]: [props.application], + }); + + return `/transaction/${props.transactionFingerprintId}?${searchParams}`; }; export function makeTransactionsColumns( @@ -141,7 +146,7 @@ export function makeTransactionsColumns( item.stats_data.statement_fingerprint_ids, statements, ) || "Transaction query unavailable.", - aggregatedTs: TimestampToString(item.stats_data.aggregated_ts), + appName: item.stats_data.app, transactionFingerprintId: item.stats_data.transaction_fingerprint_id.toString(), search, 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 3b42ce258ea1..e2ece781f3fc 100644 --- a/pkg/ui/workspaces/db-console/src/views/transactions/transactionDetails.tsx +++ b/pkg/ui/workspaces/db-console/src/views/transactions/transactionDetails.tsx @@ -17,8 +17,8 @@ import { refreshUserSQLRoles, } from "src/redux/apiReducers"; import { AdminUIState } from "src/redux/state"; -import { txnFingerprintIdAttr } from "src/util/constants"; -import { getMatchParamByName } from "src/util/query"; +import { appNamesAttr, txnFingerprintIdAttr, unset } from "src/util/constants"; +import { getMatchParamByName, queryByName } from "src/util/query"; import { nodeRegionsByIDSelector } from "src/redux/nodes"; import { reqSortSetting, @@ -48,16 +48,23 @@ export const selectTransaction = createSelector( isValid: transactionState.valid, }; } + + const apps = queryByName(props.location, appNamesAttr) + ?.split(",") + .map(s => s.trim()); + const txnFingerprintId = getMatchParamByName( props.match, txnFingerprintIdAttr, ); - const transaction = transactions.filter( + const transaction = transactions.find( txn => - txn.stats_data.transaction_fingerprint_id.toString() == - txnFingerprintId, - )[0]; + txn.stats_data.transaction_fingerprint_id.toString() === + txnFingerprintId && + (apps?.length ? apps.includes(txn.stats_data.app ?? unset) : true), + ); + return { isLoading: transactionState.inFlight, transaction: transaction, From 7d077f20f278d8c014a6728d4871afb4ca3647b0 Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Thu, 30 Mar 2023 13:20:09 -0400 Subject: [PATCH 4/5] ui: remove `totalFingerprints` prop from statement page This commit removes the unused prop `totalFingerprints` and its relevant selectors. Epic: none Release note: None --- .../statementsPage/statementsPage.fixture.ts | 1 - .../statementsPage.selectors.ts | 12 -------- .../src/statementsPage/statementsPage.tsx | 1 - .../statementsPageConnected.tsx | 2 -- .../src/views/statements/statements.spec.tsx | 28 +------------------ .../src/views/statements/statementsPage.tsx | 13 --------- 6 files changed, 1 insertion(+), 56 deletions(-) diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.fixture.ts b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.fixture.ts index 1a527ac140a4..24f02795fb25 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.fixture.ts +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.fixture.ts @@ -885,7 +885,6 @@ const statementsPagePropsFixture: StatementsPageProps = { key: "Custom", }, apps: ["$ internal", "movr", "$ cockroach demo"], - totalFingerprints: 95, lastReset: "2020-04-13 07:22:23", columns: null, isTenant: false, diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts index 440a62e6dbcb..be76155428c7 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts @@ -45,18 +45,6 @@ export const selectDatabases = createSelector( }, ); -// selectTotalFingerprints returns the count of distinct statement fingerprints -// present in the data. -export const selectTotalFingerprints = createSelector( - sqlStatsSelector, - state => { - if (!state?.data) { - return 0; - } - return state.data.statements?.length ?? 0; - }, -); - // selectLastReset returns a string displaying the last time the statement // statistics were reset. export const selectLastReset = createSelector(sqlStatsSelector, state => { diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.tsx index 2adb82764213..c59600149880 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.tsx @@ -141,7 +141,6 @@ export interface StatementsPageStateProps { statementsError: Error | null; apps: string[]; databases: string[]; - totalFingerprints: number; lastReset: string; columns: string[]; nodeRegions: { [key: string]: string }; diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx index a5c1c37ede31..c6544f70368b 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx @@ -33,7 +33,6 @@ import { selectStatementsDataValid, selectStatementsDataInFlight, selectStatementsLastError, - selectTotalFingerprints, selectColumns, selectTimeScale, selectSortSetting, @@ -117,7 +116,6 @@ export const ConnectedStatementsPage = withRouter( isReqInFlight: selectStatementsDataInFlight(state), lastUpdated: selectStatementsLastUpdated(state), statementsError: selectStatementsLastError(state), - totalFingerprints: selectTotalFingerprints(state), limit: selectStmtsPageLimit(state), reqSortSetting: selectStmtsPageReqSort(state), stmtsTotalRuntimeSecs: diff --git a/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx b/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx index fac47a7a9de9..b91992204a08 100644 --- a/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx +++ b/pkg/ui/workspaces/db-console/src/views/statements/statements.spec.tsx @@ -22,11 +22,7 @@ import { statementAttr, unset, } from "src/util/constants"; -import { - selectStatements, - selectTotalFingerprints, - selectLastReset, -} from "./statementsPage"; +import { selectStatements, selectLastReset } from "./statementsPage"; import { selectStatementDetails } from "./statementDetails"; import ISensitiveInfo = protos.cockroach.sql.ISensitiveInfo; import { AdminUIState, createAdminUIStore } from "src/redux/state"; @@ -36,7 +32,6 @@ import { util, selectStmtsAllApps as selectApps, } from "@cockroachlabs/cluster-ui"; -import { allSchedulesFixture } from "@cockroachlabs/cluster-ui/dist/types/schedules/schedulesPage/schedulesPage.fixture"; const { generateStmtDetailsToID, longToInt } = util; @@ -208,27 +203,6 @@ describe("selectApps", () => { }); }); -describe("selectTotalFingerprints", () => { - it("returns zero if the statements data is invalid", () => { - const state = makeInvalidState(); - - const result = selectTotalFingerprints(state); - - expect(result).toBe(0); - }); - - it("returns the number of statement fingerprints", () => { - const state = makeStateWithStatements( - [makeFingerprint(1), makeFingerprint(2), makeFingerprint(3)], - timeScale, - ); - - const result = selectTotalFingerprints(state); - - expect(result).toBe(3); - }); -}); - describe("selectLastReset", () => { it('returns "unknown" if the statements data is invalid', () => { const state = makeInvalidState(); 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 0beaa0566b83..dfd9b49fe2c1 100644 --- a/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx +++ b/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx @@ -106,18 +106,6 @@ export const selectDatabases = createSelector( }, ); -// selectTotalFingerprints returns the count of distinct statement fingerprints -// present in the data. -export const selectTotalFingerprints = createSelector( - (state: AdminUIState) => state.cachedData.statements, - (state: CachedDataReducerState) => { - if (!state?.data) { - return 0; - } - return state.data.statements?.length ?? 0; - }, -); - // selectLastReset returns a string displaying the last time the statement // statistics were reset. export const selectLastReset = createSelector( @@ -258,7 +246,6 @@ export default withRouter( isReqInFlight: selectStatementsDataInFlight(state), lastUpdated: selectStatementsLastUpdated(state), statementsError: state.cachedData.statements.lastError, - totalFingerprints: selectTotalFingerprints(state), hasViewActivityRedactedRole: selectHasViewActivityRedactedRole(state), hasAdminRole: selectHasAdminRole(state), limit: limitSetting.selector(state), From 87ebb296835ebb6f6f517f664cbf7b782f83c106 Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Mon, 17 Apr 2023 14:52:31 -0400 Subject: [PATCH 5/5] ui: add aggregation step for cases when we have an older server version For CC clusters we may be in a situation where we have the latest cluster-ui version for a major version, but an older server version. In those cases, the stmt stats data returned by the server will be grouped on more than just the stmt fingerprint id and app name, so we should put an additional grouping step in the UI payload handler to ensure that the data is aligned with what is expected by the newer UI version. Epic: none Release note: None --- .../cluster-ui/src/api/statementsApi.spec.ts | 93 +++++++++++++++++++ .../cluster-ui/src/api/statementsApi.ts | 8 +- .../cluster-ui/src/util/appStats/appStats.ts | 2 +- 3 files changed, 98 insertions(+), 5 deletions(-) diff --git a/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.spec.ts b/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.spec.ts index 13137dbecff8..190becccfd4b 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.spec.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.spec.ts @@ -9,6 +9,7 @@ // licenses/APL.txt. import Long from "long"; import { + aggregateOnStmtFingerprintAndAppName, createCombinedStmtsRequest, getCombinedStatements, getFlushedTxnStatsApi, @@ -433,3 +434,95 @@ describe("getFlushedTxnStatsApi", () => { }, ); }); + +describe("aggregateOnStmtFingerprintAndAppName", () => { + it("should aggregate stmts on stmt fingerprint ID and app name only", () => { + const stmts = [ + { + id: 1, + appName: "cockroach", + count: 3, + }, + { + id: 1, + appName: "cockroach", + count: 1, + }, + { + id: 1, + appName: "not_cockroach", + count: 1, + }, + { + id: 2, + appName: "cockroach", + count: 2, + }, + { + id: 2, + appName: "cockroach", + count: 1, + }, + { + id: 2, + appName: "cockroach", + count: 1, + }, + { + id: 3, + appName: "myApp", + count: 1, + }, + ].map(stmt => + mockStmtStats({ + id: Long.fromInt(stmt.id), + key: { + key_data: { + app: stmt.appName, + }, + }, + stats: { + count: Long.fromInt(stmt.count), + }, + }), + ); + + const aggregatedStmts = aggregateOnStmtFingerprintAndAppName(stmts) + .sort((stmtA, stmtB) => { + const comp = stmtA.id.compare(stmtB.id); + if (comp === 0) + return stmtA.key.key_data.app.localeCompare(stmtB.key.key_data.app); + return comp; + }) + .map(stmt => ({ + id: stmt.id.toInt(), + appName: stmt.key.key_data.app, + count: stmt.stats.count.toInt(), + })); + + const expectedSortedRes = [ + { + id: 1, + appName: "cockroach", + count: 4, + }, + { + id: 1, + appName: "not_cockroach", + count: 1, + }, + { + id: 2, + appName: "cockroach", + count: 4, + }, + { + id: 3, + appName: "myApp", + count: 1, + }, + ]; + + expect(aggregatedStmts).toEqual(expectedSortedRes); + }); +}); diff --git a/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts index b0a57a6dcfb9..fd0a15cc6a2c 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/statementsApi.ts @@ -134,9 +134,6 @@ export function sortAndTruncateStmtsResponse( return; } - // Aggregate on only stmt fingerprint id and app name. - res.statements = aggregateOnStmtFingerprintAndAppName(res.statements); - switch (sort) { case SqlStatsSortOptions.SERVICE_LAT: res.statements.sort((stmtA: Stmt, stmtB: Stmt): number => { @@ -190,11 +187,14 @@ export const getCombinedStatements = ( null, "10M", ).then(res => { + // Data from older server versions may be aggregated on more criteria. + // As of 23.1, we should now only aggregate on only stmt fingerprint id and app name. + res.statements = aggregateOnStmtFingerprintAndAppName(res.statements); + // 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; diff --git a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts index 2a22f5f39ee0..1357c60220b4 100644 --- a/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts +++ b/pkg/ui/workspaces/cluster-ui/src/util/appStats/appStats.ts @@ -193,7 +193,7 @@ export function addStatementStats( a.last_exec_timestamp.seconds > b.last_exec_timestamp.seconds ? a.last_exec_timestamp : b.last_exec_timestamp, - nodes: uniqueLong([...a.nodes, ...b.nodes]), + nodes: uniqueLong([...(a?.nodes ?? []), ...(b?.nodes ?? [])]), plan_gists: planGists, index_recommendations: indexRec, };