Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
61359: tracing: use byte-limits for logs/structured events per span r=irfansharif a=irfansharif

Touches #59188. Follow-on work from #60678. We can introduce byte-limits for
verbose logging and structured events, instead of limiting things based on
count.

This PR also:
- adds a _dropped tag to recordings with dropped logs/structured events.
- squashes a bug where reset spans (as used in SessionTracing) still
  held onto earlier structured events
- moves away from the internal usage of the opentracing.LogRecord type,
  it's unnecessary

Release justification: low risk, high benefit changes to existing
functionality

Release note: None

---

+cc @knz / @erikgrinaker / @angelapwen for pod-visibility.

61482: jobs: add job metrics per-type to track success, failure, and cancel r=fqazi a=fqazi

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

61491: sqlsmith: add support for computed columns r=RaduBerinde a=RaduBerinde

This changes the random table generator to also create computed
columns (either STORED or VIRTUAL). Some example of definitions:
 - `col1_14 STRING NOT NULL AS (lower(CAST(col1_8 AS STRING))) VIRTUAL`
 - `col2_6 DECIMAL NOT NULL AS (col2_2 + 1:::DECIMAL) STORED`
 - `col1_13 INT4 AS (col1_0 + col1_10) STORED`

Release justification: non-production code change.

Release note: None

Informs #57608.

61509: sql: add a regression test r=RaduBerinde a=RaduBerinde

This commit adds a regression test for #58104 (the problem was already
fixed).

Resolves #58104.

Release justification: non-production code change.

Release note: None

61522: opt: fix fetch scope in UPDATE..FROM statements r=mgartner a=mgartner

Previously, the fetch scope incorrectly included columns in the FROM
clause of an UPDATE..FROM statement. As a result, column names shared by
the FROM clause and the mutating table lead to ambiguity when resolving
partial index DEL column expressions. This commit ensures that the fetch
scope does not include columns in the FROM clause.

Fixes #61284

Release justification: This is a low-risk bug fix to existing
functionality.

Release note (bug fix): An UPDATE..FROM statement where the FROM clause
contained column names that match table column names erred if the table
had a partial index predicate referencing those columns. This bug,
present since partial indexes were released in version 20.2.0, has been
fixed.

61553: ccl,server: error.Wrap on previously handled errors r=[dt,miretskiy] a=stevendanna

These errors.Wrap calls are wrapping errors that are nil and thus will
always return a nil error.

Release justification: Minor error handling fixes
Release note: None

Co-authored-by: irfan sharif <[email protected]>
Co-authored-by: Faizan Qazi <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Steven Danna <[email protected]>
  • Loading branch information
6 people committed Mar 5, 2021
7 parents 05a7bec + 2b71afb + 1b39a7f + d9949a3 + b97e82c + 99a11f7 + cda1912 commit 2264789
Show file tree
Hide file tree
Showing 23 changed files with 598 additions and 91 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -1034,7 +1034,7 @@ func backupPlanHook(
return errors.Wrap(err, "invalid previous backups")
}
if coveredTime != startTime {
return errors.Wrapf(err, "expected previous backups to cover until time %v, got %v", startTime, coveredTime)
return errors.Errorf("expected previous backups to cover until time %v, got %v", startTime, coveredTime)
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/read_import_workload.go
Original file line number Diff line number Diff line change
Expand Up @@ -157,7 +157,7 @@ func (w *workloadReader) readFiles(
}
}
if t.Name == `` {
return errors.Wrapf(err, `unknown table %s for generator %s`, conf.Table, meta.Name)
return errors.Errorf(`unknown table %s for generator %s`, conf.Table, meta.Name)
}

wc := NewWorkloadKVConverter(
Expand Down
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
2 changes: 1 addition & 1 deletion pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -233,7 +233,7 @@ func bootstrapCluster(
if i == 0 {
bootstrapVersion = cv
} else if bootstrapVersion != cv {
return nil, errors.Wrapf(err, "found cluster versions %s and %s", bootstrapVersion, cv)
return nil, errors.Errorf("found cluster versions %s and %s", bootstrapVersion, cv)
}

sIdent := roachpb.StoreIdent{
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
14 changes: 14 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/partial_index
Original file line number Diff line number Diff line change
Expand Up @@ -1795,3 +1795,17 @@ CREATE TABLE t61414_c (

statement ok
UPSERT INTO t61414_c (k, a, b, d) VALUES (1, 2, 3, 4)

# Regression test for #61284. When building partial index DEL column
# expressions, there should not be ambiguous column errors if there exists
# columns in an UPDATE FROM clause that match column names in the partial index
# predicate.

statement ok
CREATE TABLE t61284 (
a INT,
INDEX (a) WHERE a > 0
)

statement ok
UPDATE t61284 SET a = v.a FROM (VALUES (1), (2)) AS v(a) WHERE t61284.a = v.a
4 changes: 4 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/select
Original file line number Diff line number Diff line change
Expand Up @@ -746,3 +746,7 @@ SELECT * FROM crdb_internal.jobs
# Cleanup
statement ok
SET disallow_full_table_scans = false

# Regression test for #58104.
statement ok
SELECT * FROM pg_catalog.pg_attrdef WHERE (adnum = 1 AND adrelid = 1) OR (adbin = 'foo' AND adrelid = 2)
10 changes: 10 additions & 0 deletions pkg/sql/mutations/mutations.go
Original file line number Diff line number Diff line change
Expand Up @@ -313,6 +313,8 @@ func statisticsMutator(
return stmts, changed
}

// foreignKeyMutator is a MultiStatementMutation implementation which adds
// foreign key references between existing columns.
func foreignKeyMutator(
rng *rand.Rand, stmts []tree.Statement,
) (mutated []tree.Statement, changed bool) {
Expand Down Expand Up @@ -369,6 +371,10 @@ func foreignKeyMutator(
// Choose a random column subset.
var fkCols []*tree.ColumnTableDef
for _, c := range cols[table.Table] {
if c.Computed.Computed {
// We don't support FK references from computed columns (#46672).
continue
}
if usedCols[table.Table][c.Name] {
continue
}
Expand Down Expand Up @@ -429,6 +435,10 @@ func foreignKeyMutator(
fkCol := fkCols[len(usingCols)]
found := false
for refI, refCol := range availCols {
if refCol.Computed.Virtual {
// We don't support FK references to virtual columns (#51296).
continue
}
fkColType := tree.MustBeStaticallyKnownType(fkCol.Type)
refColType := tree.MustBeStaticallyKnownType(refCol.Type)
if fkColType.Equivalent(refColType) && colinfo.ColumnTypeIsIndexable(refColType) {
Expand Down
14 changes: 10 additions & 4 deletions pkg/sql/opt/optbuilder/mutation_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -287,10 +287,9 @@ func (mb *mutationBuilder) buildInputForUpdate(
noRowLocking,
inScope,
)
mb.outScope = mb.fetchScope

// Set list of columns that will be fetched by the input expression.
mb.setFetchColIDs(mb.outScope.cols)
mb.setFetchColIDs(mb.fetchScope.cols)

// If there is a FROM clause present, we must join all the tables
// together with the table being updated.
Expand All @@ -299,18 +298,25 @@ func (mb *mutationBuilder) buildInputForUpdate(
fromScope := mb.b.buildFromTables(from, noRowLocking, inScope)

// Check that the same table name is not used multiple times.
mb.b.validateJoinTableNames(mb.outScope, fromScope)
mb.b.validateJoinTableNames(mb.fetchScope, fromScope)

// The FROM table columns can be accessed by the RETURNING clause of the
// query and so we have to make them accessible.
mb.extraAccessibleCols = fromScope.cols

// Add the columns in the FROM scope.
// We create a new scope so that fetchScope is not modified. It will be
// used later to build partial index predicate expressions, and we do
// not want ambiguities with column names in the FROM clause.
mb.outScope = mb.fetchScope.replace()
mb.outScope.appendColumnsFromScope(mb.fetchScope)
mb.outScope.appendColumnsFromScope(fromScope)

left := mb.outScope.expr.(memo.RelExpr)
left := mb.fetchScope.expr.(memo.RelExpr)
right := fromScope.expr.(memo.RelExpr)
mb.outScope.expr = mb.b.factory.ConstructInnerJoin(left, right, memo.TrueFilter, memo.EmptyJoinPrivate)
} else {
mb.outScope = mb.fetchScope
}

// WHERE
Expand Down
53 changes: 53 additions & 0 deletions pkg/sql/opt/optbuilder/testdata/partial-indexes
Original file line number Diff line number Diff line change
Expand Up @@ -339,6 +339,59 @@ update partial_indexes
├── c:7 = 'delete-only' [as=partial_index_put3:13]
└── c:7 = 'write-only' [as=partial_index_put4:14]

build
UPDATE partial_indexes SET a = v.a FROM (VALUES (1), (2)) AS v(a) WHERE partial_indexes.a = v.a
----
update partial_indexes
├── columns: <none>
├── fetch columns: a:5 b:6 c:7
├── update-mapping:
│ └── column1:9 => a:1
├── partial index put columns: partial_index_put1:10 partial_index_put2:11 partial_index_put3:13 partial_index_put4:14
├── partial index del columns: partial_index_put1:10 partial_index_del2:12 partial_index_put3:13 partial_index_put4:14
└── project
├── columns: partial_index_put1:10 partial_index_put2:11 partial_index_del2:12 partial_index_put3:13 partial_index_put4:14 a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8 column1:9!null
├── distinct-on
│ ├── columns: a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8 column1:9!null
│ ├── grouping columns: a:5!null
│ ├── select
│ │ ├── columns: a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8 column1:9!null
│ │ ├── inner-join (cross)
│ │ │ ├── columns: a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8 column1:9!null
│ │ │ ├── scan partial_indexes
│ │ │ │ ├── columns: a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8
│ │ │ │ └── partial index predicates
│ │ │ │ ├── secondary: filters
│ │ │ │ │ └── c:7 = 'foo'
│ │ │ │ ├── secondary: filters
│ │ │ │ │ └── (a:5 > b:6) AND (c:7 = 'bar')
│ │ │ │ ├── b: filters
│ │ │ │ │ └── c:7 = 'delete-only'
│ │ │ │ └── b: filters
│ │ │ │ └── c:7 = 'write-only'
│ │ │ ├── values
│ │ │ │ ├── columns: column1:9!null
│ │ │ │ ├── (1,)
│ │ │ │ └── (2,)
│ │ │ └── filters (true)
│ │ └── filters
│ │ └── a:5 = column1:9
│ └── aggregations
│ ├── first-agg [as=b:6]
│ │ └── b:6
│ ├── first-agg [as=c:7]
│ │ └── c:7
│ ├── first-agg [as=crdb_internal_mvcc_timestamp:8]
│ │ └── crdb_internal_mvcc_timestamp:8
│ └── first-agg [as=column1:9]
│ └── column1:9
└── projections
├── c:7 = 'foo' [as=partial_index_put1:10]
├── (column1:9 > b:6) AND (c:7 = 'bar') [as=partial_index_put2:11]
├── (a:5 > b:6) AND (c:7 = 'bar') [as=partial_index_del2:12]
├── c:7 = 'delete-only' [as=partial_index_put3:13]
└── c:7 = 'write-only' [as=partial_index_put4:14]

# Do not error with "column reference is ambiguous" when table column names
# match synthesized column names.
build
Expand Down
Loading

0 comments on commit 2264789

Please sign in to comment.