Skip to content

Commit

Permalink
kv: increasing the minimal max_range_size to 64MiB
Browse files Browse the repository at this point in the history
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.

Epic: https://cockroachlabs.atlassian.net/browse/CRDB-24182
Fixes: #96549
  • Loading branch information
shralex committed Feb 10, 2023
1 parent f640acb commit 938d746
Show file tree
Hide file tree
Showing 17 changed files with 107 additions and 93 deletions.
2 changes: 1 addition & 1 deletion pkg/base/constants.go
Expand Up @@ -51,5 +51,5 @@ const (
InflightTraceDir = "inflight_trace_dump"

// MinRangeMaxBytes is the minimum value for range max bytes.
MinRangeMaxBytes = 64 << 10 // 64 KB
MinRangeMaxBytes = 64 << 20 // 64 MiB, half of the default range_min_bytes
)
2 changes: 1 addition & 1 deletion pkg/bench/rttanalysis/alter_table_bench_test.go
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",
},
})
}
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/backup_test.go
Expand Up @@ -9325,7 +9325,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 @@ -9409,7 +9409,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
17 changes: 7 additions & 10 deletions pkg/ccl/backupccl/utils_test.go
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 @@ -623,18 +624,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
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
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
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
Expand Up @@ -71,15 +71,15 @@ translate database=db table=t
# Configure a zone config field on the table, so that it is no longer a
# placeholder zone config.
exec-sql
ALTER TABLE db.t CONFIGURE ZONE USING range_min_bytes = 1000, range_max_bytes=100000;
ALTER TABLE db.t CONFIGURE ZONE USING range_min_bytes = 1000, range_max_bytes=100000000;
----

query-sql
SHOW ZONE CONFIGURATION FOR INDEX db.t@idx
----
INDEX db.public.t@idx ALTER INDEX db.public.t@idx CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 25,
num_replicas = 7,
num_voters = 5,
Expand All @@ -89,6 +89,6 @@ INDEX db.public.t@idx ALTER INDEX db.public.t@idx CONFIGURE ZONE USING

translate database=db table=t
----
/Table/106{-/2} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Table/106/{2-3} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5
/Table/10{6/3-7} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Table/106{-/2} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Table/106/{2-3} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5
/Table/10{6/3-7} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
Expand Up @@ -71,15 +71,15 @@ translate database=db table=t
# Configure a zone config field on the table, so that it is no longer a
# placeholder zone config.
exec-sql
ALTER TABLE db.t CONFIGURE ZONE USING range_min_bytes = 1000, range_max_bytes=100000;
ALTER TABLE db.t CONFIGURE ZONE USING range_min_bytes = 1000, range_max_bytes=100000000;
----

query-sql
SHOW ZONE CONFIGURATION FOR INDEX db.t@idx
----
INDEX db.public.t@idx ALTER INDEX db.public.t@idx CONFIGURE ZONE USING
range_min_bytes = 1000,
range_max_bytes = 100000,
range_max_bytes = 100000000,
gc.ttlseconds = 25,
num_replicas = 7,
num_voters = 5,
Expand All @@ -89,9 +89,9 @@ INDEX db.public.t@idx ALTER INDEX db.public.t@idx CONFIGURE ZONE USING

translate database=db table=t
----
/Tenant/10/Table/106{-/2} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{2-3} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5
/Tenant/10/Table/10{6/3-7} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106{-/2} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{2-3} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5
/Tenant/10/Table/10{6/3-7} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7

block-gc-jobs
----
Expand All @@ -105,12 +105,12 @@ ALTER INDEX db.t@idx2 CONFIGURE ZONE USING gc.ttlseconds = 1;
# Both the newly added index and the temporary index have the configured zone configuration.
translate database=db table=t
----
/Tenant/10/Table/106{-/2} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{2-3} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5
/Tenant/10/Table/106/{3-4} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{4-5} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=1 num_replicas=7
/Tenant/10/Table/106/{5-6} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=1 num_replicas=7
/Tenant/10/Table/10{6/6-7} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106{-/2} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{2-3} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5
/Tenant/10/Table/106/{3-4} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{4-5} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=1 num_replicas=7
/Tenant/10/Table/106/{5-6} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=1 num_replicas=7
/Tenant/10/Table/10{6/6-7} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7

unblock-gc-jobs
----
Expand All @@ -122,11 +122,11 @@ SHOW JOBS WHEN COMPLETE (SELECT job_id FROM [SHOW JOBS])
# The zone configuration for the temporary index is cleaned up
translate database=db table=t
----
/Tenant/10/Table/106{-/2} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{2-3} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5
/Tenant/10/Table/106/{3-4} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{4-5} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=1 num_replicas=7
/Tenant/10/Table/10{6/5-7} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106{-/2} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{2-3} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5
/Tenant/10/Table/106/{3-4} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{4-5} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=1 num_replicas=7
/Tenant/10/Table/10{6/5-7} range_max_bytes=100000000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7

# Create and drop an index inside the same transaction. The related
# zone configuration should also be cleaned up.
Expand Down

0 comments on commit 938d746

Please sign in to comment.