Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
97138: ui: add error code to stmt and txn insights details pages r=gtr a=gtr

Part of: #87785.

Previously, the stmt and txn insights details pages did not show any
further information for failed executions. This commit adds an "error
code" column to the insights table for a failed execution in the stmt
and txn insights details pages. Additionally, a "status" column was
added to the stmt and txn workload insights tables which is either
"Completed" or "Failed".

Future work involves adding the error message string in addition to the
error code but it needs to be redacted first. Additionally, the txn
status is missing the implementation of a "Cancelled" status.

Note to reviewers: only consider the second commit, as the first is 
required to get the txn status.

- Loom [demo](https://www.loom.com/share/e82b97ff9f034d82b98640170eb54408).

Release note (ui change): Adds error code column to the insights table
for a failed execution in the stmt and txn insights details page. Adds
status column to the stmt and txn workload insights tables.

98410: cluster-ui: tenants use sqlstats-supplied regions r=matthewtodd a=matthewtodd

Fixes #98056.

As of #95449, the SQL Activity pages in the DB Console can draw regions information directly from the sqlstats tables, rather than having to translate node IDs to regions on page load.
    
Here, we make that switch, but for non-system tenants only, because:
    
1. The ephemeral nature of serverless nodes made this view-time mapping especially problematic in that context. (See further notes in #95449.)
    
2. The system-tenant views also include KV node IDs in a special Regions/Nodes column, which we are unable to recreate given the backend storage structure. (Future design work might suggest removing these node IDs altogether, for a unified UI.)

# Screenshots!
## Statements, with and without regions filter
<img width="1372" alt="statements" src="https://user-images.githubusercontent.com/5261/225033247-739df90a-9173-4aab-a666-a61a1ceeb579.png">
<img width="1372" alt="statements - filtered" src="https://user-images.githubusercontent.com/5261/225033271-1c0d0f82-3dd4-48ea-bdef-11f19af97a85.png">

## Statement details
<img width="1372" alt="statement details" src="https://user-images.githubusercontent.com/5261/225033338-6dff4a6e-a4a3-48c6-863a-84f1375b0a61.png">

## Transactions, with and without regions filter
<img width="1372" alt="transactions" src="https://user-images.githubusercontent.com/5261/225033366-65f44e95-3549-47cc-b0f2-67ad48a1a1fa.png">
<img width="1372" alt="transactions - filtered" src="https://user-images.githubusercontent.com/5261/225033391-50b9a2dc-e9a1-457b-84b1-837426eba35e.png">

## Transaction details
<img width="1372" alt="transaction details" src="https://user-images.githubusercontent.com/5261/225033505-3fdeceef-35dc-4e06-af25-ab4d0c53518f.png">

Release note: None

98464: jobs,upgrades: add migration to backfill job_info table r=dt a=adityamaru

This change adds a migration and corresponding cluster version
after which every job entry in the system.jobs table will have its
Payload and Progress written to two rows in the system.job_info table.

Informs: #97762

Release note: None

98510: backupccl: update restore/nodeshutdown tests to use new roachtest framework r=adityamaru a=msbutler

The restore/nodeshutdown tests have been using a very old workload that will not be restorable when #93804 lands. This patch changes the restore/nodeshutdown workload to a 80GB tpce restore and moves the tests to run on aws instead of gcp.

Release note: None

Epic: None

98579: upgrade/upgrades: skip TestUpgradeSchemaChangerElements r=smg260 a=smg260

Refs: #98062

Reason: flaky test

Generated by bin/skip-test.

Release justification: non-production code changes

Release note: None
Epic: None

Co-authored-by: gtr <gerardo@cockroachlabs.com>
Co-authored-by: Matthew Todd <todd@cockroachlabs.com>
Co-authored-by: adityamaru <adityamaru@gmail.com>
Co-authored-by: Michael Butler <butler@cockroachlabs.com>
Co-authored-by: Miral Gadani <miral@cockroachlabs.com>
  • Loading branch information
6 people committed Mar 14, 2023
6 parents 094df2b + 45a1b58 + 7ea327a + 08f0288 + 903bd5e + b03f9f1 commit 421d136
Show file tree
Hide file tree
Showing 35 changed files with 438 additions and 239 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -299,4 +299,4 @@ trace.opentelemetry.collector string address of an OpenTelemetry trace collecto
trace.snapshot.rate duration 0s if non-zero, interval at which background trace snapshots are captured
trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https://<ui>/#/debug/tracez
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.
version version 1000022.2-78 set the active cluster version in the format '<major>.<minor>'
version version 1000022.2-80 set the active cluster version in the format '<major>.<minor>'
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -240,6 +240,6 @@
<tr><td><div id="setting-trace-snapshot-rate" class="anchored"><code>trace.snapshot.rate</code></div></td><td>duration</td><td><code>0s</code></td><td>if non-zero, interval at which background trace snapshots are captured</td></tr>
<tr><td><div id="setting-trace-span-registry-enabled" class="anchored"><code>trace.span_registry.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>if set, ongoing traces can be seen at https://&lt;ui&gt;/#/debug/tracez</td></tr>
<tr><td><div id="setting-trace-zipkin-collector" class="anchored"><code>trace.zipkin.collector</code></div></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as &lt;host&gt;:&lt;port&gt;. If no port is specified, 9411 will be used.</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000022.2-78</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000022.2-80</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td></tr>
</tbody>
</table>
2 changes: 1 addition & 1 deletion pkg/cli/testdata/declarative-rules/deprules
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
dep
----
debug declarative-print-rules 1000022.2-78 dep
debug declarative-print-rules 1000022.2-80 dep
deprules
----
- name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED'
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/declarative-rules/oprules
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
op
----
debug declarative-print-rules 1000022.2-78 op
debug declarative-print-rules 1000022.2-80 op
rules
----
[]
9 changes: 9 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -479,6 +479,11 @@ const (
// since we've made mixed-version clusters tolerate new privileges.
V23_1AllowNewSystemPrivileges

// V23_1JobInfoTableIsBackfilled is a version gate after which the
// system.jobs_info table has been backfilled with rows for the payload and
// progress of each job in the system.jobs table.
V23_1JobInfoTableIsBackfilled

// *************************************************
// Step (1): Add new versions here.
// Do not add new versions to a patch release.
Expand Down Expand Up @@ -828,6 +833,10 @@ var rawVersionsSingleton = keyedVersions{
Key: V23_1AllowNewSystemPrivileges,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 78},
},
{
Key: V23_1JobInfoTableIsBackfilled,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 80},
},

// *************************************************
// Step (2): Add new versions here.
Expand Down
102 changes: 34 additions & 68 deletions pkg/cmd/roachtest/tests/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/httputil"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/workload/histogram"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -235,91 +234,56 @@ func (dul *DiskUsageLogger) Runner(ctx context.Context) error {
}
}
func registerRestoreNodeShutdown(r registry.Registry) {
sp := restoreSpecs{
hardware: makeHardwareSpecs(hardwareSpecs{}),
backup: makeBackupSpecs(
backupSpecs{workload: tpceRestore{customers: 5000},
version: "v22.2.1"}),
timeout: 1 * time.Hour,
}

makeRestoreStarter := func(ctx context.Context, t test.Test, c cluster.Cluster, gatewayNode int) jobStarter {
return func(c cluster.Cluster, t test.Test) (string, error) {
t.L().Printf("connecting to gateway")
gatewayDB := c.Conn(ctx, t.L(), gatewayNode)
defer gatewayDB.Close()

t.L().Printf("creating bank database")
if _, err := gatewayDB.Exec("CREATE DATABASE bank"); err != nil {
return "", err
}

errCh := make(chan error, 1)
go func() {
defer close(errCh)

// 10 GiB restore.
restoreQuery := `RESTORE bank.bank FROM
'gs://cockroach-fixtures/workload/bank/version=1.0.0,payload-bytes=100,ranges=10,rows=10000000,seed=1/bank?AUTH=implicit'`

t.L().Printf("starting to run the restore job")
if _, err := gatewayDB.Exec(restoreQuery); err != nil {
errCh <- err
}
t.L().Printf("done running restore job")
}()

// Wait for the job.
retryOpts := retry.Options{
MaxRetries: 50,
InitialBackoff: 1 * time.Second,
MaxBackoff: 5 * time.Second,
}
for r := retry.StartWithCtx(ctx, retryOpts); r.Next(); {
var jobCount int
if err := gatewayDB.QueryRowContext(ctx, "SELECT count(*) FROM [SHOW JOBS] WHERE job_type = 'RESTORE'").Scan(&jobCount); err != nil {
return "", err
}

select {
case err := <-errCh:
// We got an error when starting the job.
return "", err
default:
}

if jobCount == 0 {
t.L().Printf("waiting for restore job")
} else if jobCount == 1 {
t.L().Printf("found restore job")
break
} else {
t.L().Printf("found multiple restore jobs -- erroring")
return "", errors.New("unexpectedly found multiple restore jobs")
}
}

var jobID string
if err := gatewayDB.QueryRowContext(ctx, "SELECT job_id FROM [SHOW JOBS] WHERE job_type = 'RESTORE'").Scan(&jobID); err != nil {
return "", errors.Wrap(err, "querying the job ID")
}
return jobID, nil
sp.getRuntimeSpecs(ctx, t, c)
jobID, err := sp.runDetached(ctx, "DATABASE tpce", gatewayNode)
return fmt.Sprintf("%d", jobID), err
}
}

r.Add(registry.TestSpec{
Name: "restore/nodeShutdown/worker",
Owner: registry.OwnerDisasterRecovery,
Cluster: r.MakeClusterSpec(4),
Cluster: sp.hardware.makeClusterSpecs(r),
Timeout: sp.timeout,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
gatewayNode := 2
nodeToShutdown := 3

if c.Spec().Cloud != sp.backup.cloud {
// For now, only run the test on the cloud provider that also stores the backup.
t.Skip("test configured to run on %s", sp.backup.cloud)
}
c.Put(ctx, t.Cockroach(), "./cockroach")
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings())

jobSurvivesNodeShutdown(ctx, t, c, nodeToShutdown, makeRestoreStarter(ctx, t, c, gatewayNode))
},
})

r.Add(registry.TestSpec{
Name: "restore/nodeShutdown/coordinator",
Owner: registry.OwnerDisasterRecovery,
Cluster: r.MakeClusterSpec(4),
Cluster: sp.hardware.makeClusterSpecs(r),
Timeout: sp.timeout,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {

gatewayNode := 2
nodeToShutdown := 2

if c.Spec().Cloud != sp.backup.cloud {
// For now, only run the test on the cloud provider that also stores the backup.
t.Skip("test configured to run on %s", sp.backup.cloud)
}

c.Put(ctx, t.Cockroach(), "./cockroach")
c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings())

Expand Down Expand Up @@ -446,7 +410,7 @@ func registerRestore(r registry.Registry) {
defer close(jobIDCh)
t.Status(`running restore`)
metricCollector := withPauseSpecs.initRestorePerfMetrics(ctx, durationGauge)
jobID, err := withPauseSpecs.runDetached(ctx, "DATABASE tpce")
jobID, err := withPauseSpecs.runDetached(ctx, "DATABASE tpce", 1)
require.NoError(t, err)
jobIDCh <- jobID

Expand Down Expand Up @@ -840,14 +804,16 @@ func (sp *restoreSpecs) run(ctx context.Context, target string) error {
return sp.c.RunE(ctx, sp.c.Node(1), sp.restoreCmd(target, ""))
}

func (sp *restoreSpecs) runDetached(ctx context.Context, target string) (jobspb.JobID, error) {
if err := sp.c.RunE(ctx, sp.c.Node(1), sp.restoreCmd(target, "WITH DETACHED")); err != nil {
func (sp *restoreSpecs) runDetached(
ctx context.Context, target string, node int,
) (jobspb.JobID, error) {
if err := sp.c.RunE(ctx, sp.c.Node(node), sp.restoreCmd(target, "WITH DETACHED")); err != nil {
return 0, err
}

db, err := sp.c.ConnE(ctx, sp.t.L(), sp.c.Node(1)[0])
db, err := sp.c.ConnE(ctx, sp.t.L(), sp.c.Node(node)[0])
if err != nil {
return 0, errors.Wrap(err, "failed to connect to node 1; running restore detached")
return 0, errors.Wrapf(err, "failed to connect to node %d; running restore detached", node)
}
var jobID jobspb.JobID
if err := db.QueryRow(`SELECT job_id FROM [SHOW JOBS] WHERE job_type = 'RESTORE'`).Scan(&jobID); err != nil {
Expand Down
16 changes: 14 additions & 2 deletions pkg/jobs/job_info_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ func (i InfoStorage) write(ctx context.Context, infoKey, value []byte) error {

// First clear out any older revisions of this info.
_, err := i.txn.ExecEx(
ctx, "job-info-write", i.txn.KV(),
ctx, "write-job-info-delete", i.txn.KV(),
sessiondata.NodeUserSessionDataOverride,
"DELETE FROM system.job_info WHERE job_id = $1 AND info_key = $2",
j.ID(), infoKey,
Expand All @@ -120,7 +120,7 @@ func (i InfoStorage) write(ctx context.Context, infoKey, value []byte) error {

// Write the new info, using the same transaction.
_, err = i.txn.ExecEx(
ctx, "job-info-write", i.txn.KV(),
ctx, "write-job-info-insert", i.txn.KV(),
sessiondata.NodeUserSessionDataOverride,
`INSERT INTO system.job_info (job_id, info_key, written, value) VALUES ($1, $2, now(), $3)`,
j.ID(), infoKey, value,
Expand Down Expand Up @@ -206,6 +206,18 @@ const (
legacyProgressKey = "legacy_progress"
)

// GetLegacyPayloadKey returns the info_key whose value is the jobspb.Payload of
// the job.
func GetLegacyPayloadKey() []byte {
return []byte(legacyPayloadKey)
}

// GetLegacyProgressKey returns the info_key whose value is the jobspb.Progress
// of the job.
func GetLegacyProgressKey() []byte {
return []byte(legacyProgressKey)
}

// GetLegacyPayload returns the job's Payload from the system.jobs_info table.
func (i InfoStorage) GetLegacyPayload(ctx context.Context) ([]byte, bool, error) {
return i.Get(ctx, []byte(legacyPayloadKey))
Expand Down
9 changes: 8 additions & 1 deletion pkg/ui/workspaces/cluster-ui/src/api/stmtInsightsApi.ts
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import {
ContentionDetails,
getInsightsFromProblemsAndCauses,
InsightExecEnum,
StatementStatus,
StmtInsightEvent,
} from "src/insights";
import moment from "moment";
Expand Down Expand Up @@ -63,6 +64,8 @@ export type StmtInsightsResponseRow = {
index_recommendations: string[];
plan_gist: string;
cpu_sql_nanos: number;
error_code: string;
status: StatementStatus;
};

const stmtColumns = `
Expand Down Expand Up @@ -90,7 +93,9 @@ causes,
problem,
index_recommendations,
plan_gist,
cpu_sql_nanos
cpu_sql_nanos,
error_code,
status
`;

const stmtInsightsOverviewQuery = (filters?: StmtInsightsReq): string => {
Expand Down Expand Up @@ -232,6 +237,8 @@ export function formatStmtInsights(
),
planGist: row.plan_gist,
cpuSQLNanos: row.cpu_sql_nanos,
errorCode: row.error_code,
status: row.status,
} as StmtInsightEvent;
});
}
9 changes: 8 additions & 1 deletion pkg/ui/workspaces/cluster-ui/src/api/txnInsightsApi.ts
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import {
getInsightsFromProblemsAndCauses,
InsightExecEnum,
InsightNameEnum,
TransactionStatus,
TxnContentionInsightDetails,
TxnInsightDetails,
TxnInsightEvent,
Expand Down Expand Up @@ -293,6 +294,8 @@ type TxnInsightsResponseRow = {
causes: string[];
stmt_execution_ids: string[];
cpu_sql_nanos: number;
last_error_code: string;
status: TransactionStatus;
};

type TxnQueryFilters = {
Expand Down Expand Up @@ -326,7 +329,9 @@ last_retry_reason,
problems,
causes,
stmt_execution_ids,
cpu_sql_nanos`;
cpu_sql_nanos,
last_error_code,
status`;

if (filters?.execID) {
return `
Expand Down Expand Up @@ -394,6 +399,8 @@ function formatTxnInsightsRow(row: TxnInsightsResponseRow): TxnInsightEvent {
insights,
stmtExecutionIDs: row.stmt_execution_ids,
cpuSQLNanos: row.cpu_sql_nanos,
errorCode: row.last_error_code,
status: row.status,
};
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@

import React from "react";
import classNames from "classnames/bind";
import { flatMap } from "lodash";
import {
ISortedTablePagination,
SortedTable,
Expand Down Expand Up @@ -462,7 +463,9 @@ export class IndexDetailsPage extends React.Component<
.map(n => Number(n))
.sort();
const regions = unique(
nodes.map(node => nodeRegions[node.toString()]),
isTenant
? flatMap(statements, statement => statement.stats.regions)
: nodes.map(node => nodeRegions[node.toString()]),
).sort();

const filteredStmts = this.filteredStatements();
Expand Down
17 changes: 17 additions & 0 deletions pkg/ui/workspaces/cluster-ui/src/insights/types.ts
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,18 @@ export enum InsightExecEnum {
STATEMENT = "statement",
}

export enum StatementStatus {
COMPLETED = "Completed",
FAILED = "Failed",
}

export enum TransactionStatus {
COMPLETED = "Completed",
FAILED = "Failed",
// Unimplemented, see https://github.com/cockroachdb/cockroach/issues/98219/.
CANCELLED = "Cancelled",
}

// Common fields for both txn and stmt insights.
export type InsightEventBase = {
application: string;
Expand All @@ -46,9 +58,11 @@ export type InsightEventBase = {
transactionExecutionID: string;
transactionFingerprintID: string;
username: string;
errorCode: string;
};

export type TxnInsightEvent = InsightEventBase & {
status: TransactionStatus;
stmtExecutionIDs: string[];
};

Expand Down Expand Up @@ -99,6 +113,7 @@ export type StmtInsightEvent = InsightEventBase & {
planGist: string;
databaseName: string;
execType?: InsightExecEnum;
status: StatementStatus;
};

export type Insight = {
Expand Down Expand Up @@ -327,6 +342,8 @@ export interface ExecutionDetails {
statementExecutionID?: string;
transactionExecutionID?: string;
execType?: InsightExecEnum;
errorCode?: string;
status?: string;
}

export interface insightDetails {
Expand Down
Loading

0 comments on commit 421d136

Please sign in to comment.