Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
106130: sql: add extra information to protocol errors in bind r=rafiss a=cucaroach

A user is running into mysterious protocol errors when using prepared
statements. Add some extra information to the error message to help
guide the investigation.

Informs: https://github.com/cockroachlabs/support/issues/2184
Release note: none
Epic: none


107912: sql: fix exec+audit logs for BEGIN, COMMIT, SET stmts r=rafiss a=rafiss

Epic: None
Release note (bug fix): Fixed a bug where BEGIN, COMMIT, SET, ROLLBACK, and SAVEPOINT statements would not be written to the execution or audit logs.

108093: jobs: avoid crdb_internal.system_jobs in gc-jobs r=miretskiy a=stevendanna

The crdb_internal.system_jobs is a virtual table that joins information from the jobs table and the jobs_info table.

For the previous query,

    SELECT id, payload, status FROM "".crdb_internal.system_jobs
    WHERE (created < $1) AND (id > $2)
    ORDER BY id
    LIMIT $3

this is a little suboptimal because:

- We don't make use of the progress column so any read of that is useless.

- While the crdb_internal.virtual table has a virtual index on job id, and EXPLAIN will even claim that it will be used:

      • limit
      │ count: 100
      │
      └── • filter
          │ filter: created < '2023-07-20 07:29:01.17001'
          │
          └── • virtual table
                table: system_jobs@system_jobs_id_idx
                spans: [/101 - ]

  This is actually a lie. A virtual index can only handle single-key spans. As a result the unconstrained query is used:
```
    WITH
        latestpayload AS (SELECT job_id, value FROM system.job_info AS payload WHERE info_key = 'legacy_payload' ORDER BY written DESC),
        latestprogress AS (SELECT job_id, value FROM system.job_info AS progress WHERE info_key = 'legacy_progress' ORDER BY written DESC)
    SELECT
       DISTINCT(id), status, created, payload.value AS payload, progress.value AS progress,
                created_by_type, created_by_id, claim_session_id, claim_instance_id, num_runs, last_run, job_type
    FROM system.jobs AS j
    INNER JOIN latestpayload AS payload ON j.id = payload.job_id
    LEFT JOIN latestprogress AS progress ON j.id = progress.job_id
```
  which has a full scan of the jobs table and 2 full scans of the info table:

      • distinct
      │ distinct on: id, value, value
      │
      └── • merge join
          │ equality: (job_id) = (id)
          │
          ├── • render
          │   │
          │   └── • filter
          │       │ estimated row count: 7,318
          │       │ filter: info_key = 'legacy_payload'
          │       │
          │       └── • scan
          │             estimated row count: 14,648 (100% of the table; stats collected 39 minutes ago; using stats forecast for 2 hours in the future)
          │             table: job_info@primary
          │             spans: FULL SCAN
          │
          └── • merge join (right outer)
              │ equality: (job_id) = (id)
              │ right cols are key
              │
              ├── • render
              │   │
              │   └── • filter
              │       │ estimated row count: 7,317
              │       │ filter: info_key = 'legacy_progress'
              │       │
              │       └── • scan
              │             estimated row count: 14,648 (100% of the table; stats collected 39 minutes ago; using stats forecast for 2 hours in the future)
              │             table: job_info@primary
              │             spans: FULL SCAN
              │
              └── • scan
                    missing stats
                    table: jobs@primary
                    spans: FULL SCAN

  Because of the limit, I don't think this ends up being as bad as it looks. But it still isn't great.

In this PR, we replace crdb_internal.jobs with a query that removes the join on the unused progress field and also constrains the query of the job_info table.

      • distinct
      │ distinct on: id, value
      │
      └── • merge join
          │ equality: (job_id) = (id)
          │ right cols are key
          │
          ├── • render
          │   │
          │   └── • filter
          │       │ estimated row count: 7,318
          │       │ filter: info_key = 'legacy_payload'
          │       │
          │       └── • scan
          │             estimated row count: 14,646 (100% of the table; stats collected 45 minutes ago; using stats forecast for 2 hours in the future)
          │             table: job_info@primary
          │             spans: [/101/'legacy_payload' - ]
          │
          └── • render
              │
              └── • limit
                  │ count: 100
                  │
                  └── • filter
                      │ filter: created < '2023-07-20 07:29:01.17001'
                      │
                      └── • scan
                            missing stats
                            table: jobs@primary
                            spans: [/101 - ]

In a local example, this does seem faster:

    > SELECT id, payload, status, created
    > FROM "".crdb_internal.system_jobs
    > WHERE (created < '2023-07-20 07:29:01.17001') AND (id > 100) ORDER BY id LIMIT 100;

    id | payload | status | created
    -----+---------+--------+----------
    (0 rows)

    Time: 183ms total (execution 183ms / network 0ms)

    > WITH
    > latestpayload AS (
    >     SELECT job_id, value
    >     FROM system.job_info AS payload
    >     WHERE job_id > 100 AND info_key = 'legacy_payload'
    >     ORDER BY written desc
    > ),
    > jobpage AS (
    >     SELECT id, status, created
    >     FROM system.jobs
    >     WHERE (created < '2023-07-20 07:29:01.17001') and (id > 100)
    >     ORDER BY id
    >     LIMIT 100
    > )
    > SELECT distinct (id), latestpayload.value AS payload, status
    > FROM jobpage AS j
    > INNER JOIN latestpayload ON j.id = latestpayload.job_id;
      id | payload | status
    -----+---------+---------
    (0 rows)

    Time: 43ms total (execution 42ms / network 0ms)

Release note: None

Epic: none

108313: cli: debug zip uses read-only range probe r=joshimhoff a=dhartunian

To reduce the chance of corruption issues when run against older clusters, the recently added use of `crdb_internal.probe_ranges` is modified to use a `read` probe instead of a `write` probe.

See discussion in #107720

Release note: None
Epic: None

108335: kvcoord: Implement CloseStream for MuxRangeFeed r=miretskiy a=miretskiy

Extend MuxRangeFeed protocol to support explicit,
caller initiated CloseStream operation.

The caller may decide to stop receiving events
for a particular stream, which is part of MuxRangeFeed. The caller may issue a request to MuxRangeFeed server to close the stream.  The server will cancel underlying range feed, and return a `RangeFeedRetryError_REASON_RANGEFEED_CLOSED` error as a response.

Note, current mux rangefeed clinet does not use this request. The code to support cancellation is added pre-emptively in case this functionality will be required in the future to support restarts due to stuck rangefeeds.

Epic: CRDB-26372
Release note: None

108355: roachtest/multitenant-upgrade: hard-code predecessor version temporarily r=knz a=healthy-pod

Hard-code the pre-decessor release to 23.1.4 until a new patch release is out (23.1.9) because the test is in-compatible with 23.1.{5,6,7,8} due to an erroneous PR merged on the 23.1 branch.

Release note: None
Epic: none

108375: changefeedccl: Increase test utility timeout r=miretskiy a=miretskiy

As observed in #108348, a test failed because
it timed out reading a row.  Yet, this flake
could not be reproduced in over 50k runs.
Bump timeout period to make this flake even
less likely.

Fixes #108348

Release note: None

Co-authored-by: Tommy Reilly <treilly@cockroachlabs.com>
Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
Co-authored-by: Steven Danna <danna@cockroachlabs.com>
Co-authored-by: David Hartunian <davidh@cockroachlabs.com>
Co-authored-by: Yevgeniy Miretskiy <yevgeniy@cockroachlabs.com>
Co-authored-by: healthy-pod <ahmad@cockroachlabs.com>
  • Loading branch information
7 people committed Aug 8, 2023
8 parents 3ef6592 + ee3a9ab + ee05a80 + b9a92b1 + 0f44cac + 72a8b4e + ed2bad4 + a1cf907 commit 0d110cd
Show file tree
Hide file tree
Showing 39 changed files with 584 additions and 169 deletions.
2 changes: 1 addition & 1 deletion pkg/acceptance/testdata/node/base-test.js
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ describe('error cases', () => {
const cases = [{
name: 'not enough params',
query: { text: 'SELECT 3', values: ['foo'] },
msg: "bind message supplies 1 parameters, but prepared statement \"\" requires 0",
msg: "bind message supplies 1 parameters, but requires 0",
code: '08P01',
}, {
name: 'invalid utf8',
Expand Down
20 changes: 18 additions & 2 deletions pkg/ccl/auditloggingccl/audit_logging_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,6 +156,22 @@ func TestSingleRoleAuditLogging(t *testing.T) {
`GRANT SELECT ON TABLE u TO root`,
// DML statement
`SELECT * FROM u`,
// The following statements are all executed specially by the conn_executor.
`SET application_name = 'test'`,
`SET CLUSTER SETTING sql.defaults.vectorize = 'on'`,
`BEGIN`,
`SHOW application_name`,
`SAVEPOINT s`,
`RELEASE SAVEPOINT s`,
`SAVEPOINT t`,
`ROLLBACK TO SAVEPOINT t`,
`COMMIT`,
`SHOW COMMIT TIMESTAMP`,
`BEGIN TRANSACTION PRIORITY LOW`,
`ROLLBACK`,
`PREPARE q AS SELECT 1`,
`EXECUTE q`,
`DEALLOCATE q`,
}
testData := []struct {
name string
Expand All @@ -167,7 +183,7 @@ func TestSingleRoleAuditLogging(t *testing.T) {
name: "test-all-stmt-types",
role: allStmtTypesRole,
queries: testQueries,
expectedNumLogs: 3,
expectedNumLogs: len(testQueries),
},
{
name: "test-no-stmt-types",
Expand All @@ -181,7 +197,7 @@ func TestSingleRoleAuditLogging(t *testing.T) {
role: "testuser",
queries: testQueries,
// One for each test query.
expectedNumLogs: 3,
expectedNumLogs: len(testQueries),
},
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/cdctest/row.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ func MakeRangeFeedValueReader(
)
require.NoError(t, err)

var timeout = 5 * time.Second
var timeout = 10 * time.Second
if util.RaceEnabled {
timeout = 3 * timeout
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/zip/partial1
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null
[cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done
[cluster] retrieving SQL data for crdb_internal.kv_system_privileges... writing output: debug/crdb_internal.kv_system_privileges.txt... done
[cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_write_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_write_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_read_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_read_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.regions... writing output: debug/crdb_internal.regions.txt... done
[cluster] retrieving SQL data for crdb_internal.schema_changes... writing output: debug/crdb_internal.schema_changes.txt... done
[cluster] retrieving SQL data for crdb_internal.super_regions... writing output: debug/crdb_internal.super_regions.txt... done
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/zip/partial1_excluded
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0
[cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done
[cluster] retrieving SQL data for crdb_internal.kv_system_privileges... writing output: debug/crdb_internal.kv_system_privileges.txt... done
[cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_write_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_write_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_read_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_read_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.regions... writing output: debug/crdb_internal.regions.txt... done
[cluster] retrieving SQL data for crdb_internal.schema_changes... writing output: debug/crdb_internal.schema_changes.txt... done
[cluster] retrieving SQL data for crdb_internal.super_regions... writing output: debug/crdb_internal.super_regions.txt... done
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/zip/partial2
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null
[cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done
[cluster] retrieving SQL data for crdb_internal.kv_system_privileges... writing output: debug/crdb_internal.kv_system_privileges.txt... done
[cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_write_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_write_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_read_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_read_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.regions... writing output: debug/crdb_internal.regions.txt... done
[cluster] retrieving SQL data for crdb_internal.schema_changes... writing output: debug/crdb_internal.schema_changes.txt... done
[cluster] retrieving SQL data for crdb_internal.super_regions... writing output: debug/crdb_internal.super_regions.txt... done
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/zip/testzip
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null
[cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done
[cluster] retrieving SQL data for crdb_internal.kv_system_privileges... writing output: debug/crdb_internal.kv_system_privileges.txt... done
[cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_write_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_write_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_read_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_read_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.regions... writing output: debug/crdb_internal.regions.txt... done
[cluster] retrieving SQL data for crdb_internal.schema_changes... writing output: debug/crdb_internal.schema_changes.txt... done
[cluster] retrieving SQL data for crdb_internal.super_regions... writing output: debug/crdb_internal.super_regions.txt... done
Expand Down
6 changes: 3 additions & 3 deletions pkg/cli/testdata/zip/testzip_concurrent
Original file line number Diff line number Diff line change
Expand Up @@ -110,9 +110,9 @@ zip
[cluster] retrieving SQL data for crdb_internal.partitions...
[cluster] retrieving SQL data for crdb_internal.partitions: done
[cluster] retrieving SQL data for crdb_internal.partitions: writing output: debug/crdb_internal.partitions.txt...
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_write_limit_100...
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_write_limit_100: done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_write_limit_100: writing output: debug/crdb_internal.probe_ranges_1s_write_limit_100.txt...
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_read_limit_100...
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_read_limit_100: done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_read_limit_100: writing output: debug/crdb_internal.probe_ranges_1s_read_limit_100.txt...
[cluster] retrieving SQL data for crdb_internal.regions...
[cluster] retrieving SQL data for crdb_internal.regions: done
[cluster] retrieving SQL data for crdb_internal.regions: writing output: debug/crdb_internal.regions.txt...
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null
[cluster] retrieving SQL data for crdb_internal.kv_store_status: creating error output: debug/crdb_internal.kv_store_status.txt.err.txt... done
[cluster] retrieving SQL data for crdb_internal.kv_system_privileges... writing output: debug/crdb_internal.kv_system_privileges.txt... done
[cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_write_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_write_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_read_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_read_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.regions... writing output: debug/crdb_internal.regions.txt... done
[cluster] retrieving SQL data for crdb_internal.schema_changes... writing output: debug/crdb_internal.schema_changes.txt... done
[cluster] retrieving SQL data for crdb_internal.super_regions... writing output: debug/crdb_internal.super_regions.txt... done
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/zip/testzip_include_range_info
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s --include-range-info /dev/nu
[cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done
[cluster] retrieving SQL data for crdb_internal.kv_system_privileges... writing output: debug/crdb_internal.kv_system_privileges.txt... done
[cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_write_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_write_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_read_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_read_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.regions... writing output: debug/crdb_internal.regions.txt... done
[cluster] retrieving SQL data for crdb_internal.schema_changes... writing output: debug/crdb_internal.schema_changes.txt... done
[cluster] retrieving SQL data for crdb_internal.super_regions... writing output: debug/crdb_internal.super_regions.txt... done
Expand Down
4 changes: 2 additions & 2 deletions pkg/cli/testdata/zip/testzip_shared_process_virtualization
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null
[cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done
[cluster] retrieving SQL data for crdb_internal.kv_system_privileges... writing output: debug/crdb_internal.kv_system_privileges.txt... done
[cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_write_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_write_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_read_limit_100... writing output: debug/crdb_internal.probe_ranges_1s_read_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.regions... writing output: debug/crdb_internal.regions.txt... done
[cluster] retrieving SQL data for crdb_internal.schema_changes... writing output: debug/crdb_internal.schema_changes.txt... done
[cluster] retrieving SQL data for crdb_internal.super_regions... writing output: debug/crdb_internal.super_regions.txt... done
Expand Down Expand Up @@ -160,7 +160,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null
[cluster] retrieving SQL data for crdb_internal.kv_store_status: creating error output: debug/cluster/test-tenant/crdb_internal.kv_store_status.txt.err.txt... done
[cluster] retrieving SQL data for crdb_internal.kv_system_privileges... writing output: debug/cluster/test-tenant/crdb_internal.kv_system_privileges.txt... done
[cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/cluster/test-tenant/crdb_internal.partitions.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_write_limit_100... writing output: debug/cluster/test-tenant/crdb_internal.probe_ranges_1s_write_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.probe_ranges_1s_read_limit_100... writing output: debug/cluster/test-tenant/crdb_internal.probe_ranges_1s_read_limit_100.txt... done
[cluster] retrieving SQL data for crdb_internal.regions... writing output: debug/cluster/test-tenant/crdb_internal.regions.txt... done
[cluster] retrieving SQL data for crdb_internal.schema_changes... writing output: debug/cluster/test-tenant/crdb_internal.schema_changes.txt... done
[cluster] retrieving SQL data for crdb_internal.super_regions... writing output: debug/cluster/test-tenant/crdb_internal.super_regions.txt... done
Expand Down
13 changes: 10 additions & 3 deletions pkg/cli/zip_table_registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -228,9 +228,16 @@ var zipInternalTablesPerCluster = DebugZipTableRegistry{
origin
FROM crdb_internal.cluster_settings`,
},
"crdb_internal.probe_ranges_1s_write_limit_100": {
customQueryRedacted: `SELECT * FROM crdb_internal.probe_ranges(INTERVAL '1000ms', 'write') WHERE error != '' ORDER BY end_to_end_latency_ms DESC LIMIT 100;`,
customQueryUnredacted: `SELECT * FROM crdb_internal.probe_ranges(INTERVAL '1000ms', 'write') WHERE error != '' ORDER BY end_to_end_latency_ms DESC LIMIT 100;`,
"crdb_internal.probe_ranges_1s_read_limit_100": {
// At time of writing, it's considered very dangerous to use
// `write` as the argument to crdb_internal.probe_ranges due to
// this corruption bug:
// https://github.com/cockroachdb/cockroach/issues/101549 Since
// this fix is unevenly distributed in deployments it's not safe to
// indiscriminately run it from the CLI client on an arbitrary
// cluster.
customQueryRedacted: `SELECT * FROM crdb_internal.probe_ranges(INTERVAL '1000ms', 'read') WHERE error != '' ORDER BY end_to_end_latency_ms DESC LIMIT 100;`,
customQueryUnredacted: `SELECT * FROM crdb_internal.probe_ranges(INTERVAL '1000ms', 'read') WHERE error != '' ORDER BY end_to_end_latency_ms DESC LIMIT 100;`,
},
"crdb_internal.cluster_transactions": {
// `last_auto_retry_reason` contains error text that may contain
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/zip_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,7 @@ ORDER BY name ASC`)
assert.NoError(t, rows.Scan(&table))
tables = append(tables, table)
}
tables = append(tables, "crdb_internal.probe_ranges_1s_write_limit_100")
tables = append(tables, "crdb_internal.probe_ranges_1s_read_limit_100")
sort.Strings(tables)

var exp []string
Expand Down
21 changes: 19 additions & 2 deletions pkg/cmd/roachtest/tests/multitenant_upgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"context"
gosql "database/sql"
"fmt"
"strconv"
"strings"
"time"

Expand Down Expand Up @@ -82,9 +83,25 @@ func runMultiTenantUpgrade(
currentBinaryMinSupportedVersion, ok := versionToMinSupportedVersion[curBinaryMajorAndMinorVersion]
require.True(t, ok, "current binary '%s' not found in 'versionToMinSupportedVersion' map", curBinaryMajorAndMinorVersion)

predecessor, err := release.LatestPredecessor(v)
require.NoError(t, err)
getPredecessorVersion := func() string {
predecessor, err := release.LatestPredecessor(v)
require.NoError(t, err)

// Hard-code the pre-decessor release to 23.1.4 if 23.1.9 is not out yet because
// the test is in-compatible with 23.1.{5,6,7,8} due to an erroneous PR merged on the 23.1 branch.
// See https://github.com/cockroachdb/cockroach/pull/108202 for more context.
parsedPredecessor := strings.Split(predecessor, ".")
major := parsedPredecessor[0]
minor := parsedPredecessor[1]
patch, err := strconv.Atoi(parsedPredecessor[2])
require.NoError(t, err)
if major == "23" && minor == "1" && patch < 9 {
predecessor = "23.1.4"
}
return predecessor
}

predecessor := getPredecessorVersion()
currentBinary := uploadVersion(ctx, t, c, c.All(), clusterupgrade.MainVersion)
predecessorBinary := uploadVersion(ctx, t, c, c.All(), predecessor)

Expand Down
1 change: 0 additions & 1 deletion pkg/jobs/helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,6 @@ func (j *Job) TestingCurrentStatus(ctx context.Context) (Status, error) {
const (
AdoptQuery = claimQuery
CancelQuery = pauseAndCancelUpdate
GcQuery = expiredJobsQuery
RemoveClaimsQuery = removeClaimsForDeadSessionsQuery
ProcessJobsQuery = processQueryWithBackoff
IntervalBaseSettingKey = intervalBaseSettingKey
Expand Down
33 changes: 29 additions & 4 deletions pkg/jobs/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -1256,20 +1256,45 @@ func (r *Registry) cleanupOldJobs(ctx context.Context, olderThan time.Time) erro

// The ordering is important as we keep track of the maximum ID we've seen.
const expiredJobsQuery = `
SELECT id, payload, status, created FROM "".crdb_internal.system_jobs
SELECT id, payload, status FROM "".crdb_internal.system_jobs
WHERE (created < $1) AND (id > $2)
ORDER BY id
LIMIT $3
`
LIMIT $3`

const expiredJobsQueryWithJobInfoTable = `
WITH
latestpayload AS (
SELECT job_id, value
FROM system.job_info AS payload
WHERE job_id > $2 AND info_key = 'legacy_payload'
ORDER BY written desc
),
jobpage AS (
SELECT id, status
FROM system.jobs
WHERE (created < $1) and (id > $2)
ORDER BY id
LIMIT $3
)
SELECT distinct (id), latestpayload.value AS payload, status
FROM jobpage AS j
INNER JOIN latestpayload ON j.id = latestpayload.job_id`

// cleanupOldJobsPage deletes up to cleanupPageSize job rows with ID > minID.
// minID is supposed to be the maximum ID returned by the previous page (0 if no
// previous page).
func (r *Registry) cleanupOldJobsPage(
ctx context.Context, olderThan time.Time, minID jobspb.JobID, pageSize int,
) (done bool, maxID jobspb.JobID, retErr error) {
var query string
if r.settings.Version.IsActive(ctx, clusterversion.V23_1JobInfoTableIsBackfilled) {
query = expiredJobsQueryWithJobInfoTable
} else {
query = expiredJobsQuery
}

it, err := r.db.Executor().QueryIterator(ctx, "gc-jobs", nil, /* txn */
expiredJobsQuery, olderThan, minID, pageSize)
query, olderThan, minID, pageSize)
if err != nil {
return false, 0, err
}
Expand Down
Loading

0 comments on commit 0d110cd

Please sign in to comment.