Skip to content

Commit

Permalink
schemachanger: add support discarding database/table zc
Browse files Browse the repository at this point in the history
This patch supports discarding a database/table zone config
in the declarative schema changer.

Informs: #133157

Release note: None
  • Loading branch information
annrpom committed Oct 30, 2024
1 parent a59d316 commit 7aba686
Show file tree
Hide file tree
Showing 23 changed files with 384 additions and 21 deletions.
28 changes: 28 additions & 0 deletions pkg/ccl/schemachangerccl/backup_base_generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

6 changes: 3 additions & 3 deletions pkg/sql/logictest/testdata/logic_test/event_log
Original file line number Diff line number Diff line change
Expand Up @@ -551,13 +551,13 @@ ORDER BY "timestamp", info

skipif config 3node-tenant-default-configs
query IT
SELECT "reportingID", "info"::JSONB - 'Timestamp' - 'DescriptorID'
SELECT "reportingID", "info"::JSONB - 'Timestamp' - 'DescriptorID' - 'Statement'
FROM system.eventlog
WHERE "eventType" = 'remove_zone_config'
ORDER BY "timestamp", info
----
1 {"EventType": "remove_zone_config", "Statement": "ALTER DATABASE test CONFIGURE ZONE DISCARD", "Tag": "CONFIGURE ZONE", "Target": "DATABASE test", "User": "root"}
1 {"EventType": "remove_zone_config", "Statement": "ALTER TABLE \"\".\"\".a CONFIGURE ZONE DISCARD", "Tag": "CONFIGURE ZONE", "Target": "TABLE test.public.a", "User": "root"}
1 {"EventType": "remove_zone_config", "Tag": "CONFIGURE ZONE", "Target": "DATABASE test", "User": "root"}
1 {"EventType": "remove_zone_config", "Tag": "CONFIGURE ZONE", "Target": "TABLE test.public.a", "User": "root"}

statement ok
DROP TABLE a
Expand Down
24 changes: 19 additions & 5 deletions pkg/sql/schemachanger/scbuild/event_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -467,17 +467,31 @@ func (pb payloadBuilder) build(b buildCtx) logpb.EventPayload {
var zcDetails eventpb.CommonZoneConfigDetails
var oldConfig string
if pb.maybePayload != nil {
payload := pb.maybePayload.(*eventpb.SetZoneConfig)
zcDetails = eventpb.CommonZoneConfigDetails{
Target: payload.Target,
Options: payload.Options,
if payload, ok := pb.maybePayload.(*eventpb.SetZoneConfig); ok {
zcDetails = eventpb.CommonZoneConfigDetails{
Target: payload.Target,
Options: payload.Options,
}
oldConfig = payload.ResolvedOldConfig
}
oldConfig = payload.ResolvedOldConfig
}
return &eventpb.SetZoneConfig{
CommonZoneConfigDetails: zcDetails,
ResolvedOldConfig: oldConfig,
}
} else {
var zcDetails eventpb.CommonZoneConfigDetails
if pb.maybePayload != nil {
if payload, ok := pb.maybePayload.(*eventpb.RemoveZoneConfig); ok {
zcDetails = eventpb.CommonZoneConfigDetails{
Target: payload.Target,
Options: payload.Options,
}
}
}
return &eventpb.RemoveZoneConfig{
CommonZoneConfigDetails: zcDetails,
}
}
case *scpb.Trigger:
if pb.TargetStatus == scpb.Status_PUBLIC {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlclustersettings"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/cockroach/pkg/util/log/logpb"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -83,6 +84,11 @@ func SetZoneConfig(b BuildCtx, n *tree.SetZoneConfig) {

var elem scpb.Element
if n.Discard {
// If we are discarding the zone config and a zone config did not previously
// exist for us to discard, then no-op.
if zco.isNoOp() {
return
}
elem = zco.getZoneConfigElem(b)
b.Drop(elem)
} else {
Expand All @@ -96,16 +102,18 @@ func SetZoneConfig(b BuildCtx, n *tree.SetZoneConfig) {
Target: tree.AsString(&n.ZoneSpecifier),
Options: optionsStr,
}
info := &eventpb.SetZoneConfig{CommonZoneConfigDetails: eventDetails,
ResolvedOldConfig: oldZone.String()}

var info logpb.EventPayload
if n.Discard {
info = &eventpb.RemoveZoneConfig{CommonZoneConfigDetails: eventDetails}
} else {
info = &eventpb.SetZoneConfig{CommonZoneConfigDetails: eventDetails,
ResolvedOldConfig: oldZone.String()}
}
b.LogEventForExistingPayload(elem, info)
}

func astToZoneConfigObject(b BuildCtx, n *tree.SetZoneConfig) (zoneConfigObject, error) {
if n.Discard {
return nil, scerrors.NotImplementedErrorf(n, "discarding zone configurations is not "+
"supported in the DSC")
}
zs := n.ZoneSpecifier
// We are a database object.
if n.Database != "" {
Expand Down Expand Up @@ -161,6 +169,13 @@ func astToZoneConfigObject(b BuildCtx, n *tree.SetZoneConfig) (zoneConfigObject,
return &tzo, nil
}

// TODO(annie): remove this when we add support for discarding subzone
// configs.
if n.Discard {
return nil, scerrors.NotImplementedErrorf(n, "discarding zone configurations on "+
"subzones are not supported in the DSC")
}

izo := indexZoneConfigObj{tableZoneConfigObj: tzo}
// We are an index object. Determine the index ID and fill this
// information out in our zoneConfigObject.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,10 @@ func (dzo *databaseZoneConfigObj) incrementSeqNum() {
dzo.seqNum += 1
}

func (dzo *databaseZoneConfigObj) isNoOp() bool {
return dzo.zoneConfig == nil
}

func (dzo *databaseZoneConfigObj) getZoneConfigElem(b BuildCtx) scpb.Element {
elem := &scpb.DatabaseZoneConfig{
DatabaseID: dzo.databaseID,
Expand Down Expand Up @@ -89,7 +93,7 @@ func (dzo *databaseZoneConfigObj) checkZoneConfigChangePermittedForMultiRegion(
return nil
}

return maybeMultiregionErrorWithHint(options)
return maybeMultiregionErrorWithHint(b, dzo, options)
}

func (dzo *databaseZoneConfigObj) getTargetID() catid.DescID {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,10 @@ type indexZoneConfigObj struct {

var _ zoneConfigObject = &indexZoneConfigObj{}

func (izo *indexZoneConfigObj) isNoOp() bool {
return izo.indexSubzone == nil
}

func (izo *indexZoneConfigObj) getTableZoneConfig() *zonepb.ZoneConfig {
return izo.tableZoneConfigObj.zoneConfig
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,10 @@ type partitionZoneConfigObj struct {

var _ zoneConfigObject = &partitionZoneConfigObj{}

func (pzo *partitionZoneConfigObj) isNoOp() bool {
return pzo.partitionSubzone == nil
}

func (pzo *partitionZoneConfigObj) getTableZoneConfig() *zonepb.ZoneConfig {
return pzo.tableZoneConfigObj.zoneConfig
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,10 @@ type tableZoneConfigObj struct {

var _ zoneConfigObject = &tableZoneConfigObj{}

func (tzo *tableZoneConfigObj) isNoOp() bool {
return tzo.zoneConfig == nil
}

func (tzo *tableZoneConfigObj) getZoneConfigElem(b BuildCtx) scpb.Element {
elem := &scpb.TableZoneConfig{
TableID: tzo.tableID,
Expand Down Expand Up @@ -88,7 +92,7 @@ func (tzo *tableZoneConfigObj) checkZoneConfigChangePermittedForMultiRegion(
return nil
}

return maybeMultiregionErrorWithHint(options)
return maybeMultiregionErrorWithHint(b, tzo, options)
}

func (tzo *tableZoneConfigObj) getTargetID() catid.DescID {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,10 @@ type zoneConfigObjBuilder interface {

// incrementSeqNum increments the seqNum by 1.
incrementSeqNum()

// isNoOp returns true if the zone config object is a no-op. This is defined
// by our object having no zone config yet.
isNoOp() bool
}

type zoneConfigRetriever interface {
Expand Down Expand Up @@ -156,8 +160,37 @@ func resolvePhysicalTableName(b BuildCtx, n *tree.SetZoneConfig) {

// maybeMultiregionErrorWithHint returns an error if the user is trying to
// update a zone config value that's protected for multi-region databases.
func maybeMultiregionErrorWithHint(options tree.KVOptions) error {
func maybeMultiregionErrorWithHint(b BuildCtx, zco zoneConfigObject, options tree.KVOptions) error {
hint := "to override this error, SET override_multi_region_zone_config = true and reissue the command"
// The request is to discard the zone configuration. Error in cases where
// the zone configuration being discarded was created by the multi-region
// abstractions.
if options == nil {
needToError := false
// Determine if this zone config that we're trying to discard is
// supposed to be there. zco is either a database or a table.
_, isDB := zco.(*databaseZoneConfigObj)
if isDB {
needToError = true
} else {
var err error
needToError, err = blockDiscardOfZoneConfigForMultiRegionObject(b, zco.getTargetID())
if err != nil {
return err
}
}

if needToError {
// User is trying to update a zone config value that's protected for
// multi-region databases. Return the constructed error.
err := errors.WithDetail(errors.Newf(
"attempting to discard the zone configuration of a multi-region entity"),
"discarding a multi-region zone configuration may result in sub-optimal performance or behavior",
)
return errors.WithHint(err, hint)
}
}

// This is clearly an n^2 operation, but since there are only a single
// digit number of zone config keys, it's likely faster to do it this way
// than incur the memory allocation of creating a map.
Expand All @@ -176,6 +209,44 @@ func maybeMultiregionErrorWithHint(options tree.KVOptions) error {
return nil
}

// blockDiscardOfZoneConfigForMultiRegionObject determines if discarding the
// zone configuration of a multi-region table, index or partition should be
// blocked. We only block the discard if the multi-region abstractions have
// created the zone configuration. Note that this function relies on internal
// knowledge of which table locality patterns write zone configurations. We do
// things this way to avoid having to read the zone configurations directly and
// do a more explicit comparison (with a generated zone configuration). If, down
// the road, the rules around writing zone configurations change, the tests in
// multi_region_zone_configs will fail and this function will need updating.
func blockDiscardOfZoneConfigForMultiRegionObject(b BuildCtx, tblID catid.DescID) (bool, error) {
tableElems := b.QueryByID(tblID)

// It's a table zone config that the user is trying to discard. This
// should only be present on GLOBAL and REGIONAL BY TABLE tables in a
// specified region.
globalElem := tableElems.FilterTableLocalityGlobal().MustGetZeroOrOneElement()
primaryRegionElem := tableElems.FilterTableLocalityPrimaryRegion().MustGetZeroOrOneElement()
secondaryRegionElem := tableElems.FilterTableLocalitySecondaryRegion().MustGetZeroOrOneElement()
RBRElem := tableElems.FilterTableLocalityRegionalByRow().MustGetZeroOrOneElement()

if globalElem != nil {
return true, nil
} else if secondaryRegionElem != nil {
return true, nil
} else if primaryRegionElem != nil {
// For REGIONAL BY TABLE tables, no need to error if a user-specified
// region does not exist.
return false, nil
} else if RBRElem != nil {
// For REGIONAL BY ROW tables, no need to error if we're setting a
// table level zone config.
return false, nil
} else {
return false, errors.AssertionFailedf(
"unknown table locality %s", b.QueryByID(tblID))
}
}

// isMultiRegionTable returns True if this table is a multi-region table,
// meaning it has locality GLOBAL, or REGIONAL BY TABLE, or REGIONAL BY ROW.
func isMultiRegionTable(b BuildCtx, tableID catid.DescID) bool {
Expand Down Expand Up @@ -1020,6 +1091,11 @@ func prepareZoneConfig(

// No zone was found. Use an empty zone config that inherits from its parent.
if partialZone == nil {
// If we are trying to discard a zone config that doesn't exist in
// system.zones, make this a no-op.
if n.Discard {
return nil, nil, nil
}
partialZone = zonepb.NewZoneConfig()
}
currentZone := protoutil.Clone(partialZone).(*zonepb.ZoneConfig)
Expand Down Expand Up @@ -1051,6 +1127,11 @@ func prepareZoneConfig(
partialZone.CopyFromZone(zoneInheritedFields, copyFromParentList)
}

if n.Discard {
partialZone.DeleteTableConfig()
return nil, partialZone, nil
}

// Determine where to load the configuration.
newZone := *completeZone

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ go_library(
"stats.go",
"table.go",
"trigger.go",
"zone_config.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec",
visibility = ["//visibility:public"],
Expand Down
22 changes: 22 additions & 0 deletions pkg/sql/schemachanger/scexec/scmutationexec/zone_config.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 CockroachDB Software License
// included in the /LICENSE file.

package scmutationexec

import (
"context"

"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop"
)

func (i *immediateVisitor) DiscardZoneConfig(ctx context.Context, op scop.DiscardZoneConfig) error {
zc := op.ZoneConfig
if zc.IsSubzonePlaceholder() && len(zc.Subzones) == 0 {
i.ImmediateMutationStateUpdater.DeleteZoneConfig(op.DescID)
} else {
i.ImmediateMutationStateUpdater.UpdateZoneConfig(op.DescID, zc)
}
return nil
}
7 changes: 7 additions & 0 deletions pkg/sql/schemachanger/scop/immediate_mutation.go
Original file line number Diff line number Diff line change
Expand Up @@ -995,6 +995,13 @@ type AddDatabaseZoneConfig struct {
ZoneConfig *zonepb.ZoneConfig
}

// DiscardZoneConfig discards the zone config for the given descriptor ID.
type DiscardZoneConfig struct {
immediateMutationOp
DescID descpb.ID
ZoneConfig *zonepb.ZoneConfig
}

// AddTableZoneConfig adds a zone config to a table.
type AddTableZoneConfig struct {
immediateMutationOp
Expand Down
Loading

0 comments on commit 7aba686

Please sign in to comment.