Skip to content

Commit

Permalink
sql: validate primary / secondary region localities at end of txn
Browse files Browse the repository at this point in the history
Previously, if a database was restored with skip_localities,
there was no way to modify this database to set the primary
region since validation would kick in too early during the
statement. This meant fixing the regions in a restored database
was impossible if the primary region was no longer valid. To
address this, this patch, delays locality validation till the
end of the transaction.

Fixes: #103290

Release note (bug fix): SET PRIMARY REGION and SET SECONDARY REGION
did not validate transactionally, which could prevent cleaning up
removed regions after a restore.
  • Loading branch information
fqazi committed May 18, 2023
1 parent a65bca2 commit 8b9f052
Show file tree
Hide file tree
Showing 11 changed files with 156 additions and 13 deletions.
14 changes: 14 additions & 0 deletions pkg/ccl/backupccl/testdata/backup-restore/multiregion
Original file line number Diff line number Diff line change
Expand Up @@ -69,10 +69,24 @@ exec-sql
RESTORE FROM LATEST IN 'nodelocal://1/full_cluster_backup/' WITH skip_localities_check;
----

exec-sql
ALTER DATABASE d SET PRIMARY REGION 'eu-central-1';
ALTER DATABASE d DROP REGION 'us-east-1';
ALTER DATABASE d DROP REGION 'us-west-1';
ALTER DATABASE d ADD REGION 'eu-north-1';
----

exec-sql
RESTORE DATABASE d FROM LATEST IN 'nodelocal://1/database_backup/' WITH skip_localities_check, new_db_name='d_new';
----

exec-sql
ALTER DATABASE d_new SET PRIMARY REGION 'eu-central-1';
ALTER DATABASE d_new DROP REGION 'us-east-1';
ALTER DATABASE d_new DROP REGION 'us-west-1';
ALTER DATABASE d_new ADD REGION 'eu-north-1';
----

exec-sql
DROP DATABASE d_new;
----
Expand Down
8 changes: 7 additions & 1 deletion pkg/sql/alter_database.go
Original file line number Diff line number Diff line change
Expand Up @@ -897,14 +897,18 @@ func (n *alterDatabasePrimaryRegionNode) switchPrimaryRegion(params runParams) e
return err
}

// Validate the final zone config at the end of the transaction, since
// we will not be validating localities right now.
*params.extendedEvalCtx.validateDbZoneConfig = true

// Update the database's zone configuration.
if err := ApplyZoneConfigFromDatabaseRegionConfig(
params.ctx,
n.desc.ID,
updatedRegionConfig,
params.p.InternalSQLTxn(),
params.p.execCfg,
true, /* validateLocalities */
false, /*validateLocalities*/
params.extendedEvalCtx.Tracing.KVTracingEnabled(),
); err != nil {
return err
Expand Down Expand Up @@ -2021,6 +2025,8 @@ func (n *alterDatabaseSecondaryRegion) startExec(params runParams) error {
return err
}

*params.extendedEvalCtx.validateDbZoneConfig = true

// Update the database's zone configuration.
if err := ApplyZoneConfigFromDatabaseRegionConfig(
params.ctx,
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/descs/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ go_library(
"uncommitted_metadata.go",
"validate.go",
"virtual_descriptors.go",
"zone_config_validator.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs",
visibility = ["//visibility:public"],
Expand Down
12 changes: 12 additions & 0 deletions pkg/sql/catalog/descs/collection.go
Original file line number Diff line number Diff line change
Expand Up @@ -702,6 +702,18 @@ func (tc *Collection) GetUncommittedTables() (tables []catalog.TableDescriptor)
return tables
}

// GetUncommittedDatabases returns all the databases updated or created in the
// transaction.
func (tc *Collection) GetUncommittedDatabases() (databases []catalog.DatabaseDescriptor) {
_ = tc.uncommitted.iterateUncommittedByID(func(desc catalog.Descriptor) error {
if database, ok := desc.(catalog.DatabaseDescriptor); ok {
databases = append(databases, database)
}
return nil
})
return databases
}

func newMutableSyntheticDescriptorAssertionError(id descpb.ID) error {
return errors.AssertionFailedf("attempted mutable access of synthetic descriptor %d", id)
}
Expand Down
32 changes: 30 additions & 2 deletions pkg/sql/catalog/descs/validate.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/internal/catkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/internal/validate"
"github.com/cockroachdb/errors"
)

// Validate returns any descriptor validation errors after validating using the
Expand Down Expand Up @@ -51,7 +52,14 @@ func (tc *Collection) Validate(
// descriptor set. We purposefully avoid using leased descriptors as those may
// be one version behind, in which case it's possible (and legitimate) that
// those are missing back-references which would cause validation to fail.
func (tc *Collection) ValidateUncommittedDescriptors(ctx context.Context, txn *kv.Txn) (err error) {
// Optionally, the zone config will be validated if validateZoneConfigs is
// set to true.
func (tc *Collection) ValidateUncommittedDescriptors(
ctx context.Context,
txn *kv.Txn,
validateZoneConfigs bool,
zoneConfigValidator ZoneConfigValidator,
) (err error) {
if tc.skipValidationOnWrite || !tc.validationModeProvider.ValidateDescriptorsOnWrite() {
return nil
}
Expand All @@ -63,7 +71,27 @@ func (tc *Collection) ValidateUncommittedDescriptors(ctx context.Context, txn *k
if len(descs) == 0 {
return nil
}
return tc.Validate(ctx, txn, catalog.ValidationWriteTelemetry, validate.Write, descs...)
if err := tc.Validate(ctx, txn, catalog.ValidationWriteTelemetry, validate.Write, descs...); err != nil {
return err
}
// Next validate any zone configs that may have been modified
// in the descriptor set, only if this type of validation is required.
// We only do this type of validation if region configs are modified.
if validateZoneConfigs {
if zoneConfigValidator == nil {
return errors.AssertionFailedf("zone config validator is required to " +
"validate zone configs")
}
for _, desc := range descs {
switch t := desc.(type) {
case catalog.DatabaseDescriptor:
if err = zoneConfigValidator.ValidateDbZoneConfig(ctx, t); err != nil {
return err
}
}
}
}
return nil
}

func (tc *Collection) newValidationDereferencer(txn *kv.Txn) validate.ValidationDereferencer {
Expand Down
22 changes: 22 additions & 0 deletions pkg/sql/catalog/descs/zone_config_validator.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package descs

import (
"context"

"github.com/cockroachdb/cockroach/pkg/sql/catalog"
)

// ZoneConfigValidator is used to validate zone configs
type ZoneConfigValidator interface {
ValidateDbZoneConfig(ctx context.Context, db catalog.DatabaseDescriptor) error
}
21 changes: 13 additions & 8 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1357,6 +1357,9 @@ type connExecutor struct {
// comprising statements.
numRows int

// validateDbZoneConfig should the DB zone config on commit.
validateDbZoneConfig bool

// txnCounter keeps track of how many SQL txns have been open since
// the start of the session. This is used for logging, to
// distinguish statements that belong to separate SQL transactions.
Expand Down Expand Up @@ -1914,6 +1917,7 @@ func (ex *connExecutor) resetExtraTxnState(ctx context.Context, ev txnEvent) {
} else {
ex.extraTxnState.descCollection.ReleaseAll(ctx)
ex.extraTxnState.jobs.reset()
ex.extraTxnState.validateDbZoneConfig = false
ex.extraTxnState.schemaChangerState.memAcc.Clear(ctx)
ex.extraTxnState.schemaChangerState = &SchemaChangerState{
mode: ex.sessionData().NewSchemaChangerMode,
Expand Down Expand Up @@ -3403,14 +3407,15 @@ func (ex *connExecutor) initEvalCtx(ctx context.Context, evalCtx *extendedEvalCo
RangeStatsFetcher: p.execCfg.RangeStatsFetcher,
JobsProfiler: p,
},
Tracing: &ex.sessionTracing,
MemMetrics: &ex.memMetrics,
Descs: ex.extraTxnState.descCollection,
TxnModesSetter: ex,
jobs: ex.extraTxnState.jobs,
statsProvider: ex.server.sqlStats,
indexUsageStats: ex.indexUsageStats,
statementPreparer: ex,
Tracing: &ex.sessionTracing,
MemMetrics: &ex.memMetrics,
Descs: ex.extraTxnState.descCollection,
TxnModesSetter: ex,
jobs: ex.extraTxnState.jobs,
validateDbZoneConfig: &ex.extraTxnState.validateDbZoneConfig,
statsProvider: ex.server.sqlStats,
indexUsageStats: ex.indexUsageStats,
statementPreparer: ex,
}
evalCtx.copyFromExecCfg(ex.server.cfg)
}
Expand Down
6 changes: 5 additions & 1 deletion pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -1322,7 +1322,11 @@ func (ex *connExecutor) commitSQLTransactionInternal(ctx context.Context) error
}

if ex.extraTxnState.descCollection.HasUncommittedDescriptors() {
if err := ex.extraTxnState.descCollection.ValidateUncommittedDescriptors(ctx, ex.state.mu.txn); err != nil {
zoneConfigValidator := newZoneConfigValidator(ex.state.mu.txn,
ex.extraTxnState.descCollection,
ex.planner.regionsProvider(),
ex.planner.execCfg)
if err := ex.extraTxnState.descCollection.ValidateUncommittedDescriptors(ctx, ex.state.mu.txn, ex.extraTxnState.validateDbZoneConfig, zoneConfigValidator); err != nil {
return err
}

Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,9 @@ type extendedEvalContext struct {
SchemaChangerState *SchemaChangerState

statementPreparer statementPreparer

// validateDbZoneConfig should the DB zone config on commit.
validateDbZoneConfig *bool
}

// copyFromExecCfg copies relevant fields from an ExecutorConfig.
Expand Down
45 changes: 45 additions & 0 deletions pkg/sql/region_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -2590,3 +2590,48 @@ func (p *planner) optimizeSystemDatabase(ctx context.Context) error {

return nil
}

// zoneConfigValidator implements descs.ZoneConfigValidator
type zoneConfigValidator struct {
txn *kv.Txn
descs *descs.Collection
regionProvider descs.RegionProvider
execCfg *ExecutorConfig
}

// newZoneConfigValidator creates a new zone config validator.
func newZoneConfigValidator(
txn *kv.Txn,
descs *descs.Collection,
regionProvider descs.RegionProvider,
execCfg *ExecutorConfig,
) descs.ZoneConfigValidator {
return &zoneConfigValidator{
txn: txn,
descs: descs,
regionProvider: regionProvider,
execCfg: execCfg,
}
}

// ValidateDbZoneConfig implements descs.ZoneConfigValidator.
func (zv *zoneConfigValidator) ValidateDbZoneConfig(
ctx context.Context, db catalog.DatabaseDescriptor,
) error {
regionConfig, err := SynthesizeRegionConfig(
ctx, zv.txn, db.GetID(), zv.descs,
)
if err != nil {
return err
}
_, err = generateAndValidateZoneConfigForMultiRegionDatabase(ctx,
zv.regionProvider,
zv.execCfg,
regionConfig,
true, /*validateLocalities*/
)
if err != nil {
return err
}
return nil
}
5 changes: 4 additions & 1 deletion pkg/sql/schemachanger/scdeps/exec_deps.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,7 +203,10 @@ func (d *txnDeps) DeleteZoneConfig(ctx context.Context, id descpb.ID) error {

// Validate implements the scexec.Catalog interface.
func (d *txnDeps) Validate(ctx context.Context) error {
return d.descsCollection.ValidateUncommittedDescriptors(ctx, d.txn.KV())
return d.descsCollection.ValidateUncommittedDescriptors(ctx,
d.txn.KV(),
false, /*validateZoneConfigs*/
nil /*zoneConfigValidator*/)
}

// Run implements the scexec.Catalog interface.
Expand Down

0 comments on commit 8b9f052

Please sign in to comment.