diff --git a/pkg/ccl/schemachangerccl/sctestbackupccl/backup_base_generated_test.go b/pkg/ccl/schemachangerccl/sctestbackupccl/backup_base_generated_test.go index 6e72e1b10caf..98fb3b46960c 100644 --- a/pkg/ccl/schemachangerccl/sctestbackupccl/backup_base_generated_test.go +++ b/pkg/ccl/schemachangerccl/sctestbackupccl/backup_base_generated_test.go @@ -512,6 +512,13 @@ func TestBackupRollbacks_base_alter_table_set_schema(t *testing.T) { sctest.BackupRollbacks(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestBackupRollbacks_base_alter_table_set_storage_param(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param" + sctest.BackupRollbacks(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestBackupRollbacks_base_alter_table_validate_constraint(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1324,6 +1331,13 @@ func TestBackupRollbacksMixedVersion_base_alter_table_set_schema(t *testing.T) { sctest.BackupRollbacksMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestBackupRollbacksMixedVersion_base_alter_table_set_storage_param(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param" + sctest.BackupRollbacksMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestBackupRollbacksMixedVersion_base_alter_table_validate_constraint(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -2136,6 +2150,13 @@ func TestBackupSuccess_base_alter_table_set_schema(t *testing.T) { sctest.BackupSuccess(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestBackupSuccess_base_alter_table_set_storage_param(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param" + sctest.BackupSuccess(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestBackupSuccess_base_alter_table_validate_constraint(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -2948,6 +2969,13 @@ func TestBackupSuccessMixedVersion_base_alter_table_set_schema(t *testing.T) { sctest.BackupSuccessMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestBackupSuccessMixedVersion_base_alter_table_set_storage_param(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param" + sctest.BackupSuccessMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestBackupSuccessMixedVersion_base_alter_table_validate_constraint(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel index a1799ca20a24..39357f58e419 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel @@ -23,6 +23,7 @@ go_library( "alter_table_rename_constraint.go", "alter_table_set_rls_mode.go", "alter_table_set_schema.go", + "alter_table_set_storage_param.go", "alter_table_validate_constraint.go", "comment_on.go", "configure_zone.go", @@ -118,6 +119,7 @@ go_library( "//pkg/sql/sqltelemetry", "//pkg/sql/storageparam", "//pkg/sql/storageparam/indexstorageparam", + "//pkg/sql/storageparam/tablestorageparam", "//pkg/sql/types", "//pkg/sql/vecindex", "//pkg/sql/vecindex/vecpb", diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go index 0d17e7800251..1c06aa6a2337 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go @@ -48,6 +48,7 @@ var supportedAlterTableStatements = map[reflect.Type]supportedAlterTableCommand{ reflect.TypeOf((*tree.AlterTableSetIdentity)(nil)): {fn: alterTableSetIdentity, on: true, checks: isV261Active}, reflect.TypeOf((*tree.AlterTableAddIdentity)(nil)): {fn: alterTableAddIdentity, on: true, checks: isV261Active}, reflect.TypeOf((*tree.AlterTableSetVisible)(nil)): {fn: alterTableAlterColumnSetVisible, on: true, checks: isV261Active}, + reflect.TypeOf((*tree.AlterTableSetStorageParams)(nil)): {fn: AlterTableSetStorageParams, on: true, checks: isV261Active}, } func init() { diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go index d168b90ee6d2..d994abdfa228 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go @@ -326,7 +326,7 @@ func walkColumnDependencies( *scpb.UniqueWithoutIndexConstraint, *scpb.CheckConstraint, *scpb.UniqueWithoutIndexConstraintUnvalidated, *scpb.CheckConstraintUnvalidated, *scpb.RowLevelTTL, *scpb.PolicyUsingExpr, *scpb.PolicyWithCheckExpr, - *scpb.TriggerDeps, *scpb.ColumnGeneratedAsIdentity, *scpb.ColumnHidden: + *scpb.TriggerDeps, *scpb.ColumnGeneratedAsIdentity, *scpb.ColumnHidden, *scpb.TableStorageParam: fn(e, op, objType) case *scpb.ColumnType: if elt.ColumnID == col.ColumnID { diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_set_storage_param.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_set_storage_param.go new file mode 100644 index 000000000000..9b244acd0537 --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_set_storage_param.go @@ -0,0 +1,127 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package scbuildstmt + +import ( + "strings" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/storageparam" + "github.com/cockroachdb/cockroach/pkg/sql/storageparam/tablestorageparam" + "github.com/cockroachdb/errors" +) + +// AlterTableSetStorageParams implements ALTER TABLE ... SET {storage_param} in the declarative schema changer. +func AlterTableSetStorageParams( + b BuildCtx, + tn *tree.TableName, + tbl *scpb.Table, + stmt tree.Statement, + t *tree.AlterTableSetStorageParams, +) { + if err := storageparam.StorageParamPreChecks( + b, + b.EvalCtx(), + false, /* isNewObject */ + t.StorageParams, + nil, /* resetParams */ + ); err != nil { + panic(err) + } + for _, param := range t.StorageParams { + val, err := tablestorageparam.ParseAndValidate(b, b.SemaCtx(), b.EvalCtx(), param) + if err != nil { + panic(err) // tried to set an invalid value for param + } + key := param.Key + if strings.HasPrefix(key, "ttl") { + panic(scerrors.NotImplementedErrorf(t, "set ttl not implemented yet")) + } + if key == catpb.RBRUsingConstraintTableSettingName { + panic(scerrors.NotImplementedErrorf(t, "set rbr not implemented yet")) + } + // schema_locked has a dedicated element and will be handled differently + if handleSetSchemaLockIfNeeded(b, tbl, key, val) { + return + } + // Do extra validation for exclude_data_from_backup + validateExcludeDataFromBackup(b, tbl, key) + currElem := b.QueryByID(tbl.TableID).FilterTableStorageParam().Filter( + func(_ scpb.Status, _ scpb.TargetStatus, e *scpb.TableStorageParam) bool { + return e.Name == key + }).MustGetZeroOrOneElement() + if currElem != nil { + b.Drop(currElem) + } + if val != "" { + newElem := scpb.TableStorageParam{ + TableID: tbl.TableID, + Name: key, + Value: val, + } + b.Add(&newElem) + } + } +} + +func validateExcludeDataFromBackup(b BuildCtx, tbl *scpb.Table, key string) { + if key != "exclude_data_from_backup" { + return + } + if tbl.IsTemporary { + panic(pgerror.Newf(pgcode.FeatureNotSupported, + "cannot set data in a temporary table to be excluded from backup")) + } + // Check that the table does not have any incoming FK references. During a + // backup, the rows of a table with ephemeral data will not be backed up, and + // could result in a violation of FK constraints on restore. To prevent this, + // we only allow a table with no incoming FK references to be marked as + // ephemeral. + if isTableReferencedByFK(b, tbl) { + panic(errors.New("cannot set data in a table with inbound foreign key constraints to be excluded from backup")) + } +} + +// isTableReferencedByFK returns the count of inbound foreign key constraints +// where tbl is the referenced table. +func isTableReferencedByFK(b BuildCtx, tbl *scpb.Table) bool { + hasInboundFK := false + backRefs := b.BackReferences(tbl.TableID) + // Check validated foreign key constraints + backRefs.FilterForeignKeyConstraint().ForEach(func(current scpb.Status, target scpb.TargetStatus, e *scpb.ForeignKeyConstraint) { + if e.ReferencedTableID == tbl.TableID { + hasInboundFK = true + } + }) + // Check unvalidated foreign key constraints + backRefs.FilterForeignKeyConstraintUnvalidated().ForEach(func(current scpb.Status, target scpb.TargetStatus, e *scpb.ForeignKeyConstraintUnvalidated) { + if e.ReferencedTableID == tbl.TableID { + hasInboundFK = true + } + }) + return hasInboundFK +} + +func handleSetSchemaLockIfNeeded(b BuildCtx, tbl *scpb.Table, key, val string) bool { + if key != "schema_locked" { + return false + } + currElem := b.QueryByID(tbl.TableID).FilterTableSchemaLocked().MustGetZeroOrOneElement() + // Drop element if schema is currently locked + if currElem != nil && val == "false" { + b.Drop(currElem) + } + // Add element if schema is currently unlocked + if currElem == nil && val == "true" { + b.Add(&scpb.TableSchemaLocked{TableID: tbl.TableID}) + } + return true +} diff --git a/pkg/sql/schemachanger/scdecomp/decomp.go b/pkg/sql/schemachanger/scdecomp/decomp.go index 58e15831f209..e80f237313a1 100644 --- a/pkg/sql/schemachanger/scdecomp/decomp.go +++ b/pkg/sql/schemachanger/scdecomp/decomp.go @@ -9,6 +9,7 @@ import ( "context" "fmt" "reflect" + "strings" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/geo/geopb" @@ -487,6 +488,31 @@ func (w *walkCtx) walkRelation(tbl catalog.TableDescriptor) { JobIDs: tbl.TableDesc().LDRJobIDs, }) } + w.walkStorageParams(tbl) +} + +// walkStorageParams walks through table storage parameters and creates +// TableStorageParam elements for each non-null parameter. +func (w *walkCtx) walkStorageParams(tbl catalog.TableDescriptor) { + tableID := tbl.GetID() + storageParams, err := tbl.GetStorageParams(false) + if err != nil { + panic(err) + } + for _, param := range storageParams { + key, value, found := strings.Cut(param, "=") + if !found { + continue + } + if key == "schema_locked" { + continue + } + w.ev(scpb.Status_PUBLIC, &scpb.TableStorageParam{ + TableID: tableID, + Name: key, + Value: value, + }) + } } func (w *walkCtx) walkLocality(tbl catalog.TableDescriptor, l *catpb.LocalityConfig) { diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel index a4b8abb0208c..de58ce793140 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel +++ b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel @@ -54,6 +54,7 @@ go_library( "//pkg/sql/sem/catid", "//pkg/sql/sem/idxtype", "//pkg/sql/sem/tree", + "//pkg/sql/storageparam/tablestorageparam", "//pkg/sql/types", "//pkg/util/iterutil", "//pkg/util/protoutil", diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/table.go b/pkg/sql/schemachanger/scexec/scmutationexec/table.go index 280c134d4e60..5941d426750a 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/table.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/table.go @@ -10,6 +10,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/storageparam/tablestorageparam" "github.com/cockroachdb/cockroach/pkg/util/protoutil" ) @@ -31,6 +32,28 @@ func (i *immediateVisitor) SetTableSchemaLocked( return nil } +func (i *immediateVisitor) SetTableStorageParam( + ctx context.Context, op scop.SetTableStorageParam, +) error { + tbl, err := i.checkOutTable(ctx, op.TableID) + if err != nil { + return err + } + setter := tablestorageparam.NewSetter(tbl, false) + return setter.SetToStringValue(ctx, op.Name, op.Value) +} + +func (i *immediateVisitor) ResetTableStorageParam( + ctx context.Context, op scop.ResetTableStorageParam, +) error { + tbl, err := i.checkOutTable(ctx, op.TableID) + if err != nil { + return err + } + setter := tablestorageparam.NewSetter(tbl, false) + return setter.ResetToStringValue(ctx, op.Name, op.Value) +} + func (d *deferredVisitor) UpdateTTLScheduleMetadata( ctx context.Context, op scop.UpdateTTLScheduleMetadata, ) error { diff --git a/pkg/sql/schemachanger/scop/immediate_mutation.go b/pkg/sql/schemachanger/scop/immediate_mutation.go index 89f98de80253..e1860ab19de2 100644 --- a/pkg/sql/schemachanger/scop/immediate_mutation.go +++ b/pkg/sql/schemachanger/scop/immediate_mutation.go @@ -1240,3 +1240,15 @@ type SetTableSchemaLocked struct { TableID descpb.ID Locked bool } + +// SetTableStorageParam will add a new generated as identity to a column. +type SetTableStorageParam struct { + immediateMutationOp + scpb.TableStorageParam +} + +// ResetTableStorageParam will remove the generated as identity from a column. +type ResetTableStorageParam struct { + immediateMutationOp + scpb.TableStorageParam +} diff --git a/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go index d54752dad974..b2a81c9b73ce 100644 --- a/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go +++ b/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go @@ -176,6 +176,8 @@ type ImmediateMutationVisitor interface { MarkRecreatedIndexesAsVisible(context.Context, MarkRecreatedIndexesAsVisible) error MarkRecreatedIndexAsVisible(context.Context, MarkRecreatedIndexAsVisible) error SetTableSchemaLocked(context.Context, SetTableSchemaLocked) error + SetTableStorageParam(context.Context, SetTableStorageParam) error + ResetTableStorageParam(context.Context, ResetTableStorageParam) error } // Visit is part of the ImmediateMutationOp interface. @@ -972,3 +974,13 @@ func (op MarkRecreatedIndexAsVisible) Visit(ctx context.Context, v ImmediateMuta func (op SetTableSchemaLocked) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.SetTableSchemaLocked(ctx, op) } + +// Visit is part of the ImmediateMutationOp interface. +func (op SetTableStorageParam) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.SetTableStorageParam(ctx, op) +} + +// Visit is part of the ImmediateMutationOp interface. +func (op ResetTableStorageParam) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.ResetTableStorageParam(ctx, op) +} diff --git a/pkg/sql/schemachanger/scpb/elements.proto b/pkg/sql/schemachanger/scpb/elements.proto index b1951e04c4a3..6c7f2333690d 100644 --- a/pkg/sql/schemachanger/scpb/elements.proto +++ b/pkg/sql/schemachanger/scpb/elements.proto @@ -95,6 +95,7 @@ message ElementProto { Policy policy = 137 [(gogoproto.moretags) = "parent:\"Table\""]; RowLevelSecurityEnabled row_level_security_enabled = 138 [(gogoproto.moretags) = "parent:\"Table\""]; RowLevelSecurityForced row_level_security_forced = 139 [(gogoproto.moretags) = "parent:\"Table\""]; + TableStorageParam table_storage_param = 142 [(gogoproto.moretags) = "parent:\"Table\""]; // Multi-region elements. TableLocalityGlobal table_locality_global = 110 [(gogoproto.moretags) = "parent:\"Table\""]; @@ -960,6 +961,13 @@ message TableSchemaLocked { uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; } +// TableStorageParam models storage parameter `schema_locked` of a table. +message TableStorageParam { + uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; + string name = 2; + string value = 3; +} + // LDRJobIDs models the field `ldr_job_ids` of a table descriptor. message LDRJobIDs { uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; diff --git a/pkg/sql/schemachanger/scpb/elements_generated.go b/pkg/sql/schemachanger/scpb/elements_generated.go index ae97564ff8f6..6f720b773cec 100644 --- a/pkg/sql/schemachanger/scpb/elements_generated.go +++ b/pkg/sql/schemachanger/scpb/elements_generated.go @@ -2785,6 +2785,43 @@ func (c *ElementCollection[E]) FilterTableSchemaLocked() *ElementCollection[*Tab return (*ElementCollection[*TableSchemaLocked])(ret) } +func (e TableStorageParam) element() {} + +// Element implements ElementGetter. +func (e * ElementProto_TableStorageParam) Element() Element { + return e.TableStorageParam +} + +// ForEachTableStorageParam iterates over elements of type TableStorageParam. +// Deprecated +func ForEachTableStorageParam( + c *ElementCollection[Element], fn func(current Status, target TargetStatus, e *TableStorageParam), +) { + c.FilterTableStorageParam().ForEach(fn) +} + +// FindTableStorageParam finds the first element of type TableStorageParam. +// Deprecated +func FindTableStorageParam( + c *ElementCollection[Element], +) (current Status, target TargetStatus, element *TableStorageParam) { + if tc := c.FilterTableStorageParam(); !tc.IsEmpty() { + var e Element + current, target, e = tc.Get(0) + element = e.(*TableStorageParam) + } + return current, target, element +} + +// TableStorageParamElements filters elements of type TableStorageParam. +func (c *ElementCollection[E]) FilterTableStorageParam() *ElementCollection[*TableStorageParam] { + ret := c.genericFilter(func(_ Status, _ TargetStatus, e Element) bool { + _, ok := e.(*TableStorageParam) + return ok + }) + return (*ElementCollection[*TableStorageParam])(ret) +} + func (e TableZoneConfig) element() {} // Element implements ElementGetter. @@ -3532,6 +3569,8 @@ func (e* ElementProto) SetElement(element Element) { e.ElementOneOf = &ElementProto_TablePartitioning{ TablePartitioning: t} case *TableSchemaLocked: e.ElementOneOf = &ElementProto_TableSchemaLocked{ TableSchemaLocked: t} + case *TableStorageParam: + e.ElementOneOf = &ElementProto_TableStorageParam{ TableStorageParam: t} case *TableZoneConfig: e.ElementOneOf = &ElementProto_TableZoneConfig{ TableZoneConfig: t} case *TemporaryIndex: @@ -3645,6 +3684,7 @@ func GetElementOneOfProtos() []interface{} { ((*ElementProto_TableLocalitySecondaryRegion)(nil)), ((*ElementProto_TablePartitioning)(nil)), ((*ElementProto_TableSchemaLocked)(nil)), + ((*ElementProto_TableStorageParam)(nil)), ((*ElementProto_TableZoneConfig)(nil)), ((*ElementProto_TemporaryIndex)(nil)), ((*ElementProto_Trigger)(nil)), @@ -3743,6 +3783,7 @@ func GetElementTypes() []interface{} { ((*TableLocalitySecondaryRegion)(nil)), ((*TablePartitioning)(nil)), ((*TableSchemaLocked)(nil)), + ((*TableStorageParam)(nil)), ((*TableZoneConfig)(nil)), ((*TemporaryIndex)(nil)), ((*Trigger)(nil)), diff --git a/pkg/sql/schemachanger/scpb/uml/table.puml b/pkg/sql/schemachanger/scpb/uml/table.puml index 7a3ca2112c4b..fb63e4ca8e68 100644 --- a/pkg/sql/schemachanger/scpb/uml/table.puml +++ b/pkg/sql/schemachanger/scpb/uml/table.puml @@ -463,6 +463,12 @@ object TableSchemaLocked TableSchemaLocked : TableID +object TableStorageParam + +TableStorageParam : TableID +TableStorageParam : Name +TableStorageParam : Value + object TableZoneConfig TableZoneConfig : TableID @@ -675,6 +681,7 @@ Table <|-- TableLocalityRegionalByRowUsingConstraint Table <|-- TableLocalitySecondaryRegion Table <|-- TablePartitioning Table <|-- TableSchemaLocked +Table <|-- TableStorageParam Table <|-- TableZoneConfig View <|-- TableZoneConfig Table <|-- TemporaryIndex diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel index db1e2d4ccb7f..f7a7d0555ebb 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel @@ -80,6 +80,7 @@ go_library( "opgen_table_locality_secondary_region.go", "opgen_table_partitioning.go", "opgen_table_schema_locked.go", + "opgen_table_storage_params.go", "opgen_table_zone_config.go", "opgen_temporary_index.go", "opgen_trigger.go", diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_table_storage_params.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_table_storage_params.go new file mode 100644 index 000000000000..50f478effdd4 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_table_storage_params.go @@ -0,0 +1,37 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package opgen + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" +) + +func init() { + opRegistry.register((*scpb.TableStorageParam)(nil), + toPublic( + scpb.Status_ABSENT, + to(scpb.Status_PUBLIC, + emit(func(this *scpb.TableStorageParam) *scop.SetTableStorageParam { + return &scop.SetTableStorageParam{ + TableStorageParam: *protoutil.Clone(this).(*scpb.TableStorageParam), + } + }), + ), + ), + toAbsent( + scpb.Status_PUBLIC, + to(scpb.Status_ABSENT, + emit(func(this *scpb.TableStorageParam) *scop.ResetTableStorageParam { + return &scop.ResetTableStorageParam{ + TableStorageParam: *protoutil.Clone(this).(*scpb.TableStorageParam), + } + }), + ), + ), + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/rules/current/BUILD.bazel index ea2610f81ad1..6b180a9bddf2 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/BUILD.bazel @@ -27,6 +27,7 @@ go_library( "dep_rename_constraint.go", "dep_rename_table.go", "dep_schema_locked.go", + "dep_storage_param.go", "dep_swap_index.go", "dep_two_version.go", "helpers.go", diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_storage_param.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_storage_param.go new file mode 100644 index 000000000000..40136a552c65 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_storage_param.go @@ -0,0 +1,34 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package current + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" +) + +// These rules ensure that when setting a storage param, the old element is dropped +// before the new one is added. This prevents conflicts and ensures proper rollback behavior. +func init() { + registerDepRule( + "old storage param is dropped before the new one is added", + scgraph.SameStagePrecedence, + "old-storage-param", "new-storage-param", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.TableStorageParam)(nil)), + to.Type((*scpb.TableStorageParam)(nil)), + JoinOnDescID(from, to, "table-id"), + from.TargetStatus(scpb.ToAbsent), + from.CurrentStatus(scpb.Status_ABSENT), + to.TargetStatus(scpb.ToPublic), + to.CurrentStatus(scpb.Status_PUBLIC), + } + }, + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules index fe8145e24020..7274a4ce075c 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules @@ -3284,7 +3284,7 @@ deprules kind: Precedence to: relation-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $relation, $relation-id) - ToPublicOrTransient($dependent-Target, $relation-Target) @@ -3582,7 +3582,7 @@ deprules to: referencing-via-attr-Node query: - $referenced-descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id) - toAbsent($referenced-descriptor-Target, $referencing-via-attr-Target) - $referenced-descriptor-Node[CurrentStatus] = DROPPED @@ -3638,7 +3638,7 @@ deprules to: dependent-Node query: - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($descriptor, $dependent, $desc-id) - toAbsent($descriptor-Target, $dependent-Target) - $descriptor-Node[CurrentStatus] = DROPPED @@ -3677,7 +3677,7 @@ deprules to: dependent-Node query: - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($relation, $dependent, $relation-id) - ToPublicOrTransient($relation-Target, $dependent-Target) - $relation-Node[CurrentStatus] = DESCRIPTOR_ADDED @@ -4268,7 +4268,7 @@ deprules kind: Precedence to: descriptor-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $descriptor, $desc-id) - toAbsent($dependent-Target, $descriptor-Target) @@ -4414,6 +4414,20 @@ deprules - joinTargetNode($new-index, $new-index-Target, $new-index-Node) - joinTargetNode($new-primary-index, $new-primary-index-Target, $new-primary-index-Node) - joinTargetNode($old-secondary-index, $old-secondary-index-Target, $old-secondary-index-Node) +- name: old storage param is dropped before the new one is added + from: old-storage-param-Node + kind: SameStagePrecedence + to: new-storage-param-Node + query: + - $old-storage-param[Type] = '*scpb.TableStorageParam' + - $new-storage-param[Type] = '*scpb.TableStorageParam' + - joinOnDescID($old-storage-param, $new-storage-param, $table-id) + - $old-storage-param-Target[TargetStatus] = ABSENT + - $old-storage-param-Node[CurrentStatus] = ABSENT + - $new-storage-param-Target[TargetStatus] = PUBLIC + - $new-storage-param-Node[CurrentStatus] = PUBLIC + - joinTargetNode($old-storage-param, $old-storage-param-Target, $old-storage-param-Node) + - joinTargetNode($new-storage-param, $new-storage-param-Target, $new-storage-param-Node) - name: old table namespace is dropped before new table namespace is added from: old-namespace-Node kind: Precedence @@ -4712,7 +4726,7 @@ deprules kind: PreviousTransactionPrecedence to: schema-locked-Node query: - - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] + - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] - $schema-locked[Type] = '*scpb.TableSchemaLocked' - joinOnDescID($descriptor-element, $schema-locked, $descID) - toPublicToTransientPublicUntyped($descriptor-element-Target, $schema-locked-Target) @@ -4725,7 +4739,7 @@ deprules kind: PreviousTransactionPrecedence to: schema-locked-Node query: - - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] + - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] - $schema-locked[Type] = '*scpb.TableSchemaLocked' - joinOnDescID($descriptor-element, $schema-locked, $descID) - toDropToTransientPublicUntyped($descriptor-element-Target, $schema-locked-Target) @@ -4739,7 +4753,7 @@ deprules to: descriptor-element-Node query: - $schema-locked[Type] = '*scpb.TableSchemaLocked' - - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] + - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] - joinOnDescID($schema-locked, $descriptor-element, $descID) - toPublicToTransientPublicUntyped($descriptor-element-Target, $schema-locked-Target) - $schema-locked-Node[CurrentStatus] = ABSENT @@ -4751,7 +4765,7 @@ deprules to: descriptor-element-Node query: - $schema-locked[Type] = '*scpb.TableSchemaLocked' - - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] + - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] - joinOnDescID($schema-locked, $descriptor-element, $descID) - toDropToTransientPublicUntyped($descriptor-element-Target, $schema-locked-Target) - $schema-locked-Node[CurrentStatus] = ABSENT @@ -8535,7 +8549,7 @@ deprules kind: Precedence to: relation-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $relation, $relation-id) - ToPublicOrTransient($dependent-Target, $relation-Target) @@ -8833,7 +8847,7 @@ deprules to: referencing-via-attr-Node query: - $referenced-descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id) - toAbsent($referenced-descriptor-Target, $referencing-via-attr-Target) - $referenced-descriptor-Node[CurrentStatus] = DROPPED @@ -8889,7 +8903,7 @@ deprules to: dependent-Node query: - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($descriptor, $dependent, $desc-id) - toAbsent($descriptor-Target, $dependent-Target) - $descriptor-Node[CurrentStatus] = DROPPED @@ -8928,7 +8942,7 @@ deprules to: dependent-Node query: - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($relation, $dependent, $relation-id) - ToPublicOrTransient($relation-Target, $dependent-Target) - $relation-Node[CurrentStatus] = DESCRIPTOR_ADDED @@ -9519,7 +9533,7 @@ deprules kind: Precedence to: descriptor-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $descriptor, $desc-id) - toAbsent($dependent-Target, $descriptor-Target) @@ -9665,6 +9679,20 @@ deprules - joinTargetNode($new-index, $new-index-Target, $new-index-Node) - joinTargetNode($new-primary-index, $new-primary-index-Target, $new-primary-index-Node) - joinTargetNode($old-secondary-index, $old-secondary-index-Target, $old-secondary-index-Node) +- name: old storage param is dropped before the new one is added + from: old-storage-param-Node + kind: SameStagePrecedence + to: new-storage-param-Node + query: + - $old-storage-param[Type] = '*scpb.TableStorageParam' + - $new-storage-param[Type] = '*scpb.TableStorageParam' + - joinOnDescID($old-storage-param, $new-storage-param, $table-id) + - $old-storage-param-Target[TargetStatus] = ABSENT + - $old-storage-param-Node[CurrentStatus] = ABSENT + - $new-storage-param-Target[TargetStatus] = PUBLIC + - $new-storage-param-Node[CurrentStatus] = PUBLIC + - joinTargetNode($old-storage-param, $old-storage-param-Target, $old-storage-param-Node) + - joinTargetNode($new-storage-param, $new-storage-param-Target, $new-storage-param-Node) - name: old table namespace is dropped before new table namespace is added from: old-namespace-Node kind: Precedence @@ -9963,7 +9991,7 @@ deprules kind: PreviousTransactionPrecedence to: schema-locked-Node query: - - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] + - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] - $schema-locked[Type] = '*scpb.TableSchemaLocked' - joinOnDescID($descriptor-element, $schema-locked, $descID) - toPublicToTransientPublicUntyped($descriptor-element-Target, $schema-locked-Target) @@ -9976,7 +10004,7 @@ deprules kind: PreviousTransactionPrecedence to: schema-locked-Node query: - - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] + - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] - $schema-locked[Type] = '*scpb.TableSchemaLocked' - joinOnDescID($descriptor-element, $schema-locked, $descID) - toDropToTransientPublicUntyped($descriptor-element-Target, $schema-locked-Target) @@ -9990,7 +10018,7 @@ deprules to: descriptor-element-Node query: - $schema-locked[Type] = '*scpb.TableSchemaLocked' - - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] + - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] - joinOnDescID($schema-locked, $descriptor-element, $descID) - toPublicToTransientPublicUntyped($descriptor-element-Target, $schema-locked-Target) - $schema-locked-Node[CurrentStatus] = ABSENT @@ -10002,7 +10030,7 @@ deprules to: descriptor-element-Node query: - $schema-locked[Type] = '*scpb.TableSchemaLocked' - - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] + - $descriptor-element[Type] IN ['*scpb.AliasType', '*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnGeneratedAsIdentity', '*scpb.ColumnHidden', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.Database', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumType', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.Function', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyDeps', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PolicyUsingExpr', '*scpb.PolicyWithCheckExpr', '*scpb.PrimaryIndex', '*scpb.RowLevelSecurityEnabled', '*scpb.RowLevelSecurityForced', '*scpb.RowLevelTTL', '*scpb.Schema', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.Sequence', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.Table', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalityRegionalByRowUsingConstraint', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableStorageParam', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges', '*scpb.View'] - joinOnDescID($schema-locked, $descriptor-element, $descID) - toDropToTransientPublicUntyped($descriptor-element-Target, $schema-locked-Target) - $schema-locked-Node[CurrentStatus] = ABSENT diff --git a/pkg/sql/schemachanger/screl/attr.go b/pkg/sql/schemachanger/screl/attr.go index 9d899dcbd054..3ee6be4357fd 100644 --- a/pkg/sql/schemachanger/screl/attr.go +++ b/pkg/sql/schemachanger/screl/attr.go @@ -535,6 +535,11 @@ var elementSchemaOptions = []rel.SchemaOption{ rel.EntityMapping(t((*scpb.TableSchemaLocked)(nil)), rel.EntityAttr(DescID, "TableID"), ), + rel.EntityMapping(t((*scpb.TableStorageParam)(nil)), + rel.EntityAttr(DescID, "TableID"), + rel.EntityAttr(Name, "Name"), + rel.EntityAttr(Value, "Value"), + ), rel.EntityMapping(t((*scpb.RowLevelSecurityEnabled)(nil)), rel.EntityAttr(DescID, "TableID"), ), diff --git a/pkg/sql/schemachanger/screl/scalars.go b/pkg/sql/schemachanger/screl/scalars.go index e3475ae8aeb4..0e782bf8b6fc 100644 --- a/pkg/sql/schemachanger/screl/scalars.go +++ b/pkg/sql/schemachanger/screl/scalars.go @@ -136,7 +136,7 @@ func VersionSupportsElementUse(el scpb.Element, version clusterversion.ClusterVe return version.IsActive(clusterversion.V25_2) case *scpb.TableLocalityRegionalByRowUsingConstraint: return version.IsActive(clusterversion.V25_3) - case *scpb.ColumnGeneratedAsIdentity, *scpb.ColumnHidden: + case *scpb.ColumnGeneratedAsIdentity, *scpb.ColumnHidden, *scpb.TableStorageParam: return version.IsActive(clusterversion.V26_1) default: panic(errors.AssertionFailedf("unknown element %T", el)) diff --git a/pkg/sql/schemachanger/sctest_generated_test.go b/pkg/sql/schemachanger/sctest_generated_test.go index 71df739a8ead..90b7fdc1f4e4 100644 --- a/pkg/sql/schemachanger/sctest_generated_test.go +++ b/pkg/sql/schemachanger/sctest_generated_test.go @@ -512,6 +512,13 @@ func TestEndToEndSideEffects_alter_table_set_schema(t *testing.T) { sctest.EndToEndSideEffects(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestEndToEndSideEffects_alter_table_set_storage_param(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param" + sctest.EndToEndSideEffects(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestEndToEndSideEffects_alter_table_validate_constraint(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1324,6 +1331,13 @@ func TestExecuteWithDMLInjection_alter_table_set_schema(t *testing.T) { sctest.ExecuteWithDMLInjection(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestExecuteWithDMLInjection_alter_table_set_storage_param(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param" + sctest.ExecuteWithDMLInjection(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestExecuteWithDMLInjection_alter_table_validate_constraint(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -2136,6 +2150,13 @@ func TestGenerateSchemaChangeCorpus_alter_table_set_schema(t *testing.T) { sctest.GenerateSchemaChangeCorpus(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestGenerateSchemaChangeCorpus_alter_table_set_storage_param(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param" + sctest.GenerateSchemaChangeCorpus(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestGenerateSchemaChangeCorpus_alter_table_validate_constraint(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -2948,6 +2969,13 @@ func TestPause_alter_table_set_schema(t *testing.T) { sctest.Pause(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestPause_alter_table_set_storage_param(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param" + sctest.Pause(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestPause_alter_table_validate_constraint(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -3760,6 +3788,13 @@ func TestPauseMixedVersion_alter_table_set_schema(t *testing.T) { sctest.PauseMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestPauseMixedVersion_alter_table_set_storage_param(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param" + sctest.PauseMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestPauseMixedVersion_alter_table_validate_constraint(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -4572,6 +4607,13 @@ func TestRollback_alter_table_set_schema(t *testing.T) { sctest.Rollback(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestRollback_alter_table_set_storage_param(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param" + sctest.Rollback(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestRollback_alter_table_validate_constraint(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param.definition b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param.definition new file mode 100644 index 000000000000..5a4a88efc719 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param.definition @@ -0,0 +1,9 @@ +setup +CREATE TABLE t (i int); +CREATE SCHEMA sc; +---- + +# note that autovacuum_enabled should not show a param because setting it is a noop +test +ALTER TABLE t SET (exclude_data_from_backup=true, autovacuum_enabled=false, sql_stats_histogram_buckets_count=64); +---- diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param.explain new file mode 100644 index 000000000000..6ad309cdc23d --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param.explain @@ -0,0 +1,48 @@ +/* setup */ +CREATE TABLE t (i int); +CREATE SCHEMA sc; + +/* test */ +EXPLAIN (DDL) ALTER TABLE t SET (exclude_data_from_backup=true, autovacuum_enabled=false, sql_stats_histogram_buckets_count=64); +---- +Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› SET ('exclude_data_from_backup' = ‹true›, 'autovacuum_enabled' = ‹false›, 'sql_stats_histogram_buckets_count' = ‹64›); + ├── StatementPhase + │ └── Stage 1 of 1 in StatementPhase + │ ├── 2 elements transitioning toward PUBLIC + │ │ ├── ABSENT → PUBLIC TableStorageParam:{DescID: 104 (t), Name: "exclude_data_from_backup", Value: "true"} + │ │ └── ABSENT → PUBLIC TableStorageParam:{DescID: 104 (t), Name: "sql_stats_histogram_buckets_count", Value: "64"} + │ ├── 1 element transitioning toward TRANSIENT_PUBLIC + │ │ └── PUBLIC → ABSENT TableSchemaLocked:{DescID: 104 (t)} + │ └── 3 Mutation operations + │ ├── SetTableSchemaLocked {"TableID":104} + │ ├── SetTableStorageParam + │ └── SetTableStorageParam + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 2 elements transitioning toward PUBLIC + │ │ │ ├── PUBLIC → ABSENT TableStorageParam:{DescID: 104 (t), Name: "exclude_data_from_backup", Value: "true"} + │ │ │ └── PUBLIC → ABSENT TableStorageParam:{DescID: 104 (t), Name: "sql_stats_histogram_buckets_count", Value: "64"} + │ │ ├── 1 element transitioning toward TRANSIENT_PUBLIC + │ │ │ └── ABSENT → PUBLIC TableSchemaLocked:{DescID: 104 (t)} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 2 elements transitioning toward PUBLIC + │ │ ├── ABSENT → PUBLIC TableStorageParam:{DescID: 104 (t), Name: "exclude_data_from_backup", Value: "true"} + │ │ └── ABSENT → PUBLIC TableStorageParam:{DescID: 104 (t), Name: "sql_stats_histogram_buckets_count", Value: "64"} + │ ├── 1 element transitioning toward TRANSIENT_PUBLIC + │ │ └── PUBLIC → ABSENT TableSchemaLocked:{DescID: 104 (t)} + │ └── 5 Mutation operations + │ ├── SetTableSchemaLocked {"TableID":104} + │ ├── SetTableStorageParam + │ ├── SetTableStorageParam + │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} + │ └── CreateSchemaChangerJob {"RunningStatus":"Pending: Updatin..."} + └── PostCommitPhase + └── Stage 1 of 1 in PostCommitPhase + ├── 1 element transitioning toward TRANSIENT_PUBLIC + │ └── ABSENT → TRANSIENT_PUBLIC TableSchemaLocked:{DescID: 104 (t)} + └── 3 Mutation operations + ├── SetTableSchemaLocked {"Locked":true,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param.explain_shape b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param.explain_shape new file mode 100644 index 000000000000..c01a6b981f21 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param.explain_shape @@ -0,0 +1,9 @@ +/* setup */ +CREATE TABLE t (i int); +CREATE SCHEMA sc; + +/* test */ +EXPLAIN (DDL, SHAPE) ALTER TABLE t SET (exclude_data_from_backup=true, autovacuum_enabled=false, sql_stats_histogram_buckets_count=64); +---- +Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› SET ('exclude_data_from_backup' = ‹true›, 'autovacuum_enabled' = ‹false›, 'sql_stats_histogram_buckets_count' = ‹64›); + └── execute 2 system table mutations transactions diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param.side_effects new file mode 100644 index 000000000000..09d8164c184c --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param.side_effects @@ -0,0 +1,150 @@ +/* setup */ +CREATE TABLE t (i int); +CREATE SCHEMA sc; +---- +... ++object {100 101 t} -> 104 ++schema {100 0 sc} -> 105 + +/* test */ +ALTER TABLE t SET (exclude_data_from_backup=true, autovacuum_enabled=false, sql_stats_histogram_buckets_count=64); +---- +begin transaction #1 +# begin StatementPhase +checking for feature: ALTER TABLE +increment telemetry for sql.schema.alter_table +increment telemetry for sql.schema.alter_table.set_storage_param +## StatementPhase stage 1 of 1 with 3 MutationType ops +upsert descriptor #104 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + excludeDataFromBackup: true + families: + - columnIds: + ... + name: primary + formatVersion: 3 + + histogramBuckets: 64 + id: 104 + modificationTime: {} + ... + replacementOf: + time: {} + - schemaLocked: true + unexposedParentSchemaId: 101 + - version: "1" + + version: "2" +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 2 with 1 MutationType op +undo all catalog changes within txn #1 +persist all catalog changes to storage +## PreCommitPhase stage 2 of 2 with 5 MutationType ops +upsert descriptor #104 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + jobId: "1" + + nameMapping: + + columns: + + "1": i + + "2": rowid + + "4294967292": crdb_internal_origin_timestamp + + "4294967293": crdb_internal_origin_id + + "4294967294": tableoid + + "4294967295": crdb_internal_mvcc_timestamp + + families: + + "0": primary + + id: 104 + + indexes: + + "1": t_pkey + + name: t + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› SET ('exclude_data_from_backup' = ‹true›, 'autovacuum_enabled' = ‹false›, 'sql_stats_histogram_buckets_count' = ‹64›) + + statement: ALTER TABLE t SET ('exclude_data_from_backup' = true, 'autovacuum_enabled' = false, 'sql_stats_histogram_buckets_count' = 64) + + statementTag: ALTER TABLE + + revertible: true + + targetRanks: + + targets: + + excludeDataFromBackup: true + families: + - columnIds: + ... + name: primary + formatVersion: 3 + + histogramBuckets: 64 + id: 104 + modificationTime: {} + ... + replacementOf: + time: {} + - schemaLocked: true + unexposedParentSchemaId: 101 + - version: "1" + + version: "2" +persist all catalog changes to storage +create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t SET ('exclude_data_from_backup' = true, 'autovacuum_enabled' = false, 'sql_stats_histogram_buckets_count' = 64)" + descriptor IDs: [104] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitPhase stage 1 of 1 with 3 MutationType ops +upsert descriptor #104 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - jobId: "1" + - nameMapping: + - columns: + - "1": i + - "2": rowid + - "4294967292": crdb_internal_origin_timestamp + - "4294967293": crdb_internal_origin_id + - "4294967294": tableoid + - "4294967295": crdb_internal_mvcc_timestamp + - families: + - "0": primary + - id: 104 + - indexes: + - "1": t_pkey + - name: t + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› SET ('exclude_data_from_backup' = ‹true›, 'autovacuum_enabled' = ‹false›, 'sql_stats_histogram_buckets_count' = ‹64›) + - statement: ALTER TABLE t SET ('exclude_data_from_backup' = true, 'autovacuum_enabled' = false, 'sql_stats_histogram_buckets_count' = 64) + - statementTag: ALTER TABLE + - revertible: true + - targetRanks: + - targets: + excludeDataFromBackup: true + families: + ... + replacementOf: + time: {} + + schemaLocked: true + unexposedParentSchemaId: 101 + - version: "2" + + version: "3" +persist all catalog changes to storage +update progress of schema change job #1: "all stages completed" +set schema change job #1 to non-cancellable +updated schema change job #1 descriptor IDs to [] +write *eventpb.FinishSchemaChange to event log: + sc: + descriptorId: 104 +commit transaction #3 +# end PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param__rollback_1_of_1.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param__rollback_1_of_1.explain new file mode 100644 index 000000000000..4a1f5f914709 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_set_storage_param/alter_table_set_storage_param__rollback_1_of_1.explain @@ -0,0 +1,26 @@ +/* setup */ +CREATE TABLE t (i int); +CREATE SCHEMA sc; + +/* test */ +ALTER TABLE t SET (exclude_data_from_backup=true, autovacuum_enabled=false, sql_stats_histogram_buckets_count=64); +EXPLAIN (DDL) rollback at post-commit stage 1 of 1; +---- +Schema change plan for rolling back ALTER TABLE defaultdb.public.t SET ('exclude_data_from_backup' = ‹true›, 'autovacuum_enabled' = ‹false›, 'sql_stats_histogram_buckets_count' = ‹64›); + └── PostCommitNonRevertiblePhase + ├── Stage 1 of 2 in PostCommitNonRevertiblePhase + │ ├── 2 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT TableStorageParam:{DescID: 104 (t), Name: "exclude_data_from_backup", Value: "true"} + │ │ └── PUBLIC → ABSENT TableStorageParam:{DescID: 104 (t), Name: "sql_stats_histogram_buckets_count", Value: "64"} + │ └── 4 Mutation operations + │ ├── ResetTableStorageParam + │ ├── ResetTableStorageParam + │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"Pending: Updatin..."} + └── Stage 2 of 2 in PostCommitNonRevertiblePhase + ├── 1 element transitioning toward TRANSIENT_PUBLIC + │ └── ABSENT → TRANSIENT_PUBLIC TableSchemaLocked:{DescID: 104 (t)} + └── 3 Mutation operations + ├── SetTableSchemaLocked {"Locked":true,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/storageparam/indexstorageparam/index_storage_param.go b/pkg/sql/storageparam/indexstorageparam/index_storage_param.go index 726478f5c3ec..7fbdb8739d8d 100644 --- a/pkg/sql/storageparam/indexstorageparam/index_storage_param.go +++ b/pkg/sql/storageparam/indexstorageparam/index_storage_param.go @@ -227,5 +227,5 @@ func (po *Setter) RunPostChecks() error { // IsNewTableObject implements the Setter interface. func (po *Setter) IsNewTableObject() bool { //Not applicable to indexes. - panic(errors.AssertionFailedf("not-implemented for indexes")) + return false } diff --git a/pkg/sql/storageparam/storage_param.go b/pkg/sql/storageparam/storage_param.go index 536a79233ab7..e11cdd78f3ab 100644 --- a/pkg/sql/storageparam/storage_param.go +++ b/pkg/sql/storageparam/storage_param.go @@ -48,7 +48,7 @@ func Set( params tree.StorageParams, setter Setter, ) error { - if err := storageParamPreChecks(ctx, evalCtx, setter, params, nil /* resetParams */); err != nil { + if err := StorageParamPreChecks(ctx, evalCtx, setter.IsNewTableObject(), params, nil /* resetParams */); err != nil { return err } for _, sp := range params { @@ -95,7 +95,7 @@ func Set( func Reset( ctx context.Context, evalCtx *eval.Context, params []string, paramObserver Setter, ) error { - if err := storageParamPreChecks(ctx, evalCtx, paramObserver, nil /* setParam */, params); err != nil { + if err := StorageParamPreChecks(ctx, evalCtx, paramObserver.IsNewTableObject(), nil /* setParam */, params); err != nil { return err } for _, p := range params { @@ -126,12 +126,12 @@ func SetFillFactor(ctx context.Context, evalCtx *eval.Context, key string, datum return nil } -// storageParamPreChecks is where we specify pre-conditions for setting/resetting +// StorageParamPreChecks is where we specify pre-conditions for setting/resetting // storage parameters `param`. -func storageParamPreChecks( +func StorageParamPreChecks( ctx context.Context, evalCtx *eval.Context, - setter Setter, + isNewTableObject bool, setParams tree.StorageParams, resetParams []string, ) error { @@ -163,7 +163,7 @@ func storageParamPreChecks( // since later operations cannot unset schema_locked (i.e. only implicit single // statement transactions are allowed to manipulate schema_locked, see // checkSchemaChangeIsAllowed). - if !setter.IsNewTableObject() && (len(keys) > 1 || !evalCtx.TxnImplicit || !evalCtx.TxnIsSingleStmt) { + if !isNewTableObject && (len(keys) > 1 || !evalCtx.TxnImplicit || !evalCtx.TxnIsSingleStmt) { return pgerror.Newf(pgcode.InvalidParameterValue, "%q can only be set/reset on "+ "its own without other parameters in a single-statement implicit transaction.", key) } diff --git a/pkg/sql/storageparam/tablestorageparam/BUILD.bazel b/pkg/sql/storageparam/tablestorageparam/BUILD.bazel index 12fa6650d00a..16a9dda4d446 100644 --- a/pkg/sql/storageparam/tablestorageparam/BUILD.bazel +++ b/pkg/sql/storageparam/tablestorageparam/BUILD.bazel @@ -7,6 +7,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/clusterversion", + "//pkg/server/telemetry", "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/tabledesc", @@ -15,8 +16,11 @@ go_library( "//pkg/sql/pgwire/pgerror", "//pkg/sql/pgwire/pgnotice", "//pkg/sql/sem/eval", + "//pkg/sql/sem/normalize", "//pkg/sql/sem/tree", + "//pkg/sql/sqltelemetry", "//pkg/sql/storageparam", + "//pkg/sql/types", "//pkg/util/duration", "//pkg/util/errorutil/unimplemented", "//pkg/util/protoutil", diff --git a/pkg/sql/storageparam/tablestorageparam/table_storage_param.go b/pkg/sql/storageparam/tablestorageparam/table_storage_param.go index 893da5fe29cc..edde22df67cf 100644 --- a/pkg/sql/storageparam/tablestorageparam/table_storage_param.go +++ b/pkg/sql/storageparam/tablestorageparam/table_storage_param.go @@ -9,10 +9,14 @@ package tablestorageparam import ( "context" + "fmt" "math" + "strconv" "strings" + "time" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -21,8 +25,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sem/normalize" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/storageparam" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -130,8 +137,10 @@ func (po *Setter) getOrCreateRowLevelTTL() *catpb.RowLevelTTL { } type tableParam struct { - onSet func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error - onReset func(ctx context.Context, po *Setter, evalCtx *eval.Context, key string) error + validateSetValue func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) + onSet func(ctx context.Context, po *Setter, key string, value string) error + getResetValue func(ctx context.Context, po *Setter, evalCtx *eval.Context, key string) (string, error) + onReset func(ctx context.Context, po *Setter, key string, value string) error } var ttlAutomaticColumnNotice = pgnotice.Newf("ttl_automatic_column is no longer used. " + @@ -142,28 +151,25 @@ var ttlRangeConcurrencyNotice = pgnotice.Newf("ttl_range_concurrency is no longe var tableParams = map[string]tableParam{ `fillfactor`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { - return storageparam.SetFillFactor(ctx, evalCtx, key, datum) + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { + // SetFillFactor validates the value and issues a notice but doesn't actually set anything. + if err := storageparam.SetFillFactor(ctx, evalCtx, key, datum); err != nil { + return "", err + } + return "", nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + return nil }, - onReset: func(_ context.Context, po *Setter, _ *eval.Context, key string) error { - // Operation is a no-op so do nothing. + onReset: func(_ context.Context, po *Setter, key string, value string) error { return nil }, }, `autovacuum_enabled`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { - var boolVal bool - if stringVal, err := paramparse.DatumAsString(ctx, evalCtx, key, datum); err == nil { - boolVal, err = paramparse.ParseBoolVar(key, stringVal) - if err != nil { - return err - } - } else { - s, err := paramparse.GetSingleBool(key, datum) - if err != nil { - return err - } - boolVal = bool(*s) + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { + boolVal, err := boolFromDatum(ctx, evalCtx, key, datum) + if err != nil { + return "", err } if !boolVal && evalCtx != nil { evalCtx.ClientNoticeSender.BufferClientNotice( @@ -171,25 +177,23 @@ var tableParams = map[string]tableParam{ pgnotice.Newf(`storage parameter "%s = %s" is ignored`, key, datum.String()), ) } + return "", nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { return nil }, - onReset: func(_ context.Context, po *Setter, _ *eval.Context, key string) error { - // Operation is a no-op so do nothing. + onReset: func(_ context.Context, po *Setter, key string, value string) error { return nil }, }, `ttl`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { setTrue, err := boolFromDatum(ctx, evalCtx, key, datum) if err != nil { - return err + return "", err } - if setTrue { - // Set the base struct, but do not populate it. - // An error from runPostChecks will appear if the requisite fields are not set. - po.getOrCreateRowLevelTTL() - } else { - return errors.WithHintf( + if !setTrue { + return "", errors.WithHintf( pgerror.Newf( pgcode.InvalidParameterValue, `setting "ttl = 'off'" is not permitted`, @@ -197,30 +201,42 @@ var tableParams = map[string]tableParam{ "use `RESET (ttl)` to remove TTL from the table", ) } + return fmt.Sprintf("%t", setTrue), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + // Set the base struct, but do not populate it. + // An error from runPostChecks will appear if the requisite fields are not set. + po.getOrCreateRowLevelTTL() return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { po.UpdatedRowLevelTTL = nil return nil }, }, `ttl_automatic_column`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { evalCtx.ClientNoticeSender.BufferClientNotice(ctx, ttlAutomaticColumnNotice) + return "", nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { return nil }, - onReset: func(ctx context.Context, po *Setter, evalCtx *eval.Context, key string) error { + getResetValue: func(ctx context.Context, po *Setter, evalCtx *eval.Context, key string) (string, error) { evalCtx.ClientNoticeSender.BufferClientNotice(ctx, ttlAutomaticColumnNotice) + return "", nil + }, + onReset: func(ctx context.Context, po *Setter, key string, value string) error { return nil }, }, `ttl_expire_after`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { var d *tree.DInterval if stringVal, err := paramparse.DatumAsString(ctx, evalCtx, key, datum); err == nil { d, err = tree.ParseDInterval(evalCtx.SessionData().GetIntervalStyle(), stringVal) if err != nil { - return pgerror.Wrapf( + return "", pgerror.Wrapf( err, pgcode.InvalidParameterValue, `value of %q must be an interval`, @@ -228,7 +244,7 @@ var tableParams = map[string]tableParam{ ) } if d == nil { - return pgerror.Newf( + return "", pgerror.Newf( pgcode.InvalidParameterValue, `value of %q must be an interval`, key, @@ -238,7 +254,7 @@ var tableParams = map[string]tableParam{ var ok bool d, ok = datum.(*tree.DInterval) if !ok || d == nil { - return pgerror.Newf( + return "", pgerror.Newf( pgcode.InvalidParameterValue, `value of %q must be an interval`, key, @@ -247,17 +263,20 @@ var tableParams = map[string]tableParam{ } if d.Duration.Compare(duration.MakeDuration(0, 0, 0)) < 0 { - return pgerror.Newf( + return "", pgerror.Newf( pgcode.InvalidParameterValue, `value of %q must be at least zero`, key, ) } + return tree.Serialize(d), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { rowLevelTTL := po.getOrCreateRowLevelTTL() - rowLevelTTL.DurationExpr = catpb.Expression(tree.Serialize(d)) + rowLevelTTL.DurationExpr = catpb.Expression(value) return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { if po.hasRowLevelTTL() { po.UpdatedRowLevelTTL.DurationExpr = "" } @@ -265,16 +284,15 @@ var tableParams = map[string]tableParam{ }, }, `ttl_expiration_expression`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { - stringVal, err := paramparse.DatumAsString(ctx, evalCtx, key, datum) - if err != nil { - return err - } + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { + return paramparse.DatumAsString(ctx, evalCtx, key, datum) + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { rowLevelTTL := po.getOrCreateRowLevelTTL() - rowLevelTTL.ExpirationExpr = catpb.Expression(stringVal) + rowLevelTTL.ExpirationExpr = catpb.Expression(value) return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { if po.hasRowLevelTTL() { po.UpdatedRowLevelTTL.ExpirationExpr = "" } @@ -282,19 +300,26 @@ var tableParams = map[string]tableParam{ }, }, `ttl_select_batch_size`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { val, err := paramparse.DatumAsInt(ctx, evalCtx, key, datum) if err != nil { - return err + return "", err } if err := tabledesc.ValidateTTLBatchSize(key, val); err != nil { + return "", err + } + return fmt.Sprintf("%d", val), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + val, err := strconv.ParseInt(value, 10, 64) + if err != nil { return err } rowLevelTTL := po.getOrCreateRowLevelTTL() rowLevelTTL.SelectBatchSize = val return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { if po.hasRowLevelTTL() { po.UpdatedRowLevelTTL.SelectBatchSize = 0 } @@ -302,19 +327,26 @@ var tableParams = map[string]tableParam{ }, }, `ttl_delete_batch_size`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { val, err := paramparse.DatumAsInt(ctx, evalCtx, key, datum) if err != nil { - return err + return "", err } if err := tabledesc.ValidateTTLBatchSize(key, val); err != nil { + return "", err + } + return fmt.Sprintf("%d", val), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + val, err := strconv.ParseInt(value, 10, 64) + if err != nil { return err } rowLevelTTL := po.getOrCreateRowLevelTTL() rowLevelTTL.DeleteBatchSize = val return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { if po.hasRowLevelTTL() { po.UpdatedRowLevelTTL.DeleteBatchSize = 0 } @@ -322,29 +354,42 @@ var tableParams = map[string]tableParam{ }, }, `ttl_range_concurrency`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { evalCtx.ClientNoticeSender.BufferClientNotice(ctx, ttlRangeConcurrencyNotice) + return "", nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { return nil }, - onReset: func(ctx context.Context, po *Setter, evalCtx *eval.Context, key string) error { + getResetValue: func(ctx context.Context, po *Setter, evalCtx *eval.Context, key string) (string, error) { evalCtx.ClientNoticeSender.BufferClientNotice(ctx, ttlRangeConcurrencyNotice) + return "", nil + }, + onReset: func(ctx context.Context, po *Setter, key string, value string) error { return nil }, }, `ttl_select_rate_limit`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { val, err := paramparse.DatumAsInt(ctx, evalCtx, key, datum) if err != nil { - return err + return "", err } if err := tabledesc.ValidateTTLRateLimit(key, val); err != nil { + return "", err + } + return fmt.Sprintf("%d", val), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + val, err := strconv.ParseInt(value, 10, 64) + if err != nil { return err } rowLevelTTL := po.getOrCreateRowLevelTTL() rowLevelTTL.SelectRateLimit = val return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { if po.hasRowLevelTTL() { po.UpdatedRowLevelTTL.SelectRateLimit = 0 } @@ -352,19 +397,26 @@ var tableParams = map[string]tableParam{ }, }, `ttl_delete_rate_limit`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { val, err := paramparse.DatumAsInt(ctx, evalCtx, key, datum) if err != nil { - return err + return "", err } if err := tabledesc.ValidateTTLRateLimit(key, val); err != nil { + return "", err + } + return fmt.Sprintf("%d", val), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + val, err := strconv.ParseInt(value, 10, 64) + if err != nil { return err } rowLevelTTL := po.getOrCreateRowLevelTTL() rowLevelTTL.DeleteRateLimit = val return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { if po.hasRowLevelTTL() { po.UpdatedRowLevelTTL.DeleteRateLimit = 0 } @@ -372,8 +424,15 @@ var tableParams = map[string]tableParam{ }, }, `ttl_label_metrics`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { val, err := boolFromDatum(ctx, evalCtx, key, datum) + if err != nil { + return "", err + } + return fmt.Sprintf("%t", val), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + val, err := strconv.ParseBool(value) if err != nil { return err } @@ -381,7 +440,7 @@ var tableParams = map[string]tableParam{ rowLevelTTL.LabelMetrics = val return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { if po.hasRowLevelTTL() { po.UpdatedRowLevelTTL.LabelMetrics = false } @@ -389,19 +448,22 @@ var tableParams = map[string]tableParam{ }, }, `ttl_job_cron`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { str, err := paramparse.DatumAsString(ctx, evalCtx, key, datum) if err != nil { - return err + return "", err } if err := tabledesc.ValidateTTLCronExpr(key, str); err != nil { - return err + return "", err } + return str, nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { rowLevelTTL := po.getOrCreateRowLevelTTL() - rowLevelTTL.DeletionCron = str + rowLevelTTL.DeletionCron = value return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { if po.hasRowLevelTTL() { po.UpdatedRowLevelTTL.DeletionCron = "" } @@ -409,8 +471,15 @@ var tableParams = map[string]tableParam{ }, }, `ttl_pause`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { b, err := boolFromDatum(ctx, evalCtx, key, datum) + if err != nil { + return "", err + } + return fmt.Sprintf("%t", b), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + b, err := strconv.ParseBool(value) if err != nil { return err } @@ -418,7 +487,7 @@ var tableParams = map[string]tableParam{ rowLevelTTL.Pause = b return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { if po.hasRowLevelTTL() { po.UpdatedRowLevelTTL.Pause = false } @@ -426,19 +495,26 @@ var tableParams = map[string]tableParam{ }, }, `ttl_row_stats_poll_interval`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { d, err := paramparse.DatumAsDuration(ctx, evalCtx, key, datum) if err != nil { - return err + return "", err } if err := tabledesc.ValidateNotNegativeInterval(key, d); err != nil { + return "", err + } + return d.String(), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + d, err := time.ParseDuration(value) + if err != nil { return err } rowLevelTTL := po.getOrCreateRowLevelTTL() rowLevelTTL.RowStatsPollInterval = d return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { if po.hasRowLevelTTL() { po.UpdatedRowLevelTTL.RowStatsPollInterval = 0 } @@ -446,8 +522,15 @@ var tableParams = map[string]tableParam{ }, }, `ttl_disable_changefeed_replication`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { b, err := boolFromDatum(ctx, evalCtx, key, datum) + if err != nil { + return "", err + } + return fmt.Sprintf("%t", b), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + b, err := strconv.ParseBool(value) if err != nil { return err } @@ -455,7 +538,7 @@ var tableParams = map[string]tableParam{ rowLevelTTL.DisableChangefeedReplication = b return nil }, - onReset: func(ctx context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(ctx context.Context, po *Setter, key string, value string) error { if po.hasRowLevelTTL() { po.UpdatedRowLevelTTL.DisableChangefeedReplication = false } @@ -463,8 +546,14 @@ var tableParams = map[string]tableParam{ }, }, `exclude_data_from_backup`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, - evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { + excludeDataFromBackup, err := boolFromDatum(ctx, evalCtx, key, datum) + if err != nil { + return "", err + } + return fmt.Sprintf("%t", excludeDataFromBackup), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { if po.TableDesc.Temporary { return pgerror.Newf(pgcode.FeatureNotSupported, "cannot set data in a temporary table to be excluded from backup") @@ -479,7 +568,7 @@ var tableParams = map[string]tableParam{ return errors.New("cannot set data in a table with inbound foreign key constraints to be excluded from backup") } - excludeDataFromBackup, err := boolFromDatum(ctx, evalCtx, key, datum) + excludeDataFromBackup, err := strconv.ParseBool(value) if err != nil { return err } @@ -491,157 +580,204 @@ var tableParams = map[string]tableParam{ po.TableDesc.ExcludeDataFromBackup = excludeDataFromBackup return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { po.TableDesc.ExcludeDataFromBackup = false return nil }, }, catpb.AutoStatsEnabledTableSettingName: { - onSet: autoStatsEnabledSettingFunc, - onReset: autoStatsTableSettingResetFunc, + validateSetValue: autoStatsEnabledPreSetFunc, + onSet: autoStatsEnabledSetFunc, + onReset: autoStatsTableSettingResetFunc, }, catpb.AutoStatsMinStaleTableSettingName: { - onSet: autoStatsMinStaleRowsSettingFunc(func(intVal int64) error { + validateSetValue: autoStatsMinStaleRowsPreSetFunc(func(intVal int64) error { if intVal < 0 { return errors.Newf("cannot be set to a negative value: %d", intVal) } return nil }), + onSet: autoStatsMinStaleRowsSetFunc, onReset: autoStatsTableSettingResetFunc, }, catpb.AutoStatsFractionStaleTableSettingName: { - onSet: autoStatsFractionStaleRowsSettingFunc(func(floatVal float64) error { + validateSetValue: autoStatsFractionStaleRowsPreSetFunc(func(floatVal float64) error { if floatVal < 0 { return errors.Newf("cannot set to a negative value: %f", floatVal) } return nil }), + onSet: autoStatsFractionStaleRowsSetFunc, onReset: autoStatsTableSettingResetFunc, }, catpb.AutoPartialStatsEnabledTableSettingName: { - onSet: autoStatsEnabledSettingFunc, - onReset: autoStatsTableSettingResetFunc, + validateSetValue: autoStatsEnabledPreSetFunc, + onSet: autoStatsEnabledSetFunc, + onReset: autoStatsTableSettingResetFunc, }, catpb.AutoFullStatsEnabledTableSettingName: { - onSet: autoStatsEnabledSettingFunc, - onReset: autoStatsTableSettingResetFunc, + validateSetValue: autoStatsEnabledPreSetFunc, + onSet: autoStatsEnabledSetFunc, + onReset: autoStatsTableSettingResetFunc, }, catpb.AutoPartialStatsMinStaleTableSettingName: { - onSet: autoStatsMinStaleRowsSettingFunc(func(intVal int64) error { + validateSetValue: autoStatsMinStaleRowsPreSetFunc(func(intVal int64) error { if intVal < 0 { return errors.Newf("cannot be set to a negative value: %d", intVal) } return nil }), + onSet: autoStatsMinStaleRowsSetFunc, onReset: autoStatsTableSettingResetFunc, }, catpb.AutoPartialStatsFractionStaleTableSettingName: { - onSet: autoStatsFractionStaleRowsSettingFunc(func(floatVal float64) error { + validateSetValue: autoStatsFractionStaleRowsPreSetFunc(func(floatVal float64) error { if floatVal < 0 { return errors.Newf("cannot set to a negative value: %f", floatVal) } return nil }), + onSet: autoStatsFractionStaleRowsSetFunc, onReset: autoStatsTableSettingResetFunc, }, `sql_stats_forecasts_enabled`: { - onSet: func( - ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum, - ) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { enabled, err := boolFromDatum(ctx, evalCtx, key, datum) + if err != nil { + return "", err + } + return fmt.Sprintf("%t", enabled), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + enabled, err := strconv.ParseBool(value) if err != nil { return err } po.TableDesc.ForecastStats = &enabled return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { po.TableDesc.ForecastStats = nil return nil }, }, `sql_stats_histogram_samples_count`: { - onSet: func( - ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum, - ) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { intVal, err := intFromDatum(ctx, evalCtx, key, datum) if err != nil { - return err + return "", err } if err := nonNegativeIntWithMaximum(math.MaxUint32)(intVal); err != nil { - return errors.Wrapf(err, "invalid integer value for %s", key) + return "", errors.Wrapf(err, "invalid integer value for %s", key) + } + return fmt.Sprintf("%d", intVal), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + intVal, err := strconv.ParseInt(value, 10, 64) + if err != nil { + return err } uint32Val := uint32(intVal) po.TableDesc.HistogramSamples = &uint32Val return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { po.TableDesc.HistogramSamples = nil return nil }, }, `sql_stats_histogram_buckets_count`: { - onSet: func( - ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum, - ) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { intVal, err := intFromDatum(ctx, evalCtx, key, datum) if err != nil { - return err + return "", err } - if err = nonNegativeIntWithMaximum(math.MaxUint32)(intVal); err != nil { - return errors.Wrapf(err, "invalid integer value for %s", key) + if err := nonNegativeIntWithMaximum(math.MaxUint32)(intVal); err != nil { + return "", errors.Wrapf(err, "invalid integer value for %s", key) + } + return fmt.Sprintf("%d", intVal), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + intVal, err := strconv.ParseInt(value, 10, 64) + if err != nil { + return err } uint32Val := uint32(intVal) po.TableDesc.HistogramBuckets = &uint32Val return nil }, - onReset: func(_ context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(_ context.Context, po *Setter, key string, value string) error { po.TableDesc.HistogramBuckets = nil return nil }, }, `schema_locked`: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { boolVal, err := boolFromDatum(ctx, evalCtx, key, datum) + if err != nil { + return "", err + } + return fmt.Sprintf("%t", boolVal), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + boolVal, err := strconv.ParseBool(value) if err != nil { return err } po.TableDesc.SchemaLocked = boolVal return nil }, - onReset: func(ctx context.Context, po *Setter, evalCtx *eval.Context, key string) error { + getResetValue: func(ctx context.Context, po *Setter, evalCtx *eval.Context, key string) (string, error) { schemaLockedDefault := evalCtx.SessionData().CreateTableWithSchemaLocked // Before 25.3 tables were never created with schema_locked by default. if !evalCtx.Settings.Version.IsActive(ctx, clusterversion.V25_3) { schemaLockedDefault = false } - po.TableDesc.SchemaLocked = schemaLockedDefault + return fmt.Sprintf("%t", schemaLockedDefault), nil + }, + onReset: func(ctx context.Context, po *Setter, key string, value string) error { + boolVal, err := strconv.ParseBool(value) + if err != nil { + return err + } + po.TableDesc.SchemaLocked = boolVal return nil }, }, catpb.RBRUsingConstraintTableSettingName: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { + // Handled by the schema changer. + return "", nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { // Handled by the schema changer. return nil }, - onReset: func(ctx context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(ctx context.Context, po *Setter, key string, value string) error { po.TableDesc.RBRUsingConstraint = descpb.ConstraintID(0) return nil }, }, catpb.CanaryStatsWindowSettingName: { - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { d, err := paramparse.DatumAsDuration(ctx, evalCtx, key, datum) if err != nil { - return err + return "", err } if err := tabledesc.ValidateNotNegativeInterval(key, d); err != nil { + return "", err + } + return d.String(), nil + }, + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + d, err := time.ParseDuration(value) + if err != nil { return err } po.TableDesc.StatsCanaryWindow = d return nil }, - onReset: func(ctx context.Context, po *Setter, evalCtx *eval.Context, key string) error { + onReset: func(ctx context.Context, po *Setter, key string, value string) error { po.TableDesc.StatsCanaryWindow = 0 return nil }, @@ -691,25 +827,35 @@ func init() { `user_catalog_table`, } { tableParams[param] = tableParam{ - onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { - return unimplemented.NewWithIssuef(43299, "storage parameter %q", key) + validateSetValue: func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { + return "", unimplemented.NewWithIssuef(43299, "storage parameter %q", key) }, - onReset: func(_ context.Context, po *Setter, _ *eval.Context, key string) error { + onSet: func(ctx context.Context, po *Setter, key string, value string) error { + return nil + }, + onReset: func(_ context.Context, po *Setter, key string, value string) error { return nil }, } } } -func autoStatsEnabledSettingFunc( +func autoStatsEnabledPreSetFunc( ctx context.Context, - po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum, -) error { +) (string, error) { boolVal, err := boolFromDatum(ctx, evalCtx, key, datum) + if err != nil { + return "", err + } + return fmt.Sprintf("%t", boolVal), nil +} + +func autoStatsEnabledSetFunc(ctx context.Context, po *Setter, key string, value string) error { + boolVal, err := strconv.ParseBool(value) if err != nil { return err } @@ -731,64 +877,79 @@ func autoStatsEnabledSettingFunc( return errors.AssertionFailedf("unable to set table setting %s", key) } -func autoStatsMinStaleRowsSettingFunc( +func autoStatsMinStaleRowsPreSetFunc( validateFunc func(v int64) error, -) func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { - return func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { +) func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { + return func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { intVal, err := intFromDatum(ctx, evalCtx, key, datum) if err != nil { - return err - } - if po.TableDesc.AutoStatsSettings == nil { - po.TableDesc.AutoStatsSettings = &catpb.AutoStatsSettings{} + return "", err } if err = validateFunc(intVal); err != nil { - return errors.Wrapf(err, "invalid integer value for %s", key) + return "", errors.Wrapf(err, "invalid integer value for %s", key) } + return fmt.Sprintf("%d", intVal), nil + } +} - switch key { - case catpb.AutoStatsMinStaleTableSettingName: - po.TableDesc.AutoStatsSettings.MinStaleRows = &intVal - return nil - case catpb.AutoPartialStatsMinStaleTableSettingName: - po.TableDesc.AutoStatsSettings.PartialMinStaleRows = &intVal - return nil - } - return errors.AssertionFailedf("unable to set table setting %s", key) +func autoStatsMinStaleRowsSetFunc(ctx context.Context, po *Setter, key string, value string) error { + intVal, err := strconv.ParseInt(value, 10, 64) + if err != nil { + return err + } + if po.TableDesc.AutoStatsSettings == nil { + po.TableDesc.AutoStatsSettings = &catpb.AutoStatsSettings{} } + + switch key { + case catpb.AutoStatsMinStaleTableSettingName: + po.TableDesc.AutoStatsSettings.MinStaleRows = &intVal + return nil + case catpb.AutoPartialStatsMinStaleTableSettingName: + po.TableDesc.AutoStatsSettings.PartialMinStaleRows = &intVal + return nil + } + return errors.AssertionFailedf("unable to set table setting %s", key) } -func autoStatsFractionStaleRowsSettingFunc( +func autoStatsFractionStaleRowsPreSetFunc( validateFunc func(v float64) error, -) func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { - return func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, - evalCtx *eval.Context, key string, datum tree.Datum) error { +) func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { + return func(ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) (string, error) { floatVal, err := floatFromDatum(ctx, evalCtx, key, datum) if err != nil { - return err - } - if po.TableDesc.AutoStatsSettings == nil { - po.TableDesc.AutoStatsSettings = &catpb.AutoStatsSettings{} + return "", err } if err = validateFunc(floatVal); err != nil { - return errors.Wrapf(err, "invalid float value for %s", key) - } - - switch key { - case catpb.AutoStatsFractionStaleTableSettingName: - po.TableDesc.AutoStatsSettings.FractionStaleRows = &floatVal - return nil - case catpb.AutoPartialStatsFractionStaleTableSettingName: - po.TableDesc.AutoStatsSettings.PartialFractionStaleRows = &floatVal - return nil + return "", errors.Wrapf(err, "invalid float value for %s", key) } - return errors.AssertionFailedf("unable to set table setting %s", key) + return fmt.Sprintf("%f", floatVal), nil } } -func autoStatsTableSettingResetFunc( - _ context.Context, po *Setter, evalCtx *eval.Context, key string, +func autoStatsFractionStaleRowsSetFunc( + ctx context.Context, po *Setter, key string, value string, ) error { + floatVal, err := strconv.ParseFloat(value, 64) + if err != nil { + return err + } + if po.TableDesc.AutoStatsSettings == nil { + po.TableDesc.AutoStatsSettings = &catpb.AutoStatsSettings{} + } + + switch key { + case catpb.AutoStatsFractionStaleTableSettingName: + po.TableDesc.AutoStatsSettings.FractionStaleRows = &floatVal + return nil + case catpb.AutoPartialStatsFractionStaleTableSettingName: + po.TableDesc.AutoStatsSettings.PartialFractionStaleRows = &floatVal + return nil + } + return errors.AssertionFailedf("unable to set table setting %s", key) +} + +func autoStatsTableSettingResetFunc(_ context.Context, po *Setter, key string, value string) error { if po.TableDesc.AutoStatsSettings == nil { return nil } @@ -834,7 +995,11 @@ func (po *Setter) Set( ) } if p, ok := tableParams[key]; ok { - return p.onSet(ctx, po, semaCtx, evalCtx, key, datum) + value, err := p.validateSetValue(ctx, semaCtx, evalCtx, key, datum) + if err != nil { + return err + } + return p.onSet(ctx, po, key, value) } return pgerror.Newf(pgcode.InvalidParameterValue, "invalid storage parameter %q", key) } @@ -848,7 +1013,78 @@ func (po *Setter) Reset(ctx context.Context, evalCtx *eval.Context, key string) ) } if p, ok := tableParams[key]; ok { - return p.onReset(ctx, po, evalCtx, key) + value := "" + var err error + if p.getResetValue != nil { + value, err = p.getResetValue(ctx, po, evalCtx, key) + if err != nil { + return err + } + } + return p.onReset(ctx, po, key, value) + } + return pgerror.Newf(pgcode.InvalidParameterValue, "invalid storage parameter %q", key) +} + +// SetToStringValue sets the param value to an already validated string representation. +// This function was introduced to be used by the declarative schema changer. +func (po *Setter) SetToStringValue(ctx context.Context, key string, value string) error { + if p, ok := tableParams[key]; ok { + return p.onSet(ctx, po, key, value) } return pgerror.Newf(pgcode.InvalidParameterValue, "invalid storage parameter %q", key) } + +// ResetToStringValue resets the param value. It will use the string value if specified. +// This function was introduced to be used by the declarative schema changer. +func (po *Setter) ResetToStringValue(ctx context.Context, key string, value string) error { + if p, ok := tableParams[key]; ok { + return p.onReset(ctx, po, key, value) + } + return pgerror.Newf(pgcode.InvalidParameterValue, "invalid storage parameter %q", key) +} + +// ParseAndValidate evaluates and validates a storage parameter value without +// applying it. It returns the validated string value that would be passed to +// onSet, allowing callers to perform validation before committing changes. +// This function was introduced to be used by the declarative schema changer. +func ParseAndValidate( + ctx context.Context, semaCtx *tree.SemaContext, evalCtx *eval.Context, param tree.StorageParam, +) (string, error) { + key := param.Key + if param.Value == nil { + return "", pgerror.Newf(pgcode.InvalidParameterValue, "storage parameter %q requires a value", key) + } + telemetry.Inc(sqltelemetry.SetTableStorageParameter(key)) + + // Expressions may be an unresolved name. + // Cast these as strings. + expr := paramparse.UnresolvedNameToStrVal(param.Value) + + // Storage params handle their own scalar arguments, with no help from the + // optimizer. As such, they cannot contain subqueries. + defer semaCtx.Properties.Restore(semaCtx.Properties) + semaCtx.Properties.Require("table storage parameters", tree.RejectSubqueries) + + // Convert the expressions to a datum. + typedExpr, err := tree.TypeCheck(ctx, expr, semaCtx, types.AnyElement) + if err != nil { + return "", err + } + if typedExpr, err = normalize.Expr(ctx, evalCtx, typedExpr); err != nil { + return "", err + } + datum, err := eval.Expr(ctx, evalCtx, typedExpr) + if err != nil { + return "", err + } + + if p, ok := tableParams[key]; ok { + value, err := p.validateSetValue(ctx, semaCtx, evalCtx, key, datum) + if err != nil { + return "", err + } + return value, nil + } + return "", pgerror.Newf(pgcode.InvalidParameterValue, "invalid storage parameter %q", key) +}