diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index a97d76ec9dd7..b5de846b294c 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -57,6 +57,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" + "github.com/cockroachdb/cockroach/pkg/sql/contentionpb" "github.com/cockroachdb/cockroach/pkg/sql/idxusage" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -6473,9 +6474,15 @@ CREATE TABLE crdb_internal.transaction_contention_events ( contention_duration INTERVAL NOT NULL, contending_key BYTES NOT NULL, - - waiting_stmt_id string NOT NULL, - waiting_stmt_fingerprint_id BYTES NOT NULL + contending_pretty_key STRING NOT NULL, + + waiting_stmt_id string NOT NULL, + waiting_stmt_fingerprint_id BYTES NOT NULL, + + database_name STRING NOT NULL, + schema_name STRING NOT NULL, + table_name STRING NOT NULL, + index_name STRING );`, generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) { // Check permission first before making RPC fanout. @@ -6541,10 +6548,13 @@ CREATE TABLE crdb_internal.transaction_contention_events ( types.DefaultIntervalTypeMetadata, ) + contendingPrettyKey := tree.NewDString("") contendingKey := tree.NewDBytes("") if !shouldRedactContendingKey { + decodedKey, _, _ := keys.DecodeTenantPrefix(resp.Events[i].BlockingEvent.Key) + contendingPrettyKey = tree.NewDString(keys.PrettyPrint(nil /* valDirs */, decodedKey)) contendingKey = tree.NewDBytes( - tree.DBytes(resp.Events[i].BlockingEvent.Key)) + tree.DBytes(decodedKey)) } waitingStmtFingerprintID := tree.NewDBytes( @@ -6552,17 +6562,27 @@ CREATE TABLE crdb_internal.transaction_contention_events ( waitingStmtId := tree.NewDString(hex.EncodeToString(resp.Events[i].WaitingStmtID.GetBytes())) + schemaName, dbName, tableName, indexName, err := getContentionEventInfo(ctx, p, resp.Events[i]) + if err != nil { + return err + } + row = row[:0] row = append(row, collectionTs, // collection_ts tree.NewDUuid(tree.DUuid{UUID: resp.Events[i].BlockingEvent.TxnMeta.ID}), // blocking_txn_id blockingFingerprintID, // blocking_fingerprint_id tree.NewDUuid(tree.DUuid{UUID: resp.Events[i].WaitingTxnID}), // waiting_txn_id - waitingFingerprintID, // waiting_fingerprint_id - contentionDuration, // contention_duration - contendingKey, // contending_key, - waitingStmtId, // waiting_stmt_id - waitingStmtFingerprintID, // waiting_stmt_fingerprint_id + waitingFingerprintID, // waiting_fingerprint_id + contentionDuration, // contention_duration + contendingKey, // contending_key, + contendingPrettyKey, // contending_pretty_key + waitingStmtId, // waiting_stmt_id + waitingStmtFingerprintID, // waiting_stmt_fingerprint_id + tree.NewDString(dbName), // database_name + tree.NewDString(schemaName), // schema_name + tree.NewDString(tableName), // table_name + tree.NewDString(indexName), // index_name ) if err = pusher.pushRow(row...); err != nil { @@ -7207,7 +7227,6 @@ CREATE TABLE crdb_internal.%s ( last_retry_reason STRING, exec_node_ids INT[] NOT NULL, contention INTERVAL, - contention_events JSONB, index_recommendations STRING[] NOT NULL, implicit_txn BOOL NOT NULL, cpu_sql_nanos INT8 @@ -7295,17 +7314,6 @@ func populateStmtInsights( ) } - contentionEvents := tree.DNull - if len(s.ContentionEvents) > 0 { - var contentionEventsJSON json.JSON - contentionEventsJSON, err = convertContentionEventsToJSON(ctx, p, s.ContentionEvents) - if err != nil { - return err - } - - contentionEvents = tree.NewDJSON(contentionEventsJSON) - } - indexRecommendations := tree.NewDArray(types.String) for _, recommendation := range s.IndexRecommendations { if err = indexRecommendations.Append(tree.NewDString(recommendation)); err != nil { @@ -7337,7 +7345,6 @@ func populateStmtInsights( autoRetryReason, execNodeIDs, contentionTime, - contentionEvents, indexRecommendations, tree.MakeDBool(tree.DBool(insight.Transaction.ImplicitTxn)), tree.NewDInt(tree.DInt(s.CPUSQLNanos)), @@ -7347,57 +7354,45 @@ func populateStmtInsights( return } -func convertContentionEventsToJSON( - ctx context.Context, p *planner, contentionEvents []roachpb.ContentionEvent, -) (json json.JSON, err error) { +func getContentionEventInfo( + ctx context.Context, p *planner, contentionEvent contentionpb.ExtendedContentionEvent, +) (schemaName, dbName, tableName, indexName string, err error) { - eventWithNames := make([]sqlstatsutil.ContentionEventWithNames, len(contentionEvents)) - for i, contentionEvent := range contentionEvents { - _, tableID, err := p.ExecCfg().Codec.DecodeTablePrefix(contentionEvent.Key) - if err != nil { - return nil, err - } - _, _, indexID, err := p.ExecCfg().Codec.DecodeIndexPrefix(contentionEvent.Key) - if err != nil { - return nil, err - } - - desc := p.Descriptors() - var tableDesc catalog.TableDescriptor - tableDesc, err = desc.ByIDWithLeased(p.txn).WithoutNonPublic().Get().Table(ctx, descpb.ID(tableID)) - if err != nil { - return nil, err - } + _, tableID, err := p.ExecCfg().Codec.DecodeTablePrefix(contentionEvent.BlockingEvent.Key) + if err != nil { + return "", "", "", "", err + } + _, _, indexID, err := p.ExecCfg().Codec.DecodeIndexPrefix(contentionEvent.BlockingEvent.Key) + if err != nil { + return "", "", "", "", err + } - idxDesc, err := catalog.MustFindIndexByID(tableDesc, descpb.IndexID(indexID)) - if err != nil { - return nil, err - } + desc := p.Descriptors() + var tableDesc catalog.TableDescriptor + tableDesc, err = desc.ByIDWithLeased(p.txn).WithoutNonPublic().Get().Table(ctx, descpb.ID(tableID)) + if err != nil { + return "", "", "", "", err + } - dbDesc, err := desc.ByIDWithLeased(p.txn).WithoutNonPublic().Get().Database(ctx, tableDesc.GetParentID()) - if err != nil { - return nil, err - } + idxDesc, err := catalog.MustFindIndexByID(tableDesc, descpb.IndexID(indexID)) + if err != nil { + return "", "", "", "", err + } - schemaDesc, err := desc.ByIDWithLeased(p.txn).WithoutNonPublic().Get().Schema(ctx, tableDesc.GetParentSchemaID()) - if err != nil { - return nil, err - } + dbDesc, err := desc.ByIDWithLeased(p.txn).WithoutNonPublic().Get().Database(ctx, tableDesc.GetParentID()) + if err != nil { + return "", "", "", "", err + } - var idxName string - if idxDesc != nil { - idxName = idxDesc.GetName() - } + schemaDesc, err := desc.ByIDWithLeased(p.txn).WithoutNonPublic().Get().Schema(ctx, tableDesc.GetParentSchemaID()) + if err != nil { + return "", "", "", "", err + } - eventWithNames[i] = sqlstatsutil.ContentionEventWithNames{ - BlockingTransactionID: contentionEvent.TxnMeta.ID.String(), - SchemaName: schemaDesc.GetName(), - DatabaseName: dbDesc.GetName(), - TableName: tableDesc.GetName(), - IndexName: idxName, - DurationInMs: float64(contentionEvent.Duration) / float64(time.Millisecond), - } + var idxName string + if idxDesc != nil { + idxName = idxDesc.GetName() } - return sqlstatsutil.BuildContentionEventsJSON(eventWithNames) + return schemaDesc.GetName(), dbDesc.GetName(), tableDesc.GetName(), idxName, nil } diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index 603c84fc5c55..f7c8c3b0c13a 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -982,7 +982,12 @@ func TestTxnContentionEventsTable(t *testing.T) { waiting_stmt_id, encode( waiting_txn_fingerprint_id, 'hex' - ) AS waiting_txn_fingerprint_id + ) AS waiting_txn_fingerprint_id, + contending_pretty_key, + database_name, + schema_name, + table_name, + index_name FROM crdb_internal.transaction_contention_events tce inner join ( select @@ -1001,7 +1006,8 @@ func TestTxnContentionEventsTable(t *testing.T) { rowCount++ var blockingTxnId, waitingTxnId, waitingStmtId, waitingStmtFingerprint string - errVerify = rows.Scan(&blockingTxnId, &waitingTxnId, &waitingStmtId, &waitingStmtFingerprint) + var prettyKey, dbName, schemaName, tableName, indexName string + errVerify = rows.Scan(&blockingTxnId, &waitingTxnId, &waitingStmtId, &waitingStmtFingerprint, &prettyKey, &dbName, &schemaName, &tableName, &indexName) if errVerify != nil { return errVerify } @@ -1015,8 +1021,24 @@ func TestTxnContentionEventsTable(t *testing.T) { return fmt.Errorf("transaction_contention_events had default waiting txn id %s, blocking txn id %s", waitingTxnId, blockingTxnId) } - if waitingStmtId == defaultIdString { - return fmt.Errorf("transaction_contention_events had default waiting stmt id %s, blocking txn id %s, waiting txn id %s", waitingStmtId, blockingTxnId, waitingTxnId) + if !strings.HasPrefix(prettyKey, "/Table/") { + return fmt.Errorf("prettyKey should be defaultdb: %s, %s, %s, %s, %s", prettyKey, dbName, schemaName, tableName, indexName) + } + + if dbName != "defaultdb" { + return fmt.Errorf("dbName should be defaultdb: %s, %s, %s, %s, %s", prettyKey, dbName, schemaName, tableName, indexName) + } + + if schemaName != "public" { + return fmt.Errorf("schemaName should be public: %s, %s, %s, %s, %s", prettyKey, dbName, schemaName, tableName, indexName) + } + + if tableName != "t" { + return fmt.Errorf("tableName should be t: %s, %s, %s, %s, %s", prettyKey, dbName, schemaName, tableName, indexName) + } + + if indexName != "t_pkey" { + return fmt.Errorf("indexName should be t_pkey: %s, %s, %s, %s, %s", prettyKey, dbName, schemaName, tableName, indexName) } } diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index e2f38e77e3b7..14f0a502a9a8 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -420,7 +420,7 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 4294967245 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "session_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "client_address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "active_queries", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "last_active_query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "num_txns_executed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "session_start", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 10, "name": "active_query_start", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "kv_txn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "alloc_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "max_alloc_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "session_end", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 4294967245, "name": "node_sessions", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967246 {"table": {"columns": [{"id": 1, "name": "id", "nullable": true, "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "session_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "start", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 5, "name": "txn_string", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "num_stmts", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "num_retries", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "num_auto_retries", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "last_auto_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967246, "name": "node_transactions", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967247 {"table": {"columns": [{"id": 1, "name": "query_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "nullable": true, "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "session_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "start", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 7, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "client_address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "distributed", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "full_scan", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "plan_gist", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "database", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967247, "name": "node_queries", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967248 {"table": {"columns": [{"id": 1, "name": "session_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "problem", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "causes", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "query", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "start_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "end_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "full_scan", "type": {"oid": 16}}, {"id": 13, "name": "user_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "plan_gist", "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "rows_read", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "rows_written", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "last_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 23, "name": "contention", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 24, "name": "contention_events", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 25, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 26, "name": "implicit_txn", "type": {"oid": 16}}, {"id": 27, "name": "cpu_sql_nanos", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967248, "name": "node_execution_insights", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967248 {"table": {"columns": [{"id": 1, "name": "session_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "problem", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "causes", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "query", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "start_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "end_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "full_scan", "type": {"oid": 16}}, {"id": 13, "name": "user_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "plan_gist", "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "rows_read", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "rows_written", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "last_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 23, "name": "contention", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 24, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 25, "name": "implicit_txn", "type": {"oid": 16}}, {"id": 26, "name": "cpu_sql_nanos", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967248, "name": "node_execution_insights", "nextColumnId": 27, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967249 {"table": {"columns": [{"id": 1, "name": "flow_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "stmt", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "since", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 5, "name": "status", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967249, "name": "node_distsql_flows", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967250 {"table": {"columns": [{"id": 1, "name": "table_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "num_contention_events", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "cumulative_contention_time", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 7, "name": "count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967250, "name": "node_contention_events", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967251 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "parent_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "expiration", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 6, "name": "deleted", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967251, "name": "leases", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} @@ -432,7 +432,7 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 4294967257 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "total_reads", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "last_read", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967257, "name": "index_usage_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967258 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967258, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["descriptor_id"], "name": "index_spans_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["index_id", "start_key", "end_key"], "version": 3}], "name": "index_spans", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967259 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "column_direction", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "implicit", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967259, "name": "index_columns", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967260 {"table": {"columns": [{"id": 1, "name": "collection_ts", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "blocking_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "blocking_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "waiting_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 5, "name": "waiting_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "contention_duration", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 7, "name": "contending_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 8, "name": "waiting_stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "waiting_stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967260, "name": "transaction_contention_events", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967260 {"table": {"columns": [{"id": 1, "name": "collection_ts", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "blocking_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "blocking_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "waiting_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 5, "name": "waiting_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "contention_duration", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 7, "name": "contending_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 8, "name": "contending_pretty_key", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "waiting_stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "waiting_stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 11, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "index_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967260, "name": "transaction_contention_events", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967261 {"table": {"columns": [{"id": 1, "name": "source_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "target_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967261, "name": "gossip_network", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967262 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "epoch", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "expiration", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "draining", "type": {"oid": 16}}, {"id": 5, "name": "decommissioning", "type": {"oid": 16}}, {"id": 6, "name": "membership", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "updated_at", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 4294967262, "name": "gossip_liveness", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967263 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "category", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "description", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "value", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967263, "name": "gossip_alerts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} @@ -454,7 +454,7 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 4294967279 {"table": {"columns": [{"id": 1, "name": "range_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "index_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "lock_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 8, "name": "lock_key_pretty", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "txn_id", "nullable": true, "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 10, "name": "ts", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "lock_strength", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "durability", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "granted", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "contended", "type": {"oid": 16}}, {"id": 15, "name": "duration", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967279, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["table_id"], "name": "cluster_locks_table_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], "storeColumnNames": ["range_id", "database_name", "schema_name", "table_name", "index_name", "lock_key", "lock_key_pretty", "txn_id", "ts", "lock_strength", "durability", "granted", "contended", "duration"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [3], "keyColumnNames": ["database_name"], "name": "cluster_locks_database_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], "storeColumnNames": ["range_id", "table_id", "schema_name", "table_name", "index_name", "lock_key", "lock_key_pretty", "txn_id", "ts", "lock_strength", "durability", "granted", "contended", "duration"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 4, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [5], "keyColumnNames": ["table_name"], "name": "cluster_locks_table_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], "storeColumnNames": ["range_id", "table_id", "database_name", "schema_name", "index_name", "lock_key", "lock_key_pretty", "txn_id", "ts", "lock_strength", "durability", "granted", "contended", "duration"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 5, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [14], "keyColumnNames": ["contended"], "name": "cluster_locks_contended_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 15], "storeColumnNames": ["range_id", "table_id", "database_name", "schema_name", "table_name", "index_name", "lock_key", "lock_key_pretty", "txn_id", "ts", "lock_strength", "durability", "granted", "duration"], "version": 3}], "name": "cluster_locks", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 6, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967280 {"table": {"columns": [{"id": 1, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "query", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "implicit_txn", "type": {"oid": 16}}, {"id": 5, "name": "session_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "start_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 7, "name": "end_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 8, "name": "user_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "rows_read", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "rows_written", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "last_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "contention", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 16, "name": "problems", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 17, "name": "causes", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 18, "name": "stmt_execution_ids", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 19, "name": "cpu_sql_nanos", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967280, "name": "node_txn_execution_insights", "nextColumnId": 20, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967281 {"table": {"columns": [{"id": 1, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "query", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "implicit_txn", "type": {"oid": 16}}, {"id": 5, "name": "session_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "start_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 7, "name": "end_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 8, "name": "user_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "rows_read", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "rows_written", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "last_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "contention", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 16, "name": "problems", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 17, "name": "causes", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 18, "name": "stmt_execution_ids", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 19, "name": "cpu_sql_nanos", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967281, "name": "cluster_txn_execution_insights", "nextColumnId": 20, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967282 {"table": {"columns": [{"id": 1, "name": "session_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "problem", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "causes", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "query", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "start_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "end_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "full_scan", "type": {"oid": 16}}, {"id": 13, "name": "user_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "plan_gist", "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "rows_read", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "rows_written", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "last_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 23, "name": "contention", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 24, "name": "contention_events", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 25, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 26, "name": "implicit_txn", "type": {"oid": 16}}, {"id": 27, "name": "cpu_sql_nanos", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967282, "name": "cluster_execution_insights", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967282 {"table": {"columns": [{"id": 1, "name": "session_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "problem", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "causes", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "query", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "start_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "end_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "full_scan", "type": {"oid": 16}}, {"id": 13, "name": "user_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "plan_gist", "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "rows_read", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "rows_written", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "last_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 23, "name": "contention", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 24, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 25, "name": "implicit_txn", "type": {"oid": 16}}, {"id": 26, "name": "cpu_sql_nanos", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967282, "name": "cluster_execution_insights", "nextColumnId": 27, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967283 {"table": {"columns": [{"id": 1, "name": "flow_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "stmt", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "since", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 5, "name": "status", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967283, "name": "cluster_distsql_flows", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967284 {"table": {"columns": [{"id": 1, "name": "table_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "num_contention_events", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "cumulative_contention_time", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 7, "name": "count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967284, "name": "cluster_contention_events", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967285 {"table": {"columns": [{"id": 1, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "num_contention_events", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967285, "name": "cluster_contended_tables", "nextColumnId": 5, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT database_name, schema_name, name, sum(num_contention_events) FROM (SELECT DISTINCT database_name, schema_name, name, index_id, num_contention_events FROM crdb_internal.cluster_contention_events JOIN crdb_internal.tables ON crdb_internal.cluster_contention_events.table_id = crdb_internal.tables.table_id) GROUP BY database_name, schema_name, name"}} diff --git a/pkg/sql/sqlstats/insights/insights.proto b/pkg/sql/sqlstats/insights/insights.proto index effb8556a8e3..36e5e76df179 100644 --- a/pkg/sql/sqlstats/insights/insights.proto +++ b/pkg/sql/sqlstats/insights/insights.proto @@ -115,8 +115,7 @@ message Statement { repeated int64 nodes = 17; google.protobuf.Duration contention = 18 [(gogoproto.stdduration) = true]; repeated string index_recommendations = 19; - // contention_events hit at the statement level. - repeated cockroach.roachpb.ContentionEvent contention_events = 20 [(gogoproto.nullable) = false]; + reserved 20; // previously contention_events Problem problem = 21; repeated Cause causes = 22; int64 cpu_sql_nanos = 23 [(gogoproto.customname) = "CPUSQLNanos"]; diff --git a/pkg/sql/sqlstats/insights/integration/insights_test.go b/pkg/sql/sqlstats/insights/integration/insights_test.go index 900a405e0a3c..2fe5aca68aac 100644 --- a/pkg/sql/sqlstats/insights/integration/insights_test.go +++ b/pkg/sql/sqlstats/insights/integration/insights_test.go @@ -372,16 +372,17 @@ func TestInsightsIntegrationForContention(t *testing.T) { testutils.SucceedsWithin(t, func() error { rows, err := conn.QueryContext(ctx, `SELECT query, - contention::FLOAT, - contention_events->0->>'durationInMs' AS durationMs, - contention_events->0->>'schemaName', - contention_events->0->>'databaseName', - contention_events->0->>'tableName', - contention_events->0->>'indexName', - txn_contention.blocking_txn_fingerprint_id + insight.contention::FLOAT, + sum(txn_contention.contention_duration)::FLOAT AS durationMs, + txn_contention.schema_name, + txn_contention.database_name, + txn_contention.table_name, + txn_contention.index_name, + txn_contention.waiting_txn_fingerprint_id FROM crdb_internal.cluster_execution_insights insight - left join crdb_internal.transaction_contention_events txn_contention on (contention_events->0->>'blockingTxnID')::uuid = txn_contention.blocking_txn_id - where query like 'UPDATE t SET s =%'`) + left join crdb_internal.transaction_contention_events txn_contention on insight.stmt_id = txn_contention.waiting_stmt_id + where query like 'UPDATE t SET s =%' + group by query, insight.contention, txn_contention.schema_name, txn_contention.database_name, txn_contention.table_name, txn_contention.index_name, waiting_txn_fingerprint_id;`) if err != nil { return err } @@ -393,19 +394,18 @@ func TestInsightsIntegrationForContention(t *testing.T) { return err } - var totalContentionFromQuerySeconds, contentionFromEventMs float64 + var totalContentionFromQueryMs, contentionFromEventMs float64 var queryText, schemaName, dbName, tableName, indexName string var blockingTxnFingerprintID gosql.NullString - err = rows.Scan(&queryText, &totalContentionFromQuerySeconds, &contentionFromEventMs, &schemaName, &dbName, &tableName, &indexName, &blockingTxnFingerprintID) + err = rows.Scan(&queryText, &totalContentionFromQueryMs, &contentionFromEventMs, &schemaName, &dbName, &tableName, &indexName, &blockingTxnFingerprintID) if err != nil { return err } - if totalContentionFromQuerySeconds < .2 { - return fmt.Errorf("contention time is %f should be greater than .2 since block is delayed by .5 seconds", totalContentionFromQuerySeconds) + if totalContentionFromQueryMs < .2 { + return fmt.Errorf("contention time is %f should be greater than .2 since block is delayed by .5 seconds", totalContentionFromQueryMs) } - totalContentionFromQueryMs := totalContentionFromQuerySeconds * 1000 diff := totalContentionFromQueryMs - contentionFromEventMs if math.Abs(diff) > .1 { return fmt.Errorf("contention time from column: %f should be the same as event value %f", totalContentionFromQueryMs, contentionFromEventMs) diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding.go index 5b6d76b5adde..2fc6be743f56 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_encoding.go @@ -311,27 +311,6 @@ func BuildStmtDetailsMetadataJSON( return (*aggregatedMetadata)(metadata).jsonFields().encodeJSON() } -// BuildContentionEventsJSON returns a json.JSON object for contention events -// roachpb.ContentionEvent. -// JSON Schema for contention events -// -// { -// "$schema": "https://json-schema.org/draft/2020-12/schema", -// "title": "system.statement_statistics.contention_events", -// "type": "object", -// [{ -// "blockingTxnID": { "type": "string" }, -// "durationInMs": { "type": "number" }, -// "schemaName": { "type": "string" }, -// "databaseName": { "type": "string" }, -// "tableName": { "type": "string" }, -// "indexName": { "type": "string" } -// }] -// } -func BuildContentionEventsJSON(events []ContentionEventWithNames) (json.JSON, error) { - return (*contentionEvents)(&events).encodeJSON() -} - // EncodeUint64ToBytes returns the []byte representation of an uint64 value. func EncodeUint64ToBytes(id uint64) []byte { result := make([]byte, 0, 8) diff --git a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go index 1587c43a4ebd..e1c7fb16dcfb 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go +++ b/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil/json_impl.go @@ -124,50 +124,6 @@ func (s *aggregatedMetadata) jsonFields() jsonFields { } } -// ContentionEventWithNames is used to serialize into -// cluster_execution_insights as a JSON array. This type -// has the names instead of ids to avoid doing joins to get -// the user-friendly names. -type ContentionEventWithNames struct { - BlockingTransactionID string - SchemaName string - DatabaseName string - TableName string - IndexName string - DurationInMs float64 -} - -type contentionEvents []ContentionEventWithNames - -func (s *contentionEvents) encodeJSON() (json.JSON, error) { - builder := json.NewArrayBuilder(len(*s)) - - for _, value := range *s { - jsVal := (*contentionEvent)(&value).jsonFields() - jsObj, err := jsVal.encodeJSON() - if err != nil { - return nil, err - } - - builder.Add(jsObj) - } - - return builder.Build(), nil -} - -type contentionEvent ContentionEventWithNames - -func (s *contentionEvent) jsonFields() jsonFields { - return jsonFields{ - {"blockingTxnID", (*jsonString)(&s.BlockingTransactionID)}, - {"durationInMs", (*jsonFloat)(&s.DurationInMs)}, - {"schemaName", (*jsonString)(&s.SchemaName)}, - {"databaseName", (*jsonString)(&s.DatabaseName)}, - {"tableName", (*jsonString)(&s.TableName)}, - {"indexName", (*jsonString)(&s.IndexName)}, - } -} - type int64Array []int64 func (a *int64Array) decodeJSON(js json.JSON) error { diff --git a/pkg/sql/sqlstats/ssmemstorage/BUILD.bazel b/pkg/sql/sqlstats/ssmemstorage/BUILD.bazel index ea7716c6aecc..7df2696d8dd0 100644 --- a/pkg/sql/sqlstats/ssmemstorage/BUILD.bazel +++ b/pkg/sql/sqlstats/ssmemstorage/BUILD.bazel @@ -12,7 +12,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/ssmemstorage", visibility = ["//visibility:public"], deps = [ - "//pkg/roachpb", "//pkg/server/serverpb", "//pkg/settings", "//pkg/settings/cluster", diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go index 2117c5c81797..4888d01475f0 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go @@ -15,7 +15,6 @@ import ( "time" "unsafe" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/execstats" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -194,11 +193,9 @@ func (s *Container) RecordStatement( } var contention *time.Duration - var contentionEvents []roachpb.ContentionEvent var cpuSQLNanos int64 if value.ExecStats != nil { contention = &value.ExecStats.ContentionTime - contentionEvents = value.ExecStats.ContentionEvents cpuSQLNanos = value.ExecStats.CPUTime.Nanoseconds() } @@ -218,7 +215,6 @@ func (s *Container) RecordStatement( RowsWritten: value.RowsWritten, Nodes: value.Nodes, Contention: contention, - ContentionEvents: contentionEvents, IndexRecommendations: value.IndexRecommendations, Database: value.Database, CPUSQLNanos: cpuSQLNanos, diff --git a/pkg/ui/workspaces/cluster-ui/src/api/contentionApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/contentionApi.ts new file mode 100644 index 000000000000..13665eca94e7 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/api/contentionApi.ts @@ -0,0 +1,173 @@ +// 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 { + executeInternalSql, + LARGE_RESULT_SIZE, + LONG_TIMEOUT, + sqlApiErrorMessage, + SqlExecutionRequest, + sqlResultsAreEmpty, +} from "./sqlApi"; +import { ContentionDetails } from "src/insights"; +import moment from "moment"; + +export type ContentionFilters = { + waitingTxnID?: string; + waitingStmtID?: string; + start?: moment.Moment; + end?: moment.Moment; +}; + +export type ContentionResponseColumns = { + waiting_txn_id: string; + waiting_txn_fingerprint_id: string; + collection_ts: string; + contention_duration: string; + blocking_txn_id: string; + blocking_txn_fingerprint_id: string; + waiting_stmt_id: string; + waiting_stmt_fingerprint_id: string; + schema_name: string; + database_name: string; + table_name: string; + index_name: string; + key: string; +}; + +export async function getContentionDetailsApi( + filters?: ContentionFilters, +): Promise { + const request: SqlExecutionRequest = { + statements: [ + { + sql: contentionDetailsQuery(filters), + }, + ], + execute: true, + max_result_size: LARGE_RESULT_SIZE, + timeout: LONG_TIMEOUT, + }; + + const result = await executeInternalSql(request); + if (result.error) { + throw new Error( + `Error while retrieving insights information: ${sqlApiErrorMessage( + result.error.message, + )}`, + ); + } + + if (sqlResultsAreEmpty(result)) { + return []; + } + + const contentionDetails: ContentionDetails[] = []; + result.execution.txn_results.forEach(x => { + x.rows.forEach(row => { + contentionDetails.push({ + blockingExecutionID: row.blocking_txn_id, + blockingTxnFingerprintID: row.blocking_txn_fingerprint_id, + blockingTxnQuery: null, + waitingTxnID: row.waiting_txn_id, + waitingTxnFingerprintID: row.waiting_txn_fingerprint_id, + waitingStmtID: row.waiting_stmt_id, + waitingStmtFingerprintID: row.waiting_stmt_fingerprint_id, + collectionTimeStamp: moment(row.collection_ts).utc(), + contendedKey: row.key, + contentionTimeMs: moment + .duration(row.contention_duration) + .asMilliseconds(), + databaseName: row.database_name, + schemaName: row.schema_name, + tableName: row.table_name, + indexName: + row.index_name && row.index_name !== "" + ? row.index_name + : "index not found", + }); + }); + }); + + return contentionDetails; +} + +function isFiltered(filters: ContentionFilters): boolean { + if (filters == null) { + return false; + } + + return ( + filters.waitingStmtID != null || + filters.waitingTxnID != null || + filters.end != null || + filters.start != null + ); +} + +function getContentionWhereClause(filters?: ContentionFilters): string { + if (!isFiltered(filters)) { + return ""; + } + const defaultWhereClause = " where "; + let whereClause = defaultWhereClause; + if (filters?.waitingStmtID) { + whereClause = + whereClause + ` waiting_stmt_id = '${filters.waitingStmtID}' `; + } + + if (filters?.waitingTxnID) { + if (whereClause != defaultWhereClause) { + whereClause += " and "; + } + whereClause = whereClause + ` waiting_txn_id >= '${filters.waitingTxnID}' `; + } + + if (filters?.start) { + if (whereClause != defaultWhereClause) { + whereClause += " and "; + } + whereClause = + whereClause + ` collection_ts >= '${filters.start.toISOString()}' `; + } + if (filters?.end) { + if (whereClause != defaultWhereClause) { + whereClause += " and "; + } + + whereClause = + whereClause + + ` (collection_ts + contention_duration) <= '${filters.end.toISOString()}' `; + } + return whereClause; +} + +// txnContentionDetailsQuery selects information about a specific transaction contention event. +function contentionDetailsQuery(filters?: ContentionFilters) { + const whereClause = getContentionWhereClause(filters); + return ` + SELECT DISTINCT collection_ts, + blocking_txn_id, + encode(blocking_txn_fingerprint_id, 'hex') AS blocking_txn_fingerprint_id, + waiting_txn_id, + encode(waiting_txn_fingerprint_id, 'hex') AS waiting_txn_fingerprint_id, + waiting_stmt_id, + encode(waiting_stmt_fingerprint_id, 'hex') AS waiting_stmt_fingerprint_id, + contention_duration, + contending_pretty_key AS key, + database_name, + schema_name, + table_name, + index_name + FROM + crdb_internal.transaction_contention_events AS tce + ${whereClause} + `; +} diff --git a/pkg/ui/workspaces/cluster-ui/src/api/stmtInsightsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/stmtInsightsApi.ts index 93c8f7479d73..f35619149aaf 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/stmtInsightsApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/stmtInsightsApi.ts @@ -18,6 +18,7 @@ import { SqlTxnResult, } from "./sqlApi"; import { + ContentionDetails, getInsightsFromProblemsAndCauses, InsightExecEnum, StmtInsightEvent, @@ -25,6 +26,7 @@ import { 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; @@ -33,15 +35,6 @@ export type StmtInsightsReq = { stmtFingerprintId?: string; }; -type InsightsContentionResponseEvent = { - blockingTxnID: string; - durationInMs: number; - schemaName: string; - databaseName: string; - tableName: string; - indexName: string; -}; - export type StmtInsightsResponseRow = { session_id: string; txn_id: string; @@ -62,7 +55,7 @@ export type StmtInsightsResponseRow = { retries: number; exec_node_ids: number[]; contention: string; // interval - contention_events: InsightsContentionResponseEvent[]; + contention_events: ContentionDetails[]; last_retry_reason?: string; causes: string[]; problem: string; @@ -91,7 +84,6 @@ priority, retries, exec_node_ids, contention, -contention_events, last_retry_reason, causes, problem, @@ -143,7 +135,7 @@ export const stmtInsightsByTxnExecutionQuery = (id: string): string => ` WHERE txn_id = '${id}' `; -export function getStmtInsightsApi( +export async function getStmtInsightsApi( req?: StmtInsightsReq, ): Promise { const request: SqlExecutionRequest = { @@ -156,21 +148,48 @@ export function getStmtInsightsApi( max_result_size: LARGE_RESULT_SIZE, timeout: LONG_TIMEOUT, }; - return executeInternalSql(request).then(result => { - if (result.error) { - throw new Error( - `Error while retrieving insights information: ${sqlApiErrorMessage( - result.error.message, - )}`, - ); - } - if (sqlResultsAreEmpty(result)) { - return []; + const result = await executeInternalSql(request); + if (result.error) { + throw new Error( + `Error while retrieving insights information: ${sqlApiErrorMessage( + result.error.message, + )}`, + ); + } + + if (sqlResultsAreEmpty(result)) { + return []; + } + + const stmtInsightEvent = formatStmtInsights(result.execution?.txn_results[0]); + await addStmtContentionInfoApi(stmtInsightEvent); + return stmtInsightEvent; +} + +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; } - return formatStmtInsights(result.execution?.txn_results[0]); - }); + event.contentionEvents = await getContentionDetailsApi({ + waitingTxnID: null, + waitingStmtID: event.statementExecutionID, + start: null, + end: null, + }); + } } export function formatStmtInsights( @@ -204,7 +223,6 @@ export function formatStmtInsights( isFullScan: row.full_scan, rowsRead: row.rows_read, rowsWritten: row.rows_written, - contentionEvents: row.contention_events, // This is the total stmt contention. contentionTime: row.contention ? moment.duration(row.contention) : null, indexRecommendations: row.index_recommendations, diff --git a/pkg/ui/workspaces/cluster-ui/src/api/txnInsightsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/txnInsightsApi.ts index 7f51fdee1bdd..d74384302fbf 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/txnInsightsApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/txnInsightsApi.ts @@ -19,7 +19,7 @@ import { sqlResultsAreEmpty, } from "./sqlApi"; import { - BlockedContentionDetails, + ContentionDetails, getInsightsFromProblemsAndCauses, InsightExecEnum, InsightNameEnum, @@ -35,6 +35,7 @@ import { 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; @@ -128,69 +129,10 @@ function createStmtFingerprintToQueryMap( return idToQuery; } -// txnContentionDetailsQuery selects information about a specific transaction contention event. -function txnContentionDetailsQuery(filters: TxnContentionDetailsRequest) { - let whereClause = ` WHERE waiting_txn_id = '${filters.txnExecutionID}'`; - if (filters?.start) { - whereClause = - whereClause + ` AND collection_ts >= '${filters.start.toISOString()}'`; - } - if (filters?.end) { - whereClause = - whereClause + - ` AND (collection_ts + contention_duration) <= '${filters.end.toISOString()}'`; - } - return ` -SELECT DISTINCT - collection_ts, - blocking_txn_id, - encode( blocking_txn_fingerprint_id, 'hex' ) AS blocking_txn_fingerprint_id, - waiting_txn_id, - encode( waiting_txn_fingerprint_id, 'hex' ) AS waiting_txn_fingerprint_id, - contention_duration, - crdb_internal.pretty_key(contending_key, 0) AS key, - database_name, - schema_name, - table_name, - index_name -FROM - crdb_internal.transaction_contention_events AS tce - JOIN [SELECT database_name, - schema_name, - name AS table_name, - table_id - FROM - "".crdb_internal.tables] AS tables ON tce.contending_key BETWEEN crdb_internal.table_span(tables.table_id)[1] - AND crdb_internal.table_span(tables.table_id)[2] - LEFT OUTER JOIN [SELECT index_name, - descriptor_id, - index_id - FROM - "".crdb_internal.table_indexes] AS indexes ON tce.contending_key BETWEEN crdb_internal.index_span( - indexes.descriptor_id, - indexes.index_id - )[1] - AND crdb_internal.index_span( - indexes.descriptor_id, - indexes.index_id - )[2] - ${whereClause} -`; -} - -type TxnContentionDetailsResponseColumns = { - waiting_txn_id: string; - waiting_txn_fingerprint_id: string; - collection_ts: string; - contention_duration: string; - blocking_txn_id: string; - blocking_txn_fingerprint_id: string; - schema_name: string; - database_name: string; - table_name: string; - index_name: string; - key: string; -}; +export type TransactionContentionEventDetails = Omit< + TxnContentionInsightDetails, + "application" | "queries" | "blockingQueries" +>; type PartialTxnContentionDetails = Omit< TxnContentionInsightDetails, @@ -198,48 +140,20 @@ type PartialTxnContentionDetails = Omit< >; function formatTxnContentionDetailsResponse( - response: SqlExecutionResponse, + response: ContentionDetails[], ): PartialTxnContentionDetails { - const resultsRows = response.execution.txn_results[0].rows; - if (!resultsRows) { + if (!response || response.length === 9) { // No data. return; } - const blockingContentionDetails = new Array( - resultsRows.length, - ); - - resultsRows.forEach((value, idx) => { - const contentionTimeInMs = moment - .duration(value.contention_duration) - .asMilliseconds(); - blockingContentionDetails[idx] = { - blockingExecutionID: value.blocking_txn_id, - blockingTxnFingerprintID: FixFingerprintHexValue( - value.blocking_txn_fingerprint_id, - ), - blockingQueries: null, - collectionTimeStamp: moment(value.collection_ts).utc(), - contentionTimeMs: contentionTimeInMs, - contendedKey: value.key, - schemaName: value.schema_name, - databaseName: value.database_name, - tableName: value.table_name, - indexName: - value.index_name && value.index_name !== "" - ? value.index_name - : "index not found", - }; - }); - - const row = resultsRows[0]; + const row = response[0]; return { - transactionExecutionID: row.waiting_txn_id, + transactionExecutionID: row.waitingTxnID, transactionFingerprintID: FixFingerprintHexValue( - row.waiting_txn_fingerprint_id, + row.waitingTxnFingerprintID, ), - blockingContentionDetails: blockingContentionDetails, + blockingContentionDetails: response, insightName: InsightNameEnum.highContention, execType: InsightExecEnum.TRANSACTION, }; @@ -263,24 +177,15 @@ export async function getTxnInsightsContentionDetailsApi( // 3. Get the query strings for ALL statements involved in the transaction. // Get contention results for requested transaction. - const contentionResults = - await executeInternalSql( - makeInsightsSqlRequest([ - txnContentionDetailsQuery({ - txnExecutionID: req.txnExecutionID, - start: req.start, - end: req.end, - }), - ]), - ); - if (contentionResults.error) { - throw new Error( - `Error while retrieving contention information: ${sqlApiErrorMessage( - contentionResults.error.message, - )}`, - ); - } - if (sqlResultsAreEmpty(contentionResults)) { + + const contentionResults = await getContentionDetailsApi({ + waitingTxnID: req.txnExecutionID, + waitingStmtID: null, + start: null, + end: null, + }); + + if (contentionResults.length === 0) { return; } const contentionDetails = @@ -358,7 +263,7 @@ function buildTxnContentionInsightDetails( return; } - blockedRow.blockingQueries = currBlockedFingerprintStmts.queryIDs.map( + blockedRow.blockingTxnQuery = currBlockedFingerprintStmts.queryIDs.map( id => stmtFingerprintToQuery.get(id) ?? "", ); }); diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/types.ts b/pkg/ui/workspaces/cluster-ui/src/insights/types.ts index e7a069dc2022..41e5bd5f99ef 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/types.ts +++ b/pkg/ui/workspaces/cluster-ui/src/insights/types.ts @@ -53,11 +53,15 @@ export type TxnInsightEvent = InsightEventBase & { }; // Information about the blocking transaction and schema. -export type BlockedContentionDetails = { +export type ContentionDetails = { collectionTimeStamp: Moment; blockingExecutionID: string; blockingTxnFingerprintID: string; - blockingQueries: string[]; + blockingTxnQuery: string[]; + waitingTxnID: string; + waitingTxnFingerprintID: string; + waitingStmtID: string; + waitingStmtFingerprintID: string; contendedKey: string; schemaName: string; databaseName: string; @@ -70,7 +74,7 @@ export type TxnContentionInsightDetails = { transactionExecutionID: string; application: string; transactionFingerprintID: string; - blockingContentionDetails: BlockedContentionDetails[]; + blockingContentionDetails: ContentionDetails[]; execType: InsightExecEnum; insightName: string; }; @@ -80,26 +84,17 @@ export type TxnInsightDetails = { // This data is segmented into 3 parts so that we can // selective fetch missing info on the details page. txnDetails?: TxnInsightEvent; - blockingContentionDetails?: BlockedContentionDetails[]; + blockingContentionDetails?: ContentionDetails[]; statements?: StmtInsightEvent[]; execType?: InsightExecEnum; }; -export type BlockedStatementContentionDetails = { - blockingTxnID: string; - durationInMs: number; - schemaName: string; - databaseName: string; - tableName: string; - indexName: string; -}; - // Shown on the stmt insights overview page. export type StmtInsightEvent = InsightEventBase & { statementExecutionID: string; statementFingerprintID: string; isFullScan: boolean; - contentionEvents?: BlockedStatementContentionDetails[]; + contentionEvents?: ContentionDetails[]; indexRecommendations: string[]; planGist: string; databaseName: string; @@ -116,6 +111,8 @@ export type Insight = { export type ContentionEvent = { executionID: string; fingerprintID: string; + waitingStmtID: string; + waitingStmtFingerprintID: string; queries: string[]; startTime: Moment; contentionTimeMs: number; @@ -124,6 +121,7 @@ export type ContentionEvent = { tableName: string; indexName: string; execType: InsightExecEnum; + stmtInsightEvent: StmtInsightEvent; }; export const highContentionInsight = ( diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/utils.spec.ts b/pkg/ui/workspaces/cluster-ui/src/insights/utils.spec.ts index 0a796f7e2213..bb8c55a19eaf 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/utils.spec.ts +++ b/pkg/ui/workspaces/cluster-ui/src/insights/utils.spec.ts @@ -10,35 +10,39 @@ import moment from "moment"; import { - filterTransactionInsights, - getAppsFromTransactionInsights, filterStatementInsights, + filterTransactionInsights, getAppsFromStatementInsights, + getAppsFromTransactionInsights, getInsightsFromProblemsAndCauses, mergeTxnInsightDetails, } from "./utils"; import { - TxnInsightEvent, - InsightNameEnum, + ContentionDetails, failedExecutionInsight, - StmtInsightEvent, - InsightExecEnum, highContentionInsight, - slowExecutionInsight, + highRetryCountInsight, + InsightExecEnum, + InsightNameEnum, planRegressionInsight, + slowExecutionInsight, + StmtInsightEvent, suboptimalPlanInsight, - highRetryCountInsight, - BlockedContentionDetails, TxnInsightDetails, + TxnInsightEvent, } from "./types"; const INTERNAL_APP_PREFIX = "$ internal"; -const blockedContentionMock: BlockedContentionDetails = { +const blockedContentionMock: ContentionDetails = { collectionTimeStamp: moment(), blockingExecutionID: "execution", blockingTxnFingerprintID: "block", - blockingQueries: ["select 1"], + blockingTxnQuery: ["select 1"], + waitingStmtFingerprintID: "waitingStmtFingerprintID", + waitingStmtID: "waitingStmtID", + waitingTxnFingerprintID: "waitingTxnFingerprintID", + waitingTxnID: "waitingTxnID", contendedKey: "key", schemaName: "schema", databaseName: "db", 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 f00c59c18623..f06e4269728c 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/insightDetailsTables.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/insightDetailsTables.tsx @@ -11,14 +11,11 @@ import React, { useState } from "react"; import { ColumnDescriptor, SortedTable, SortSetting } from "src/sortedtable"; import { DATE_WITH_SECONDS_AND_MILLISECONDS_FORMAT, Duration } from "src/util"; -import { - BlockedStatementContentionDetails, - ContentionEvent, - InsightExecEnum, -} from "../types"; +import { ContentionDetails, ContentionEvent, InsightExecEnum } from "../types"; import { insightsTableTitles, QueriesCell, + StatementDetailsLink, TransactionDetailsLink, } from "../workloadInsights/util"; import { TimeScale } from "../../timeScaleDropdown"; @@ -51,6 +48,23 @@ export function makeInsightDetailsColumns( ), sort: (item: ContentionEvent) => item.fingerprintID, }, + { + name: "waitingStmtId", + title: insightsTableTitles.waitingID(InsightExecEnum.STATEMENT), + cell: (item: ContentionEvent) => String(item.waitingStmtID), + sort: (item: ContentionEvent) => item.waitingStmtID, + }, + { + name: "waitingStmtFingerprintID", + title: insightsTableTitles.waitingFingerprintID( + InsightExecEnum.STATEMENT, + ), + cell: (item: ContentionEvent) => + item.stmtInsightEvent + ? StatementDetailsLink(item.stmtInsightEvent, setTimeScale) + : item.waitingStmtFingerprintID, + sort: (item: ContentionEvent) => item.waitingStmtFingerprintID, + }, { name: "query", title: insightsTableTitles.query(execType), @@ -116,59 +130,72 @@ export const WaitTimeDetailsTable: React.FC< ); }; -export function makeInsightStatementContentionColumns(): ColumnDescriptor[] { +export function makeInsightStatementContentionColumns( + setTimeScale: (ts: TimeScale) => void, +): ColumnDescriptor[] { const execType = InsightExecEnum.STATEMENT; return [ { name: "executionID", title: insightsTableTitles.executionID(InsightExecEnum.TRANSACTION), - cell: (item: BlockedStatementContentionDetails) => item.blockingTxnID, - sort: (item: BlockedStatementContentionDetails) => item.blockingTxnID, + cell: (item: ContentionDetails) => item.blockingExecutionID, + sort: (item: ContentionDetails) => item.blockingExecutionID, }, { - name: "duration", - title: insightsTableTitles.contention(execType), - cell: (item: BlockedStatementContentionDetails) => - Duration(item.durationInMs * 1e6), - sort: (item: BlockedStatementContentionDetails) => item.durationInMs, + name: "fingerprintId", + title: insightsTableTitles.fingerprintID(InsightExecEnum.TRANSACTION), + cell: (item: ContentionDetails) => + TransactionDetailsLink( + item.blockingTxnFingerprintID, + item.collectionTimeStamp, + setTimeScale, + ), + sort: (item: ContentionDetails) => item.blockingTxnFingerprintID, }, { - name: "schemaName", - title: insightsTableTitles.schemaName(execType), - cell: (item: BlockedStatementContentionDetails) => item.schemaName, - sort: (item: BlockedStatementContentionDetails) => item.schemaName, + name: "duration", + title: insightsTableTitles.contention(execType), + cell: (item: ContentionDetails) => Duration(item.contentionTimeMs * 1e6), + sort: (item: ContentionDetails) => item.contentionTimeMs, }, { name: "databaseName", title: insightsTableTitles.databaseName(execType), - cell: (item: BlockedStatementContentionDetails) => item.databaseName, - sort: (item: BlockedStatementContentionDetails) => item.databaseName, + cell: (item: ContentionDetails) => item.databaseName, + sort: (item: ContentionDetails) => item.databaseName, + }, + { + name: "schemaName", + title: insightsTableTitles.schemaName(execType), + cell: (item: ContentionDetails) => item.schemaName, + sort: (item: ContentionDetails) => item.schemaName, }, { name: "tableName", title: insightsTableTitles.tableName(execType), - cell: (item: BlockedStatementContentionDetails) => item.tableName, - sort: (item: BlockedStatementContentionDetails) => item.tableName, + cell: (item: ContentionDetails) => item.tableName, + sort: (item: ContentionDetails) => item.tableName, }, { name: "indexName", title: insightsTableTitles.indexName(execType), - cell: (item: BlockedStatementContentionDetails) => item.indexName, - sort: (item: BlockedStatementContentionDetails) => item.indexName, + cell: (item: ContentionDetails) => item.indexName, + sort: (item: ContentionDetails) => item.indexName, }, ]; } interface InsightContentionTableProps { - data: BlockedStatementContentionDetails[]; + data: ContentionDetails[]; sortSetting?: SortSetting; onChangeSortSetting?: (ss: SortSetting) => void; + setTimeScale: (ts: TimeScale) => void; } export const ContentionStatementDetailsTable: React.FC< InsightContentionTableProps > = props => { - const columns = makeInsightStatementContentionColumns(); + const columns = makeInsightStatementContentionColumns(props.setTimeScale); return ( @@ -85,6 +85,7 @@ export const StatementInsightDetailsOverviewTab: React.FC< data={insightDetails.contentionEvents} sortSetting={insightsDetailsContentionSortSetting} onChangeSortSetting={setDetailsContentionSortSetting} + setTimeScale={setTimeScale} /> diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsOverviewTab.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsOverviewTab.tsx index 02ad4db39d92..00db10d01a77 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsOverviewTab.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsightDetails/transactionInsightDetailsOverviewTab.tsx @@ -28,12 +28,12 @@ import { } from "src/insightsTable/insightsTable"; import { WaitTimeDetailsTable } from "./insightDetailsTables"; import { - BlockedContentionDetails, + ContentionDetails, ContentionEvent, - TxnInsightEvent, InsightExecEnum, - StmtInsightEvent, InsightNameEnum, + StmtInsightEvent, + TxnInsightEvent, } from "../types"; import classNames from "classnames/bind"; @@ -55,7 +55,7 @@ const tableCx = classNames.bind(insightTableStyles); type Props = { txnDetails: TxnInsightEvent | null; statements: StmtInsightEvent[] | null; - contentionDetails?: BlockedContentionDetails[]; + contentionDetails?: ContentionDetails[]; setTimeScale: (ts: TimeScale) => void; hasAdminRole: boolean; errors: TxnInsightDetailsReqErrs | null; @@ -87,20 +87,28 @@ export const TransactionInsightDetailsOverviewTab: React.FC = ({ true, ); - const blockingExecutions: ContentionEvent[] = contentionDetails?.map(x => { - return { - executionID: x.blockingExecutionID, - fingerprintID: x.blockingTxnFingerprintID, - queries: x.blockingQueries, - startTime: x.collectionTimeStamp, - contentionTimeMs: x.contentionTimeMs, - execType: InsightExecEnum.TRANSACTION, - schemaName: x.schemaName, - databaseName: x.databaseName, - tableName: x.tableName, - indexName: x.indexName, - }; - }); + 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 = diff --git a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/util/insightsColumns.tsx b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/util/insightsColumns.tsx index 1a57c3ed9b97..d6bf4f7f3625 100644 --- a/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/util/insightsColumns.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/insights/workloadInsights/util/insightsColumns.tsx @@ -18,6 +18,8 @@ import { contentionTime, readFromDisk, writtenToDisk } from "../../../util"; export const insightsColumnLabels = { executionID: "Execution ID", latestExecutionID: "Latest Execution ID", + waitingID: "Waiting Execution ID", + waitingFingerprintID: "Waiting Fingerprint ID", query: "Execution", insights: "Insights", startTime: "Start Time (UTC)", @@ -90,6 +92,20 @@ export const insightsTableTitles: InsightsTableTitleType = { execType, ); }, + waitingFingerprintID: (execType: InsightExecEnum) => { + return makeToolTip( +

The {execType} fingerprint ID.

, + "waitingFingerprintID", + execType, + ); + }, + waitingID: (execType: InsightExecEnum) => { + return makeToolTip( +

The ID of the waiting {execType}.

, + "waitingID", + execType, + ); + }, latestExecutionID: (execType: InsightExecEnum) => { return makeToolTip(