Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
96725: sql: increase minimal max_range_size to 64MiB r=shralex a=shralex

We've seen that small range sizes can be detrimental to various components. This PR makes it so users can't lower max_range_size below 64MiB (half of the default min_range_size), instead of 64KiB previously.

Release note: Small ranges have been known to cause problems in various CRDB subsystems. This PR prevents setting max_range_size below COCKROACH_MIN_RANGE_MAX_BYTES, an environment variable which defaults to 64MiB (half of the default minimum range size). 

Epic: https://cockroachlabs.atlassian.net/browse/CRDB-24182
Fixes: #96549

98759: backupccl: create stripped crdb_internal.fingerprint overload r=stevendanna a=msbutler

This patch adds a variant of crdb_internal.fingerprint(), which creates a "stripped" fingerprint of the target span. Namely, `crdb_internal.fingerping(span,true)` will return a fingerprint that is agnostic to the mvcc timestamps and the index prefix of the key, and considers only the latest mvcc history of the key span.

For example, suppose the user fingerprinted a table at some system time, then backed up and restored it to that same system time. The restored table should have the same fingerprint!

This fingerprint variant is signicantly more scalable than SHOW EXPERIMENTAL FINGERPRINT, as it uses export requests compared to a simple scan.

Fixes #98570

Release note: None

99371: roachtest: lower default timeout r=srosenberg a=renatolabs

The previous default of 10 hours if unnecessarily long. As a consequence, if a test has a bug that causes it to hang, it will take 10h's worth of cluster usage, even if it's a test that generally succeeds in half an hour.

This should hopefully help with the timeouts we have been seeing in roachtest nightly runs.

Epic: none

Release note: None

Co-authored-by: shralex <[email protected]>
Co-authored-by: Michael Butler <[email protected]>
Co-authored-by: Renato Costa <[email protected]>
  • Loading branch information
4 people committed Mar 25, 2023
4 parents 0d84ad5 + a37e053 + cf91202 + 3950cab commit 86ad731
Show file tree
Hide file tree
Showing 43 changed files with 617 additions and 273 deletions.
2 changes: 2 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -3127,6 +3127,8 @@ may increase either contention or retry errors, or both.</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="crdb_internal.fingerprint"></a><code>crdb_internal.fingerprint(span: <a href="bytes.html">bytes</a>[], start_time: <a href="timestamp.html">timestamptz</a>, all_revisions: <a href="bool.html">bool</a>) &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>This function is used only by CockroachDB’s developers for testing purposes.</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="crdb_internal.fingerprint"></a><code>crdb_internal.fingerprint(span: <a href="bytes.html">bytes</a>[], stripped: <a href="bool.html">bool</a>) &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>This function is used only by CockroachDB’s developers for testing purposes.</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="crdb_internal.force_assertion_error"></a><code>crdb_internal.force_assertion_error(msg: <a href="string.html">string</a>) &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>This function is used only by CockroachDB’s developers for testing purposes.</p>
</span></td><td>Volatile</td></tr>
<tr><td><a name="crdb_internal.force_error"></a><code>crdb_internal.force_error(errorCode: <a href="string.html">string</a>, msg: <a href="string.html">string</a>) &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>This function is used only by CockroachDB’s developers for testing purposes.</p>
Expand Down
3 changes: 0 additions & 3 deletions pkg/base/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,9 +50,6 @@ const (
// when a job opts in to dumping its execution traces.
InflightTraceDir = "inflight_trace_dump"

// MinRangeMaxBytes is the minimum value for range max bytes.
MinRangeMaxBytes = 64 << 10 // 64 KB

// ObsServiceEmbedFlagValue is the special value of the --obsservice-addr flag
// configuring the CRDB node to run the Obs Service internally.
ObsServiceEmbedFlagValue = "embed"
Expand Down
2 changes: 1 addition & 1 deletion pkg/bench/rttanalysis/alter_table_bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,7 @@ func init() {
Name: "alter table configure zone ranges",
Setup: `CREATE TABLE alter_table(a INT);`,
Stmt: "ALTER TABLE alter_table CONFIGURE ZONE USING " +
"range_min_bytes = 0, range_max_bytes = 90000",
"range_min_bytes = 0, range_max_bytes = 500000000",
},
})
}
11 changes: 8 additions & 3 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1080,6 +1080,8 @@ SELECT payload FROM "".crdb_internal.system_jobs ORDER BY created DESC LIMIT 10

sqlDB.Exec(t, incBackupQuery, queryArgs...)
}
bankTableID := sqlutils.QueryTableID(t, conn, "data", "public", "bank")
backupTableFingerprint := sqlutils.FingerprintTable(t, sqlDB, bankTableID)

sqlDB.Exec(t, `DROP DATABASE data CASCADE`)

Expand All @@ -1100,7 +1102,7 @@ SELECT payload FROM "".crdb_internal.system_jobs ORDER BY created DESC LIMIT 10
restoreQuery = fmt.Sprintf("%s WITH kms = %s", restoreQuery, kmsURIFmtString)
}
queryArgs := append(restoreURIArgs, kmsURIArgs...)
verifyRestoreData(t, sqlDB, storageSQLDB, restoreQuery, queryArgs, numAccounts)
verifyRestoreData(t, sqlDB, storageSQLDB, restoreQuery, queryArgs, numAccounts, backupTableFingerprint)
}

func verifyRestoreData(
Expand All @@ -1110,6 +1112,7 @@ func verifyRestoreData(
restoreQuery string,
restoreURIArgs []interface{},
numAccounts int,
bankStrippedFingerprint int,
) {
var unused string
var restored struct {
Expand Down Expand Up @@ -1160,6 +1163,8 @@ func verifyRestoreData(
t.Fatal("unexpected span start at primary index")
}
}
restorebankID := sqlutils.QueryTableID(t, sqlDB.DB, "data", "public", "bank")
require.Equal(t, bankStrippedFingerprint, sqlutils.FingerprintTable(t, sqlDB, restorebankID))
}

func TestBackupRestoreSystemTables(t *testing.T) {
Expand Down Expand Up @@ -9417,7 +9422,7 @@ func TestExportRequestBelowGCThresholdOnDataExcludedFromBackup(t *testing.T) {
_, err = conn.Exec("SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'") // speeds up the test
require.NoError(t, err)

const tableRangeMaxBytes = 1 << 18
const tableRangeMaxBytes = 100 << 20
_, err = conn.Exec("ALTER TABLE foo CONFIGURE ZONE USING "+
"gc.ttlseconds = 1, range_max_bytes = $1, range_min_bytes = 1<<10;", tableRangeMaxBytes)
require.NoError(t, err)
Expand Down Expand Up @@ -9501,7 +9506,7 @@ func TestExcludeDataFromBackupDoesNotHoldupGC(t *testing.T) {
// Exclude the table from backup so that it does not hold up GC.
runner.Exec(t, `ALTER TABLE test.foo SET (exclude_data_from_backup = true)`)

const tableRangeMaxBytes = 1 << 18
const tableRangeMaxBytes = 100 << 20
runner.Exec(t, "ALTER TABLE test.foo CONFIGURE ZONE USING "+
"gc.ttlseconds = 1, range_max_bytes = $1, range_min_bytes = 1<<10;", tableRangeMaxBytes)

Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/backuprand/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ go_test(
"//pkg/util/log",
"//pkg/util/randutil",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
],
)

Expand Down
9 changes: 6 additions & 3 deletions pkg/ccl/backupccl/backuprand/backup_rand_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)

// TestBackupRestoreRandomDataRoundtrips conducts backup/restore roundtrips on
Expand Down Expand Up @@ -97,12 +98,13 @@ database_name = 'rand' AND schema_name = 'public'`)
}

expectedCreateTableStmt := make(map[string]string)
expectedData := make(map[string][][]string)
expectedData := make(map[string]int)
for _, tableName := range tableNames {
expectedCreateTableStmt[tableName] = sqlDB.QueryStr(t,
fmt.Sprintf(`SELECT create_statement FROM [SHOW CREATE TABLE %s]`, tree.NameString(tableName)))[0][0]
if runSchemaOnlyExtension == "" {
expectedData[tableName] = sqlDB.QueryStr(t, fmt.Sprintf(`SELECT * FROM %s`, tree.NameString(tableName)))
tableID := sqlutils.QueryTableID(t, sqlDB.DB, "rand", "public", tableName)
expectedData[tableName] = sqlutils.FingerprintTable(t, sqlDB, tableID)
}
}

Expand Down Expand Up @@ -135,7 +137,8 @@ database_name = 'rand' AND schema_name = 'public'`)
assert.Equal(t, expectedCreateTableStmt[tableName], createStmt,
"SHOW CREATE %s not equal after RESTORE", tableName)
if runSchemaOnlyExtension == "" {
sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT * FROM %s`, restoreTable), expectedData[tableName])
tableID := sqlutils.QueryTableID(t, sqlDB.DB, "restoredb", "public", tableName)
require.Equal(t, expectedData[tableName], sqlutils.FingerprintTable(t, sqlDB, tableID))
} else {
sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT count(*) FROM %s`, restoreTable),
[][]string{{"0"}})
Expand Down
17 changes: 7 additions & 10 deletions pkg/ccl/backupccl/utils_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/workload/workloadsql"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -629,18 +630,14 @@ func upsertUntilBackpressure(
t *testing.T, rRand *rand.Rand, conn *gosql.DB, database, table string,
) {
t.Helper()
testutils.SucceedsSoon(t, func() error {
for i := 1; i < 50; i++ {
_, err := conn.Exec(fmt.Sprintf("UPSERT INTO %s.%s VALUES (1, $1)", database, table),
randutil.RandBytes(rRand, 1<<15))
if err == nil {
return errors.New("expected `backpressure` error")
}

if !testutils.IsError(err, "backpressure") {
return errors.NewAssertionErrorWithWrappedErrf(err, "expected `backpressure` error")
randutil.RandBytes(rRand, 5<<20))
if testutils.IsError(err, "backpressure") {
return
}
return nil
})
}
assert.Fail(t, "expected `backpressure` error")
}

// requireRecoveryEvent fetches all available log entries on disk after
Expand Down
10 changes: 5 additions & 5 deletions pkg/ccl/logictestccl/testdata/logic_test/distsql_partitioning
Original file line number Diff line number Diff line change
Expand Up @@ -345,13 +345,13 @@ statement ok
ALTER INDEX partitioning.inheritance@inheritance_pkey PARTITION BY LIST (x) ( PARTITION p1 VALUES IN (1))

statement ok
ALTER DATABASE partitioning CONFIGURE ZONE USING range_min_bytes=64000, range_max_bytes=75000
ALTER DATABASE partitioning CONFIGURE ZONE USING range_min_bytes=64000, range_max_bytes=75000000

query TTTTTTTTT
SHOW PARTITIONS FROM TABLE partitioning.inheritance
----
partitioning inheritance p1 NULL x inheritance@inheritance_pkey (1) NULL range_min_bytes = 64000,
range_max_bytes = 75000,
range_max_bytes = 75000000,
gc.ttlseconds = 14400,
num_replicas = 3,
constraints = '[]',
Expand All @@ -364,7 +364,7 @@ query TTTTTTTTT
SHOW PARTITIONS FROM TABLE partitioning.inheritance
----
partitioning inheritance p1 NULL x inheritance@inheritance_pkey (1) NULL range_min_bytes = 64000,
range_max_bytes = 75000,
range_max_bytes = 75000000,
gc.ttlseconds = 80000,
num_replicas = 3,
constraints = '[]',
Expand All @@ -377,7 +377,7 @@ query TTTTTTTTT
SHOW PARTITIONS FROM TABLE partitioning.inheritance
----
partitioning inheritance p1 NULL x inheritance@inheritance_pkey (1) NULL range_min_bytes = 64000,
range_max_bytes = 75000,
range_max_bytes = 75000000,
gc.ttlseconds = 80000,
num_replicas = 5,
constraints = '[]',
Expand All @@ -390,7 +390,7 @@ query TTTTTTTTT
SHOW PARTITIONS FROM TABLE partitioning.inheritance
----
partitioning inheritance p1 NULL x inheritance@inheritance_pkey (1) constraints = '[+dc=dc1]' range_min_bytes = 64000,
range_max_bytes = 75000,
range_max_bytes = 75000000,
gc.ttlseconds = 80000,
num_replicas = 5,
constraints = '[+dc=dc1]',
Expand Down
36 changes: 18 additions & 18 deletions pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs
Original file line number Diff line number Diff line change
Expand Up @@ -123,14 +123,14 @@ statement ok
ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING gc.ttlseconds = 5

statement ok
ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING range_min_bytes = 1000, range_max_bytes = 100000
ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING range_min_bytes = 1000, range_max_bytes = 100000000

query TT
SHOW ZONE CONFIGURATION FOR DATABASE "mr-zone-configs"
----
DATABASE "mr-zone-configs" ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 5,
num_replicas = 5,
num_voters = 3,
Expand All @@ -149,7 +149,7 @@ SHOW ZONE CONFIGURATION FOR DATABASE "mr-zone-configs"
----
DATABASE "mr-zone-configs" ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 5,
num_replicas = 5,
num_voters = 3,
Expand All @@ -170,7 +170,7 @@ SHOW ZONE CONFIGURATION FOR DATABASE "mr-zone-configs"
----
DATABASE "mr-zone-configs" ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 5,
num_replicas = 5,
num_voters = 5,
Expand Down Expand Up @@ -243,7 +243,7 @@ SHOW ZONE CONFIGURATION FOR DATABASE "mr-zone-configs"
----
DATABASE "mr-zone-configs" ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 5,
num_replicas = 5,
num_voters = 3,
Expand Down Expand Up @@ -302,7 +302,7 @@ SHOW ZONE CONFIGURATION FOR DATABASE "mr-zone-configs"
----
DATABASE "mr-zone-configs" ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 100000,
num_replicas = 4,
num_voters = 3,
Expand All @@ -323,7 +323,7 @@ SHOW ZONE CONFIGURATION FOR DATABASE "mr-zone-configs"
----
DATABASE "mr-zone-configs" ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 100000,
num_replicas = 3,
num_voters = 3,
Expand All @@ -344,7 +344,7 @@ SHOW ZONE CONFIGURATION FOR DATABASE "mr-zone-configs"
----
DATABASE "mr-zone-configs" ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 100000,
num_replicas = 3,
num_voters = 3,
Expand All @@ -365,7 +365,7 @@ SHOW ZONE CONFIGURATION FOR DATABASE "mr-zone-configs"
----
DATABASE "mr-zone-configs" ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 100000,
num_replicas = 3,
constraints = '[]',
Expand Down Expand Up @@ -443,7 +443,7 @@ SHOW ZONE CONFIGURATION FOR TABLE regional_by_row
----
TABLE regional_by_row ALTER TABLE regional_by_row CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 10,
num_replicas = 10,
num_voters = 3,
Expand Down Expand Up @@ -504,7 +504,7 @@ SHOW ZONE CONFIGURATION FOR INDEX regional_by_row@regional_by_row_pkey
----
TABLE regional_by_row ALTER TABLE regional_by_row CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 10,
num_replicas = 5,
num_voters = 3,
Expand All @@ -517,7 +517,7 @@ SHOW ZONE CONFIGURATION FOR INDEX regional_by_row@regional_by_row_i_idx
----
TABLE regional_by_row ALTER TABLE regional_by_row CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 10,
num_replicas = 5,
num_voters = 3,
Expand All @@ -530,7 +530,7 @@ SHOW ZONE CONFIGURATION FOR TABLE regional_by_row
----
TABLE regional_by_row ALTER TABLE regional_by_row CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 10,
num_replicas = 5,
num_voters = 3,
Expand Down Expand Up @@ -598,7 +598,7 @@ SHOW ZONE CONFIGURATION FOR TABLE regional_by_row
----
TABLE regional_by_row ALTER TABLE regional_by_row CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 100000,
global_reads = false,
num_replicas = 5,
Expand All @@ -617,7 +617,7 @@ SHOW ZONE CONFIGURATION FOR TABLE regional_by_row
----
TABLE regional_by_row ALTER TABLE regional_by_row CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 100000,
global_reads = true,
num_replicas = 5,
Expand Down Expand Up @@ -667,7 +667,7 @@ SHOW ZONE CONFIGURATION FOR INDEX regional_by_row_as@regional_by_row_as_pkey
----
INDEX regional_by_row_as@regional_by_row_as_pkey ALTER INDEX regional_by_row_as@regional_by_row_as_pkey CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 100000,
num_replicas = 10,
num_voters = 3,
Expand All @@ -688,7 +688,7 @@ SHOW ZONE CONFIGURATION FOR INDEX regional_by_row_as@regional_by_row_as_pkey
----
DATABASE "mr-zone-configs" ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 100000,
num_replicas = 5,
num_voters = 3,
Expand Down Expand Up @@ -737,7 +737,7 @@ SHOW ZONE CONFIGURATION FOR DATABASE "mr-zone-configs"
----
DATABASE "mr-zone-configs" ALTER DATABASE "mr-zone-configs" CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 100000,
num_replicas = 5,
num_voters = 3,
Expand Down
9 changes: 6 additions & 3 deletions pkg/ccl/partitionccl/drop_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -217,7 +217,8 @@ SELECT job_id
PARTITION c VALUES IN ('c')
)`)
tdb.Exec(t, `CREATE INDEX idx ON t (e)`)
tdb.Exec(t, `ALTER PARTITION a OF TABLE t CONFIGURE ZONE USING range_min_bytes = 123456, range_max_bytes = 654321`)
tdb.Exec(t, `ALTER PARTITION a OF TABLE t CONFIGURE ZONE USING range_min_bytes = 123456,
range_max_bytes = 654321000`)
tdb.Exec(t, `ALTER INDEX t@idx CONFIGURE ZONE USING gc.ttlseconds = 1`)
tdb.Exec(t, `DROP INDEX t@idx`)

Expand Down Expand Up @@ -271,8 +272,10 @@ SELECT job_id
PARTITION ci VALUES IN ('c')
)`,
)
tdb.Exec(t, `ALTER PARTITION ai OF INDEX t@idx CONFIGURE ZONE USING range_min_bytes = 123456,range_max_bytes = 654321`)
tdb.Exec(t, `ALTER PARTITION a OF TABLE t CONFIGURE ZONE USING range_min_bytes = 123456, range_max_bytes = 654321`)
tdb.Exec(t, `ALTER PARTITION ai OF INDEX t@idx CONFIGURE ZONE USING range_min_bytes = 123456,
range_max_bytes = 654321000`)
tdb.Exec(t, `ALTER PARTITION a OF TABLE t CONFIGURE ZONE USING range_min_bytes = 123456,
range_max_bytes = 654321000`)
tdb.Exec(t, `ALTER INDEX t@idx CONFIGURE ZONE USING gc.ttlseconds = 1`)
tdb.Exec(t, `DROP INDEX t@idx`)
tdb.Exec(t, `DROP TABLE t`)
Expand Down
Loading

0 comments on commit 86ad731

Please sign in to comment.