diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index f645c5d3eed7..11ef33ad129e 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -2025,19 +2025,20 @@ func TestShowAutomaticJobs(t *testing.T) { } for _, in := range rows { - // system.jobs is part proper SQL columns, part protobuf, so we can't use the - // row struct directly. - inPayload, err := protoutil.Marshal(&jobspb.Payload{ + rawPayload := &jobspb.Payload{ UsernameProto: username.RootUserName().EncodeProto(), Details: jobspb.WrapPayloadDetails(in.details), - }) + } + // system.jobs is part proper SQL columns, part protobuf, so we can't use the + // row struct directly. + inPayload, err := protoutil.Marshal(rawPayload) if err != nil { t.Fatal(err) } sqlDB.Exec(t, - `INSERT INTO system.jobs (id, status, payload) VALUES ($1, $2, $3)`, - in.id, in.status, inPayload, + `INSERT INTO system.jobs (id, status, payload, job_type) VALUES ($1, $2, $3, $4)`, + in.id, in.status, inPayload, rawPayload.Type().String(), ) } diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index 0f6548e4e033..9b67d813dc7e 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -1688,8 +1688,8 @@ func TestAdminAPIJobs(t *testing.T) { t.Fatal(err) } sqlDB.Exec(t, - `INSERT INTO system.jobs (id, status, payload, progress, num_runs, last_run) VALUES ($1, $2, $3, $4, $5, $6)`, - job.id, job.status, payloadBytes, progressBytes, job.numRuns, job.lastRun, + `INSERT INTO system.jobs (id, status, payload, progress, num_runs, last_run, job_type) VALUES ($1, $2, $3, $4, $5, $6, $7)`, + job.id, job.status, payloadBytes, progressBytes, job.numRuns, job.lastRun, payload.Type().String(), ) } diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 13291c29bb0d..7f3052893cbc 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1036,13 +1036,22 @@ const ( // Note that we are querying crdb_internal.system_jobs instead of system.jobs directly. // The former has access control built in and will filter out jobs that the // user is not allowed to see. - jobsQFrom = ` FROM crdb_internal.system_jobs` - jobsBackoffArgs = `(SELECT $1::FLOAT AS initial_delay, $2::FLOAT AS max_delay) args` - jobsStatusFilter = ` WHERE status = $3` - jobsQuery = jobsQSelect + `, last_run, COALESCE(num_runs, 0), ` + jobs.NextRunClause + + jobsQFrom = ` FROM crdb_internal.system_jobs` + jobsBackoffArgs = `(SELECT $1::FLOAT AS initial_delay, $2::FLOAT AS max_delay) args` + jobsStatusFilter = ` WHERE status = $3` + oldJobsTypeFilter = ` WHERE crdb_internal.job_payload_type(payload) = $3` + jobsTypeFilter = ` WHERE job_type = $3` + jobsQuery = jobsQSelect + `, last_run, COALESCE(num_runs, 0), ` + jobs.NextRunClause + ` as next_run` + jobsQFrom + ", " + jobsBackoffArgs ) +func getCRDBInternalJobsTableTypeFilter(ctx context.Context, version clusterversion.Handle) string { + if !version.IsActive(ctx, clusterversion.V23_1BackfillTypeColumnInJobsTable) { + return oldJobsTypeFilter + } + return jobsTypeFilter +} + // TODO(tbg): prefix with kv_. var crdbInternalJobsTable = virtualSchemaTable{ schema: ` @@ -1069,7 +1078,8 @@ CREATE TABLE crdb_internal.jobs ( num_runs INT, execution_errors STRING[], execution_events JSONB, - INDEX(status) + INDEX(status), + INDEX(job_type) )`, comment: `decoded job metadata from crdb_internal.system_jobs (KV scan)`, indexes: []virtualIndex{{ @@ -1078,6 +1088,12 @@ CREATE TABLE crdb_internal.jobs ( targetStatus := tree.MustBeDString(unwrappedConstraint) return makeJobsTableRows(ctx, p, addRow, q, p.execCfg.JobRegistry.RetryInitialDelay(), p.execCfg.JobRegistry.RetryMaxDelay(), targetStatus) }, + }, { + populate: func(ctx context.Context, unwrappedConstraint tree.Datum, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) (matched bool, err error) { + q := jobsQuery + getCRDBInternalJobsTableTypeFilter(ctx, p.execCfg.Settings.Version) + targetStatus := tree.MustBeDString(unwrappedConstraint) + return makeJobsTableRows(ctx, p, addRow, q, p.execCfg.JobRegistry.RetryInitialDelay(), p.execCfg.JobRegistry.RetryMaxDelay(), targetStatus) + }, }}, populate: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { _, err := makeJobsTableRows(ctx, p, addRow, jobsQuery, p.execCfg.JobRegistry.RetryInitialDelay(), p.execCfg.JobRegistry.RetryMaxDelay()) diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index 2a9dc5586355..039db9092168 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -431,7 +431,7 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 4294967252 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "attrs", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 4, "name": "capacity", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "available", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "used", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "logical_bytes", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "range_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "lease_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "writes_per_second", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "bytes_per_replica", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 12, "name": "writes_per_replica", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 13, "name": "metrics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 14, "name": "properties", "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967252, "name": "kv_store_status", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967253 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "network", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "attrs", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 5, "name": "locality", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "server_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "go_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "tag", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "time", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "revision", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "cgo_compiler", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "platform", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "distribution", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "dependencies", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "started_at", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 17, "name": "updated_at", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 18, "name": "metrics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 19, "name": "args", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 20, "name": "env", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 21, "name": "activity", "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967253, "name": "kv_node_status", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967254 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"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": 4294967254, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "system_jobs_id_idx", "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"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [12], "keyColumnNames": ["job_type"], "name": "system_jobs_job_type_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11], "storeColumnNames": ["id", "status", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 4, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["status"], "name": "system_jobs_status_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], "storeColumnNames": ["id", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run", "job_type"], "version": 3}], "name": "system_jobs", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 5, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967255 {"table": {"columns": [{"id": 1, "name": "job_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "job_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "descriptor_ids", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 7, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "running_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "created", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 10, "name": "started", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "finished", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "modified", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 13, "name": "fraction_completed", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "high_water_timestamp", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 15, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "coordinator_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "trace_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "last_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 19, "name": "next_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 20, "name": "num_runs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "execution_errors", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 22, "name": "execution_events", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967255, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [7], "keyColumnNames": ["status"], "name": "jobs_status_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22], "storeColumnNames": ["job_id", "job_type", "description", "statement", "user_name", "descriptor_ids", "running_status", "created", "started", "finished", "modified", "fraction_completed", "high_water_timestamp", "error", "coordinator_id", "trace_id", "last_run", "next_run", "num_runs", "execution_errors", "execution_events"], "version": 3}], "name": "jobs", "nextColumnId": 23, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967255 {"table": {"columns": [{"id": 1, "name": "job_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "job_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "descriptor_ids", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 7, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "running_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "created", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 10, "name": "started", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "finished", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "modified", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 13, "name": "fraction_completed", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "high_water_timestamp", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 15, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "coordinator_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "trace_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "last_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 19, "name": "next_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 20, "name": "num_runs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "execution_errors", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 22, "name": "execution_events", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967255, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [7], "keyColumnNames": ["status"], "name": "jobs_status_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22], "storeColumnNames": ["job_id", "job_type", "description", "statement", "user_name", "descriptor_ids", "running_status", "created", "started", "finished", "modified", "fraction_completed", "high_water_timestamp", "error", "coordinator_id", "trace_id", "last_run", "next_run", "num_runs", "execution_errors", "execution_events"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["job_type"], "name": "jobs_job_type_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22], "storeColumnNames": ["job_id", "description", "statement", "user_name", "descriptor_ids", "status", "running_status", "created", "started", "finished", "modified", "fraction_completed", "high_water_timestamp", "error", "coordinator_id", "trace_id", "last_run", "next_run", "num_runs", "execution_errors", "execution_events"], "version": 3}], "name": "jobs", "nextColumnId": 23, "nextConstraintId": 2, "nextIndexId": 4, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967256 {"table": {"columns": [{"id": 1, "name": "trace_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_span_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "span_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "goroutine_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "finished", "type": {"oid": 16}}, {"id": 6, "name": "start_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "duration", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 8, "name": "operation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967256, "name": "node_inflight_trace_spans", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967257 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "total_reads", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "last_read", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967257, "name": "index_usage_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967258 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967258, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["descriptor_id"], "name": "index_spans_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["index_id", "start_key", "end_key"], "version": 3}], "name": "index_spans", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} diff --git a/pkg/sql/opt/exec/execbuilder/testdata/virtual b/pkg/sql/opt/exec/execbuilder/testdata/virtual index 24ed374e59ad..be3a0eee20db 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/virtual +++ b/pkg/sql/opt/exec/execbuilder/testdata/virtual @@ -271,6 +271,19 @@ SELECT info FROM [EXPLAIN SELECT count(*) FROM crdb_internal.jobs WHERE status = table: jobs@jobs_status_idx spans: [/'paused' - /'paused'] +# Validate that the virtual index on 'job_type' works. +# Vectorized execution may be on or off during tests, so exclude it from the output. +query T +SELECT info FROM [EXPLAIN SELECT count(*) FROM crdb_internal.jobs WHERE job_type = 'CHANGEFEED'] WHERE info NOT LIKE 'vectorized%' +---- + distribution: local + · + • group (scalar) + │ + └── • virtual table + table: jobs@jobs_job_type_idx + spans: [/'CHANGEFEED' - /'CHANGEFEED'] + # Regression test for not projecting away looked up columns by the left semi # virtual lookup join (#91012). statement ok