From 819d0aaee2fa828b10180ddc22e39b8e584342ea Mon Sep 17 00:00:00 2001 From: Jayant Shrivastava Date: Tue, 6 Dec 2022 10:16:46 -0500 Subject: [PATCH 1/2] jobs: introduce job_type column to system.jobs This change introduces an upgrade to add the `job_type` column to `system.jobs` as a nullable string. This upgrade also creates an index for the column and backfills it. With this change, new jobs being inserted into the jobs table by the jobs registry will populate the `job_type` column. To make migrations easier, this change adds a new builtin, `crdb_internal.job_payload_type`, which can be used to determine the type of a jobs payload. Release note: None --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- docs/generated/sql/functions.md | 2 + pkg/clusterversion/cockroach_versions.go | 16 + pkg/jobs/BUILD.bazel | 1 + pkg/jobs/jobspb/wrap.go | 54 ++++ pkg/jobs/registry.go | 46 ++- pkg/sql/catalog/systemschema/system.go | 23 +- .../systemschema_test/testdata/bootstrap | 6 +- .../testdata/logic_test/crdb_internal_catalog | 2 +- .../testdata/logic_test/information_schema | 1 + .../logictest/testdata/logic_test/pg_catalog | 2 + pkg/sql/logictest/testdata/logic_test/system | 24 +- pkg/sql/sem/builtins/builtins.go | 31 ++ pkg/sql/sem/builtins/fixed_oids.go | 1 + pkg/upgrade/upgrades/BUILD.bazel | 2 + .../upgrades/alter_jobs_add_job_type.go | 76 +++++ .../upgrades/alter_jobs_add_job_type_test.go | 280 ++++++++++++++++++ pkg/upgrade/upgrades/upgrades.go | 12 + 19 files changed, 555 insertions(+), 28 deletions(-) create mode 100644 pkg/upgrade/upgrades/alter_jobs_add_job_type.go create mode 100644 pkg/upgrade/upgrades/alter_jobs_add_job_type_test.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 81fce5491a57..754f4b178120 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -297,4 +297,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 1000022.2-16 set the active cluster version in the format '.' +version version 1000022.2-20 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index b54d9968cfc6..d8f4ec70f636 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -233,6 +233,6 @@ trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabledbooleantrueif set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion1000022.2-16set the active cluster version in the format '.' +versionversion1000022.2-20set the active cluster version in the format '.' diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 0f39d4b2e180..ca033c123415 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -1014,6 +1014,8 @@ available replica will error.

Stable array_to_json(array: anyelement[], pretty_bool: bool) → jsonb

Returns the array as JSON or JSONB.

Stable +crdb_internal.job_payload_type(data: bytes) → string

Reads the type from the jobspb.Payload protocol message.

+
Immutable crdb_internal.json_to_pb(pbname: string, json: jsonb) → bytes

Convert JSONB data to protocol message bytes

Immutable crdb_internal.pb_to_json(pbname: string, data: bytes) → jsonb

Converts protocol message to its JSONB representation.

diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index db38e9e92a05..47c7a1ae4fba 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -354,6 +354,14 @@ const ( // supported through `CREATE SCHEDULE FOR CHANGEFEED` statement. V23_1ScheduledChangefeeds + // V23_1AddTypeColumnToJobsTable adds the nullable job_type + // column to the system.jobs table. + V23_1AddTypeColumnToJobsTable + + // V23_1BackfillTypeColumnInJobsTable backfills the job_type + // column in the system.jobs table. + V23_1BackfillTypeColumnInJobsTable + // ************************************************* // Step (1): Add new versions here. // Do not add new versions to a patch release. @@ -612,6 +620,14 @@ var rawVersionsSingleton = keyedVersions{ Key: V23_1ScheduledChangefeeds, Version: roachpb.Version{Major: 22, Minor: 2, Internal: 16}, }, + { + Key: V23_1AddTypeColumnToJobsTable, + Version: roachpb.Version{Major: 22, Minor: 2, Internal: 18}, + }, + { + Key: V23_1BackfillTypeColumnInJobsTable, + Version: roachpb.Version{Major: 22, Minor: 2, Internal: 20}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/jobs/BUILD.bazel b/pkg/jobs/BUILD.bazel index de1954b5b779..98be8e7ae7e2 100644 --- a/pkg/jobs/BUILD.bazel +++ b/pkg/jobs/BUILD.bazel @@ -29,6 +29,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/clusterversion", "//pkg/jobs/jobspb", "//pkg/kv", "//pkg/multitenant", diff --git a/pkg/jobs/jobspb/wrap.go b/pkg/jobs/jobspb/wrap.go index eb1beb599642..16863259e41e 100644 --- a/pkg/jobs/jobspb/wrap.go +++ b/pkg/jobs/jobspb/wrap.go @@ -141,6 +141,41 @@ func DetailsType(d isPayload_Details) Type { } } +// ForEachType executes f for each job Type. +func ForEachType(f func(typ Type), includeTypeUnspecified bool) { + start := TypeBackup + if includeTypeUnspecified { + start = TypeUnspecified + } + for typ := start; typ < NumJobTypes; typ++ { + f(typ) + } +} + +// JobDetailsForEveryJobType is an array of Details keyed by every job type, +// except for jobspb.TypeUnspecified. +var JobDetailsForEveryJobType = map[Type]Details{ + TypeBackup: BackupDetails{}, + TypeRestore: RestoreDetails{}, + TypeSchemaChange: SchemaChangeDetails{}, + TypeImport: ImportDetails{}, + TypeChangefeed: ChangefeedDetails{}, + TypeCreateStats: CreateStatsDetails{}, + TypeAutoCreateStats: CreateStatsDetails{ + Name: AutoStatsName, + }, + TypeSchemaChangeGC: SchemaChangeGCDetails{}, + TypeTypeSchemaChange: TypeSchemaChangeDetails{}, + TypeStreamIngestion: StreamIngestionDetails{}, + TypeNewSchemaChange: NewSchemaChangeDetails{}, + TypeMigration: MigrationDetails{}, + TypeAutoSpanConfigReconciliation: AutoSpanConfigReconciliationDetails{}, + TypeAutoSQLStatsCompaction: AutoSQLStatsCompactionDetails{}, + TypeStreamReplication: StreamReplicationDetails{}, + TypeRowLevelTTL: RowLevelTTLDetails{}, + TypeAutoSchemaTelemetry: SchemaTelemetryDetails{}, +} + // WrapProgressDetails wraps a ProgressDetails object in the protobuf wrapper // struct necessary to make it usable as the Details field of a Progress. // @@ -375,12 +410,31 @@ func (m *ChangefeedDetails) MarshalJSONPB(marshaller *jsonpb.Marshaler) ([]byte, // DescRewriteMap maps old descriptor IDs to new descriptor and parent IDs. type DescRewriteMap map[descpb.ID]*DescriptorRewrite +// assertDetailsMap asserts that the entries in JobDetailsForEveryJobType are correct. +func assertDetailsMap() { + if len(JobDetailsForEveryJobType) != NumJobTypes-1 { + panic("JobDetailsForEveryJobType does not have an entry for each Type") + } + ForEachType( + func(typ Type) { + payload := Payload{ + Details: WrapPayloadDetails(JobDetailsForEveryJobType[typ]), + } + if typ != payload.Type() { + panic(fmt.Errorf("JobDetailsForEveryJobType has the incorrect entry for type %s", typ)) + } + }, false, + ) +} + func init() { if len(Type_name) != NumJobTypes { panic(fmt.Errorf("NumJobTypes (%d) does not match generated job type name map length (%d)", NumJobTypes, len(Type_name))) } + assertDetailsMap() + protoreflect.RegisterShorthands((*Progress)(nil), "progress") protoreflect.RegisterShorthands((*Payload)(nil), "payload") protoreflect.RegisterShorthands((*ScheduleDetails)(nil), "schedule", "schedule_details") diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index 5452edda908a..a4a4c7bd0c9c 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -20,6 +20,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/multitenant" @@ -378,8 +379,8 @@ func (r *Registry) batchJobInsertStmt( ctx context.Context, sessionID sqlliveness.SessionID, records []*Record, modifiedMicros int64, ) (string, []interface{}, []jobspb.JobID, error) { instanceID := r.ID() - const numColumns = 7 - columns := [numColumns]string{`id`, `created`, `status`, `payload`, `progress`, `claim_session_id`, `claim_instance_id`} + columns := []string{`id`, `created`, `status`, `payload`, `progress`, `claim_session_id`, `claim_instance_id`, `job_type`} + numColumns := len(columns) marshalPanic := func(m protoutil.Message) []byte { data, err := protoutil.Marshal(m) if err != nil { @@ -408,7 +409,17 @@ func (r *Registry) batchJobInsertStmt( progress.ModifiedMicros = modifiedMicros return marshalPanic(&progress) }, + `job_type`: func(rec *Record) interface{} { + return (&jobspb.Payload{Details: jobspb.WrapPayloadDetails(rec.Details)}).Type().String() + }, + } + + // To run the upgrade below, migration and schema change jobs will need to be + // created using the old schema, which does not have the job_type column. + if !r.settings.Version.IsActive(ctx, clusterversion.V23_1AddTypeColumnToJobsTable) { + numColumns -= 1 } + appendValues := func(rec *Record, vals *[]interface{}) (err error) { defer func() { switch r := recover(); r.(type) { @@ -419,7 +430,8 @@ func (r *Registry) batchJobInsertStmt( panic(r) } }() - for _, c := range columns { + for j := 0; j < numColumns; j++ { + c := columns[j] *vals = append(*vals, valueFns[c](rec)) } return nil @@ -436,7 +448,7 @@ func (r *Registry) batchJobInsertStmt( buf.WriteString(", ") } buf.WriteString("(") - for j := range columns { + for j := 0; j < numColumns; j++ { if j > 0 { buf.WriteString(", ") } @@ -473,6 +485,7 @@ func (r *Registry) CreateJobWithTxn( if txn != nil { start = txn.ReadTimestamp().GoTime() } + jobType := j.mu.payload.Type() j.mu.progress.ModifiedMicros = timeutil.ToUnixMicros(start) payloadBytes, err := protoutil.Marshal(&j.mu.payload) if err != nil { @@ -482,9 +495,28 @@ func (r *Registry) CreateJobWithTxn( if err != nil { return nil, err } - if _, err = j.registry.ex.Exec(ctx, "job-row-insert", txn, ` -INSERT INTO system.jobs (id, status, payload, progress, claim_session_id, claim_instance_id) -VALUES ($1, $2, $3, $4, $5, $6)`, jobID, StatusRunning, payloadBytes, progressBytes, s.ID().UnsafeBytes(), r.ID(), + + cols := [7]string{"id", "status", "payload", "progress", "claim_session_id", "claim_instance_id", "job_type"} + numCols := len(cols) + vals := [7]interface{}{jobID, StatusRunning, payloadBytes, progressBytes, s.ID().UnsafeBytes(), r.ID(), jobType.String()} + placeholders := func() string { + var p strings.Builder + for i := 0; i < numCols; i++ { + if i > 0 { + p.WriteByte(',') + } + p.WriteByte('$') + p.WriteString(strconv.Itoa(i + 1)) + } + return p.String() + } + // To run the upgrade below, migration and schema change jobs will need + // to be created using the old schema of the jobs table. + if !r.settings.Version.IsActive(ctx, clusterversion.V23_1AddTypeColumnToJobsTable) { + numCols -= 1 + } + insertStmt := fmt.Sprintf(`INSERT INTO system.jobs (%s) VALUES (%s)`, strings.Join(cols[:numCols], ","), placeholders()) + if _, err = j.registry.ex.Exec(ctx, "job-row-insert", txn, insertStmt, vals[:numCols]..., ); err != nil { return nil, err } diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index c0c4193c2e19..8201467c41ab 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -207,6 +207,7 @@ CREATE TABLE system.jobs ( claim_instance_id INT8, num_runs INT8, last_run TIMESTAMP, + job_type STRING, CONSTRAINT "primary" PRIMARY KEY (id), INDEX (status, created), INDEX (created_by_type, created_by_id) STORING (status), @@ -216,7 +217,8 @@ CREATE TABLE system.jobs ( created ) STORING(last_run, num_runs, claim_instance_id) WHERE ` + JobsRunStatsIdxPredicate + `, - FAMILY fam_0_id_status_created_payload (id, status, created, payload, created_by_type, created_by_id), + INDEX jobs_job_type_idx (job_type), + FAMILY fam_0_id_status_created_payload (id, status, created, payload, created_by_type, created_by_id, job_type), FAMILY progress (progress), FAMILY claim (claim_session_id, claim_instance_id, num_runs, last_run) );` @@ -1401,16 +1403,17 @@ var ( {Name: "claim_instance_id", ID: 9, Type: types.Int, Nullable: true}, {Name: "num_runs", ID: 10, Type: types.Int, Nullable: true}, {Name: "last_run", ID: 11, Type: types.Timestamp, Nullable: true}, + {Name: "job_type", ID: 12, Type: types.String, Nullable: true}, }, []descpb.ColumnFamilyDescriptor{ { - // NB: We are using family name that existed prior to adding created_by_type and - // created_by_id columns. This is done to minimize and simplify migration work + // NB: We are using family name that existed prior to adding created_by_type, + // created_by_id, and job_type columns. This is done to minimize and simplify migration work // that needed to be done. Name: "fam_0_id_status_created_payload", ID: 0, - ColumnNames: []string{"id", "status", "created", "payload", "created_by_type", "created_by_id"}, - ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 6, 7}, + ColumnNames: []string{"id", "status", "created", "payload", "created_by_type", "created_by_id", "job_type"}, + ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 6, 7, 12}, }, { Name: "progress", @@ -1462,6 +1465,16 @@ var ( Version: descpb.StrictIndexColumnIDGuaranteesVersion, Predicate: JobsRunStatsIdxPredicate, }, + descpb.IndexDescriptor{ + Name: "jobs_job_type_idx", + ID: 5, + Unique: false, + KeyColumnNames: []string{"job_type"}, + KeyColumnDirections: []catpb.IndexColumn_Direction{catpb.IndexColumn_ASC}, + KeyColumnIDs: []descpb.ColumnID{12}, + KeySuffixColumnIDs: []descpb.ColumnID{1}, + Version: descpb.StrictIndexColumnIDGuaranteesVersion, + }, )) // WebSessions table to authenticate sessions over stateless connections. diff --git a/pkg/sql/catalog/systemschema_test/testdata/bootstrap b/pkg/sql/catalog/systemschema_test/testdata/bootstrap index b482e1a2fe35..46b7791628d4 100644 --- a/pkg/sql/catalog/systemschema_test/testdata/bootstrap +++ b/pkg/sql/catalog/systemschema_test/testdata/bootstrap @@ -101,11 +101,13 @@ CREATE TABLE public.jobs ( claim_instance_id INT8 NULL, num_runs INT8 NULL, last_run TIMESTAMP NULL, + job_type STRING NULL, CONSTRAINT "primary" PRIMARY KEY (id ASC), INDEX jobs_status_created_idx (status ASC, created ASC), INDEX jobs_created_by_type_created_by_id_idx (created_by_type ASC, created_by_id ASC) STORING (status), INDEX jobs_run_stats_idx (claim_session_id ASC, status ASC, created ASC) STORING (last_run, num_runs, claim_instance_id) WHERE status IN ('running':::STRING, 'reverting':::STRING, 'pending':::STRING, 'pause-requested':::STRING, 'cancel-requested':::STRING), - FAMILY fam_0_id_status_created_payload (id, status, created, payload, created_by_type, created_by_id), + INDEX jobs_job_type_idx (job_type ASC), + FAMILY fam_0_id_status_created_payload (id, status, created, payload, created_by_type, created_by_id, job_type), FAMILY progress (progress), FAMILY claim (claim_session_id, claim_instance_id, num_runs, last_run) ); @@ -419,7 +421,7 @@ schema_telemetry {"table":{"name":"eventlog","id":12,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"timestamp","id":1,"type":{"family":"TimestampFamily","oid":1114}},{"name":"eventType","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"targetID","id":3,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"reportingID","id":4,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"info","id":5,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"uniqueID","id":6,"type":{"family":"BytesFamily","oid":17},"defaultExpr":"uuid_v4()"}],"nextColumnId":7,"families":[{"name":"primary","columnNames":["timestamp","uniqueID"],"columnIds":[1,6]},{"name":"fam_2_eventType","id":2,"columnNames":["eventType"],"columnIds":[2],"defaultColumnId":2},{"name":"fam_3_targetID","id":3,"columnNames":["targetID"],"columnIds":[3],"defaultColumnId":3},{"name":"fam_4_reportingID","id":4,"columnNames":["reportingID"],"columnIds":[4],"defaultColumnId":4},{"name":"fam_5_info","id":5,"columnNames":["info"],"columnIds":[5],"defaultColumnId":5}],"nextFamilyId":6,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["timestamp","uniqueID"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["eventType","targetID","reportingID","info"],"keyColumnIds":[1,6],"storeColumnIds":[2,3,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":480,"withGrantOption":480},{"userProto":"root","privileges":480,"withGrantOption":480}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} {"table":{"name":"external_connections","id":52,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"connection_name","id":1,"type":{"family":"StringFamily","oid":25}},{"name":"created","id":2,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"updated","id":3,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"connection_type","id":4,"type":{"family":"StringFamily","oid":25}},{"name":"connection_details","id":5,"type":{"family":"BytesFamily","oid":17}},{"name":"owner","id":6,"type":{"family":"StringFamily","oid":25}}],"nextColumnId":7,"families":[{"name":"primary","columnNames":["connection_name","created","updated","connection_type","connection_details","owner"],"columnIds":[1,2,3,4,5,6]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["connection_name"],"keyColumnDirections":["ASC"],"storeColumnNames":["created","updated","connection_type","connection_details","owner"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":480,"withGrantOption":480},{"userProto":"root","privileges":480,"withGrantOption":480}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} {"table":{"name":"job_info","id":53,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"job_id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"info_key","id":2,"type":{"family":"BytesFamily","oid":17}},{"name":"written","id":3,"type":{"family":"TimestampTZFamily","oid":1184},"defaultExpr":"now():::TIMESTAMPTZ"},{"name":"value","id":4,"type":{"family":"BytesFamily","oid":17},"nullable":true}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["job_id","info_key","written","value"],"columnIds":[1,2,3,4],"defaultColumnId":4}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["job_id","info_key","written"],"keyColumnDirections":["ASC","ASC","DESC"],"storeColumnNames":["value"],"keyColumnIds":[1,2,3],"storeColumnIds":[4],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":480,"withGrantOption":480},{"userProto":"root","privileges":480,"withGrantOption":480}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} -{"table":{"name":"jobs","id":15,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"unique_rowid()"},{"name":"status","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"created","id":3,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"payload","id":4,"type":{"family":"BytesFamily","oid":17}},{"name":"progress","id":5,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"created_by_type","id":6,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"created_by_id","id":7,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true},{"name":"claim_session_id","id":8,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"claim_instance_id","id":9,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true},{"name":"num_runs","id":10,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true},{"name":"last_run","id":11,"type":{"family":"TimestampFamily","oid":1114},"nullable":true}],"nextColumnId":12,"families":[{"name":"fam_0_id_status_created_payload","columnNames":["id","status","created","payload","created_by_type","created_by_id"],"columnIds":[1,2,3,4,6,7]},{"name":"progress","id":1,"columnNames":["progress"],"columnIds":[5],"defaultColumnId":5},{"name":"claim","id":2,"columnNames":["claim_session_id","claim_instance_id","num_runs","last_run"],"columnIds":[8,9,10,11]}],"nextFamilyId":3,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["status","created","payload","progress","created_by_type","created_by_id","claim_session_id","claim_instance_id","num_runs","last_run"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6,7,8,9,10,11],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"jobs_status_created_idx","id":2,"version":3,"keyColumnNames":["status","created"],"keyColumnDirections":["ASC","ASC"],"keyColumnIds":[2,3],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"jobs_created_by_type_created_by_id_idx","id":3,"version":3,"keyColumnNames":["created_by_type","created_by_id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["status"],"keyColumnIds":[6,7],"keySuffixColumnIds":[1],"storeColumnIds":[2],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"jobs_run_stats_idx","id":4,"version":3,"keyColumnNames":["claim_session_id","status","created"],"keyColumnDirections":["ASC","ASC","ASC"],"storeColumnNames":["last_run","num_runs","claim_instance_id"],"keyColumnIds":[8,2,3],"keySuffixColumnIds":[1],"storeColumnIds":[11,10,9],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"status IN ('_':::STRING, '_':::STRING, '_':::STRING, '_':::STRING, '_':::STRING)"}],"nextIndexId":5,"privileges":{"users":[{"userProto":"admin","privileges":480,"withGrantOption":480},{"userProto":"root","privileges":480,"withGrantOption":480}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} +{"table":{"name":"jobs","id":15,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"unique_rowid()"},{"name":"status","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"created","id":3,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"payload","id":4,"type":{"family":"BytesFamily","oid":17}},{"name":"progress","id":5,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"created_by_type","id":6,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"created_by_id","id":7,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true},{"name":"claim_session_id","id":8,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"claim_instance_id","id":9,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true},{"name":"num_runs","id":10,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true},{"name":"last_run","id":11,"type":{"family":"TimestampFamily","oid":1114},"nullable":true},{"name":"job_type","id":12,"type":{"family":"StringFamily","oid":25},"nullable":true}],"nextColumnId":13,"families":[{"name":"fam_0_id_status_created_payload","columnNames":["id","status","created","payload","created_by_type","created_by_id","job_type"],"columnIds":[1,2,3,4,6,7,12]},{"name":"progress","id":1,"columnNames":["progress"],"columnIds":[5],"defaultColumnId":5},{"name":"claim","id":2,"columnNames":["claim_session_id","claim_instance_id","num_runs","last_run"],"columnIds":[8,9,10,11]}],"nextFamilyId":3,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["status","created","payload","progress","created_by_type","created_by_id","claim_session_id","claim_instance_id","num_runs","last_run","job_type"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6,7,8,9,10,11,12],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"jobs_status_created_idx","id":2,"version":3,"keyColumnNames":["status","created"],"keyColumnDirections":["ASC","ASC"],"keyColumnIds":[2,3],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"jobs_created_by_type_created_by_id_idx","id":3,"version":3,"keyColumnNames":["created_by_type","created_by_id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["status"],"keyColumnIds":[6,7],"keySuffixColumnIds":[1],"storeColumnIds":[2],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"jobs_run_stats_idx","id":4,"version":3,"keyColumnNames":["claim_session_id","status","created"],"keyColumnDirections":["ASC","ASC","ASC"],"storeColumnNames":["last_run","num_runs","claim_instance_id"],"keyColumnIds":[8,2,3],"keySuffixColumnIds":[1],"storeColumnIds":[11,10,9],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"status IN ('_':::STRING, '_':::STRING, '_':::STRING, '_':::STRING, '_':::STRING)"},{"name":"jobs_job_type_idx","id":5,"version":3,"keyColumnNames":["job_type"],"keyColumnDirections":["ASC"],"keyColumnIds":[12],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}}],"nextIndexId":6,"privileges":{"users":[{"userProto":"admin","privileges":480,"withGrantOption":480},{"userProto":"root","privileges":480,"withGrantOption":480}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} {"table":{"name":"join_tokens","id":41,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"UuidFamily","oid":2950}},{"name":"secret","id":2,"type":{"family":"BytesFamily","oid":17}},{"name":"expiration","id":3,"type":{"family":"TimestampTZFamily","oid":1184}}],"nextColumnId":4,"families":[{"name":"primary","columnNames":["id","secret","expiration"],"columnIds":[1,2,3]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["secret","expiration"],"keyColumnIds":[1],"storeColumnIds":[2,3],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":480,"withGrantOption":480},{"userProto":"root","privileges":480,"withGrantOption":480}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} {"table":{"name":"lease","id":11,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"descID","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"version","id":2,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"nodeID","id":3,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"expiration","id":4,"type":{"family":"TimestampFamily","oid":1114}}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["descID","version","nodeID","expiration"],"columnIds":[1,2,3,4]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["descID","version","expiration","nodeID"],"keyColumnDirections":["ASC","ASC","ASC","ASC"],"keyColumnIds":[1,2,4,3],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":480,"withGrantOption":480},{"userProto":"root","privileges":480,"withGrantOption":480}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} {"table":{"name":"locations","id":21,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"localityKey","id":1,"type":{"family":"StringFamily","oid":25}},{"name":"localityValue","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"latitude","id":3,"type":{"family":"DecimalFamily","width":15,"precision":18,"oid":1700}},{"name":"longitude","id":4,"type":{"family":"DecimalFamily","width":15,"precision":18,"oid":1700}}],"nextColumnId":5,"families":[{"name":"fam_0_localityKey_localityValue_latitude_longitude","columnNames":["localityKey","localityValue","latitude","longitude"],"columnIds":[1,2,3,4]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["localityKey","localityValue"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["latitude","longitude"],"keyColumnIds":[1,2],"storeColumnIds":[3,4],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":480,"withGrantOption":480},{"userProto":"root","privileges":480,"withGrantOption":480}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index 69d11667ee44..a222dc0289ff 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -114,7 +114,7 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 12 {"table": {"columns": [{"id": 1, "name": "timestamp", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 2, "name": "eventType", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "targetID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "reportingID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "info", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "uuid_v4()", "id": 6, "name": "uniqueID", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 12, "name": "eventlog", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [1, 6], "keyColumnNames": ["timestamp", "uniqueID"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5], "storeColumnNames": ["eventType", "targetID", "reportingID", "info"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": 480, "userProto": "admin", "withGrantOption": 480}, {"privileges": 480, "userProto": "root", "withGrantOption": 480}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 13 {"table": {"columns": [{"id": 1, "name": "timestamp", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 2, "name": "rangeID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "storeID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "eventType", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "otherRangeID", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "info", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "unique_rowid()", "id": 7, "name": "uniqueID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 13, "name": "rangelog", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [1, 7], "keyColumnNames": ["timestamp", "uniqueID"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6], "storeColumnNames": ["rangeID", "storeID", "eventType", "otherRangeID", "info"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": 480, "userProto": "admin", "withGrantOption": 480}, {"privileges": 480, "userProto": "root", "withGrantOption": 480}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 14 {"table": {"columns": [{"id": 1, "name": "key", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "lastUpdated", "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 14, "name": "ui", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["key"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3], "storeColumnNames": ["value", "lastUpdated"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": 480, "userProto": "admin", "withGrantOption": 480}, {"privileges": 480, "userProto": "root", "withGrantOption": 480}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} -15 {"table": {"columns": [{"defaultExpr": "unique_rowid()", "id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "now():::TIMESTAMP", "id": 3, "name": "created", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 4, "name": "payload", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "progress", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "created_by_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "created_by_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "claim_session_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "claim_instance_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "num_runs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "last_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 15, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [2, 3], "keyColumnNames": ["status", "created"], "keySuffixColumnIds": [1], "name": "jobs_status_created_idx", "partitioning": {}, "sharded": {}, "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [6, 7], "keyColumnNames": ["created_by_type", "created_by_id"], "keySuffixColumnIds": [1], "name": "jobs_created_by_type_created_by_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["status"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 4, "interleave": {}, "keyColumnDirections": ["ASC", "ASC", "ASC"], "keyColumnIds": [8, 2, 3], "keyColumnNames": ["claim_session_id", "status", "created"], "keySuffixColumnIds": [1], "name": "jobs_run_stats_idx", "partitioning": {}, "predicate": "status IN ('running':::STRING, 'reverting':::STRING, 'pending':::STRING, 'pause-requested':::STRING, 'cancel-requested':::STRING)", "sharded": {}, "storeColumnIds": [11, 10, 9], "storeColumnNames": ["last_run", "num_runs", "claim_instance_id"], "version": 3}], "name": "jobs", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 5, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11], "storeColumnNames": ["status", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": 480, "userProto": "admin", "withGrantOption": 480}, {"privileges": 480, "userProto": "root", "withGrantOption": 480}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} +15 {"table": {"columns": [{"defaultExpr": "unique_rowid()", "id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "now():::TIMESTAMP", "id": 3, "name": "created", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 4, "name": "payload", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "progress", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "created_by_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "created_by_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "claim_session_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "claim_instance_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "num_runs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "last_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "job_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 15, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [2, 3], "keyColumnNames": ["status", "created"], "keySuffixColumnIds": [1], "name": "jobs_status_created_idx", "partitioning": {}, "sharded": {}, "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [6, 7], "keyColumnNames": ["created_by_type", "created_by_id"], "keySuffixColumnIds": [1], "name": "jobs_created_by_type_created_by_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["status"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 4, "interleave": {}, "keyColumnDirections": ["ASC", "ASC", "ASC"], "keyColumnIds": [8, 2, 3], "keyColumnNames": ["claim_session_id", "status", "created"], "keySuffixColumnIds": [1], "name": "jobs_run_stats_idx", "partitioning": {}, "predicate": "status IN ('running':::STRING, 'reverting':::STRING, 'pending':::STRING, 'pause-requested':::STRING, 'cancel-requested':::STRING)", "sharded": {}, "storeColumnIds": [11, 10, 9], "storeColumnNames": ["last_run", "num_runs", "claim_instance_id"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 5, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [12], "keyColumnNames": ["job_type"], "keySuffixColumnIds": [1], "name": "jobs_job_type_idx", "partitioning": {}, "sharded": {}, "version": 3}], "name": "jobs", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 6, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], "storeColumnNames": ["status", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run", "job_type"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": 480, "userProto": "admin", "withGrantOption": 480}, {"privileges": 480, "userProto": "root", "withGrantOption": 480}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 19 {"table": {"columns": [{"defaultExpr": "unique_rowid()", "id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "hashedSecret", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "username", "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "now():::TIMESTAMP", "id": 4, "name": "createdAt", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 5, "name": "expiresAt", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 6, "name": "revokedAt", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"defaultExpr": "now():::TIMESTAMP", "id": 7, "name": "lastUsedAt", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 8, "name": "auditInfo", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 19, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [5], "keyColumnNames": ["expiresAt"], "keySuffixColumnIds": [1], "name": "web_sessions_expiresAt_idx", "partitioning": {}, "sharded": {}, "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [4], "keyColumnNames": ["createdAt"], "keySuffixColumnIds": [1], "name": "web_sessions_createdAt_idx", "partitioning": {}, "sharded": {}, "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 4, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [6], "keyColumnNames": ["revokedAt"], "keySuffixColumnIds": [1], "name": "web_sessions_revokedAt_idx", "partitioning": {}, "sharded": {}, "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 5, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [7], "keyColumnNames": ["lastUsedAt"], "keySuffixColumnIds": [1], "name": "web_sessions_lastUsedAt_idx", "partitioning": {}, "sharded": {}, "version": 3}], "name": "web_sessions", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 6, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8], "storeColumnNames": ["hashedSecret", "username", "createdAt", "expiresAt", "revokedAt", "lastUsedAt", "auditInfo"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": 480, "userProto": "admin", "withGrantOption": 480}, {"privileges": 480, "userProto": "root", "withGrantOption": 480}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 20 {"table": {"columns": [{"id": 1, "name": "tableID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"defaultExpr": "unique_rowid()", "id": 2, "name": "statisticID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "columnIDs", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"defaultExpr": "now():::TIMESTAMP", "id": 5, "name": "createdAt", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 6, "name": "rowCount", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "distinctCount", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "nullCount", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "histogram", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"defaultExpr": "0:::INT8", "id": 10, "name": "avgSize", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "partialPredicate", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 20, "name": "table_statistics", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [1, 2], "keyColumnNames": ["tableID", "statisticID"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [3, 4, 5, 6, 7, 8, 9, 10, 11], "storeColumnNames": ["name", "columnIDs", "createdAt", "rowCount", "distinctCount", "nullCount", "histogram", "avgSize", "partialPredicate"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": 480, "userProto": "admin", "withGrantOption": 480}, {"privileges": 480, "userProto": "root", "withGrantOption": 480}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 21 {"table": {"columns": [{"id": 1, "name": "localityKey", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "localityValue", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "latitude", "type": {"family": "DecimalFamily", "oid": 1700, "precision": 18, "width": 15}}, {"id": 4, "name": "longitude", "type": {"family": "DecimalFamily", "oid": 1700, "precision": 18, "width": 15}}], "formatVersion": 3, "id": 21, "name": "locations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [1, 2], "keyColumnNames": ["localityKey", "localityValue"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [3, 4], "storeColumnNames": ["latitude", "longitude"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": 480, "userProto": "admin", "withGrantOption": 480}, {"privileges": 480, "userProto": "root", "withGrantOption": 480}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 17c643d6135b..9c5b9a0beb05 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -2118,6 +2118,7 @@ system public jobs created system public jobs created_by_id 7 system public jobs created_by_type 6 system public jobs id 1 +system public jobs job_type 12 system public jobs last_run 11 system public jobs num_runs 10 system public jobs payload 4 diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 832d272cf09e..9f4c8d6eeb00 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1153,6 +1153,7 @@ indexrelid indrelid indnatts indisunique indisprimary indisexclusion indim 2621181441 15 3 false false false false false true false false true false 6 7 2 3403232968 0 0 0 2 2 NULL NULL 2 2621181443 15 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 2621181446 15 6 false false false false false true false false true false 8 2 3 11 10 9 0 3403232968 0 0 0 0 2 2 2 NULL status IN ('running'::STRING, 'reverting'::STRING, 'pending'::STRING, 'pause-requested'::STRING, 'cancel-requested'::STRING) 3 +2621181447 15 1 false false false false false true false false true false 12 3403232968 0 2 NULL NULL 1 2667577107 31 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 2834522046 34 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 2880917710 50 2 true true false true false true false false true false 1 2 0 3403232968 0 0 2 2 NULL NULL 2 @@ -1254,6 +1255,7 @@ indexrelid operator_argument_type_oid operator_argument_position 2621181446 0 1 2621181446 0 2 2621181446 0 3 +2621181447 0 1 2667577107 0 1 2834522046 0 1 2880917710 0 1 diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system index e5921ff1e29d..7257117151f1 100644 --- a/pkg/sql/logictest/testdata/logic_test/system +++ b/pkg/sql/logictest/testdata/logic_test/system @@ -284,17 +284,19 @@ lastUpdated TIMESTAMP false NULL · {primary} false query TTBTTTB SHOW COLUMNS FROM system.jobs ---- -id INT8 false unique_rowid() · {jobs_created_by_type_created_by_id_idx,jobs_run_stats_idx,jobs_status_created_idx,primary} false -status STRING false NULL · {jobs_created_by_type_created_by_id_idx,jobs_run_stats_idx,jobs_status_created_idx,primary} false -created TIMESTAMP false now():::TIMESTAMP · {jobs_run_stats_idx,jobs_status_created_idx,primary} false -payload BYTES false NULL · {primary} false -progress BYTES true NULL · {primary} false -created_by_type STRING true NULL · {jobs_created_by_type_created_by_id_idx,primary} false -created_by_id INT8 true NULL · {jobs_created_by_type_created_by_id_idx,primary} false -claim_session_id BYTES true NULL · {jobs_run_stats_idx,primary} false -claim_instance_id INT8 true NULL · {jobs_run_stats_idx,primary} false -num_runs INT8 true NULL · {jobs_run_stats_idx,primary} false -last_run TIMESTAMP true NULL · {jobs_run_stats_idx,primary} false +id INT8 false unique_rowid() · {jobs_created_by_type_created_by_id_idx,jobs_job_type_idx,jobs_run_stats_idx,jobs_status_created_idx,primary} false +status STRING false NULL · {jobs_created_by_type_created_by_id_idx,jobs_run_stats_idx,jobs_status_created_idx,primary} false +created TIMESTAMP false now():::TIMESTAMP · {jobs_run_stats_idx,jobs_status_created_idx,primary} false +payload BYTES false NULL · {primary} false +progress BYTES true NULL · {primary} false +created_by_type STRING true NULL · {jobs_created_by_type_created_by_id_idx,primary} false +created_by_id INT8 true NULL · {jobs_created_by_type_created_by_id_idx,primary} false +claim_session_id BYTES true NULL · {jobs_run_stats_idx,primary} false +claim_instance_id INT8 true NULL · {jobs_run_stats_idx,primary} false +num_runs INT8 true NULL · {jobs_run_stats_idx,primary} false +last_run TIMESTAMP true NULL · {jobs_run_stats_idx,primary} false +job_type STRING true NULL · {jobs_job_type_idx,primary} false + query TTBTTTB SHOW COLUMNS FROM system.settings diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 0bab3a0886dc..b1c3a234d2d1 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -38,6 +38,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" @@ -86,6 +87,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/ipaddr" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/timeofday" "github.com/cockroachdb/cockroach/pkg/util/timetz" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -4081,6 +4083,35 @@ value if you rely on the HLC for accuracy.`, } }()...), + "crdb_internal.job_payload_type": makeBuiltin( + jsonProps(), + func() []tree.Overload { + returnType := tree.FixedReturnType(types.String) + getJobPayloadType := func(data []byte) (tree.Datum, error) { + var msg jobspb.Payload + if err := protoutil.Unmarshal(data, &msg); err != nil { + return nil, err + } + return tree.NewDString(msg.Type().String()), nil + } + + return []tree.Overload{ + { + Info: "Reads the type from the jobspb.Payload protocol message.", + Volatility: volatility.Immutable, + Types: tree.ParamTypes{ + {Name: "data", Typ: types.Bytes}, + }, + ReturnType: returnType, + Fn: func(_ context.Context, _ *eval.Context, args tree.Datums) (tree.Datum, error) { + return getJobPayloadType( + []byte(tree.MustBeDBytes(args[0])), + ) + }, + }, + } + }()...), + "crdb_internal.json_to_pb": makeBuiltin( jsonProps(), tree.Overload{ diff --git a/pkg/sql/sem/builtins/fixed_oids.go b/pkg/sql/sem/builtins/fixed_oids.go index c3076cf1418c..94d7e65f9f95 100644 --- a/pkg/sql/sem/builtins/fixed_oids.go +++ b/pkg/sql/sem/builtins/fixed_oids.go @@ -443,6 +443,7 @@ var builtinOidsBySignature = map[string]oid.Oid{ `crdb_internal.num_inverted_index_entries(val: jsonb, version: int) -> int`: 1336, `crdb_internal.num_inverted_index_entries(val: string, version: int) -> int`: 1337, `crdb_internal.num_inverted_index_entries(val: anyelement[], version: int) -> int`: 1338, + `crdb_internal.job_payload_type(data: bytes) -> string`: 2061, `crdb_internal.payloads_for_span(span_id: int) -> tuple{string AS payload_type, jsonb AS payload_jsonb}`: 349, `crdb_internal.payloads_for_trace(trace_id: int) -> tuple{int AS span_id, string AS payload_type, jsonb AS payload_jsonb}`: 350, `crdb_internal.pb_to_json(pbname: string, data: bytes) -> jsonb`: 1270, diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index d46c8347d341..4431d6d5d555 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -4,6 +4,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "upgrades", srcs = [ + "alter_jobs_add_job_type.go", "alter_sql_instances_locality.go", "alter_statement_statistics_index_recommendations.go", "alter_table_statistics_partial_predicate.go", @@ -76,6 +77,7 @@ go_test( name = "upgrades_test", size = "large", srcs = [ + "alter_jobs_add_job_type_test.go", "alter_sql_instances_locality_test.go", "alter_statement_statistics_index_recommendations_test.go", "alter_table_statistics_partial_predicate_test.go", diff --git a/pkg/upgrade/upgrades/alter_jobs_add_job_type.go b/pkg/upgrade/upgrades/alter_jobs_add_job_type.go new file mode 100644 index 000000000000..e36026b0e84a --- /dev/null +++ b/pkg/upgrade/upgrades/alter_jobs_add_job_type.go @@ -0,0 +1,76 @@ +// 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. + +package upgrades + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/upgrade" +) + +const addTypeColumnStmt = ` +ALTER TABLE system.jobs +ADD COLUMN IF NOT EXISTS job_type STRING +FAMILY "fam_0_id_status_created_payload" +` + +const addTypeColumnIdxStmt = ` +CREATE INDEX IF NOT EXISTS jobs_job_type_idx +ON system.jobs (job_type) +` + +const backfillTypeColumnStmt = ` +UPDATE system.jobs +SET job_type = crdb_internal.job_payload_type(payload) +WHERE job_type IS NULL +` + +func alterSystemJobsAddJobType( + ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps, +) error { + schemaChangeOps := []operation{ + { + name: "add-jobs-type-col", + schemaList: []string{"type"}, + query: addTypeColumnStmt, + schemaExistsFn: hasColumn, + }, + { + name: "add-jobs-type-col-idx", + schemaList: []string{"jobs_job_type_idx"}, + query: addTypeColumnIdxStmt, + schemaExistsFn: hasIndex, + }, + } + + for _, op := range schemaChangeOps { + if err := migrateTable(ctx, cs, d, op, keys.JobsTableID, systemschema.JobsTable); err != nil { + return err + } + } + + return nil +} + +func backfillJobTypeColumn( + ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps, +) error { + ie := d.InternalExecutorFactory.MakeInternalExecutorWithoutTxn() + _, err := ie.Exec(ctx, "backfill-jobs-type-column", nil /* txn */, backfillTypeColumnStmt, username.RootUser) + if err != nil { + return err + } + return nil +} diff --git a/pkg/upgrade/upgrades/alter_jobs_add_job_type_test.go b/pkg/upgrade/upgrades/alter_jobs_add_job_type_test.go new file mode 100644 index 000000000000..1f42bafd00fc --- /dev/null +++ b/pkg/upgrade/upgrades/alter_jobs_add_job_type_test.go @@ -0,0 +1,280 @@ +// 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. + +package upgrades_test + +import ( + "context" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +var _ jobs.Resumer = &fakeResumer{} + +type fakeResumer struct { +} + +// Resume implements the jobs.Resumer interface. +func (*fakeResumer) Resume(ctx context.Context, execCtx interface{}) error { + return nil +} + +// Resume implements the jobs.Resumer interface. +func (*fakeResumer) OnFailOrCancel(ctx context.Context, execCtx interface{}, jobErr error) error { + return jobErr +} + +// TestAlterSystemJobsTableAddJobTypeColumn verifies that the migrations that add & backfill +// the type column to system.jobs succeed. +func TestAlterSystemJobsTableAddJobTypeColumn(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + clusterArgs := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: clusterversion.ByKey( + clusterversion.V23_1AddTypeColumnToJobsTable - 1), + }, + }, + }, + } + + var ( + ctx = context.Background() + tc = testcluster.StartTestCluster(t, 1, clusterArgs) + s = tc.Server(0) + sqlDB = tc.ServerConn(0) + ) + defer tc.Stopper().Stop(ctx) + + var ( + validationSchemas = []upgrades.Schema{ + {Name: "job_type", ValidationFn: upgrades.HasColumn}, + {Name: "fam_0_id_status_created_payload", ValidationFn: upgrades.HasColumnFamily}, + {Name: "jobs_job_type_idx", ValidationFn: upgrades.HasIndex}, + } + ) + + // Inject the old copy of the descriptor and validate that the schema matches the old version. + upgrades.InjectLegacyTable(ctx, t, s, systemschema.JobsTable, getJobsTableDescriptorPriorToV23_1AddTypeColumnToJobsTable) + upgrades.ValidateSchemaExists( + ctx, + t, + s, + sqlDB, + keys.JobsTableID, + systemschema.JobsTable, + []string{}, + validationSchemas, + false, /* expectExists */ + ) + + // Start a job of each type. + registry := s.JobRegistry().(*jobs.Registry) + registry.TestingResumerCreationKnobs = map[jobspb.Type]func(raw jobs.Resumer) jobs.Resumer{} + jobspb.ForEachType(func(typ jobspb.Type) { + // The upgrade creates migration and schemachange jobs, so we do not + // need to create more. We should not override resumers for these job types, + // otherwise the upgrade will hang. + if typ != jobspb.TypeMigration && typ != jobspb.TypeSchemaChange { + registry.TestingResumerCreationKnobs[typ] = func(r jobs.Resumer) jobs.Resumer { + return &fakeResumer{} + } + + record := jobs.Record{ + Details: jobspb.JobDetailsForEveryJobType[typ], + // We must pass a progress struct to be able to create a job. Since we + // override the resumer for each type to be a fake resumer, the type of + // progess we pass in does not matter. + Progress: jobspb.ImportProgress{}, + } + + _, err := registry.CreateJobWithTxn(ctx, record, registry.MakeJobID(), nil /* txn */) + require.NoError(t, err) + } + }, false) + + // Run the upgrade which adds the job_type column. + upgrades.Upgrade( + t, + sqlDB, + clusterversion.V23_1AddTypeColumnToJobsTable, + nil, /* done */ + false, /* expectError */ + ) + + // Check that the type column exists but has not been populated yet. + var count int + row := sqlDB.QueryRow("SELECT count(*) FROM system.jobs WHERE job_type IS NOT NULL") + err := row.Scan(&count) + require.NoError(t, err) + assert.Equal(t, count, 0) + + upgrades.ValidateSchemaExists( + ctx, + t, + s, + sqlDB, + keys.JobsTableID, + systemschema.JobsTable, + []string{}, + validationSchemas, + true, /* expectExists */ + ) + + // Run the upgrade which backfills the job_type column. + upgrades.Upgrade( + t, + sqlDB, + clusterversion.V23_1BackfillTypeColumnInJobsTable, + nil, /* done */ + false, /* expectError */ + ) + + // Assert that we backfill the job_type column correctly for each type of job. + row = sqlDB.QueryRow("SELECT count(*) FROM system.jobs WHERE job_type IS NULL") + err = row.Scan(&count) + require.NoError(t, err) + assert.Equal(t, count, 0) + + var typStr string + rows, err := sqlDB.Query("SELECT distinct(job_type) FROM system.jobs") + require.NoError(t, err) + seenTypes := util.FastIntSet{} + for rows.Next() { + err = rows.Scan(&typStr) + require.NoError(t, err) + seenTypes.Add(int(jobspb.Type_value[strings.ReplaceAll(typStr, " ", "_")])) + } + jobspb.ForEachType(func(typ jobspb.Type) { + assert.True(t, seenTypes.Contains(int(typ))) + }, false) +} + +func getJobsTableDescriptorPriorToV23_1AddTypeColumnToJobsTable() *descpb.TableDescriptor { + defaultID := "unique_rowid()" + defaultCreated := "now():::TIMESTAMP" + return &descpb.TableDescriptor{ + Name: string(catconstants.JobsTableName), + ID: keys.JobsTableID, + ParentID: keys.SystemDatabaseID, + UnexposedParentSchemaID: keys.PublicSchemaID, + Version: 1, + Columns: []descpb.ColumnDescriptor{ + {Name: "id", ID: 1, Type: types.Int, DefaultExpr: &defaultID}, + {Name: "status", ID: 2, Type: types.String}, + {Name: "created", ID: 3, Type: types.Timestamp, DefaultExpr: &defaultCreated}, + {Name: "payload", ID: 4, Type: types.Bytes}, + {Name: "progress", ID: 5, Type: types.Bytes}, + {Name: "created_by_type", ID: 6, Type: types.String, Nullable: true}, + {Name: "created_by_id", ID: 7, Type: types.Int, Nullable: true}, + {Name: "claim_session_id", ID: 8, Type: types.Bytes, Nullable: true}, + {Name: "claim_instance_id", ID: 9, Type: types.Int, Nullable: true}, + {Name: "num_runs", ID: 10, Type: types.Int, Nullable: true}, + {Name: "last_run", ID: 11, Type: types.Timestamp, Nullable: true}, + }, + NextColumnID: 12, + Families: []descpb.ColumnFamilyDescriptor{ + { + Name: "fam_0_id_status_created_payload", + ID: 0, + ColumnNames: []string{"id", "status", "created", "payload", "created_by_type", "created_by_id"}, + ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 6, 7}, + }, + { + Name: "progress", + ID: 1, + ColumnNames: []string{"progress"}, + ColumnIDs: []descpb.ColumnID{5}, + DefaultColumnID: 5, + }, + { + Name: "claim", + ID: 2, + ColumnNames: []string{"claim_session_id", "claim_instance_id", "num_runs", "last_run"}, + ColumnIDs: []descpb.ColumnID{8, 9, 10, 11}, + }, + }, + NextFamilyID: 3, + PrimaryIndex: descpb.IndexDescriptor{ + Name: "id", + ID: 1, + Unique: true, + KeyColumnNames: []string{"id"}, + KeyColumnDirections: []catpb.IndexColumn_Direction{catpb.IndexColumn_ASC}, + KeyColumnIDs: []descpb.ColumnID{1}, + }, + Indexes: []descpb.IndexDescriptor{ + { + Name: "jobs_status_created_idx", + ID: 2, + Unique: false, + KeyColumnNames: []string{"status", "created"}, + KeyColumnDirections: []catpb.IndexColumn_Direction{catpb.IndexColumn_ASC, catpb.IndexColumn_ASC}, + KeyColumnIDs: []descpb.ColumnID{2, 3}, + KeySuffixColumnIDs: []descpb.ColumnID{1}, + Version: descpb.StrictIndexColumnIDGuaranteesVersion, + }, + { + Name: "jobs_created_by_type_created_by_id_idx", + ID: 3, + Unique: false, + KeyColumnNames: []string{"created_by_type", "created_by_id"}, + KeyColumnDirections: []catpb.IndexColumn_Direction{catpb.IndexColumn_ASC, catpb.IndexColumn_ASC}, + KeyColumnIDs: []descpb.ColumnID{6, 7}, + StoreColumnIDs: []descpb.ColumnID{2}, + StoreColumnNames: []string{"status"}, + KeySuffixColumnIDs: []descpb.ColumnID{1}, + Version: descpb.StrictIndexColumnIDGuaranteesVersion, + }, + { + Name: "jobs_run_stats_idx", + ID: 4, + Unique: false, + KeyColumnNames: []string{"claim_session_id", "status", "created"}, + KeyColumnDirections: []catpb.IndexColumn_Direction{catpb.IndexColumn_ASC, catpb.IndexColumn_ASC, catpb.IndexColumn_ASC}, + KeyColumnIDs: []descpb.ColumnID{8, 2, 3}, + StoreColumnNames: []string{"last_run", "num_runs", "claim_instance_id"}, + StoreColumnIDs: []descpb.ColumnID{11, 10, 9}, + KeySuffixColumnIDs: []descpb.ColumnID{1}, + Version: descpb.StrictIndexColumnIDGuaranteesVersion, + Predicate: systemschema.JobsRunStatsIdxPredicate, + }, + }, + NextIndexID: 5, + Privileges: catpb.NewCustomSuperuserPrivilegeDescriptor(privilege.ReadWriteData, username.NodeUserName()), + NextMutationID: 1, + FormatVersion: 3, + } +} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index b286b945eb3a..463f5dfbae96 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -228,6 +228,18 @@ var upgrades = []upgradebase.Upgrade{ upgrade.NoPrecondition, backfillSystemRoleMembersIDColumns, ), + upgrade.NewTenantUpgrade( + "add job_type column to system.jobs table", + toCV(clusterversion.V23_1AddTypeColumnToJobsTable), + upgrade.NoPrecondition, + alterSystemJobsAddJobType, + ), + upgrade.NewTenantUpgrade( + "backfill job_type column in system.jobs table", + toCV(clusterversion.V23_1BackfillTypeColumnInJobsTable), + upgrade.NoPrecondition, + backfillJobTypeColumn, + ), } func init() { From 987af0183d86dd623ebc1781080cc59252bca4e4 Mon Sep 17 00:00:00 2001 From: Renato Costa Date: Fri, 16 Dec 2022 15:52:06 -0500 Subject: [PATCH 2/2] kv/kvserver: skip TestProtectedTimestamps It's flaky and failing very frequently. Epic: none Release note: None --- pkg/kv/kvserver/client_protectedts_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/kv/kvserver/client_protectedts_test.go b/pkg/kv/kvserver/client_protectedts_test.go index d14d6424d28f..d49261ac1f99 100644 --- a/pkg/kv/kvserver/client_protectedts_test.go +++ b/pkg/kv/kvserver/client_protectedts_test.go @@ -47,6 +47,7 @@ import ( func TestProtectedTimestamps(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + skip.WithIssue(t, 93497, "flaky test") ctx := context.Background() // This test is too slow to run with race.