diff --git a/pkg/ccl/logictestccl/testdata/logic_test/multi_region b/pkg/ccl/logictestccl/testdata/logic_test/multi_region index 176bb8ae8257..0692a05fd303 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/multi_region +++ b/pkg/ccl/logictestccl/testdata/logic_test/multi_region @@ -1015,7 +1015,7 @@ statement ok CREATE DATABASE drop_region_db PRIMARY REGION "ca-central-1" REGIONS "ap-southeast-2", "us-east-1"; USE drop_region_db -statement error pq: cannot drop region "ca-central-1"\nHINT: You must designate another region as the primary region using ALTER DATABASE drop_region_db PRIMARY REGION or remove all other regions before attempting to drop region "ca-central-1" +statement error pgcode 42P12 cannot drop region "ca-central-1"\nHINT: You must designate another region as the primary region using ALTER DATABASE drop_region_db PRIMARY REGION or remove all other regions before attempting to drop region "ca-central-1" ALTER DATABASE drop_region_db DROP REGION "ca-central-1" statement ok @@ -1108,7 +1108,7 @@ CREATE TABLE start_off_non_multi_region.public.t(a INT); ALTER DATABASE start_off_non_multi_region PRIMARY REGION "ca-central-1"; ALTER DATABASE start_off_non_multi_region ADD REGION "ap-southeast-2" -statement error pq: cannot drop region "ca-central-1"\nHINT: You must designate another region as the primary region using ALTER DATABASE start_off_non_multi_region PRIMARY REGION or remove all other regions before attempting to drop region "ca-central-1" +statement error pgcode 42P12 cannot drop region "ca-central-1"\nHINT: You must designate another region as the primary region using ALTER DATABASE start_off_non_multi_region PRIMARY REGION or remove all other regions before attempting to drop region "ca-central-1" ALTER DATABASE start_off_non_multi_region DROP REGION "ca-central-1" statement ok @@ -1203,7 +1203,7 @@ statement ok ALTER DATABASE drop_primary_regions_db DROP REGION "ca-central-1" # Cannot drop the primary region yet, as there are other regions in the db. -statement error pq: cannot drop region "us-east-1"\nHINT: You must designate another region as the primary region using ALTER DATABASE drop_primary_regions_db PRIMARY REGION or remove all other regions before attempting to drop region "us-east-1" +statement error pgcode 42P12 cannot drop region "us-east-1"\nHINT: You must designate another region as the primary region using ALTER DATABASE drop_primary_regions_db PRIMARY REGION or remove all other regions before attempting to drop region "us-east-1" ALTER DATABASE drop_primary_regions_db DROP REGION "us-east-1" statement ok diff --git a/pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs index bbf0f73406b8..1a58b4d1c307 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs +++ b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs @@ -484,13 +484,13 @@ SET override_multi_region_zone_config = true; ALTER index regional_by_row@primary CONFIGURE ZONE USING num_replicas = 10; SET override_multi_region_zone_config = false -statement error attempting to update zone config which contains an extra zone configuration for index regional_by_row@"primary" +statement error attempting to update zone config which contains an extra zone configuration for index regional_by_row@"primary" with field num_replicas populated ALTER TABLE regional_by_row SET LOCALITY GLOBAL -statement error extraneous zone configuration for index regional_by_row@"primary" +statement error extraneous zone configuration for index regional_by_row@"primary" with field num_replicas populated SELECT crdb_internal.validate_multi_region_zone_configs() -statement error attempting to update zone config which contains an extra zone configuration for index regional_by_row@"primary" +statement error attempting to update zone config which contains an extra zone configuration for index regional_by_row@"primary" with field num_replicas populated ALTER TABLE regional_by_row SET LOCALITY REGIONAL BY ROW statement ok @@ -525,7 +525,7 @@ INDEX regional_by_row_as@primary ALTER INDEX regional_by_row_as@primary CONFIGU voter_constraints = '[+region=us-east-1]', lease_preferences = '[[+region=us-east-1]]' -statement error attempting to update zone config which contains an extra zone configuration for index regional_by_row_as@"primary" +statement error attempting to update zone config which contains an extra zone configuration for index regional_by_row_as@"primary" with field num_replicas populated ALTER TABLE regional_by_row_as SET LOCALITY REGIONAL BY ROW statement ok @@ -629,3 +629,79 @@ statement ok SET override_multi_region_zone_config = true; ALTER PARTITION "ca-central-1" OF INDEX regional_by_row@primary CONFIGURE ZONE DISCARD; SET override_multi_region_zone_config = false + +# Test validation for initial SET PRIMARY REGION +statement ok +CREATE DATABASE initial_multiregion_db; +USE initial_multiregion_db; +CREATE TABLE tbl (a INT PRIMARY KEY, INDEX a_idx (a)); +ALTER DATABASE initial_multiregion_db CONFIGURE ZONE USING gc.ttlseconds = 5; +ALTER TABLE tbl CONFIGURE ZONE USING gc.ttlseconds = 5; +ALTER INDEX tbl@a_idx CONFIGURE ZONE USING gc.ttlseconds = 5 + +statement ok +ALTER DATABASE initial_multiregion_db SET PRIMARY REGION "us-east-1" + +query TT +SHOW ZONE CONFIGURATION FOR DATABASE initial_multiregion_db +---- +DATABASE initial_multiregion_db ALTER DATABASE initial_multiregion_db CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 5, + num_replicas = 3, + num_voters = 3, + constraints = '{+region=us-east-1: 1}', + voter_constraints = '[+region=us-east-1]', + lease_preferences = '[[+region=us-east-1]]' + +query TT +SHOW ZONE CONFIGURATION FOR TABLE tbl +---- +TABLE tbl ALTER TABLE tbl CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 5, + num_replicas = 3, + num_voters = 3, + constraints = '{+region=us-east-1: 1}', + voter_constraints = '[+region=us-east-1]', + lease_preferences = '[[+region=us-east-1]]' + +query TT +SHOW ZONE CONFIGURATION FOR INDEX tbl@a_idx +---- +INDEX tbl@a_idx ALTER INDEX tbl@a_idx CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 5, + num_replicas = 3, + num_voters = 3, + constraints = '{+region=us-east-1: 1}', + voter_constraints = '[+region=us-east-1]', + lease_preferences = '[[+region=us-east-1]]' + +statement ok +ALTER DATABASE initial_multiregion_db DROP REGION "us-east-1"; +ALTER INDEX tbl@a_idx CONFIGURE ZONE USING num_replicas = 10 + +statement error zone configuration for index tbl@a_idx has field "num_replicas" set which will be overwritten when setting the initial PRIMARY REGION\nHINT: discard the zone config using CONFIGURE ZONE DISCARD before continuing +ALTER DATABASE initial_multiregion_db SET PRIMARY REGION "us-east-1" + +statement ok +ALTER INDEX tbl@a_idx CONFIGURE ZONE DISCARD; +ALTER TABLE tbl CONFIGURE ZONE USING num_replicas = 10 + +statement error zone configuration for table tbl has field "num_replicas" set which will be overwritten when setting the the initial PRIMARY REGION\nHINT: discard the zone config using CONFIGURE ZONE DISCARD before continuing +ALTER DATABASE initial_multiregion_db SET PRIMARY REGION "us-east-1" + +statement ok +ALTER TABLE tbl CONFIGURE ZONE DISCARD; +ALTER DATABASE initial_multiregion_db CONFIGURE ZONE USING num_replicas = 10; + +statement error zone configuration for database initial_multiregion_db has field "num_replicas" set which will be overwritten when setting the the initial PRIMARY REGION\nHINT: discard the zone config using CONFIGURE ZONE DISCARD before continuing +ALTER DATABASE initial_multiregion_db SET PRIMARY REGION "us-east-1" + +statement ok +ALTER DATABASE initial_multiregion_db CONFIGURE ZONE DISCARD; +ALTER DATABASE initial_multiregion_db SET PRIMARY REGION "us-east-1" diff --git a/pkg/col/coldata/BUILD.bazel b/pkg/col/coldata/BUILD.bazel index b28a0bdc2c7d..9153b4693c4a 100644 --- a/pkg/col/coldata/BUILD.bazel +++ b/pkg/col/coldata/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/sql/types", "//pkg/util", "//pkg/util/duration", + "//pkg/util/uuid", "@com_github_cockroachdb_apd_v2//:apd", "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", diff --git a/pkg/col/coldata/bytes.go b/pkg/col/coldata/bytes.go index 6be38f46979a..9789d12840ab 100644 --- a/pkg/col/coldata/bytes.go +++ b/pkg/col/coldata/bytes.go @@ -45,12 +45,24 @@ const BytesInitialAllocationFactor = 64 // NewBytes returns a Bytes struct with enough capacity for n zero-length // []byte values. It is legal to call Set on the returned Bytes at this point, // but Get is undefined until at least one element is Set. +// BytesInitialAllocationFactor number of bytes are allocated initially for each +// []byte element. func NewBytes(n int) *Bytes { + return NewBytesWithAvgLength(n, BytesInitialAllocationFactor) +} + +// NewBytesWithAvgLength returns a Bytes struct with enough capacity for n +// []byte values with the average length of avgElementLength. It is legal to +// call Set on the returned Bytes at this point, but Get is undefined until at +// least one element is Set. +// - avgElementLength determines the average length of a single []byte element +// that will be added to this Bytes. +func NewBytesWithAvgLength(n int, avgElementLength int) *Bytes { return &Bytes{ // Given that the []byte slices are of variable length, we multiply the // number of elements by some constant factor. // TODO(asubiotto): Make this tunable. - data: make([]byte, 0, n*BytesInitialAllocationFactor), + data: make([]byte, 0, n*avgElementLength), offsets: make([]int32, n+1), } } diff --git a/pkg/col/coldata/vec.go b/pkg/col/coldata/vec.go index e239817e1be9..e8ca71bc2357 100644 --- a/pkg/col/coldata/vec.go +++ b/pkg/col/coldata/vec.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/uuid" ) // Column is an interface that represents a raw array of a Go native type. @@ -162,6 +163,9 @@ func (cf *defaultColumnFactory) MakeColumn(t *types.T, length int) Column { case types.BoolFamily: return make(Bools, length) case types.BytesFamily: + if t.Family() == types.UuidFamily { + return NewBytesWithAvgLength(length, uuid.Size) + } return NewBytes(length) case types.IntFamily: switch t.Width() { diff --git a/pkg/config/zonepb/zone.go b/pkg/config/zonepb/zone.go index 670c0c10f8b5..b584ccc0d9e3 100644 --- a/pkg/config/zonepb/zone.go +++ b/pkg/config/zonepb/zone.go @@ -611,8 +611,6 @@ type DiffWithZoneMismatch struct { // PartitionName represents a subzone with a mismatching partitionName. PartitionName string - // NOTE: only one of the below fields is set. - // IsMissingSubzone indicates a subzone is missing. IsMissingSubzone bool // IsExtraSubzone indicates we have an extraneous subzone. @@ -782,6 +780,7 @@ func (z *ZoneConfig) DiffWithZone( IndexID: s.IndexID, PartitionName: s.PartitionName, IsExtraSubzone: true, + Field: subzoneMismatch.Field, }, nil } continue @@ -822,6 +821,7 @@ func (z *ZoneConfig) DiffWithZone( IndexID: o.IndexID, PartitionName: o.PartitionName, IsMissingSubzone: true, + Field: subzoneMismatch.Field, }, nil } } diff --git a/pkg/sql/alter_database.go b/pkg/sql/alter_database.go index 875df6c690bc..87b492738535 100644 --- a/pkg/sql/alter_database.go +++ b/pkg/sql/alter_database.go @@ -332,7 +332,11 @@ func (p *planner) AlterDatabaseDropRegion( } if len(regions) != 1 { return nil, errors.WithHintf( - errors.Newf("cannot drop region %q", dbDesc.RegionConfig.PrimaryRegion), + pgerror.Newf( + pgcode.InvalidDatabaseDefinition, + "cannot drop region %q", + dbDesc.RegionConfig.PrimaryRegion, + ), "You must designate another region as the primary region using "+ "ALTER DATABASE %s PRIMARY REGION or remove all other regions before "+ "attempting to drop region %q", dbDesc.GetName(), n.Region, @@ -425,7 +429,7 @@ func (p *planner) checkPrivilegesForMultiRegionOp( func (p *planner) checkPrivilegesForRepartitioningRegionalByRowTables( ctx context.Context, dbDesc *dbdesc.Immutable, ) error { - return p.forEachTableInMultiRegionDatabase(ctx, dbDesc, + return p.forEachMutableTableInDatabase(ctx, dbDesc, func(ctx context.Context, tbDesc *tabledesc.Mutable) error { if tbDesc.IsLocalityRegionalByRow() { err := p.checkPrivilegesForMultiRegionOp(ctx, tbDesc) @@ -455,7 +459,7 @@ func removeLocalityConfigFromAllTablesInDB( ) } b := p.Txn().NewBatch() - if err := p.forEachTableInMultiRegionDatabase(ctx, desc, func(ctx context.Context, tbDesc *tabledesc.Mutable) error { + if err := p.forEachMutableTableInDatabase(ctx, desc, func(ctx context.Context, tbDesc *tabledesc.Mutable) error { // The user must either be an admin or have the requisite privileges. if err := p.checkPrivilegesForMultiRegionOp(ctx, tbDesc); err != nil { return err @@ -694,7 +698,7 @@ func addDefaultLocalityConfigToAllTables( ) } b := p.Txn().NewBatch() - if err := p.forEachTableInMultiRegionDatabase(ctx, dbDesc, func(ctx context.Context, tbDesc *tabledesc.Mutable) error { + if err := p.forEachMutableTableInDatabase(ctx, dbDesc, func(ctx context.Context, tbDesc *tabledesc.Mutable) error { if err := p.checkPrivilegesForMultiRegionOp(ctx, tbDesc); err != nil { return err } @@ -756,7 +760,6 @@ func checkCanConvertTableToMultiRegion( ) } } - // TODO(#57668): check zone configurations are not set here return nil } @@ -775,6 +778,15 @@ func (n *alterDatabasePrimaryRegionNode) setInitialPrimaryRegion(params runParam return err } + // Check we are writing valid zone configurations. + if err := params.p.validateAllMultiRegionZoneConfigsInDatabase( + params.ctx, + &n.desc.Immutable, + &zoneConfigForMultiRegionValidatorSetInitialRegion{}, + ); err != nil { + return err + } + // Set the region config on the database descriptor. if err := n.desc.SetInitialMultiRegionConfig(regionConfig); err != nil { return err diff --git a/pkg/sql/colmem/BUILD.bazel b/pkg/sql/colmem/BUILD.bazel index f6b113c82c2d..4e445e2ef6a2 100644 --- a/pkg/sql/colmem/BUILD.bazel +++ b/pkg/sql/colmem/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "//pkg/sql/types", "//pkg/util/duration", "//pkg/util/mon", + "//pkg/util/uuid", "@com_github_cockroachdb_apd_v2//:apd", "@com_github_cockroachdb_errors//:errors", ], diff --git a/pkg/sql/colmem/allocator.go b/pkg/sql/colmem/allocator.go index 1e9775539546..02d482b0dc6d 100644 --- a/pkg/sql/colmem/allocator.go +++ b/pkg/sql/colmem/allocator.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" ) @@ -370,12 +371,19 @@ func EstimateBatchSizeBytes(vecTypes []*types.T, batchLength int) int { // (excluding any Bytes vectors, those are tracked separately). acc := 0 numBytesVectors := 0 + // We will track Uuid vectors separately because they use smaller initial + // allocation factor. + numUUIDVectors := 0 for _, t := range vecTypes { switch typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) { case types.BoolFamily: acc += sizeOfBool case types.BytesFamily: - numBytesVectors++ + if t.Family() == types.UuidFamily { + numUUIDVectors++ + } else { + numBytesVectors++ + } case types.IntFamily: switch t.Width() { case 16: @@ -416,15 +424,20 @@ func EstimateBatchSizeBytes(vecTypes []*types.T, batchLength int) int { colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", t)) } } - // For byte arrays, we initially allocate BytesInitialAllocationFactor - // number of bytes (plus an int32 for the offset) for each row, so we use - // the sum of two values as the estimate. However, later, the exact - // memory footprint will be used: whenever a modification of Bytes takes - // place, the Allocator will measure the old footprint and the updated - // one and will update the memory account accordingly. We also account for - // the overhead and for the additional offset value that are needed for - // Bytes vectors (to be in line with coldata.Bytes.Size() method). - bytesVectorsSize := numBytesVectors * (int(coldata.FlatBytesOverhead) + - coldata.BytesInitialAllocationFactor*batchLength + sizeOfInt32*(batchLength+1)) + // For byte arrays, we initially allocate a constant number of bytes (plus + // an int32 for the offset) for each row, so we use the sum of two values as + // the estimate. However, later, the exact memory footprint will be used: + // whenever a modification of Bytes takes place, the Allocator will measure + // the old footprint and the updated one and will update the memory account + // accordingly. We also account for the overhead and for the additional + // offset value that are needed for Bytes vectors (to be in line with + // coldata.Bytes.Size() method). + var bytesVectorsSize int + // Add the overhead. + bytesVectorsSize += (numBytesVectors + numUUIDVectors) * (int(coldata.FlatBytesOverhead)) + // Add the data for both Bytes and Uuids. + bytesVectorsSize += (numBytesVectors*coldata.BytesInitialAllocationFactor + numUUIDVectors*uuid.Size) * batchLength + // Add the offsets. + bytesVectorsSize += (numBytesVectors + numUUIDVectors) * sizeOfInt32 * (batchLength + 1) return acc*batchLength + bytesVectorsSize } diff --git a/pkg/sql/database.go b/pkg/sql/database.go index 2df5b7ff7c3f..2a5af562fe39 100644 --- a/pkg/sql/database.go +++ b/pkg/sql/database.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -94,3 +95,30 @@ func (p *planner) writeDatabaseChangeToBatch( b, ) } + +// forEachMutableTableInDatabase calls the given function on every table +// descriptor inside the given database. Tables that have been +// dropped are skipped. +func (p *planner) forEachMutableTableInDatabase( + ctx context.Context, + dbDesc *dbdesc.Immutable, + fn func(ctx context.Context, tbDesc *tabledesc.Mutable) error, +) error { + allDescs, err := p.Descriptors().GetAllDescriptors(ctx, p.txn) + if err != nil { + return err + } + + lCtx := newInternalLookupCtx(ctx, allDescs, dbDesc, nil /* fallback */) + for _, tbID := range lCtx.tbIDs { + desc := lCtx.tbDescs[tbID] + if desc.Dropped() { + continue + } + mutable := tabledesc.NewBuilder(desc.TableDesc()).BuildExistingMutableTable() + if err := fn(ctx, mutable); err != nil { + return err + } + } + return nil +} diff --git a/pkg/sql/logictest/testdata/logic_test/set b/pkg/sql/logictest/testdata/logic_test/set index 3fff592cc902..7e98e4740c9a 100644 --- a/pkg/sql/logictest/testdata/logic_test/set +++ b/pkg/sql/logictest/testdata/logic_test/set @@ -292,12 +292,12 @@ SET statement_timeout = '0ms' # Test that statement_timeout can be set with an interval string, defaulting to # milliseconds as a unit. statement ok -SET statement_timeout = '100' +SET statement_timeout = '10000' query T SHOW statement_timeout ---- -100 +10000 # Set the statement timeout to something absurdly small, so that no query would # presumably be able to go through. It should still be possible to get out of diff --git a/pkg/sql/region_util.go b/pkg/sql/region_util.go index 28a506ac89dd..2d5f5e4a68c7 100644 --- a/pkg/sql/region_util.go +++ b/pkg/sql/region_util.go @@ -132,7 +132,7 @@ func makeRequiredConstraintForRegion(r descpb.RegionName) zonepb.Constraint { // are set the way they are. func zoneConfigForMultiRegionDatabase( regionConfig multiregion.RegionConfig, -) (*zonepb.ZoneConfig, error) { +) (zonepb.ZoneConfig, error) { numVoters, numReplicas := getNumVotersAndNumReplicas(regionConfig) constraints := make([]zonepb.ConstraintsConjunction, len(regionConfig.Regions())) for i, region := range regionConfig.Regions() { @@ -145,10 +145,10 @@ func zoneConfigForMultiRegionDatabase( voterConstraints, err := synthesizeVoterConstraints(regionConfig.PrimaryRegion(), regionConfig) if err != nil { - return nil, err + return zonepb.ZoneConfig{}, err } - return &zonepb.ZoneConfig{ + return zonepb.ZoneConfig{ NumReplicas: &numReplicas, NumVoters: &numVoters, LeasePreferences: []zonepb.LeasePreference{ @@ -662,7 +662,7 @@ func discardMultiRegionFieldsForDatabaseZoneConfig( return applyZoneConfigForMultiRegionDatabase( ctx, dbID, - zonepb.NewZoneConfig(), + *zonepb.NewZoneConfig(), txn, execConfig, ) @@ -671,7 +671,7 @@ func discardMultiRegionFieldsForDatabaseZoneConfig( func applyZoneConfigForMultiRegionDatabase( ctx context.Context, dbID descpb.ID, - mergeZoneConfig *zonepb.ZoneConfig, + mergeZoneConfig zonepb.ZoneConfig, txn *kv.Txn, execConfig *ExecutorConfig, ) error { @@ -684,7 +684,7 @@ func applyZoneConfigForMultiRegionDatabase( newZoneConfig = *currentZoneConfig } newZoneConfig.CopyFromZone( - *mergeZoneConfig, + mergeZoneConfig, zonepb.MultiRegionZoneConfigFields, ) // If the new zone config is the same as a blank zone config, delete it. @@ -712,40 +712,10 @@ func applyZoneConfigForMultiRegionDatabase( return nil } -// forEachTableInMultiRegionDatabase calls the given function on every table -// descriptor inside the given multi-region database. Tables that have been -// dropped are skipped. -func (p *planner) forEachTableInMultiRegionDatabase( - ctx context.Context, - dbDesc *dbdesc.Immutable, - fn func(ctx context.Context, tbDesc *tabledesc.Mutable) error, -) error { - if !dbDesc.IsMultiRegion() { - return errors.AssertionFailedf("db %q is not multi-region", dbDesc.Name) - } - allDescs, err := p.Descriptors().GetAllDescriptors(ctx, p.txn) - if err != nil { - return err - } - - lCtx := newInternalLookupCtx(ctx, allDescs, dbDesc, nil /* fallback */) - for _, tbID := range lCtx.tbIDs { - desc := lCtx.tbDescs[tbID] - if desc.Dropped() { - continue - } - mutable := tabledesc.NewBuilder(desc.TableDesc()).BuildExistingMutableTable() - if err := fn(ctx, mutable); err != nil { - return err - } - } - return nil -} - // updateZoneConfigsForAllTables loops through all of the tables in the // specified database and refreshes the zone configs for all tables. func (p *planner) updateZoneConfigsForAllTables(ctx context.Context, desc *dbdesc.Mutable) error { - return p.forEachTableInMultiRegionDatabase( + return p.forEachMutableTableInDatabase( ctx, &desc.Immutable, func(ctx context.Context, tbDesc *tabledesc.Mutable) error { @@ -844,23 +814,65 @@ func (p *planner) ValidateAllMultiRegionZoneConfigsInCurrentDatabase(ctx context if !dbDesc.IsMultiRegion() { return nil } + regionConfig, err := SynthesizeRegionConfigForZoneConfigValidation(ctx, p.txn, dbDesc.ID, p.Descriptors()) + if err != nil { + return err + } + return p.validateAllMultiRegionZoneConfigsInDatabase( + ctx, + dbDesc, + &zoneConfigForMultiRegionValidatorValidation{ + zoneConfigForMultiRegionValidatorExistingMultiRegionObject: zoneConfigForMultiRegionValidatorExistingMultiRegionObject{ + regionConfig: regionConfig, + }, + }, + ) +} - if err := p.validateZoneConfigForMultiRegionDatabase( +func (p *planner) validateAllMultiRegionZoneConfigsInDatabase( + ctx context.Context, + dbDesc *dbdesc.Immutable, + zoneConfigForMultiRegionValidator zoneConfigForMultiRegionValidator, +) error { + var ids []descpb.ID + if err := p.forEachMutableTableInDatabase( + ctx, + dbDesc, + func(ctx context.Context, tbDesc *tabledesc.Mutable) error { + ids = append(ids, tbDesc.GetID()) + return nil + }, + ); err != nil { + return err + } + ids = append(ids, dbDesc.GetID()) + + zoneConfigs, err := getZoneConfigRawBatch( ctx, + p.txn, + p.ExecCfg().Codec, + ids, + ) + if err != nil { + return err + } + + if err := p.validateZoneConfigForMultiRegionDatabase( dbDesc, - &validateZoneConfigForMultiRegionErrorHandlerValidation{}, + zoneConfigs[dbDesc.GetID()], + zoneConfigForMultiRegionValidator, ); err != nil { return err } - return p.forEachTableInMultiRegionDatabase( + + return p.forEachMutableTableInDatabase( ctx, dbDesc, func(ctx context.Context, tbDesc *tabledesc.Mutable) error { return p.validateZoneConfigForMultiRegionTable( - ctx, - dbDesc, tbDesc, - &validateZoneConfigForMultiRegionErrorHandlerValidation{}, + zoneConfigs[tbDesc.GetID()], + zoneConfigForMultiRegionValidator, ) }, ) @@ -1112,20 +1124,127 @@ func (p *planner) CheckZoneConfigChangePermittedForMultiRegion( return nil } -// validateZoneConfigForMultiRegionErrorHandler is an interface representing -// an error to generate if validating a zone config for multi-region -// fails. -type validateZoneConfigForMultiRegionErrorHandler interface { +// zoneConfigForMultiRegionValidator is an interface representing +// actions to take when validating a zone config for multi-region +// purposes. +type zoneConfigForMultiRegionValidator interface { + getExpectedDatabaseZoneConfig() (zonepb.ZoneConfig, error) + getExpectedTableZoneConfig(desc catalog.TableDescriptor) (zonepb.ZoneConfig, error) + newMismatchFieldError(descType string, descName string, field string) error - newMissingSubzoneError(descType string, descName string) error - newExtraSubzoneError(descType string, descName string) error + newMissingSubzoneError(descType string, descName string, field string) error + newExtraSubzoneError(descType string, descName string, field string) error } -// validateZoneConfigForMultiRegionErrorHandlerModifiedByUser implements -// interface validateZoneConfigForMultiRegionErrorHandler. -type validateZoneConfigForMultiRegionErrorHandlerModifiedByUser struct{} +// zoneConfigForMultiRegionValidatorSetInitialRegion implements +// interface zoneConfigForMultiRegionValidator. +type zoneConfigForMultiRegionValidatorSetInitialRegion struct{} + +var _ zoneConfigForMultiRegionValidator = (*zoneConfigForMultiRegionValidatorSetInitialRegion)(nil) + +func (v *zoneConfigForMultiRegionValidatorSetInitialRegion) getExpectedDatabaseZoneConfig() ( + zonepb.ZoneConfig, + error, +) { + // For set initial region, we want no multi-region fields to be set. + return *zonepb.NewZoneConfig(), nil +} + +func (v *zoneConfigForMultiRegionValidatorSetInitialRegion) getExpectedTableZoneConfig( + desc catalog.TableDescriptor, +) (zonepb.ZoneConfig, error) { + // For set initial region, we want no multi-region fields to be set. + return *zonepb.NewZoneConfig(), nil +} + +func (v *zoneConfigForMultiRegionValidatorSetInitialRegion) wrapErr(err error) error { + // We currently do not allow "inherit from parent" behavior, so one must + // discard the zone config before continuing. + // COPY FROM PARENT copies the value but does not inherit. + // This can be replaced with the override session variable hint when it is + // available. + return errors.WithHintf( + err, + "discard the zone config using CONFIGURE ZONE DISCARD before continuing", + ) +} -func (v *validateZoneConfigForMultiRegionErrorHandlerModifiedByUser) newMismatchFieldError( +func (v *zoneConfigForMultiRegionValidatorSetInitialRegion) newMismatchFieldError( + descType string, descName string, field string, +) error { + return v.wrapErr( + pgerror.Newf( + pgcode.InvalidObjectDefinition, + "zone configuration for %s %s has field %q set which will be overwritten when setting the the initial PRIMARY REGION", + descType, + descName, + field, + ), + ) +} + +func (v *zoneConfigForMultiRegionValidatorSetInitialRegion) newMissingSubzoneError( + descType string, descName string, field string, +) error { + // There can never be a missing subzone as we only compare against + // blank zone configs. + return errors.AssertionFailedf( + "unexpected missing subzone for %s %s", + descType, + descName, + ) +} + +func (v *zoneConfigForMultiRegionValidatorSetInitialRegion) newExtraSubzoneError( + descType string, descName string, field string, +) error { + return v.wrapErr( + pgerror.Newf( + pgcode.InvalidObjectDefinition, + "zone configuration for %s %s has field %q set which will be overwritten when setting the initial PRIMARY REGION", + descType, + descName, + field, + ), + ) +} + +// zoneConfigForMultiRegionValidatorExistingMultiRegionObject partially implements +// the zoneConfigForMultiRegionValidator interface. +type zoneConfigForMultiRegionValidatorExistingMultiRegionObject struct { + regionConfig multiregion.RegionConfig +} + +func (v *zoneConfigForMultiRegionValidatorExistingMultiRegionObject) getExpectedDatabaseZoneConfig() ( + zonepb.ZoneConfig, + error, +) { + return zoneConfigForMultiRegionDatabase(v.regionConfig) +} + +func (v *zoneConfigForMultiRegionValidatorExistingMultiRegionObject) getExpectedTableZoneConfig( + desc catalog.TableDescriptor, +) (zonepb.ZoneConfig, error) { + _, expectedZoneConfig, err := ApplyZoneConfigForMultiRegionTableOptionTableAndIndexes( + *zonepb.NewZoneConfig(), + v.regionConfig, + desc, + ) + if err != nil { + return zonepb.ZoneConfig{}, err + } + return expectedZoneConfig, err +} + +// zoneConfigForMultiRegionValidatorModifiedByUser implements +// interface zoneConfigForMultiRegionValidator. +type zoneConfigForMultiRegionValidatorModifiedByUser struct { + zoneConfigForMultiRegionValidatorExistingMultiRegionObject +} + +var _ zoneConfigForMultiRegionValidator = (*zoneConfigForMultiRegionValidatorModifiedByUser)(nil) + +func (v *zoneConfigForMultiRegionValidatorModifiedByUser) newMismatchFieldError( descType string, descName string, field string, ) error { return v.wrapErr( @@ -1139,7 +1258,7 @@ func (v *validateZoneConfigForMultiRegionErrorHandlerModifiedByUser) newMismatch ) } -func (v *validateZoneConfigForMultiRegionErrorHandlerModifiedByUser) wrapErr(err error) error { +func (v *zoneConfigForMultiRegionValidatorModifiedByUser) wrapErr(err error) error { err = errors.WithDetail( err, "the attempted operation will overwrite a user modified field", @@ -1151,8 +1270,8 @@ func (v *validateZoneConfigForMultiRegionErrorHandlerModifiedByUser) wrapErr(err ) } -func (v *validateZoneConfigForMultiRegionErrorHandlerModifiedByUser) newMissingSubzoneError( - descType string, descName string, +func (v *zoneConfigForMultiRegionValidatorModifiedByUser) newMissingSubzoneError( + descType string, descName string, field string, ) error { return v.wrapErr( pgerror.Newf( @@ -1164,24 +1283,29 @@ func (v *validateZoneConfigForMultiRegionErrorHandlerModifiedByUser) newMissingS ) } -func (v *validateZoneConfigForMultiRegionErrorHandlerModifiedByUser) newExtraSubzoneError( - descType string, descName string, +func (v *zoneConfigForMultiRegionValidatorModifiedByUser) newExtraSubzoneError( + descType string, descName string, field string, ) error { return v.wrapErr( pgerror.Newf( pgcode.InvalidObjectDefinition, - "attempting to update zone config which contains an extra zone configuration for %s %s", + "attempting to update zone config which contains an extra zone configuration for %s %s with field %s populated", descType, descName, + field, ), ) } -// validateZoneConfigForMultiRegionErrorHandlerValidation implements -// interface validateZoneConfigForMultiRegionErrorHandler. -type validateZoneConfigForMultiRegionErrorHandlerValidation struct{} +// zoneConfigForMultiRegionValidatorValidation implements +// interface zoneConfigForMultiRegionValidator. +type zoneConfigForMultiRegionValidatorValidation struct { + zoneConfigForMultiRegionValidatorExistingMultiRegionObject +} + +var _ zoneConfigForMultiRegionValidator = (*zoneConfigForMultiRegionValidatorValidation)(nil) -func (v *validateZoneConfigForMultiRegionErrorHandlerValidation) newMismatchFieldError( +func (v *zoneConfigForMultiRegionValidatorValidation) newMismatchFieldError( descType string, descName string, field string, ) error { return pgerror.Newf( @@ -1193,8 +1317,8 @@ func (v *validateZoneConfigForMultiRegionErrorHandlerValidation) newMismatchFiel ) } -func (v *validateZoneConfigForMultiRegionErrorHandlerValidation) newMissingSubzoneError( - descType string, descName string, +func (v *zoneConfigForMultiRegionValidatorValidation) newMissingSubzoneError( + descType string, descName string, field string, ) error { return pgerror.Newf( pgcode.InvalidObjectDefinition, @@ -1204,14 +1328,15 @@ func (v *validateZoneConfigForMultiRegionErrorHandlerValidation) newMissingSubzo ) } -func (v *validateZoneConfigForMultiRegionErrorHandlerValidation) newExtraSubzoneError( - descType string, descName string, +func (v *zoneConfigForMultiRegionValidatorValidation) newExtraSubzoneError( + descType string, descName string, field string, ) error { return pgerror.Newf( pgcode.InvalidObjectDefinition, - "extraneous zone configuration for %s %s", + "extraneous zone configuration for %s %s with field %s populated", descType, descName, + field, ) } @@ -1227,36 +1352,42 @@ func (p *planner) validateZoneConfigForMultiRegionDatabaseWasNotModifiedByUser( if p.SessionData().OverrideMultiRegionZoneConfigEnabled { return nil } + currentZoneConfig, err := getZoneConfigRaw(ctx, p.txn, p.ExecCfg().Codec, dbDesc.ID) + if err != nil { + return err + } + regionConfig, err := SynthesizeRegionConfigForZoneConfigValidation(ctx, p.txn, dbDesc.ID, p.Descriptors()) + if err != nil { + return err + } return p.validateZoneConfigForMultiRegionDatabase( - ctx, dbDesc, - &validateZoneConfigForMultiRegionErrorHandlerModifiedByUser{}, + currentZoneConfig, + &zoneConfigForMultiRegionValidatorModifiedByUser{ + zoneConfigForMultiRegionValidatorExistingMultiRegionObject: zoneConfigForMultiRegionValidatorExistingMultiRegionObject{ + regionConfig: regionConfig, + }, + }, ) } // validateZoneConfigForMultiRegionDatabase validates that the zone config // for the databases matches as the multi-region database definition. func (p *planner) validateZoneConfigForMultiRegionDatabase( - ctx context.Context, dbDesc *dbdesc.Immutable, - validateZoneConfigForMultiRegionErrorHandler validateZoneConfigForMultiRegionErrorHandler, + currentZoneConfig *zonepb.ZoneConfig, + zoneConfigForMultiRegionValidator zoneConfigForMultiRegionValidator, ) error { - regionConfig, err := SynthesizeRegionConfigForZoneConfigValidation(ctx, p.txn, dbDesc.ID, p.Descriptors()) - if err != nil { - return err - } - expectedZoneConfig, err := zoneConfigForMultiRegionDatabase(regionConfig) - if err != nil { - return err + if currentZoneConfig == nil { + currentZoneConfig = zonepb.NewZoneConfig() } - - currentZoneConfig, err := getZoneConfigRaw(ctx, p.txn, p.ExecCfg().Codec, dbDesc.ID) + expectedZoneConfig, err := zoneConfigForMultiRegionValidator.getExpectedDatabaseZoneConfig() if err != nil { return err } same, mismatch, err := currentZoneConfig.DiffWithZone( - *expectedZoneConfig, + expectedZoneConfig, zonepb.MultiRegionZoneConfigFields, ) if err != nil { @@ -1264,7 +1395,7 @@ func (p *planner) validateZoneConfigForMultiRegionDatabase( } if !same { dbName := tree.Name(dbDesc.GetName()) - return validateZoneConfigForMultiRegionErrorHandler.newMismatchFieldError( + return zoneConfigForMultiRegionValidator.newMismatchFieldError( "database", dbName.String(), mismatch.Field, @@ -1291,41 +1422,41 @@ func (p *planner) validateZoneConfigForMultiRegionTableWasNotModifiedByUser( if p.SessionData().OverrideMultiRegionZoneConfigEnabled || desc.GetLocalityConfig() == nil { return nil } + currentZoneConfig, err := getZoneConfigRaw(ctx, p.txn, p.ExecCfg().Codec, desc.GetID()) + if err != nil { + return err + } + regionConfig, err := SynthesizeRegionConfig(ctx, p.txn, dbDesc.ID, p.Descriptors()) + if err != nil { + return err + } return p.validateZoneConfigForMultiRegionTable( - ctx, - dbDesc, desc, - &validateZoneConfigForMultiRegionErrorHandlerModifiedByUser{}, + currentZoneConfig, + &zoneConfigForMultiRegionValidatorModifiedByUser{ + zoneConfigForMultiRegionValidatorExistingMultiRegionObject: zoneConfigForMultiRegionValidatorExistingMultiRegionObject{ + regionConfig: regionConfig, + }, + }, ) } // validateZoneConfigForMultiRegionTableOptions validates that -// the table's zone configuration matches exactly what is expected. +// the multi-region fields of the table's zone configuration +// matches what is expected for the given table. func (p *planner) validateZoneConfigForMultiRegionTable( - ctx context.Context, - dbDesc *dbdesc.Immutable, desc catalog.TableDescriptor, - validateZoneConfigForMultiRegionErrorHandler validateZoneConfigForMultiRegionErrorHandler, + currentZoneConfig *zonepb.ZoneConfig, + zoneConfigForMultiRegionValidator zoneConfigForMultiRegionValidator, ) error { - currentZoneConfig, err := getZoneConfigRaw(ctx, p.txn, p.ExecCfg().Codec, desc.GetID()) - if err != nil { - return err - } if currentZoneConfig == nil { currentZoneConfig = zonepb.NewZoneConfig() } - regionConfig, err := SynthesizeRegionConfig(ctx, p.txn, dbDesc.ID, p.Descriptors()) - if err != nil { - return err - } - tableName := tree.Name(desc.GetName()) - _, expectedZoneConfig, err := ApplyZoneConfigForMultiRegionTableOptionTableAndIndexes( - *zonepb.NewZoneConfig(), - regionConfig, + expectedZoneConfig, err := zoneConfigForMultiRegionValidator.getExpectedTableZoneConfig( desc, ) if err != nil { @@ -1404,18 +1535,20 @@ func (p *planner) validateZoneConfigForMultiRegionTable( } if mismatch.IsMissingSubzone { - return validateZoneConfigForMultiRegionErrorHandler.newMissingSubzoneError( + return zoneConfigForMultiRegionValidator.newMissingSubzoneError( descType, name, + mismatch.Field, ) } if mismatch.IsExtraSubzone { - return validateZoneConfigForMultiRegionErrorHandler.newExtraSubzoneError( + return zoneConfigForMultiRegionValidator.newExtraSubzoneError( descType, name, + mismatch.Field, ) } - return validateZoneConfigForMultiRegionErrorHandler.newMismatchFieldError( + return zoneConfigForMultiRegionValidator.newMismatchFieldError( descType, name, mismatch.Field, diff --git a/pkg/sql/region_util_test.go b/pkg/sql/region_util_test.go index 8bc31fe74c4b..7c5140047a72 100644 --- a/pkg/sql/region_util_test.go +++ b/pkg/sql/region_util_test.go @@ -27,7 +27,7 @@ func TestZoneConfigForMultiRegionDatabase(t *testing.T) { testCases := []struct { desc string regionConfig multiregion.RegionConfig - expected *zonepb.ZoneConfig + expected zonepb.ZoneConfig }{ { desc: "one region, zone survival", @@ -39,7 +39,7 @@ func TestZoneConfigForMultiRegionDatabase(t *testing.T) { descpb.SurvivalGoal_ZONE_FAILURE, descpb.InvalidID, ), - expected: &zonepb.ZoneConfig{ + expected: zonepb.ZoneConfig{ NumReplicas: proto.Int32(3), NumVoters: proto.Int32(3), LeasePreferences: []zonepb.LeasePreference{ @@ -77,7 +77,7 @@ func TestZoneConfigForMultiRegionDatabase(t *testing.T) { descpb.SurvivalGoal_ZONE_FAILURE, descpb.InvalidID, ), - expected: &zonepb.ZoneConfig{ + expected: zonepb.ZoneConfig{ NumReplicas: proto.Int32(4), NumVoters: proto.Int32(3), LeasePreferences: []zonepb.LeasePreference{ @@ -122,7 +122,7 @@ func TestZoneConfigForMultiRegionDatabase(t *testing.T) { descpb.SurvivalGoal_ZONE_FAILURE, descpb.InvalidID, ), - expected: &zonepb.ZoneConfig{ + expected: zonepb.ZoneConfig{ NumReplicas: proto.Int32(5), NumVoters: proto.Int32(3), LeasePreferences: []zonepb.LeasePreference{ @@ -173,7 +173,7 @@ func TestZoneConfigForMultiRegionDatabase(t *testing.T) { descpb.SurvivalGoal_REGION_FAILURE, descpb.InvalidID, ), - expected: &zonepb.ZoneConfig{ + expected: zonepb.ZoneConfig{ NumReplicas: proto.Int32(5), NumVoters: proto.Int32(5), LeasePreferences: []zonepb.LeasePreference{ @@ -225,7 +225,7 @@ func TestZoneConfigForMultiRegionDatabase(t *testing.T) { descpb.SurvivalGoal_ZONE_FAILURE, descpb.InvalidID, ), - expected: &zonepb.ZoneConfig{ + expected: zonepb.ZoneConfig{ NumReplicas: proto.Int32(6), NumVoters: proto.Int32(3), LeasePreferences: []zonepb.LeasePreference{ @@ -283,7 +283,7 @@ func TestZoneConfigForMultiRegionDatabase(t *testing.T) { descpb.SurvivalGoal_REGION_FAILURE, descpb.InvalidID, ), - expected: &zonepb.ZoneConfig{ + expected: zonepb.ZoneConfig{ NumReplicas: proto.Int32(5), NumVoters: proto.Int32(5), LeasePreferences: []zonepb.LeasePreference{ diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index a5e250fdb3a4..aa2d468671d4 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -1005,6 +1005,41 @@ func getZoneConfigRaw( return &zone, nil } +// getZoneConfigRawBatch looks up the zone config with the given IDs. +// Unlike getZoneConfig, it does not attempt to ascend the zone config hierarchy. +// If no zone config exists for the given ID, the map entry is not provided. +func getZoneConfigRawBatch( + ctx context.Context, txn *kv.Txn, codec keys.SQLCodec, ids []descpb.ID, +) (map[descpb.ID]*zonepb.ZoneConfig, error) { + if !codec.ForSystemTenant() { + // Secondary tenants do not have zone configs for individual objects. + return nil, nil + } + b := txn.NewBatch() + for _, id := range ids { + b.Get(config.MakeZoneKey(config.SystemTenantObjectID(id))) + } + if err := txn.Run(ctx, b); err != nil { + return nil, err + } + ret := make(map[descpb.ID]*zonepb.ZoneConfig, len(b.Results)) + for idx, r := range b.Results { + if r.Err != nil { + return nil, r.Err + } + var zone zonepb.ZoneConfig + row := r.Rows[0] + if row.Value == nil { + continue + } + if err := row.ValueProto(&zone); err != nil { + return nil, err + } + ret[ids[idx]] = &zone + } + return ret, nil +} + // RemoveIndexZoneConfigs removes the zone configurations for some // indexes being dropped. It is a no-op if there is no zone // configuration, there's no index zone configs to be dropped, diff --git a/pkg/sql/type_change.go b/pkg/sql/type_change.go index 5ece1344834a..47f7fe14b135 100644 --- a/pkg/sql/type_change.go +++ b/pkg/sql/type_change.go @@ -494,7 +494,7 @@ func repartitionRegionalByRowTables( } b := txn.NewBatch() - err = localPlanner.forEachTableInMultiRegionDatabase(ctx, dbDesc, + err = localPlanner.forEachMutableTableInDatabase(ctx, dbDesc, func(ctx context.Context, tableDesc *tabledesc.Mutable) error { if !tableDesc.IsLocalityRegionalByRow() || tableDesc.Dropped() { // We only need to re-partition REGIONAL BY ROW tables. Even then, we