-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
101607: jobs,sql: introduce SHOW JOB...WITH EXECUTION DETAILS r=miretskiy a=adityamaru This change introduces a new option to the SHOW JOB and SHOW JOBS statements. By specifying `WITH EXECUTION DETAILS` users can render additional observability information about a job that is collected during the job's lifetime. Currently, the only piece of information is the URL to the latest DistSQL plan being run by the job. Going forward `EXECUTION DETAILS` is expected to be a relatively expensive operation that will scan select rows of the job_info table as determined by a specific job type. The SHOW JOB query delegates to crdb_internal.job_execution_details that renders a JSONB of all the execution details collected over the lifetime of the job. Release note: None Informs: #101743 Epic: CRDB-8964 Co-authored-by: adityamaru <[email protected]>
- Loading branch information
Showing
24 changed files
with
433 additions
and
37 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,11 +1,14 @@ | ||
show_jobs_stmt ::= | ||
'SHOW' 'AUTOMATIC' 'JOBS' | ||
| 'SHOW' 'JOBS' | ||
| 'SHOW' 'JOBS' 'WITH' show_job_options_list | ||
| 'SHOW' 'CHANGEFEED' 'JOBS' | ||
| 'SHOW' 'JOBS' select_stmt | ||
| 'SHOW' 'JOBS' select_stmt 'WITH' show_job_options_list | ||
| 'SHOW' 'JOBS' 'WHEN' 'COMPLETE' select_stmt | ||
| 'SHOW' 'JOBS' for_schedules_clause | ||
| 'SHOW' 'CHANGEFEED' 'JOBS' select_stmt | ||
| 'SHOW' 'JOB' job_id | ||
| 'SHOW' 'JOB' job_id 'WITH' show_job_options_list | ||
| 'SHOW' 'CHANGEFEED' 'JOB' job_id | ||
| 'SHOW' 'JOB' 'WHEN' 'COMPLETE' job_id |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,11 @@ | ||
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") | ||
load("@io_bazel_rules_go//go:def.bzl", "go_library") | ||
|
||
go_library( | ||
name = "profilerconstants", | ||
srcs = ["constants.go"], | ||
importpath = "github.com/cockroachdb/cockroach/pkg/jobs/jobsprofiler/profilerconstants", | ||
visibility = ["//visibility:public"], | ||
) | ||
|
||
get_x_data(name = "get_x_data") |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,15 @@ | ||
// Copyright 2023 The Cockroach Authors. | ||
// | ||
// Use of this software is governed by the Business Source License | ||
// included in the file licenses/BSL.txt. | ||
// | ||
// As of the Change Date specified in that file, in accordance with | ||
// the Business Source License, use of this software will be governed | ||
// by the Apache License, Version 2.0, included in the file | ||
// licenses/APL.txt. | ||
|
||
package profilerconstants | ||
|
||
// DSPDiagramInfoKeyPrefix is the prefix of the info key used for rows that | ||
// store the DistSQL plan diagram for a job. | ||
const DSPDiagramInfoKeyPrefix = "dsp-diag-url-" |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,75 @@ | ||
// Copyright 2023 The Cockroach Authors. | ||
// | ||
// Use of this software is governed by the Business Source License | ||
// included in the file licenses/BSL.txt. | ||
// | ||
// As of the Change Date specified in that file, in accordance with | ||
// the Business Source License, use of this software will be governed | ||
// by the Apache License, Version 2.0, included in the file | ||
// licenses/APL.txt. | ||
|
||
package sql | ||
|
||
import ( | ||
"context" | ||
gojson "encoding/json" | ||
|
||
"github.com/cockroachdb/cockroach/pkg/jobs" | ||
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb" | ||
"github.com/cockroachdb/cockroach/pkg/jobs/jobsprofiler/profilerconstants" | ||
"github.com/cockroachdb/cockroach/pkg/sql/isql" | ||
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval" | ||
) | ||
|
||
// GenerateExecutionDetailsJSON implements the Profiler interface. | ||
func (p *planner) GenerateExecutionDetailsJSON( | ||
ctx context.Context, evalCtx *eval.Context, jobID jobspb.JobID, | ||
) ([]byte, error) { | ||
execCfg := evalCtx.Planner.ExecutorConfig().(*ExecutorConfig) | ||
j, err := execCfg.JobRegistry.LoadJob(ctx, jobID) | ||
if err != nil { | ||
return nil, err | ||
} | ||
|
||
var executionDetailsJSON []byte | ||
payload := j.Payload() | ||
switch payload.Type() { | ||
// TODO(adityamaru): This allows different job types to implement custom | ||
// execution detail aggregation. | ||
default: | ||
executionDetailsJSON, err = constructDefaultExecutionDetails(ctx, jobID, execCfg.InternalDB) | ||
} | ||
|
||
return executionDetailsJSON, err | ||
} | ||
|
||
// defaultExecutionDetails is a JSON serializable struct that captures the | ||
// execution details that are not specific to a particular job type. | ||
type defaultExecutionDetails struct { | ||
// PlanDiagram is the URL to render the latest DistSQL plan that is being | ||
// executed by the job. | ||
PlanDiagram string `json:"plan_diagram"` | ||
} | ||
|
||
func constructDefaultExecutionDetails( | ||
ctx context.Context, jobID jobspb.JobID, db isql.DB, | ||
) ([]byte, error) { | ||
executionDetails := &defaultExecutionDetails{} | ||
err := db.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { | ||
// Populate the latest DSP diagram URL. | ||
infoStorage := jobs.InfoStorageForJob(txn, jobID) | ||
err := infoStorage.GetLast(ctx, profilerconstants.DSPDiagramInfoKeyPrefix, func(infoKey string, value []byte) error { | ||
executionDetails.PlanDiagram = string(value) | ||
return nil | ||
}) | ||
if err != nil { | ||
return err | ||
} | ||
return nil | ||
}) | ||
if err != nil { | ||
return nil, err | ||
} | ||
j, err := gojson.Marshal(executionDetails) | ||
return j, err | ||
} |
Oops, something went wrong.