Skip to content

Commit

Permalink
jobs: add job metrics per-type to track success, failure, and cancel
Browse files Browse the repository at this point in the history
Fixes: #59711

Previously, there were only over all counters tracking how many
jobs were completed, cancelled, or failed. This was inadequate
because it didn't make it easy to tell in aggregate what job
types they were. To address this, this patch will add counters
for different job types for tracking success, failure, and
cancellation.

Release justification: Low risk change only adding a metric inside
the crdb_internal.feature_usage table
Release note: None
  • Loading branch information
fqazi committed Mar 5, 2021
1 parent c140198 commit 1b39a7f
Show file tree
Hide file tree
Showing 6 changed files with 79 additions and 0 deletions.
1 change: 1 addition & 0 deletions pkg/jobs/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ go_library(
"//pkg/roachpb",
"//pkg/scheduledjobs",
"//pkg/security",
"//pkg/server/telemetry",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql/catalog",
Expand Down
37 changes: 37 additions & 0 deletions pkg/jobs/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/util/metric"
io_prometheus_client "github.com/prometheus/client_model/go"
)
Expand Down Expand Up @@ -150,3 +151,39 @@ func (m *Metrics) init(histogramWindowInterval time.Duration) {
// MakeChangefeedMetricsHook allows for registration of changefeed metrics from
// ccl code.
var MakeChangefeedMetricsHook func(time.Duration) metric.Struct

// JobTelemetryMetrics is a telemetry metrics for individual job types.
type JobTelemetryMetrics struct {
Successful telemetry.Counter
Failed telemetry.Counter
Canceled telemetry.Counter
}

// newJobTelemetryMetrics creates a new JobTelemetryMetrics object
// for a given job type name.
func newJobTelemetryMetrics(jobName string) *JobTelemetryMetrics {
return &JobTelemetryMetrics{
Successful: telemetry.GetCounterOnce(fmt.Sprintf("job.%s.successful", jobName)),
Failed: telemetry.GetCounterOnce(fmt.Sprintf("job.%s.failed", jobName)),
Canceled: telemetry.GetCounterOnce(fmt.Sprintf("job.%s.canceled", jobName)),
}
}

// getJobTelemetryMetricsArray initializes an array of job related telemetry
// metrics
func getJobTelemetryMetricsArray() [jobspb.NumJobTypes]*JobTelemetryMetrics {
var metrics [jobspb.NumJobTypes]*JobTelemetryMetrics
for i := 0; i < jobspb.NumJobTypes; i++ {
jt := jobspb.Type(i)
if jt == jobspb.TypeUnspecified { // do not track TypeUnspecified
continue
}
typeStr := strings.ToLower(strings.Replace(jt.String(), " ", "_", -1))
metrics[i] = newJobTelemetryMetrics(typeStr)
}
return metrics
}

// TelemetryMetrics contains telemetry metrics for different
// job types.
var TelemetryMetrics = getJobTelemetryMetricsArray()
4 changes: 4 additions & 0 deletions pkg/jobs/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand Down Expand Up @@ -1218,6 +1219,7 @@ func (r *Registry) stepThroughStateMachine(
// restarted during the next adopt loop and reverting will be retried.
return errors.Wrapf(err, "job %d: could not mark as canceled: %v", job.ID(), jobErr)
}
telemetry.Inc(TelemetryMetrics[jobType].Canceled)
return errors.WithSecondaryError(errors.Errorf("job %s", status), jobErr)
case StatusSucceeded:
if jobErr != nil {
Expand All @@ -1232,6 +1234,7 @@ func (r *Registry) stepThroughStateMachine(
// better.
return r.stepThroughStateMachine(ctx, execCtx, resumer, job, StatusReverting, errors.Wrapf(err, "could not mark job %d as succeeded", job.ID()))
}
telemetry.Inc(TelemetryMetrics[jobType].Successful)
return nil
case StatusReverting:
if err := job.reverted(ctx, nil /* txn */, jobErr, nil /* fn */); err != nil {
Expand Down Expand Up @@ -1286,6 +1289,7 @@ func (r *Registry) stepThroughStateMachine(
// restarted during the next adopt loop and reverting will be retried.
return errors.Wrapf(err, "job %d: could not mark as failed: %s", job.ID(), jobErr)
}
telemetry.Inc(TelemetryMetrics[jobType].Failed)
return jobErr
default:
return errors.NewAssertionErrorWithWrappedErrf(jobErr,
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/alter_table
Original file line number Diff line number Diff line change
Expand Up @@ -1688,3 +1688,14 @@ SELECT count(descriptor_id)
WHERE descriptor_id = ('test.public.t45985'::REGCLASS)::INT8;
----
0

# Validate that the schema_change_successful metric
query T
SELECT feature_name FROM crdb_internal.feature_usage
WHERE feature_name IN ('job.schema_change.successful',
'job.schema_change.failed') AND
usage_count > 0
ORDER BY feature_name DESC
----
job.schema_change.successful
job.schema_change.failed
15 changes: 15 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/distsql_stats
Original file line number Diff line number Diff line change
Expand Up @@ -1065,3 +1065,18 @@ SHOW STATISTICS USING JSON FOR TABLE greeting_stats

statement ok
ALTER TABLE greeting_stats INJECT STATISTICS '$stats'

# Validate that the schema_change_successful metric
query T
SELECT feature_name FROM crdb_internal.feature_usage
WHERE feature_name in ('job.typedesc_schema_change.successful',
'job.schema_change.successful',
'job.create_stats.successful',
'job.auto_create_stats.successful') AND
usage_count > 0
ORDER BY feature_name DESC
----
job.typedesc_schema_change.successful
job.schema_change.successful
job.create_stats.successful
job.auto_create_stats.successful
11 changes: 11 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/jobs
Original file line number Diff line number Diff line change
Expand Up @@ -130,3 +130,14 @@ user testuser
# testuser should no longer have the ability to control jobs.
statement error pq: user testuser does not have CONTROLJOB privilege
PAUSE JOB (SELECT job_id FROM [SHOW JOBS] WHERE user_name = 'testuser2' AND job_type = 'SCHEMA CHANGE GC')

user root

# Validate that the schema_change_successful metric
query T
SELECT feature_name FROM crdb_internal.feature_usage
WHERE feature_name in ('job.schema_change.successful') AND
usage_count > 0
ORDER BY feature_name DESC
----
job.schema_change.successful

0 comments on commit 1b39a7f

Please sign in to comment.