From 18c69179f1383c84879aea72880818ea63abfb9a Mon Sep 17 00:00:00 2001 From: maryliag Date: Thu, 16 Feb 2023 14:29:20 -0500 Subject: [PATCH] ui: show data for txn insights when max size reached Previously, when the sql api returned a max size reached error, we were just showing the error, but not the data that was also being returned. This commit updates the Insights Workload > Transaction page with the new behaviour. Part Of: #96184 Release note (ui change): Still show data on the console (with a warning) for Transaction Insights when we reach a "max size exceed" error from the sql api. --- .../workspaces/cluster-ui/src/api/sqlApi.ts | 8 +- .../cluster-ui/src/api/stmtInsightsApi.ts | 235 ++++++++ .../cluster-ui/src/api/txnInsightsApi.ts | 554 ++++++++++++++++++ .../transactionInsightDetails.tsx | 21 + .../transactionInsightDetailsConnected.tsx | 5 + .../transactionInsightDetailsOverviewTab.tsx | 252 ++++++++ .../transactionInsightsView.tsx | 19 + .../workloadInsightsPageConnected.tsx | 2 + .../transactionInsightDetails.reducer.ts | 11 +- .../transactionInsightDetails.sagas.ts | 6 +- .../transactionInsightDetails.selectors.ts | 5 + .../statementFingerprintInsights.reducer.ts | 68 +++ .../statementInsights.reducer.ts | 6 +- .../statementInsights.selectors.ts | 2 +- .../transactionInsights.reducer.ts | 6 +- .../transactionInsights.selectors.ts | 6 +- .../db-console/src/redux/apiReducers.ts | 6 +- .../src/views/insights/insightsSelectors.ts | 17 +- .../transactionInsightDetailsPage.tsx | 5 + .../views/insights/workloadInsightsPage.tsx | 2 + 20 files changed, 1212 insertions(+), 24 deletions(-) create mode 100644 pkg/ui/workspaces/cluster-ui/src/api/stmtInsightsApi.ts create mode 100644 pkg/ui/workspaces/cluster-ui/src/api/txnInsightsApi.ts create mode 100644 pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsOverviewTab.tsx create mode 100644 pkg/ui/workspaces/cluster-ui/src/store/insights/statementFingerprintInsights/statementFingerprintInsights.reducer.ts diff --git a/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts index 53ba67928b85..382bd7cb1a40 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts @@ -60,9 +60,9 @@ export type SqlExecutionErrorMessage = { source: { file: string; line: number; function: "string" }; }; -export type ApiResponse = { +export type SqlApiResponse = { maxSizeReached: boolean; - results: Array; + results: ResultType; }; export const SQL_API_PATH = "/api/v2/sql/"; @@ -147,7 +147,7 @@ export function sqlApiErrorMessage(message: string): string { return message; } -function isMaxSizeError(message: string): boolean { +export function isMaxSizeError(message: string): boolean { return !!message?.includes("max result size exceeded"); } @@ -155,7 +155,7 @@ export function formatApiResult( results: Array, error: SqlExecutionErrorMessage, errorMessageContext: string, -): ApiResponse { +): SqlApiResponse { const maxSizeError = isMaxSizeError(error?.message); if (error && !maxSizeError) { diff --git a/pkg/ui/workspaces/cluster-ui/src/api/stmtInsightsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/stmtInsightsApi.ts new file mode 100644 index 000000000000..fbb28f153f1f --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/api/stmtInsightsApi.ts @@ -0,0 +1,235 @@ +// 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 { + SqlApiResponse, + executeInternalSql, + formatApiResult, + LARGE_RESULT_SIZE, + LONG_TIMEOUT, + SqlExecutionRequest, + sqlResultsAreEmpty, + SqlTxnResult, +} from "./sqlApi"; +import { + ContentionDetails, + getInsightsFromProblemsAndCauses, + InsightExecEnum, + StmtInsightEvent, +} from "src/insights"; +import moment from "moment"; +import { INTERNAL_APP_NAME_PREFIX } from "src/recentExecutions/recentStatementUtils"; +import { FixFingerprintHexValue } from "../util"; +import { getContentionDetailsApi } from "./contentionApi"; + +export type StmtInsightsReq = { + start?: moment.Moment; + end?: moment.Moment; + stmtExecutionID?: string; + stmtFingerprintId?: string; +}; + +export type StmtInsightsResponseRow = { + session_id: string; + txn_id: string; + txn_fingerprint_id: string; // hex string + implicit_txn: boolean; + stmt_id: string; + stmt_fingerprint_id: string; // hex string + query: string; + start_time: string; // Timestamp + end_time: string; // Timestamp + full_scan: boolean; + user_name: string; + app_name: string; + database_name: string; + rows_read: number; + rows_written: number; + priority: string; + retries: number; + exec_node_ids: number[]; + contention: string; // interval + contention_events: ContentionDetails[]; + last_retry_reason?: string; + causes: string[]; + problem: string; + index_recommendations: string[]; + plan_gist: string; + cpu_sql_nanos: number; +}; + +const stmtColumns = ` +session_id, +txn_id, +txn_fingerprint_id, +implicit_txn, +stmt_id, +stmt_fingerprint_id, +query, +start_time, +end_time, +full_scan, +user_name, +app_name, +database_name, +rows_read, +rows_written, +priority, +retries, +exec_node_ids, +contention, +last_retry_reason, +causes, +problem, +index_recommendations, +plan_gist, +cpu_sql_nanos +`; + +const stmtInsightsOverviewQuery = (filters?: StmtInsightsReq): string => { + if (filters?.stmtExecutionID) { + return ` +SELECT ${stmtColumns} FROM crdb_internal.cluster_execution_insights +WHERE stmt_id = '${filters.stmtExecutionID}'`; + } + + let whereClause = ` + WHERE app_name NOT LIKE '${INTERNAL_APP_NAME_PREFIX}%' + AND problem != 'None' + AND txn_id != '00000000-0000-0000-0000-000000000000'`; + if (filters?.start) { + whereClause = + whereClause + ` AND start_time >= '${filters.start.toISOString()}'`; + } + if (filters?.end) { + whereClause = + whereClause + ` AND end_time <= '${filters.end.toISOString()}'`; + } + if (filters?.stmtFingerprintId) { + whereClause = + whereClause + + ` AND encode(stmt_fingerprint_id, 'hex') = '${filters.stmtFingerprintId}'`; + } + + return ` +SELECT ${stmtColumns} FROM + ( + SELECT DISTINCT ON (stmt_fingerprint_id, problem, causes) + * + FROM + crdb_internal.cluster_execution_insights + ${whereClause} + ORDER BY stmt_fingerprint_id, problem, causes, end_time DESC + )`; +}; + +export const stmtInsightsByTxnExecutionQuery = (id: string): string => ` + SELECT ${stmtColumns} + FROM crdb_internal.cluster_execution_insights + WHERE txn_id = '${id}' +`; + +export async function getStmtInsightsApi( + req?: StmtInsightsReq, +): Promise> { + const request: SqlExecutionRequest = { + statements: [ + { + sql: stmtInsightsOverviewQuery(req), + }, + ], + execute: true, + max_result_size: LARGE_RESULT_SIZE, + timeout: LONG_TIMEOUT, + }; + + const result = await executeInternalSql(request); + + if (sqlResultsAreEmpty(result)) { + return formatApiResult([], result.error, "retrieving insights information"); + } + const stmtInsightEvent = formatStmtInsights(result.execution?.txn_results[0]); + await addStmtContentionInfoApi(stmtInsightEvent); + return formatApiResult( + stmtInsightEvent, + result.error, + "retrieving insights information", + ); +} + +async function addStmtContentionInfoApi( + input: StmtInsightEvent[], +): Promise { + if (!input || input.length === 0) { + return; + } + + for (let i = 0; i < input.length; i++) { + const event = input[i]; + if ( + event.contentionTime == null || + event.contentionTime.asMilliseconds() <= 0 + ) { + continue; + } + + event.contentionEvents = await getContentionDetailsApi({ + waitingTxnID: null, + waitingStmtID: event.statementExecutionID, + start: null, + end: null, + }); + } +} + +export function formatStmtInsights( + response: SqlTxnResult, +): StmtInsightEvent[] { + if (!response?.rows?.length) { + return []; + } + + return response.rows.map((row: StmtInsightsResponseRow) => { + const start = moment.utc(row.start_time); + const end = moment.utc(row.end_time); + + return { + transactionExecutionID: row.txn_id, + transactionFingerprintID: FixFingerprintHexValue(row.txn_fingerprint_id), + implicitTxn: row.implicit_txn, + databaseName: row.database_name, + application: row.app_name, + username: row.user_name, + sessionID: row.session_id, + priority: row.priority, + retries: row.retries, + lastRetryReason: row.last_retry_reason, + query: row.query, + startTime: start, + endTime: end, + elapsedTimeMillis: end.diff(start, "milliseconds"), + statementExecutionID: row.stmt_id, + statementFingerprintID: FixFingerprintHexValue(row.stmt_fingerprint_id), + isFullScan: row.full_scan, + rowsRead: row.rows_read, + rowsWritten: row.rows_written, + // This is the total stmt contention. + contentionTime: row.contention ? moment.duration(row.contention) : null, + indexRecommendations: row.index_recommendations, + insights: getInsightsFromProblemsAndCauses( + [row.problem], + row.causes, + InsightExecEnum.STATEMENT, + ), + planGist: row.plan_gist, + cpuSQLNanos: row.cpu_sql_nanos, + } as StmtInsightEvent; + }); +} diff --git a/pkg/ui/workspaces/cluster-ui/src/api/txnInsightsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/txnInsightsApi.ts new file mode 100644 index 000000000000..2aa02f3f24b4 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/api/txnInsightsApi.ts @@ -0,0 +1,554 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { + SqlApiResponse, + executeInternalSql, + formatApiResult, + INTERNAL_SQL_API_APP, + LARGE_RESULT_SIZE, + LONG_TIMEOUT, + sqlApiErrorMessage, + SqlExecutionRequest, + SqlExecutionResponse, + sqlResultsAreEmpty, + isMaxSizeError, +} from "./sqlApi"; +import { + ContentionDetails, + getInsightsFromProblemsAndCauses, + InsightExecEnum, + InsightNameEnum, + TxnContentionInsightDetails, + TxnInsightDetails, + TxnInsightEvent, +} from "src/insights"; +import moment from "moment"; +import { FixFingerprintHexValue } from "../util"; +import { + formatStmtInsights, + stmtInsightsByTxnExecutionQuery, + StmtInsightsResponseRow, +} from "./stmtInsightsApi"; +import { INTERNAL_APP_NAME_PREFIX } from "src/recentExecutions/recentStatementUtils"; +import { getContentionDetailsApi } from "./contentionApi"; + +export const TXN_QUERY_PREVIEW_MAX = 800; +export const QUERY_MAX = 1500; +export const TXN_INSIGHTS_TABLE_NAME = + "crdb_internal.cluster_txn_execution_insights"; + +const makeInsightsSqlRequest = ( + queries: Array, +): SqlExecutionRequest => ({ + statements: queries.filter(q => q).map(query => ({ sql: query })), + execute: true, + max_result_size: LARGE_RESULT_SIZE, + timeout: LONG_TIMEOUT, +}); + +export type TxnWithStmtFingerprints = { + application: string; + transactionFingerprintID: string; + queryIDs: string[]; // Statement fingerprint IDs. +}; + +type TxnStmtFingerprintsResponseColumns = { + transaction_fingerprint_id: string; + query_ids: string[]; // Statement Fingerprint IDs. + app_name: string; +}; + +// txnStmtFingerprintsQuery selects all statement fingerprints for each +// requested transaction fingerprint. +const txnStmtFingerprintsQuery = (txnFingerprintIDs: string[]) => ` +SELECT + DISTINCT ON (fingerprint_id) encode(fingerprint_id, 'hex') AS transaction_fingerprint_id, + app_name, + ARRAY( SELECT jsonb_array_elements_text(metadata -> 'stmtFingerprintIDs' )) AS query_ids +FROM crdb_internal.transaction_statistics +WHERE app_name != '${INTERNAL_SQL_API_APP}' + AND encode(fingerprint_id, 'hex') = + ANY ARRAY[ ${txnFingerprintIDs.map(id => `'${id}'`).join(",")} ]`; + +function formatTxnFingerprintsResults( + response: SqlExecutionResponse, +): TxnWithStmtFingerprints[] { + if (sqlResultsAreEmpty(response)) { + return []; + } + + return response.execution.txn_results[0].rows.map(row => ({ + transactionFingerprintID: FixFingerprintHexValue( + row.transaction_fingerprint_id, + ), + queryIDs: row.query_ids, + application: row.app_name, + })); +} + +type StmtFingerprintToQueryRecord = Map< + string, // Key = Stmt fingerprint ID + string // Value = query string +>; + +type FingerprintStmtsResponseColumns = { + statement_fingerprint_id: string; + query: string; +}; + +// Query to select all statement queries for each requested statement +// fingerprint. +const fingerprintStmtsQuery = (stmtFingerprintIDs: string[]): string => ` +SELECT + DISTINCT ON (fingerprint_id) encode(fingerprint_id, 'hex') AS statement_fingerprint_id, + prettify_statement(metadata ->> 'query', 108, 1, 1) AS query +FROM crdb_internal.statement_statistics +WHERE encode(fingerprint_id, 'hex') = + ANY ARRAY[ ${stmtFingerprintIDs.map(id => `'${id}'`).join(",")} ]`; + +function createStmtFingerprintToQueryMap( + response: SqlExecutionResponse, +): StmtFingerprintToQueryRecord { + const idToQuery: Map = new Map(); + if (sqlResultsAreEmpty(response)) { + // No statement fingerprint results. + return idToQuery; + } + response.execution.txn_results[0].rows.forEach(row => { + idToQuery.set( + FixFingerprintHexValue(row.statement_fingerprint_id), + row.query, + ); + }); + + return idToQuery; +} + +type PartialTxnContentionDetails = Omit< + TxnContentionInsightDetails, + "application" | "queries" +>; + +function formatTxnContentionDetailsResponse( + response: ContentionDetails[], +): PartialTxnContentionDetails { + if (!response || response.length === 9) { + // No data. + return; + } + + const row = response[0]; + return { + transactionExecutionID: row.waitingTxnID, + transactionFingerprintID: FixFingerprintHexValue( + row.waitingTxnFingerprintID, + ), + blockingContentionDetails: response, + insightName: InsightNameEnum.highContention, + execType: InsightExecEnum.TRANSACTION, + }; +} + +export async function getTxnInsightsContentionDetailsApi( + req: TxnInsightDetailsRequest, +): Promise { + // Note that any errors encountered fetching these results are caught + // earlier in the call stack. + // + // There are 3 api requests/queries in this process. + // 1. Get contention insight for the requested transaction. + // 2. Get the stmt fingerprints for ALL transactions involved in the contention. + // 3. Get the query strings for ALL statements involved in the transaction. + + // Get contention results for requested transaction. + + const contentionResults = await getContentionDetailsApi({ + waitingTxnID: req.txnExecutionID, + waitingStmtID: null, + start: null, + end: null, + }); + + if (contentionResults.length === 0) { + return; + } + const contentionDetails = + formatTxnContentionDetailsResponse(contentionResults); + + // Collect all blocking txn fingerprints involved. + const txnFingerprintIDs: string[] = []; + contentionDetails.blockingContentionDetails.forEach(x => + txnFingerprintIDs.push(x.blockingTxnFingerprintID), + ); + + // Request all blocking stmt fingerprint ids involved. + const getStmtFingerprintsResponse = + await executeInternalSql( + makeInsightsSqlRequest([txnStmtFingerprintsQuery(txnFingerprintIDs)]), + ); + if (getStmtFingerprintsResponse.error) { + throw new Error( + `Error while retrieving statements information: ${sqlApiErrorMessage( + getStmtFingerprintsResponse.error.message, + )}`, + ); + } + const txnsWithStmtFingerprints = formatTxnFingerprintsResults( + getStmtFingerprintsResponse, + ); + + const stmtFingerprintIDs = new Set(); + txnsWithStmtFingerprints.forEach(txnFingerprint => + txnFingerprint.queryIDs.forEach(id => stmtFingerprintIDs.add(id)), + ); + + // Request query string from stmt fingerprint ids. + const stmtQueriesResponse = + await executeInternalSql( + makeInsightsSqlRequest([ + fingerprintStmtsQuery(Array.from(stmtFingerprintIDs)), + ]), + ); + if (stmtQueriesResponse.error) { + throw new Error( + `Error while retrieving statements information: ${sqlApiErrorMessage( + stmtQueriesResponse.error.message, + )}`, + ); + } + + return buildTxnContentionInsightDetails( + contentionDetails, + txnsWithStmtFingerprints, + createStmtFingerprintToQueryMap(stmtQueriesResponse), + ); +} + +function buildTxnContentionInsightDetails( + partialTxnContentionDetails: PartialTxnContentionDetails, + txnsWithStmtFingerprints: TxnWithStmtFingerprints[], + stmtFingerprintToQuery: StmtFingerprintToQueryRecord, +): TxnContentionInsightDetails { + if ( + !partialTxnContentionDetails && + !txnsWithStmtFingerprints.length && + !stmtFingerprintToQuery.size + ) { + return null; + } + + partialTxnContentionDetails.blockingContentionDetails.forEach(blockedRow => { + const currBlockedFingerprintStmts = txnsWithStmtFingerprints.find( + txn => + txn.transactionFingerprintID === blockedRow.blockingTxnFingerprintID, + ); + + if (!currBlockedFingerprintStmts) { + return; + } + + blockedRow.blockingTxnQuery = currBlockedFingerprintStmts.queryIDs.map( + id => stmtFingerprintToQuery.get(id) ?? "", + ); + }); + + const waitingTxn = txnsWithStmtFingerprints.find( + txn => + txn.transactionFingerprintID === + partialTxnContentionDetails.transactionFingerprintID, + ); + + return { + ...partialTxnContentionDetails, + application: waitingTxn?.application, + }; +} + +type TxnInsightsResponseRow = { + session_id: string; + txn_id: string; + txn_fingerprint_id: string; // Hex string + implicit_txn: boolean; + query: string; + start_time: string; + end_time: string; + app_name: string; + user_name: string; + rows_read: number; + rows_written: number; + priority: string; + retries: number; + last_retry_reason?: string; + contention: string; // Duration. + problems: string[]; + causes: string[]; + stmt_execution_ids: string[]; + cpu_sql_nanos: number; +}; + +type TxnQueryFilters = { + execID?: string; + fingerprintID?: string; + start?: moment.Moment; + end?: moment.Moment; +}; + +// We only surface the most recently observed problem for a given +// transaction. +const createTxnInsightsQuery = (filters?: TxnQueryFilters) => { + const queryLimit = filters.execID ? QUERY_MAX : TXN_QUERY_PREVIEW_MAX; + + const txnColumns = ` +session_id, +txn_id, +encode(txn_fingerprint_id, 'hex') AS txn_fingerprint_id, +implicit_txn, +rpad(query, ${queryLimit}, '') AS query, +start_time, +end_time, +app_name, +user_name, +rows_read, +rows_written, +priority, +retries, +contention, +last_retry_reason, +problems, +causes, +stmt_execution_ids, +cpu_sql_nanos`; + + if (filters?.execID) { + return ` +SELECT + ${txnColumns} +FROM ${TXN_INSIGHTS_TABLE_NAME} +WHERE txn_id = '${filters.execID}' +`; + } + + let whereClause = ` +WHERE app_name NOT LIKE '${INTERNAL_APP_NAME_PREFIX}%' +AND txn_id != '00000000-0000-0000-0000-000000000000'`; + + if (filters?.start) { + whereClause += ` AND start_time >= '${filters.start.toISOString()}'`; + } + + if (filters?.end) { + whereClause += ` AND end_time <= '${filters.end.toISOString()}'`; + } + + if (filters?.fingerprintID) { + whereClause += ` AND encode(txn_fingerprint_id, 'hex') = '${filters.fingerprintID}'`; + } + + return ` +SELECT ${txnColumns} FROM + ( + SELECT DISTINCT ON (txn_fingerprint_id, problems, causes) + * + FROM + ${TXN_INSIGHTS_TABLE_NAME} + ${whereClause} + ORDER BY txn_fingerprint_id, problems, causes, end_time DESC + ) +`; +}; + +function formatTxnInsightsRow(row: TxnInsightsResponseRow): TxnInsightEvent { + const startTime = moment.utc(row.start_time); + const endTime = moment.utc(row.end_time); + const insights = getInsightsFromProblemsAndCauses( + row.problems, + row.causes, + InsightExecEnum.TRANSACTION, + ); + return { + sessionID: row.session_id, + transactionExecutionID: row.txn_id, + transactionFingerprintID: row.txn_fingerprint_id, + implicitTxn: row.implicit_txn, + query: row.query.split(" ; ").join("\n"), + startTime, + endTime, + elapsedTimeMillis: endTime.diff(startTime, "milliseconds"), + application: row.app_name, + username: row.user_name, + rowsRead: row.rows_read, + rowsWritten: row.rows_written, + priority: row.priority, + retries: row.retries, + lastRetryReason: row.last_retry_reason, + contentionTime: moment.duration(row.contention ?? 0), + insights, + stmtExecutionIDs: row.stmt_execution_ids, + cpuSQLNanos: row.cpu_sql_nanos, + }; +} + +export type TxnInsightsRequest = { + txnExecutionID?: string; + txnFingerprintID?: string; + start?: moment.Moment; + end?: moment.Moment; +}; + +export async function getTxnInsightsApi( + req?: TxnInsightsRequest, +): Promise> { + const filters: TxnQueryFilters = { + start: req?.start, + end: req?.end, + execID: req?.txnExecutionID, + fingerprintID: req?.txnFingerprintID, + }; + const request = makeInsightsSqlRequest([createTxnInsightsQuery(filters)]); + const result = await executeInternalSql(request); + + if (sqlResultsAreEmpty(result)) { + return formatApiResult([], result.error, "retrieving insights information"); + } + + return formatApiResult( + result.execution.txn_results[0].rows.map(formatTxnInsightsRow), + result.error, + "retrieving insights information", + ); +} + +export type TxnInsightDetailsRequest = { + txnExecutionID: string; + excludeStmts?: boolean; + excludeTxn?: boolean; + excludeContention?: boolean; + mergeResultWith?: TxnInsightDetails; + start?: moment.Moment; + end?: moment.Moment; +}; + +export type TxnInsightDetailsReqErrs = { + txnDetailsErr: Error | null; + contentionErr: Error | null; + statementsErr: Error | null; +}; + +export type TxnInsightDetailsResponse = { + txnExecutionID: string; + result: TxnInsightDetails; + errors: TxnInsightDetailsReqErrs; +}; + +export async function getTxnInsightDetailsApi( + req: TxnInsightDetailsRequest, +): Promise> { + // All queries in this request read from virtual tables, which is an + // expensive operation. To reduce the number of RPC fanouts, we have the + // caller specify which parts of the txn details we should return, since + // some parts may be available in the cache or are unnecessary to fetch + // (e.g. when there is no high contention to report). + // + // Note the way we construct the object below is important. We spread the + // the existing object fields into a new object in order to ensure a new + // reference is returned so that components will be notified that there + // was a change. However, we want the internal objects (e.g. txnDetails) + // should only change when they are re-fetched so that components don't update + // unnecessarily. + const txnInsightDetails: TxnInsightDetails = { ...req.mergeResultWith }; + const errors: TxnInsightDetailsReqErrs = { + txnDetailsErr: null, + contentionErr: null, + statementsErr: null, + }; + + let maxSizeReached = false; + if (!req.excludeTxn) { + const request = makeInsightsSqlRequest([ + createTxnInsightsQuery({ + execID: req?.txnExecutionID, + start: req?.start, + end: req?.end, + }), + ]); + + try { + const result = await executeInternalSql(request); + maxSizeReached = isMaxSizeError(result.error?.message); + + if (result.error && !maxSizeReached) { + throw new Error( + `Error while retrieving insights information: ${sqlApiErrorMessage( + result.error.message, + )}`, + ); + } + + const txnDetailsRes = result.execution.txn_results[0]; + if (txnDetailsRes.rows?.length) { + const txnDetails = formatTxnInsightsRow(txnDetailsRes.rows[0]); + txnInsightDetails.txnDetails = txnDetails; + } + } catch (e) { + errors.txnDetailsErr = e; + } + } + + if (!req.excludeStmts) { + try { + const request = makeInsightsSqlRequest([ + stmtInsightsByTxnExecutionQuery(req.txnExecutionID), + ]); + + const result = await executeInternalSql(request); + const maxSizeStmtReached = isMaxSizeError(result.error?.message); + + if (result.error && !maxSizeStmtReached) { + throw new Error( + `Error while retrieving insights information: ${sqlApiErrorMessage( + result.error.message, + )}`, + ); + } + maxSizeReached = maxSizeReached || maxSizeStmtReached; + + const stmts = result.execution.txn_results[0]; + if (stmts.rows?.length) { + txnInsightDetails.statements = formatStmtInsights(stmts); + } + } catch (e) { + errors.statementsErr = e; + } + } + + const highContention = txnInsightDetails.txnDetails?.insights?.some( + insight => insight.name === InsightNameEnum.highContention, + ); + + try { + if (!req.excludeContention && highContention) { + const contentionInfo = await getTxnInsightsContentionDetailsApi(req); + txnInsightDetails.blockingContentionDetails = + contentionInfo?.blockingContentionDetails; + } + } catch (e) { + errors.contentionErr = e; + } + + return { + maxSizeReached: maxSizeReached, + results: { + txnExecutionID: req.txnExecutionID, + result: txnInsightDetails, + errors, + }, + }; +} diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetails.tsx index 11c3029c4f67..72fa503930af 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetails.tsx @@ -15,6 +15,7 @@ import { Heading } from "@cockroachlabs/ui-components"; import { Col, Row } from "antd"; import "antd/lib/col/style"; import "antd/lib/row/style"; +import "antd/lib/tabs/style"; import { Button } from "src/button"; import { Loading } from "src/loading"; import { SqlBox, SqlBoxSize } from "src/sql"; @@ -48,6 +49,9 @@ import { InsightsError } from "../insightsErrorComponent"; import { TransactionDetailsLink } from "../workloadInsights/util"; import { TimeScale } from "../../timeScaleDropdown"; import { idAttr } from "src/util"; +import { InlineAlert } from "@cockroachlabs/ui-components"; +import { insights } from "src/util"; +import { Anchor } from "src/anchor"; const tableCx = classNames.bind(insightTableStyles); @@ -70,10 +74,12 @@ function insightsTableData( }); } + export interface TransactionInsightDetailsStateProps { insightEventDetails: TransactionInsightEventDetailsResponse; insightError: Error | null; hasAdminRole: boolean; + maxSizeApiReached?: boolean; } export interface TransactionInsightDetailsDispatchProps { @@ -100,6 +106,7 @@ export const TransactionInsightDetails: React.FC< match, hasAdminRole, refreshUserSQLRoles, + maxSizeApiReached, }) => { const [insightsSortSetting, setInsightsSortSetting] = useState({ ascending: false, @@ -244,6 +251,20 @@ export const TransactionInsightDetails: React.FC< )} + {maxSizeApiReached && ( + + Not all statements are displayed because the maximum + number of statements was reached in the console.  + + Learn more + + + } + /> + )} diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsConnected.tsx index 8bc362a75be7..1fe1fdddba0d 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsConnected.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsConnected.tsx @@ -19,6 +19,7 @@ import { selectTransactionInsightDetails, selectTransactionInsightDetailsError, actions, + selectTransactionInsightDetailsMaxSizeReached, } from "src/store/insightDetails/transactionInsightDetails"; import { TimeScale } from "../../timeScaleDropdown"; import { actions as sqlStatsActions } from "../../store/sqlStats"; @@ -37,6 +38,10 @@ const mapStateToProps = ( insightEventDetails: insightDetails, insightError: insightError, hasAdminRole: selectHasAdminRole(state), + maxSizeApiReached: selectTransactionInsightDetailsMaxSizeReached( + state, + props, + ), }; }; diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsOverviewTab.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsOverviewTab.tsx new file mode 100644 index 000000000000..3cb2d4910f6b --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsOverviewTab.tsx @@ -0,0 +1,252 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import React, { useContext, useState } from "react"; +import { Heading } from "@cockroachlabs/ui-components"; +import { Col, Row } from "antd"; +import "antd/lib/col/style"; +import "antd/lib/row/style"; +import { SqlBox, SqlBoxSize } from "src/sql"; +import { SummaryCard, SummaryCardItem } from "src/summaryCard"; +import { + Count, + DATE_WITH_SECONDS_AND_MILLISECONDS_FORMAT_24_UTC, + Duration, +} from "src/util/format"; +import { WaitTimeInsightsLabels } from "src/detailsPanels/waitTimeInsightsPanel"; +import { NO_SAMPLES_FOUND } from "src/util"; +import { + InsightsSortedTable, + makeInsightsColumns, +} from "src/insightsTable/insightsTable"; +import { WaitTimeDetailsTable } from "./insightDetailsTables"; +import { + ContentionDetails, + ContentionEvent, + InsightExecEnum, + InsightNameEnum, + StmtInsightEvent, + TxnInsightEvent, +} from "../types"; + +import classNames from "classnames/bind"; +import { CockroachCloudContext } from "../../contexts"; +import { TransactionDetailsLink } from "../workloadInsights/util"; +import { TimeScale } from "../../timeScaleDropdown"; +import { getTxnInsightRecommendations } from "../utils"; +import { SortSetting } from "../../sortedtable"; +import { TxnInsightDetailsReqErrs } from "src/api"; +import { Loading } from "src/loading"; + +import insightTableStyles from "src/insightsTable/insightsTable.module.scss"; +import insightsDetailsStyles from "src/insights/workloadInsightDetails/insightsDetails.module.scss"; +import { InsightsError } from "../insightsErrorComponent"; + +const cx = classNames.bind(insightsDetailsStyles); +const tableCx = classNames.bind(insightTableStyles); + +type Props = { + txnDetails: TxnInsightEvent | null; + statements: StmtInsightEvent[] | null; + contentionDetails?: ContentionDetails[]; + setTimeScale: (ts: TimeScale) => void; + hasAdminRole: boolean; + errors: TxnInsightDetailsReqErrs | null; + maxRequestsReached: boolean; + maxApiSizeReached: boolean; +}; + +export const TransactionInsightDetailsOverviewTab: React.FC = ({ + errors, + contentionDetails, + txnDetails, + statements, + setTimeScale, + hasAdminRole, + maxRequestsReached, +}) => { + const [insightsSortSetting, setInsightsSortSetting] = useState({ + ascending: false, + columnTitle: "insights", + }); + const isCockroachCloud = useContext(CockroachCloudContext); + + const queryFromStmts = statements?.map(s => s.query)?.join("\n"); + let insightQueries = queryFromStmts?.length + ? queryFromStmts + : txnDetails?.query ?? "Insight not found."; + if (maxRequestsReached) { + insightQueries = `${insightQueries} \n\nNot all statements are displayed because +the maximum number of statements was reached in the console.`; + } + const insightsColumns = makeInsightsColumns( + isCockroachCloud, + hasAdminRole, + true, + ); + + const blockingExecutions: ContentionEvent[] = contentionDetails?.map( + event => { + const stmtInsight = statements.find( + stmt => stmt.statementExecutionID == event.waitingStmtID, + ); + return { + executionID: event.blockingExecutionID, + fingerprintID: event.blockingTxnFingerprintID, + waitingStmtID: event.waitingStmtID, + waitingStmtFingerprintID: event.waitingStmtFingerprintID, + queries: event.blockingTxnQuery, + startTime: event.collectionTimeStamp, + contentionTimeMs: event.contentionTimeMs, + execType: InsightExecEnum.TRANSACTION, + schemaName: event.schemaName, + databaseName: event.databaseName, + tableName: event.tableName, + indexName: event.indexName, + stmtInsightEvent: stmtInsight, + }; + }, + ); + + const insightRecs = getTxnInsightRecommendations(txnDetails); + const hasContentionInsights = + txnDetails?.insights.find(i => i.name === InsightNameEnum.highContention) != + null; + + return ( +
+
+ InsightsError(errors?.txnDetailsErr?.message)} + > + + + + + + {txnDetails && ( + <> + + + + + + + + + + + + + + + + {txnDetails.lastRetryReason && ( + + )} + + + + + + + + + + + + + )} + +
+ {hasContentionInsights && ( + InsightsError(errors?.contentionErr?.message)} + > +
+ + + + {WaitTimeInsightsLabels.BLOCKED_TXNS_TABLE_TITLE( + txnDetails?.transactionExecutionID, + InsightExecEnum.TRANSACTION, + )} + +
+ +
+ +
+
+
+ )} +
+ ); +}; diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/transactionInsights/transactionInsightsView.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/transactionInsights/transactionInsightsView.tsx index 618213c7e05b..e197421b5174 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/transactionInsights/transactionInsightsView.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/transactionInsights/transactionInsightsView.tsx @@ -43,6 +43,9 @@ import { InsightsError } from "../../insightsErrorComponent"; import styles from "src/statementsPage/statementsPage.module.scss"; import sortableTableStyles from "src/sortedtable/sortedtable.module.scss"; import { TimeScale } from "../../../timeScaleDropdown"; +import { InlineAlert } from "@cockroachlabs/ui-components"; +import { insights } from "src/util"; +import { Anchor } from "src/anchor"; const cx = classNames.bind(styles); const sortableTableCx = classNames.bind(sortableTableStyles); @@ -53,6 +56,7 @@ export type TransactionInsightsViewStateProps = { filters: WorkloadInsightEventFilters; sortSetting: SortSetting; dropDownSelect?: React.ReactElement; + maxSizeApiReached?: boolean; }; export type TransactionInsightsViewDispatchProps = { @@ -81,6 +85,7 @@ export const TransactionInsightsView: React.FC = ( onSortChange, setTimeScale, dropDownSelect, + maxSizeApiReached, } = props; const [pagination, setPagination] = useState({ @@ -251,6 +256,20 @@ export const TransactionInsightsView: React.FC = ( total={filteredTransactions?.length} onChange={onChangePage} /> + {maxSizeApiReached && ( + + Not all insights are displayed because the maximum number of + insights was reached in the console.  + + Learn more + + + } + /> + )} diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/workloadInsightsPageConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/workloadInsightsPageConnected.tsx index cfc7fcebf7e2..d6cd9db54c1f 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/workloadInsightsPageConnected.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/workloadInsightsPageConnected.tsx @@ -39,6 +39,7 @@ import { selectTransactionInsightsError, selectFilters, selectSortSetting, + selectTransactionInsightsMaxApiReached, } from "src/store/insights/transactionInsights"; import { Dispatch } from "redux"; import { TimeScale } from "../../timeScaleDropdown"; @@ -54,6 +55,7 @@ const transactionMapStateToProps = ( transactionsError: selectTransactionInsightsError(state), filters: selectFilters(state), sortSetting: selectSortSetting(state), + maxSizeApiReached: selectTransactionInsightsMaxApiReached(state), }); const statementMapStateToProps = ( diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.reducer.ts index 7daa181e97eb..cc5227282cc1 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.reducer.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.reducer.ts @@ -16,12 +16,14 @@ import { TransactionInsightEventDetailsRequest, TransactionInsightEventDetailsResponse, } from "src/api/insightsApi"; +import { SqlApiResponse } from "src/api"; export type TransactionInsightDetailsState = { data: TransactionInsightEventDetailsResponse | null; lastUpdated: Moment | null; lastError: Error; valid: boolean; + maxSizeReached: boolean; }; const txnInitialState: TransactionInsightDetailsState = { @@ -47,11 +49,11 @@ const transactionInsightDetailsSlice = createSlice({ reducers: { received: ( state, - action: PayloadAction, + action: PayloadAction>, ) => { - if (action?.payload?.executionID) { - state.cachedData[action.payload.executionID] = { - data: action.payload, + if (action?.payload?.results?.executionID) { + state.cachedData[action.payload.results.executionID] = { + data: action.payload.results, valid: true, lastError: null, lastUpdated: moment.utc(), @@ -64,6 +66,7 @@ const transactionInsightDetailsSlice = createSlice({ valid: false, lastError: action?.payload?.err, lastUpdated: null, + maxSizeReached: false, }; }, invalidated: (state, action: PayloadAction<{ key: string }>) => { diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.sagas.ts index 689fa94899aa..2e6b2694e8f8 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.sagas.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.sagas.ts @@ -17,7 +17,7 @@ import { TransactionInsightEventDetailsResponse, } from "src/api/insightsApi"; import { PayloadAction } from "@reduxjs/toolkit"; -import { ErrorWithKey } from "src/api"; +import { ErrorWithKey, SqlApiResponse } from "src/api"; export function* refreshTransactionInsightDetailsSaga( action: PayloadAction, @@ -48,9 +48,9 @@ const CACHE_INVALIDATION_PERIOD = 5 * 60 * 1000; // 5 minutes in ms const timeoutsByExecID = new Map(); export function receivedTxnInsightsDetailsSaga( - action: PayloadAction, + action: PayloadAction>, ) { - const execID = action.payload.executionID; + const execID = action.payload.results.executionID; clearTimeout(timeoutsByExecID.get(execID)); const id = setTimeout(() => { actions.invalidated({ key: execID }); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.selectors.ts index ab5666e39fbb..5a0397aad057 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insightDetails/transactionInsightDetails/transactionInsightDetails.selectors.ts @@ -29,3 +29,8 @@ export const selectTransactionInsightDetailsError = createSelector( selectTransactionInsightDetailsState, state => state?.lastError, ); + +export const selectTransactionInsightDetailsMaxSizeReached = createSelector( + selectTransactionInsightDetailsState, + state => state?.maxSizeReached, +); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/statementFingerprintInsights/statementFingerprintInsights.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementFingerprintInsights/statementFingerprintInsights.reducer.ts new file mode 100644 index 000000000000..9bf74468e78f --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementFingerprintInsights/statementFingerprintInsights.reducer.ts @@ -0,0 +1,68 @@ +// 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 moment, { Moment } from "moment"; +import { SqlApiResponse, ErrorWithKey, StmtInsightsReq } from "src/api"; +import { StmtInsightEvent } from "../../../insights"; + +export type StatementFingerprintInsightsState = { + data: SqlApiResponse | null; + lastUpdated: Moment | null; + lastError: Error; + valid: boolean; +}; + +export type StatementFingerprintInsightsCachedState = { + cachedData: { [id: string]: StatementFingerprintInsightsState }; +}; + +export type FingerprintInsightResponseWithKey = { + response: SqlApiResponse; + key: string; +}; + +const initialState: StatementFingerprintInsightsCachedState = { + cachedData: {}, +}; + +const statementFingerprintInsightsSlice = createSlice({ + name: `${DOMAIN_NAME}/statementFingerprintInsightsSlice`, + initialState, + reducers: { + received: ( + state, + action: PayloadAction, + ) => { + state.cachedData[action.payload.key] = { + data: action.payload.response, + valid: true, + lastError: null, + lastUpdated: moment.utc(), + }; + }, + failed: (state, action: PayloadAction) => { + state.cachedData[action.payload.key] = { + data: null, + valid: false, + lastError: action.payload.err, + lastUpdated: null, + }; + }, + invalidated: (state, action: PayloadAction<{ key: string }>) => { + delete state.cachedData[action.payload.key]; + }, + refresh: (_, _action: PayloadAction) => {}, + request: (_, _action: PayloadAction) => {}, + }, +}); + +export const { reducer, actions } = statementFingerprintInsightsSlice; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.reducer.ts index 629005f125c6..2d3c31ffb9f0 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.reducer.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.reducer.ts @@ -12,10 +12,10 @@ import { createSlice, PayloadAction } from "@reduxjs/toolkit"; import { DOMAIN_NAME, noopReducer } from "../../utils"; import moment, { Moment } from "moment"; import { StatementInsights } from "src/api/insightsApi"; -import { ApiResponse } from "src/api"; +import { SqlApiResponse } from "src/api"; export type StatementInsightsState = { - data: ApiResponse; + data: SqlApiResponse; lastUpdated: Moment; lastError: Error; valid: boolean; @@ -32,7 +32,7 @@ const statementInsightsSlice = createSlice({ name: `${DOMAIN_NAME}/statementInsightsSlice`, initialState, reducers: { - received: (state, action: PayloadAction>) => { + received: (state, action: PayloadAction>) => { state.data = action.payload; state.valid = true; state.lastError = null; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.selectors.ts index f449a75819e5..c528425eb4de 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insights/statementInsights/statementInsights.selectors.ts @@ -26,7 +26,7 @@ export const selectStatementInsightsError = (state: AppState) => state.adminUI.statementInsights?.lastError; export const selectStmtInsightsMaxApiReached = (state: AppState): boolean => - state.adminUI.statementInsights?.data?.maxSizeReached; + !!state.adminUI.statementInsights?.data?.maxSizeReached; export const selectStatementInsightDetails = createSelector( selectStatementInsights, diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.reducer.ts index c2a31ecd1cc1..66db4d7e3364 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.reducer.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.reducer.ts @@ -11,10 +11,10 @@ import { createSlice, PayloadAction } from "@reduxjs/toolkit"; import { DOMAIN_NAME, noopReducer } from "src/store/utils"; import moment, { Moment } from "moment"; -import { TransactionInsightEventsResponse } from "src/api/insightsApi"; +import { SqlApiResponse, TransactionInsightEventsResponse } from "src/api"; export type TransactionInsightsState = { - data: TransactionInsightEventsResponse; + data: SqlApiResponse; lastUpdated: Moment; lastError: Error; valid: boolean; @@ -33,7 +33,7 @@ const transactionInsightsSlice = createSlice({ reducers: { received: ( state, - action: PayloadAction, + action: PayloadAction>, ) => { state.data = action.payload; state.valid = true; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.selectors.ts index add64c317bb5..d3dce382a693 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/insights/transactionInsights/transactionInsights.selectors.ts @@ -14,7 +14,7 @@ import { selectTxnInsightsCombiner } from "src/selectors/insightsCommon.selector import { localStorageSelector } from "src/store/utils/selectors"; const selectTransactionInsightsData = (state: AppState) => - state.adminUI.transactionInsights.data; + state.adminUI.transactionInsights.data?.results; export const selectTransactionInsights = createSelector( selectTransactionInsightsData, @@ -24,6 +24,10 @@ export const selectTransactionInsights = createSelector( export const selectTransactionInsightsError = (state: AppState) => state.adminUI.transactionInsights?.lastError; +export const selectTransactionInsightsMaxApiReached = ( + state: AppState, +): boolean => !!state.adminUI.transactionInsights?.data?.maxSizeReached; + export const selectSortSetting = createSelector( localStorageSelector, localStorage => localStorage["sortSetting/InsightsPage"], diff --git a/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts b/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts index 269598af403f..bf965d7cf2f7 100644 --- a/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts +++ b/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts @@ -501,9 +501,9 @@ export interface APIReducersState { userSQLRoles: CachedDataReducerState; hotRanges: PaginatedCachedDataReducerState; clusterLocks: CachedDataReducerState; - transactionInsights: CachedDataReducerState; - transactionInsightDetails: KeyedCachedDataReducerState; - statementInsights: CachedDataReducerState>; + transactionInsights: CachedDataReducerState>; + transactionInsightDetails: KeyedCachedDataReducerState>; + statementInsights: CachedDataReducerState>; schemaInsights: CachedDataReducerState; schedules: KeyedCachedDataReducerState; schedule: KeyedCachedDataReducerState; diff --git a/pkg/ui/workspaces/db-console/src/views/insights/insightsSelectors.ts b/pkg/ui/workspaces/db-console/src/views/insights/insightsSelectors.ts index bfe91e196742..6f7be9ee31c7 100644 --- a/pkg/ui/workspaces/db-console/src/views/insights/insightsSelectors.ts +++ b/pkg/ui/workspaces/db-console/src/views/insights/insightsSelectors.ts @@ -40,7 +40,7 @@ export const sortSettingLocalSetting = new LocalSetting< }); export const selectTransactionInsights = createSelector( - (state: AdminUIState) => state.cachedData.transactionInsights?.data, + (state: AdminUIState) => state.cachedData.transactionInsights?.data?.results, selectTxnInsightsCombiner, ); @@ -53,7 +53,7 @@ export const selectTransactionInsightDetails = createSelector( if (!insight) { return null; } - return insight[insightId]?.data; + return insight[insightId]?.data?.results; }, ); @@ -61,8 +61,14 @@ export const selectStmtInsightsMaxApiReached = ( state: AdminUIState, ): boolean => { return !!state.cachedData.statementInsights?.data?.maxSizeReached; + +export const selectTxnInsightsMaxApiReached = ( + state: AdminUIState, +): boolean => { + return !!state.cachedData.transactionInsights?.data?.maxSizeReached; }; + export const selectTransactionInsightDetailsError = createSelector( (state: AdminUIState) => state.cachedData.transactionInsightDetails, selectID, @@ -79,6 +85,13 @@ export const selectStatementInsights = createSelector( selectStatementInsightsCombiner, ); +export const selectTransactionInsightDetailsMaxSizeReached = createSelector( + (state: AdminUIState) => state.cachedData.transactionInsightDetails, + selectID, + (insights, insightId: string): boolean => + insights[insightId]?.data?.maxSizeReached, +); + export const selectStatementInsightDetails = createSelector( selectStatementInsights, selectID, diff --git a/pkg/ui/workspaces/db-console/src/views/insights/transactionInsightDetailsPage.tsx b/pkg/ui/workspaces/db-console/src/views/insights/transactionInsightDetailsPage.tsx index a61cc9ab5975..84260c780d56 100644 --- a/pkg/ui/workspaces/db-console/src/views/insights/transactionInsightDetailsPage.tsx +++ b/pkg/ui/workspaces/db-console/src/views/insights/transactionInsightDetailsPage.tsx @@ -22,6 +22,7 @@ import { AdminUIState } from "src/redux/state"; import { selectTransactionInsightDetails, selectTransactionInsightDetailsError, + selectTransactionInsightDetailsMaxSizeReached, } from "src/views/insights/insightsSelectors"; import { setGlobalTimeScaleAction } from "src/redux/statements"; import { selectHasAdminRole } from "src/redux/user"; @@ -34,6 +35,10 @@ const mapStateToProps = ( insightEventDetails: selectTransactionInsightDetails(state, props), insightError: selectTransactionInsightDetailsError(state, props), hasAdminRole: selectHasAdminRole(state), + maxSizeApiReached: selectTransactionInsightDetailsMaxSizeReached( + state, + props, + ), }; }; diff --git a/pkg/ui/workspaces/db-console/src/views/insights/workloadInsightsPage.tsx b/pkg/ui/workspaces/db-console/src/views/insights/workloadInsightsPage.tsx index 08af9b3f98c6..63a30548d77a 100644 --- a/pkg/ui/workspaces/db-console/src/views/insights/workloadInsightsPage.tsx +++ b/pkg/ui/workspaces/db-console/src/views/insights/workloadInsightsPage.tsx @@ -31,6 +31,7 @@ import { sortSettingLocalSetting, selectTransactionInsights, selectStmtInsightsMaxApiReached, + selectTxnInsightsMaxApiReached, } from "src/views/insights/insightsSelectors"; import { bindActionCreators } from "redux"; import { LocalSetting } from "src/redux/localsettings"; @@ -53,6 +54,7 @@ const transactionMapStateToProps = ( transactionsError: state.cachedData?.transactionInsights.lastError, filters: filtersLocalSetting.selector(state), sortSetting: sortSettingLocalSetting.selector(state), + maxSizeApiReached: selectTxnInsightsMaxApiReached(state), }); const statementMapStateToProps = (