Skip to content

Commit

Permalink
sql: add support for dropping regions from multi-region databases
Browse files Browse the repository at this point in the history
This patch builds upon the drop enum infrastructure to add the ability
to drop regions from a multi-region database. It adds support for dropping
any region that isn't the primary region. That change is forthcoming.

When dropping a region, the multi-region enum entry is moved to
"READ_ONLY" capability. This is followed by validation in the type
schema changer to ascertain that no table is using the region value
being dropped in one of its rows. The database descriptor, which
unfortunately duplicates the list of regions, also loses the region
entry when the value is moved to "READ ONLY" mode.

If validation is successful, the region is removed from the
multi-region enum. If, however, validation is unsuccessful, the region
value is moved back to `PUBLIC` and the entry is restored on the
database descriptor.

Special behavior around REGIONAL BY TABLE tables and the effect of that
on type descriptor dependencies is also worth calling out here.
Previously, these tables did not capture their "implicit" bidirectional
dependency with the multi-region type descriptor. The dependency is
described as implicit because even though no column on the table uses
the multi-region type descriptor to store the homing region, a value
from the type descriptor is stored on the table's locality config. This
dependency must be checked against when dropping regions or the type
descriptor itself.

Before this patch, all dependencies a table had with any type descriptor
could be constructed from the column descriptors on the table
descriptor. As mentioned previously, this is no longer possible now
that we have REGIONAL BY TABLE tables in the mix. As such, this requires
modifying how type references are constructed for a table descriptor. In
particular, if a table is a REGIONAL BY TABLE table homed in an explicit
(non-primary) region, the multi-region enum ID is read off the database
descriptor and added to the list of types referenced by the table
descriptor.

Now that a particular locality config state may have this implicit
dependency, switching locality patterns now requires us to be cognizant
of it. This is to say, when moving from a locality pattern that has a
dependency on the multi-region type descriptor to one that does not,
the back reference must be removed (and vice-versa). As such, modifying
the descriptor directly is discourage.Instead, new methods of the form
`alterTableDescLocality...` are provided which negotiate the dependency
linking/unlinking appropriately.

Lastly, this patch also adds validation to ensure that every
REGIONAL BY TABLE table that is homed in an explicit region stores
a reference to the multi-region type descriptor. The validation
semantics around REGIONAL BY TABLE tables and GLOBAL tables are also
handled appropriately.

Informs #58333
Closes #57389

Release note (sql change): ALTER DATABASE ... DROP REGION is now
implemented.
  • Loading branch information
arulajmani authored and otan committed Feb 10, 2021
1 parent 3a3565a commit 86b2eec
Show file tree
Hide file tree
Showing 28 changed files with 1,685 additions and 545 deletions.
23 changes: 23 additions & 0 deletions docs/generated/eventlog.md
Original file line number Diff line number Diff line change
Expand Up @@ -280,6 +280,29 @@ An event of type `alter_database_add_region` is recorded when a region is added
| `RegionName` | The region being added. | yes |


#### Common fields

| Field | Description | Sensitive |
|--|--|--|
| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no |
| `EventType` | The type of the event. | no |
| `Statement` | A normalized copy of the SQL statement that triggered the event. | yes |
| `User` | The user account that triggered the event. | yes |
| `DescriptorID` | The primary object descriptor affected by the operation. Set to zero for operations that don't affect descriptors. | no |
| `ApplicationName` | The application name for the session where the event was emitted. This is included in the event to ease filtering of logging output by application. | yes |
| `PlaceholderValues` | The mapping of SQL placeholders to their values, for prepared statements. | yes |

### `alter_database_drop_region`

AlterDatabaseAddRegion is recorded when a region is added to a database.


| Field | Description | Sensitive |
|--|--|--|
| `DatabaseName` | The name of the database. | yes |
| `RegionName` | The region being dropped. | yes |


#### Common fields

| Field | Description | Sensitive |
Expand Down
8 changes: 6 additions & 2 deletions pkg/ccl/backupccl/backupbase/targets.go
Original file line number Diff line number Diff line change
Expand Up @@ -392,7 +392,9 @@ func DescriptorsMatchingTargets(
return ret, err
}
// Get all the types used by this table.
typeIDs, err := tableDesc.GetAllReferencedTypeIDs(getTypeByID)
desc := resolver.DescByID[tableDesc.GetParentID()]
dbDesc := desc.(catalog.DatabaseDescriptor)
typeIDs, err := tableDesc.GetAllReferencedTypeIDs(dbDesc, getTypeByID)
if err != nil {
return ret, err
}
Expand Down Expand Up @@ -465,7 +467,9 @@ func DescriptorsMatchingTargets(
}
}
// Get all the types used by this table.
typeIDs, err := desc.GetAllReferencedTypeIDs(getTypeByID)
dbRaw := resolver.DescByID[desc.GetParentID()]
dbDesc := dbRaw.(catalog.DatabaseDescriptor)
typeIDs, err := desc.GetAllReferencedTypeIDs(dbDesc, getTypeByID)
if err != nil {
return ret, err
}
Expand Down
21 changes: 19 additions & 2 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -1066,7 +1066,15 @@ func createImportingDescriptors(
// to the new tables being restored.
for _, table := range mutableTables {
// Collect all types used by this table.
typeIDs, err := table.GetAllReferencedTypeIDs(func(id descpb.ID) (catalog.TypeDescriptor, error) {
dbDesc, err := descsCol.GetImmutableDatabaseByID(
ctx, txn, table.GetParentID(), tree.DatabaseLookupFlags{
AvoidCached: true,
IncludeOffline: true,
})
if err != nil {
return err
}
typeIDs, err := table.GetAllReferencedTypeIDs(dbDesc, func(id descpb.ID) (catalog.TypeDescriptor, error) {
return typesByID[id], nil
})
if err != nil {
Expand Down Expand Up @@ -1794,8 +1802,17 @@ func (r *restoreResumer) removeExistingTypeBackReferences(
return typ, nil
}

dbDesc, err := descsCol.GetImmutableDatabaseByID(
ctx, txn, tbl.GetParentID(), tree.DatabaseLookupFlags{
AvoidCached: true,
IncludeOffline: true,
})
if err != nil {
return err
}

// Get all types that this descriptor references.
referencedTypes, err := tbl.GetAllReferencedTypeIDs(lookup)
referencedTypes, err := tbl.GetAllReferencedTypeIDs(dbDesc, lookup)
if err != nil {
return err
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -438,6 +438,7 @@ go_test(
"materialized_view_test.go",
"metric_test.go",
"metric_util_test.go",
"multiregion_test.go",
"mutation_test.go",
"namespace_test.go",
"old_foreign_key_desc_test.go",
Expand Down
155 changes: 134 additions & 21 deletions pkg/sql/alter_database.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -96,8 +95,8 @@ func (p *planner) checkCanAlterDatabaseAndSetNewOwner(
privs := desc.GetPrivileges()
privs.SetOwner(newOwner)

// Log Alter Database Owner event. This is an auditable log event and is recorded
// in the same transaction as the table descriptor update.
// Log Alter Database Owner event. This is an auditable log event and is
// recorded in the same transaction as the table descriptor update.
return p.logEvent(ctx,
desc.GetID(),
&eventpb.AlterDatabaseOwner{
Expand Down Expand Up @@ -161,7 +160,7 @@ func (n *alterDatabaseAddRegionNode) startExec(params runParams) error {
// Add the region to the database descriptor. This function validates that the region
// we're adding is an active member of the cluster and isn't already present in the
// RegionConfig.
if err := params.p.addRegionToRegionConfig(params.ctx, n.desc, n.n); err != nil {
if err := params.p.addActiveRegionToRegionConfig(params.ctx, n.desc, n.n); err != nil {
return err
}

Expand Down Expand Up @@ -234,9 +233,9 @@ func (n *alterDatabaseAddRegionNode) startExec(params runParams) error {
return err
}

// Log Alter Database Add Region event. This is an auditable log event and is recorded
// in the same transaction as the database descriptor, type descriptor, and zone
// configuration updates.
// Log Alter Database Add Region event. This is an auditable log event and is
// recorded in the same transaction as the database descriptor, type
// descriptor, and zone configuration updates.
return params.p.logEvent(params.ctx,
n.desc.GetID(),
&eventpb.AlterDatabaseAddRegion{
Expand All @@ -249,6 +248,11 @@ func (n *alterDatabaseAddRegionNode) Next(runParams) (bool, error) { return fals
func (n *alterDatabaseAddRegionNode) Values() tree.Datums { return tree.Datums{} }
func (n *alterDatabaseAddRegionNode) Close(context.Context) {}

type alterDatabaseDropRegionNode struct {
n *tree.AlterDatabaseDropRegion
desc *dbdesc.Mutable
}

// AlterDatabaseDropRegion transforms a tree.AlterDatabaseDropRegion into a plan node.
func (p *planner) AlterDatabaseDropRegion(
ctx context.Context, n *tree.AlterDatabaseDropRegion,
Expand All @@ -260,9 +264,96 @@ func (p *planner) AlterDatabaseDropRegion(
); err != nil {
return nil, err
}
return nil, unimplemented.NewWithIssue(58333, "implementation pending")
_, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, n.Name.String(),
tree.DatabaseLookupFlags{Required: true})
if err != nil {
return nil, err
}

// To drop the region, the user has to have CREATEDB privileges,
// or be an admin user.
if err := p.CheckRoleOption(ctx, roleoption.CREATEDB); err != nil {
return nil, err
}

if !dbDesc.IsMultiRegion() {
return nil, pgerror.New(pgcode.InvalidDatabaseDefinition, "database has no regions to drop")
}

if dbDesc.RegionConfig.PrimaryRegion == descpb.RegionName(n.Region) {
return nil, errors.WithHintf(
errors.Newf("cannot drop region %q", dbDesc.RegionConfig.PrimaryRegion),
"You must designate another region as the primary region or remove all "+
"other regions before attempting to drop region %q", n.Region,
)
}

return &alterDatabaseDropRegionNode{n, dbDesc}, nil
}

func (n *alterDatabaseDropRegionNode) startExec(params runParams) error {
typeDesc, err := params.p.Descriptors().GetMutableTypeVersionByID(
params.ctx,
params.p.txn,
n.desc.RegionConfig.RegionEnumID,
)
if err != nil {
return err
}

// dropEnumValue tries to remove the region value from the multi-region type
// descriptor. Among other things, it validates that the region is not in
// use by any tables. A region is considered "in use" if either a REGIONAL BY
// TABLE table is explicitly homed in that region or a row in a REGIONAL BY
// ROW table is homed in that region. The type schema changer is responsible
// for all the requisite validation.
if err := params.p.dropEnumValue(params.ctx, typeDesc, tree.EnumValue(n.n.Region)); err != nil {
return err
}

idx := 0
found := false
for i, region := range n.desc.RegionConfig.Regions {
if region.Name == descpb.RegionName(n.n.Region) {
idx = i
found = true
break
}
}
if !found {
// This shouldn't happen and is simply a sanity check to ensure the database
// descriptor regions and multi-region enum regions are indeed consistent.
return errors.AssertionFailedf(
"attempting to drop region %s not on database descriptor %d but found on type descriptor",
n.n.Region, n.desc.GetID(),
)
}

n.desc.RegionConfig.Regions = append(n.desc.RegionConfig.Regions[:idx],
n.desc.RegionConfig.Regions[idx+1:]...)

if err := params.p.writeNonDropDatabaseChange(
params.ctx,
n.desc,
tree.AsStringWithFQNames(n.n, params.Ann()),
); err != nil {
return err
}

// Log Alter Database Drop Region event. This is an auditable log event and is
// recorded in the same transaction as the table descriptor update.
return params.p.logEvent(params.ctx,
n.desc.GetID(),
&eventpb.AlterDatabaseDropRegion{
DatabaseName: n.desc.GetName(),
RegionName: n.n.Region.String(),
})
}

func (n *alterDatabaseDropRegionNode) Next(runParams) (bool, error) { return false, nil }
func (n *alterDatabaseDropRegionNode) Values() tree.Datums { return tree.Datums{} }
func (n *alterDatabaseDropRegionNode) Close(context.Context) {}

type alterDatabasePrimaryRegionNode struct {
n *tree.AlterDatabasePrimaryRegion
desc *dbdesc.Mutable
Expand Down Expand Up @@ -359,12 +450,19 @@ func (n *alterDatabasePrimaryRegionNode) switchPrimaryRegion(params runParams) e
return nil
}

// addDefaultLocalityConfigToAllTables adds a locality config representing
// regional by table table's with affinity to the primary region to all table's
// inside the supplied database.
// addDefaultLocalityConfigToAllTables adds a default locality config to all
// tables inside the supplied database. The default locality config indicates
// that the table is a REGIONAL BY TABLE table homed in the primary region of
// the database.
func addDefaultLocalityConfigToAllTables(
ctx context.Context, p *planner, desc *dbdesc.Immutable,
ctx context.Context, p *planner, desc *dbdesc.Immutable, regionEnumID descpb.ID,
) error {
if !desc.IsMultiRegion() {
return errors.AssertionFailedf(
"cannot add locality config to tables in non multi-region database with ID %d",
desc.GetID(),
)
}
b := p.Txn().NewBatch()
if err := forEachTableDesc(ctx, p, desc, hideVirtual,
func(immutable *dbdesc.Immutable, _ string, desc catalog.TableDescriptor) error {
Expand All @@ -374,7 +472,13 @@ func addDefaultLocalityConfigToAllTables(
if err != nil {
return err
}
mutDesc.SetTableLocalityRegionalByTable(tree.PrimaryRegionLocalityName)

if err := p.alterTableDescLocalityToRegionalByTable(
ctx, tree.PrimaryRegionLocalityName, mutDesc, regionEnumID,
); err != nil {
return err
}

if err := p.writeSchemaChangeToBatch(ctx, mutDesc, b); err != nil {
return err
}
Expand All @@ -385,8 +489,8 @@ func addDefaultLocalityConfigToAllTables(
return p.Txn().Run(ctx, b)
}

// setInitialPrimaryRegion sets the primary region in cases where the database is already
// a multi-region database.
// setInitialPrimaryRegion sets the primary region in cases where the database
// is already a multi-region database.
func (n *alterDatabasePrimaryRegionNode) setInitialPrimaryRegion(params runParams) error {
// Create the region config structure to be added to the database descriptor.
regionConfig, err := params.p.createRegionConfig(
Expand All @@ -399,12 +503,19 @@ func (n *alterDatabasePrimaryRegionNode) setInitialPrimaryRegion(params runParam
return err
}

if err := addDefaultLocalityConfigToAllTables(params.ctx, params.p, &n.desc.Immutable); err != nil {
// Set the region config on the database descriptor.
n.desc.RegionConfig = regionConfig

if err := addDefaultLocalityConfigToAllTables(
params.ctx,
params.p,
&n.desc.Immutable,
regionConfig.RegionEnumID,
); err != nil {
return err
}

// Write the modified database descriptor.
n.desc.RegionConfig = regionConfig
if err := params.p.writeNonDropDatabaseChange(
params.ctx,
n.desc,
Expand Down Expand Up @@ -453,8 +564,9 @@ func (n *alterDatabasePrimaryRegionNode) startExec(params runParams) error {
}
}

// Log Alter Database Primary Region event. This is an auditable log event and is recorded
// in the same transaction as the database descriptor, and zone configuration updates.
// Log Alter Database Primary Region event. This is an auditable log event and
// is recorded in the same transaction as the database descriptor, and zone
// configuration updates.
return params.p.logEvent(params.ctx,
n.desc.GetID(),
&eventpb.AlterDatabasePrimaryRegion{
Expand Down Expand Up @@ -560,8 +672,9 @@ func (n *alterDatabaseSurvivalGoalNode) startExec(params runParams) error {
return err
}

// Log Alter Database Survival Goal event. This is an auditable log event and is recorded
// in the same transaction as the database descriptor, and zone configuration updates.
// Log Alter Database Survival Goal event. This is an auditable log event and
// is recorded in the same transaction as the database descriptor, and zone
// configuration updates.
return params.p.logEvent(params.ctx,
n.desc.GetID(),
&eventpb.AlterDatabaseSurvivalGoal{
Expand Down
Loading

0 comments on commit 86b2eec

Please sign in to comment.