From 372b2f49792e26d2b4b24715567b973302d661be Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Tue, 31 Jan 2023 09:40:26 -0500 Subject: [PATCH 1/9] sql/schemachanger: forward fit compatibility changes for 22.2 rules Previously, some of the constraint related rules in the 22.2 set were incorrectly using logic for 23.X. This patch addresses those to get compatibility back. Additionally, some minor clean up in rules related helpers to ensure proper compatibility. With this change a manual diff shows both branches are now equal in terms of rules. Epic: none Release note: None --- .../scplan/internal/rules/helpers.go | 27 +++++++++++++++++++ .../rules/release_22_2/dep_add_column.go | 11 -------- .../rules/release_22_2/dep_drop_column.go | 2 +- .../rules/release_22_2/dep_drop_constraint.go | 2 +- .../rules/release_22_2/dep_drop_index.go | 2 +- .../internal/rules/release_22_2/helpers.go | 7 +++++ .../internal/rules/release_22_2/op_drop.go | 2 -- .../rules/release_22_2/testdata/deprules | 19 +++++++------ .../rules/release_22_2/testdata/oprules | 2 +- 9 files changed, 47 insertions(+), 27 deletions(-) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go index e26ac11c2f6a..d7dbfd0f434d 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go @@ -31,6 +31,8 @@ func join(a, b NodeVars, attr rel.Attr, eqVarName rel.Var) rel.Clause { var _ = join +// JoinOn joins on two node variable attributes, requiring them to have +// the same value. func JoinOn(a NodeVars, aAttr rel.Attr, b NodeVars, bAttr rel.Attr, eqVarName rel.Var) rel.Clause { return rel.And( a.El.AttrEqVar(aAttr, eqVarName), @@ -38,14 +40,20 @@ func JoinOn(a NodeVars, aAttr rel.Attr, b NodeVars, bAttr rel.Attr, eqVarName re ) } +// FilterElements is used to construct a clause which runs an arbitrary predicate +// // over variables. func FilterElements(name string, a, b NodeVars, fn interface{}) rel.Clause { return rel.Filter(name, a.El, b.El)(fn) } +// ToPublicOrTransient is used to construct a clause that will require both +// elements to be targeting a public/transient state. func ToPublicOrTransient(from, to NodeVars) rel.Clause { return toPublicOrTransientUntyped(from.Target, to.Target) } +// StatusesToPublicOrTransient requires that elements have a target of +// ToPublicOrTransient and that the current status is fromStatus, toStatus. func StatusesToPublicOrTransient( from NodeVars, fromStatus scpb.Status, to NodeVars, toStatus scpb.Status, ) rel.Clause { @@ -60,6 +68,8 @@ func toAbsent(from, to NodeVars) rel.Clause { return toAbsentUntyped(from.Target, to.Target) } +// StatusesToAbsent requires that elements have a target of +// toAbsent and that the current status is fromStatus/toStatus. func StatusesToAbsent( from NodeVars, fromStatus scpb.Status, to NodeVars, toStatus scpb.Status, ) rel.Clause { @@ -74,6 +84,8 @@ func transient(from, to NodeVars) rel.Clause { return transientUntyped(from.Target, to.Target) } +// StatusesTransient requires that elements have a target of +// transient and that the current status is fromStatus/toStatus. func StatusesTransient( from NodeVars, fromStatus scpb.Status, to NodeVars, toStatus scpb.Status, ) rel.Clause { @@ -84,32 +96,40 @@ func StatusesTransient( ) } +// JoinOnDescID joins elements on descriptor ID. func JoinOnDescID(a, b NodeVars, descriptorIDVar rel.Var) rel.Clause { return JoinOnDescIDUntyped(a.El, b.El, descriptorIDVar) } +// JoinReferencedDescID joins elements on referenced descriptor ID. func JoinReferencedDescID(a, b NodeVars, descriptorIDVar rel.Var) rel.Clause { return joinReferencedDescIDUntyped(a.El, b.El, descriptorIDVar) } +// JoinOnColumnID joins elements on column ID. func JoinOnColumnID(a, b NodeVars, relationIDVar, columnIDVar rel.Var) rel.Clause { return joinOnColumnIDUntyped(a.El, b.El, relationIDVar, columnIDVar) } +// JoinOnIndexID joins elements on index ID. func JoinOnIndexID(a, b NodeVars, relationIDVar, indexIDVar rel.Var) rel.Clause { return joinOnIndexIDUntyped(a.El, b.El, relationIDVar, indexIDVar) } +// JoinOnConstraintID joins elements on constraint ID. func JoinOnConstraintID(a, b NodeVars, relationIDVar, constraintID rel.Var) rel.Clause { return joinOnConstraintIDUntyped(a.El, b.El, relationIDVar, constraintID) } +// ColumnInIndex requires that a column exists within an index. func ColumnInIndex( indexColumn, index NodeVars, relationIDVar, columnIDVar, indexIDVar rel.Var, ) rel.Clause { return columnInIndexUntyped(indexColumn.El, index.El, relationIDVar, columnIDVar, indexIDVar) } +// ColumnInSwappedInPrimaryIndex requires that a column exists within a +// primary index being swapped. func ColumnInSwappedInPrimaryIndex( indexColumn, index NodeVars, relationIDVar, columnIDVar, indexIDVar rel.Var, ) rel.Clause { @@ -156,6 +176,8 @@ var ( referenced.AttrEqVar(screl.DescID, id), } }) + // JoinOnDescIDUntyped joins on descriptor ID, in an unsafe non-type safe + // manner. JoinOnDescIDUntyped = screl.Schema.Def3( "joinOnDescID", "a", "b", "id", func( a, b, id rel.Var, @@ -231,6 +253,7 @@ var ( }) ) +// ForEachElement executes a function for each element type. func ForEachElement(fn func(element scpb.Element) error) error { var ep scpb.ElementProto vep := reflect.ValueOf(ep) @@ -243,6 +266,8 @@ func ForEachElement(fn func(element scpb.Element) error) error { return nil } +// ForEachElementInActiveVersion executes a function for each element supported within +// the current active version. func ForEachElementInActiveVersion( version clusterversion.ClusterVersion, fn func(element scpb.Element) error, ) error { @@ -261,6 +286,7 @@ func ForEachElementInActiveVersion( type elementTypePredicate = func(e scpb.Element) bool +// Or or's a series of element type predicates. func Or(predicates ...elementTypePredicate) elementTypePredicate { return func(e scpb.Element) bool { for _, p := range predicates { @@ -272,6 +298,7 @@ func Or(predicates ...elementTypePredicate) elementTypePredicate { } } +// Not not's a element type predicate. func Not(predicate elementTypePredicate) elementTypePredicate { return func(e scpb.Element) bool { return !predicate(e) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_column.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_column.go index 1149dd21d7fe..167553c9df6a 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_column.go @@ -98,17 +98,6 @@ func init() { status := rel.Var("status") return rel.Clauses{ from.Type((*scpb.Column)(nil)), - // Join first on the target and node to only explore all columns - // which are being added as opposed to all columns. If we joined - // first on the columns, we'd be filtering the cross product of - // table columns. If a relation has a lot of columns, this can hurt. - // It's less likely that we have a very large number of columns which - // are being added. We'll want to do something else here when we start - // creating tables and all the columns are being added. - // - // The "right" answer is to push ordering predicates into rel; it also - // is maintaining sorted data structures. - from.JoinTargetNode(), to.Type((*scpb.Column)(nil)), JoinOnDescID(from, to, "table-id"), ToPublicOrTransient(from, to), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_column.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_column.go index c54cf575bd98..4d4b1856958d 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_column.go @@ -83,7 +83,7 @@ func init() { // which also hold references to other descriptors. The rule prior to this one // ensures that they transition to ABSENT before scpb.ColumnType does. registerDepRule( - "column type removed right before column when rules.Not dropping relation", + "column type removed right before column when not dropping relation", scgraph.SameStagePrecedence, "column-type", "column", func(from, to NodeVars) rel.Clauses { diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_constraint.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_constraint.go index 30d73e248c86..c7a1ae6b19f1 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_constraint.go @@ -23,7 +23,7 @@ func init() { registerDepRuleForDrop( "constraint dependent absent right before constraint", - scgraph.Precedence, + scgraph.SameStagePrecedence, "dependent", "constraint", scpb.Status_ABSENT, scpb.Status_ABSENT, func(from, to NodeVars) rel.Clauses { diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index.go index 858391fadb12..dd0ebe8588de 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index.go @@ -91,7 +91,7 @@ func init() { // should be able to express the _absence_ of a target element as a query // clause. registerDepRuleForDrop( - "partial predicate removed right before secondary index when rules.Not dropping relation", + "partial predicate removed right before secondary index when not dropping relation", scgraph.SameStagePrecedence, "partial-predicate", "index", scpb.Status_ABSENT, scpb.Status_ABSENT, diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/helpers.go index 48c3bf3e4346..bb8f04cc5531 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/helpers.go @@ -55,6 +55,12 @@ func isDescriptor(e scpb.Element) bool { return false } +// IsDescriptor returns true for a descriptor-element, i.e. an element which +// owns its corresponding descriptor. This is only used for exports +func IsDescriptor(e scpb.Element) bool { + return isDescriptor(e) +} + func isSubjectTo2VersionInvariant(e scpb.Element) bool { // TODO(ajwerner): This should include constraints and enum values but it // currently does not because we do not support dropping them unless we're @@ -62,6 +68,7 @@ func isSubjectTo2VersionInvariant(e scpb.Element) bool { return IsIndex(e) || isColumn(e) } +// IsIndex returns if an element is an index type. func IsIndex(e scpb.Element) bool { switch e.(type) { case *scpb.PrimaryIndex, *scpb.SecondaryIndex, *scpb.TemporaryIndex: diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_drop.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_drop.go index 7010ebcd3bb3..35462e80405e 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_drop.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_drop.go @@ -169,8 +169,6 @@ func init() { (*scpb.View)(nil), ), constraint.Type( - (*scpb.CheckConstraint)(nil), - (*scpb.ForeignKeyConstraint)(nil), (*scpb.UniqueWithoutIndexConstraint)(nil), ), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/deprules index 7be3fc7afd5b..dfb0a5e4aeb5 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/deprules @@ -1395,7 +1395,7 @@ deprules - $column-type-Node[CurrentStatus] = ABSENT - joinTargetNode($dependent, $dependent-Target, $dependent-Node) - joinTargetNode($column-type, $column-type-Target, $column-type-Node) -- name: column type removed right before column when rules.Not dropping relation +- name: column type removed right before column when not dropping relation from: column-type-Node kind: SameStagePrecedence to: column-Node @@ -1411,7 +1411,7 @@ deprules - joinTargetNode($column, $column-Target, $column-Node) - name: constraint dependent absent right before constraint from: dependent-Node - kind: Precedence + kind: SameStagePrecedence to: constraint-Node query: - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] @@ -1424,7 +1424,7 @@ deprules - joinTargetNode($constraint, $constraint-Target, $constraint-Node) - name: constraint dependent absent right before constraint from: dependent-Node - kind: Precedence + kind: SameStagePrecedence to: constraint-Node query: - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] @@ -1437,7 +1437,7 @@ deprules - joinTargetNode($constraint, $constraint-Target, $constraint-Node) - name: constraint dependent absent right before constraint from: dependent-Node - kind: Precedence + kind: SameStagePrecedence to: constraint-Node query: - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] @@ -1451,7 +1451,7 @@ deprules - joinTargetNode($constraint, $constraint-Target, $constraint-Node) - name: constraint dependent absent right before constraint from: dependent-Node - kind: Precedence + kind: SameStagePrecedence to: constraint-Node query: - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] @@ -1740,7 +1740,6 @@ deprules to: earlier-column-Node query: - $later-column[Type] = '*scpb.Column' - - joinTargetNode($later-column, $later-column-Target, $later-column-Node) - $earlier-column[Type] = '*scpb.Column' - joinOnDescID($later-column, $earlier-column, $table-id) - ToPublicOrTransient($later-column-Target, $earlier-column-Target) @@ -1951,7 +1950,7 @@ deprules - $new-primary-index-Node[CurrentStatus] = PUBLIC - joinTargetNode($old-primary-index, $old-primary-index-Target, $old-primary-index-Node) - joinTargetNode($new-primary-index, $new-primary-index-Target, $new-primary-index-Node) -- name: partial predicate removed right before secondary index when rules.Not dropping relation +- name: partial predicate removed right before secondary index when not dropping relation from: partial-predicate-Node kind: SameStagePrecedence to: index-Node @@ -1965,7 +1964,7 @@ deprules - $index-Node[CurrentStatus] = ABSENT - joinTargetNode($partial-predicate, $partial-predicate-Target, $partial-predicate-Node) - joinTargetNode($index, $index-Target, $index-Node) -- name: partial predicate removed right before secondary index when rules.Not dropping relation +- name: partial predicate removed right before secondary index when not dropping relation from: partial-predicate-Node kind: SameStagePrecedence to: index-Node @@ -1979,7 +1978,7 @@ deprules - $index-Node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($partial-predicate, $partial-predicate-Target, $partial-predicate-Node) - joinTargetNode($index, $index-Target, $index-Node) -- name: partial predicate removed right before secondary index when rules.Not dropping relation +- name: partial predicate removed right before secondary index when not dropping relation from: partial-predicate-Node kind: SameStagePrecedence to: index-Node @@ -1994,7 +1993,7 @@ deprules - $index-Node[CurrentStatus] = ABSENT - joinTargetNode($partial-predicate, $partial-predicate-Target, $partial-predicate-Node) - joinTargetNode($index, $index-Target, $index-Node) -- name: partial predicate removed right before secondary index when rules.Not dropping relation +- name: partial predicate removed right before secondary index when not dropping relation from: partial-predicate-Node kind: SameStagePrecedence to: index-Node diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/oprules b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/oprules index 1ef348ddbb18..9a2331cd6e52 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/oprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/oprules @@ -149,7 +149,7 @@ oprules from: constraint-Node query: - $relation[Type] IN ['*scpb.Table', '*scpb.View'] - - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.UniqueWithoutIndexConstraint'] + - $constraint[Type] = '*scpb.UniqueWithoutIndexConstraint' - joinOnDescID($relation, $constraint, $relation-id) - joinTarget($relation, $relation-Target) - $relation-Target[TargetStatus] = ABSENT From ffad33a5afef5fa894d255406816cfdae5413ca6 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Tue, 31 Jan 2023 11:48:28 -0500 Subject: [PATCH 2/9] schemachanger: Improve handling of FKs This commit changed sc operations emmitted on the adding/dropping path of FK constraint. The major change is to recognize that FK constraints are "cross-descriptor" constraints and thus we need to make accompanying changes in the referenced tables whenever we make changes to the referencing table. --- pkg/sql/catalog/tabledesc/constraint.go | 13 ++++ pkg/sql/catalog/tabledesc/validate.go | 14 ++-- .../scexec/scmutationexec/constraint.go | 77 ++++++++++++++----- .../scexec/scmutationexec/references.go | 30 +------- 4 files changed, 82 insertions(+), 52 deletions(-) diff --git a/pkg/sql/catalog/tabledesc/constraint.go b/pkg/sql/catalog/tabledesc/constraint.go index b7e4f0551af2..91ba1472568c 100644 --- a/pkg/sql/catalog/tabledesc/constraint.go +++ b/pkg/sql/catalog/tabledesc/constraint.go @@ -403,6 +403,11 @@ func newConstraintCache( } else { c.all = append(c.all, ck) c.checks = append(c.checks, ck) + // TODO (xiang): If `m.WriteAndDeleteOnly()`, then `ck` is enforced and + // we need to add it to `c.allEnforced` and `c.checksEnforced`. We don't + // do it right now because it will break some existing unit test about + // legacy schema changer, due to different semantics of when a check + // is enforced. } } } @@ -430,6 +435,10 @@ func newConstraintCache( } else { c.all = append(c.all, fk) c.fks = append(c.fks, fk) + if m.WriteAndDeleteOnly() { + c.allEnforced = append(c.allEnforced, fk) + c.fksEnforced = append(c.fksEnforced, fk) + } } } } @@ -457,6 +466,10 @@ func newConstraintCache( } else { c.all = append(c.all, uwoi) c.uwois = append(c.uwois, uwoi) + if m.WriteAndDeleteOnly() { + c.allEnforced = append(c.allEnforced, uwoi) + c.uwoisEnforced = append(c.uwoisEnforced, uwoi) + } } } } diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index 64cfb0335141..71d849600f43 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -68,11 +68,11 @@ func (desc *wrapper) GetReferencedDescIDs() (catalog.DescriptorIDSet, error) { ids.Add(desc.GetParentSchemaID()) } // Collect referenced table IDs in foreign keys. - for _, fk := range desc.OutboundFKs { - ids.Add(fk.ReferencedTableID) + for _, fk := range desc.OutboundForeignKeys() { + ids.Add(fk.GetReferencedTableID()) } - for _, fk := range desc.InboundFKs { - ids.Add(fk.OriginTableID) + for _, fk := range desc.InboundForeignKeys() { + ids.Add(fk.GetOriginTableID()) } // Collect user defined type Oids and sequence references in columns. for _, col := range desc.DeletableColumns() { @@ -205,9 +205,9 @@ func (desc *wrapper) ValidateForwardReferences( } } - // Check foreign keys. - for i := range desc.OutboundFKs { - vea.Report(desc.validateOutboundFK(&desc.OutboundFKs[i], vdg)) + // Check enforced outbound foreign keys. + for _, fk := range desc.EnforcedOutboundForeignKeys() { + vea.Report(desc.validateOutboundFK(fk.ForeignKeyDesc(), vdg)) } // Check partitioning is correctly set. diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go b/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go index 6689975fbf41..c616dbc55e70 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go @@ -37,6 +37,19 @@ func (i *immediateVisitor) SetConstraintName(ctx context.Context, op scop.SetCon constraint.AsCheck().CheckDesc().Name = op.Name } else if constraint.AsForeignKey() != nil { constraint.AsForeignKey().ForeignKeyDesc().Name = op.Name + // Also attempt to set the FK constraint name in the referenced table. + // This is needed on the dropping path (i.e. when dropping an existing + // FK constraint). + referencedTable, err := i.checkOutTable(ctx, constraint.AsForeignKey().GetReferencedTableID()) + if err != nil || referencedTable.Dropped() { + return err + } + for _, inboundFK := range referencedTable.InboundForeignKeys() { + if inboundFK.GetOriginTableID() == op.TableID && inboundFK.GetConstraintID() == op.ConstraintID { + inboundFK.ForeignKeyDesc().Name = op.Name + break + } + } } else { return errors.AssertionFailedf("unknown constraint type") } @@ -366,6 +379,7 @@ func (i *immediateVisitor) MakeAbsentForeignKeyConstraintWriteOnly( out.NextConstraintID = op.ConstraintID + 1 } + // Enqueue a mutation in `out` to signal this mutation is now enforced. fk := &descpb.ForeignKeyConstraint{ OriginTableID: op.TableID, OriginColumnIDs: op.ColumnIDs, @@ -380,6 +394,12 @@ func (i *immediateVisitor) MakeAbsentForeignKeyConstraintWriteOnly( } enqueueNonIndexMutation(out, out.AddForeignKeyMutation, fk, descpb.DescriptorMutation_ADD) out.Mutations[len(out.Mutations)-1].State = descpb.DescriptorMutation_WRITE_ONLY + // Add an entry in "InboundFKs" in the referenced table as company. + in, err := i.checkOutTable(ctx, op.ReferencedTableID) + if err != nil { + return err + } + in.InboundFKs = append(in.InboundFKs, *fk) return nil } @@ -400,19 +420,19 @@ func (i *immediateVisitor) MakeValidatedForeignKeyConstraintPublic( if c := mutation.GetConstraint(); c != nil && c.ConstraintType == descpb.ConstraintToUpdate_FOREIGN_KEY && c.ForeignKey.ConstraintID == op.ConstraintID { + // Complete this mutation by marking the validity as validated, + // removing it from the mutations slice, and publishing it into + // OutboundFKs slice. + c.ForeignKey.Validity = descpb.ConstraintValidity_Validated out.OutboundFKs = append(out.OutboundFKs, c.ForeignKey) - in.InboundFKs = append(in.InboundFKs, c.ForeignKey) + out.Mutations = append(out.Mutations[:idx], out.Mutations[idx+1:]...) - // Remove the mutation from the mutation slice. The `MakeMutationComplete` - // call will also mark the above added check as VALIDATED. - // If this is a rollback of a drop, we are trying to add the foreign key constraint - // back, so swap the direction before making it complete. - mutation.Direction = descpb.DescriptorMutation_ADD - err = out.MakeMutationComplete(mutation) - if err != nil { - return err + // Update the back-reference in the referenced table. + for i, inboundFK := range in.InboundFKs { + if inboundFK.OriginTableID == out.GetID() && inboundFK.ConstraintID == op.ConstraintID { + in.InboundFKs[i].Validity = descpb.ConstraintValidity_Validated + } } - out.Mutations = append(out.Mutations[:idx], out.Mutations[idx+1:]...) found = true break @@ -434,23 +454,44 @@ func (i *immediateVisitor) MakeValidatedForeignKeyConstraintPublic( func (i *immediateVisitor) MakePublicForeignKeyConstraintValidated( ctx context.Context, op scop.MakePublicForeignKeyConstraintValidated, ) error { - tbl, err := i.checkOutTable(ctx, op.TableID) + // A helper function to update the inbound FK in referenced table to DROPPING. + updateInboundFKAsDropping := func(referencedTableID descpb.ID) error { + foundInReferencedTable := false + in, err := i.checkOutTable(ctx, referencedTableID) + if err != nil { + return err + } + for idx, inboundFk := range in.InboundFKs { + if inboundFk.OriginTableID == op.TableID && inboundFk.ConstraintID == op.ConstraintID { + in.InboundFKs[idx].Validity = descpb.ConstraintValidity_Dropping + foundInReferencedTable = true + break + } + } + if !foundInReferencedTable { + return errors.AssertionFailedf("failed to find accompanying inbound FK (%v) in"+ + " referenced table %v (%v)", op.ConstraintID, in.Name, in.ID) + } + return nil + } + + out, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } - for i, fk := range tbl.OutboundFKs { + for idx, fk := range out.OutboundFKs { if fk.ConstraintID == op.ConstraintID { - tbl.OutboundFKs = append(tbl.OutboundFKs[:i], tbl.OutboundFKs[i+1:]...) - if len(tbl.OutboundFKs) == 0 { - tbl.OutboundFKs = nil + out.OutboundFKs = append(out.OutboundFKs[:idx], out.OutboundFKs[idx+1:]...) + if len(out.OutboundFKs) == 0 { + out.OutboundFKs = nil } fk.Validity = descpb.ConstraintValidity_Dropping - enqueueNonIndexMutation(tbl, tbl.AddForeignKeyMutation, &fk, descpb.DescriptorMutation_DROP) - return nil + enqueueNonIndexMutation(out, out.AddForeignKeyMutation, &fk, descpb.DescriptorMutation_DROP) + return updateInboundFKAsDropping(fk.ReferencedTableID) } } - return errors.AssertionFailedf("failed to find FK constraint %d in descriptor %v", op.ConstraintID, tbl) + return errors.AssertionFailedf("failed to find FK constraint %d in descriptor %v", op.ConstraintID, out) } func (i *immediateVisitor) MakeAbsentUniqueWithoutIndexConstraintWriteOnly( diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/references.go b/pkg/sql/schemachanger/scexec/scmutationexec/references.go index de1db4a88909..ececa4a92cf7 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/references.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/references.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/iterutil" - "github.com/cockroachdb/errors" ) func (i *immediateVisitor) RemoveSchemaParent( @@ -74,29 +73,6 @@ func (i *immediateVisitor) RemoveForeignKeyBackReference( // Exit early if the foreign key back-reference holder is getting dropped. return err } - // Retrieve foreign key name in origin table to identify it in the referenced - // table. - var name string - { - out, err := i.getDescriptor(ctx, op.OriginTableID) - if err != nil { - return err - } - tbl, err := catalog.AsTableDescriptor(out) - if err != nil { - return err - } - for _, fk := range tbl.OutboundForeignKeys() { - if fk.GetConstraintID() == op.OriginConstraintID { - name = fk.GetName() - break - } - } - if name == "" { - return errors.AssertionFailedf("foreign key with ID %d not found in origin table %q (%d)", - op.OriginConstraintID, out.GetName(), out.GetID()) - } - } // Attempt to remove back reference. // Note how we // 1. only check to remove from `in.InboundFKs` but not from `in.Mutations`: @@ -106,9 +82,9 @@ func (i *immediateVisitor) RemoveForeignKeyBackReference( // 2. only attempt to remove (i.e. we do not panic when it's not found): // this is because if we roll back before the adding FK is published in `out`, // such a back-reference won't exist in `in` yet. - for i, fk := range in.InboundFKs { - if fk.OriginTableID == op.OriginTableID && fk.Name == name { - in.InboundFKs = append(in.InboundFKs[:i], in.InboundFKs[i+1:]...) + for idx, fk := range in.InboundFKs { + if fk.OriginTableID == op.OriginTableID && fk.ConstraintID == op.OriginConstraintID { + in.InboundFKs = append(in.InboundFKs[:idx], in.InboundFKs[idx+1:]...) if len(in.InboundFKs) == 0 { in.InboundFKs = nil } From e046e46dd83b7f0895c0c1ad36e32d70f0f407af Mon Sep 17 00:00:00 2001 From: healthy-pod Date: Wed, 25 Jan 2023 17:46:59 -0800 Subject: [PATCH 3/9] roachtest: normalize versions in multitenant-upgrade roachtest If the multitenant-upgrade roachtest uses a mix of release/non-release binaries, it may be using versions that are technically the same but fail to confirm that because version in test binaries are incremented by 1M. This code change fixes the issue by normalizing versions before comparing them. Closes #95648 Epic: none Release note: None --- pkg/cmd/roachtest/tests/BUILD.bazel | 1 + .../roachtest/tests/multitenant_upgrade.go | 58 +++++++++++++++---- 2 files changed, 48 insertions(+), 11 deletions(-) diff --git a/pkg/cmd/roachtest/tests/BUILD.bazel b/pkg/cmd/roachtest/tests/BUILD.bazel index 74d51884652b..1998595b6f64 100644 --- a/pkg/cmd/roachtest/tests/BUILD.bazel +++ b/pkg/cmd/roachtest/tests/BUILD.bazel @@ -177,6 +177,7 @@ go_library( "//pkg/cloud", "//pkg/cloud/amazon", "//pkg/cloud/gcp", + "//pkg/clusterversion", "//pkg/cmd/cmpconn", "//pkg/cmd/roachtest/cluster", "//pkg/cmd/roachtest/clusterstats", diff --git a/pkg/cmd/roachtest/tests/multitenant_upgrade.go b/pkg/cmd/roachtest/tests/multitenant_upgrade.go index c9312ef49d6d..3a55e5ead85b 100644 --- a/pkg/cmd/roachtest/tests/multitenant_upgrade.go +++ b/pkg/cmd/roachtest/tests/multitenant_upgrade.go @@ -17,11 +17,13 @@ import ( "runtime" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/version" @@ -107,7 +109,8 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster, verifySQL(t, tenant11.pgURL, mkStmt("SHOW CLUSTER SETTING version"). - withResults([][]string{{initialVersion}}), + withResults([][]string{{initialVersion}}). + withCustomResultsEqualFn(requireEqualVersionsIgnoreDevOffset), ) t.Status("preserving downgrade option on system tenant") @@ -149,7 +152,8 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster, mkStmt(`SELECT * FROM foo LIMIT 1`). withResults([][]string{{"1", "bar"}}), mkStmt("SHOW CLUSTER SETTING version"). - withResults([][]string{{initialVersion}}), + withResults([][]string{{initialVersion}}). + withCustomResultsEqualFn(requireEqualVersionsIgnoreDevOffset), ) t.Status("creating a new tenant 13") @@ -171,7 +175,8 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster, mkStmt(`SELECT * FROM foo LIMIT 1`). withResults([][]string{{"1", "bar"}}), mkStmt("SHOW CLUSTER SETTING version"). - withResults([][]string{{initialVersion}}), + withResults([][]string{{initialVersion}}). + withCustomResultsEqualFn(requireEqualVersionsIgnoreDevOffset), ) t.Status("stopping the tenant 11 server ahead of upgrading") @@ -186,7 +191,9 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster, mkStmt(`SELECT * FROM foo LIMIT 1`). withResults([][]string{{"1", "bar"}}), mkStmt("SHOW CLUSTER SETTING version"). - withResults([][]string{{initialVersion}})) + withResults([][]string{{initialVersion}}). + withCustomResultsEqualFn(requireEqualVersionsIgnoreDevOffset), + ) } t.Status("attempting to upgrade tenant 11 before storage cluster is finalized and expecting a failure") @@ -206,7 +213,8 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster, mkStmt(`SELECT * FROM foo LIMIT 1`). withResults([][]string{{"1", "bar"}}), mkStmt("SHOW CLUSTER SETTING version"). - withResults([][]string{{initialVersion}}), + withResults([][]string{{initialVersion}}). + withCustomResultsEqualFn(requireEqualVersionsIgnoreDevOffset), mkStmt("SET CLUSTER SETTING version = crdb_internal.node_executable_version()"), mkStmt("SELECT version = crdb_internal.node_executable_version() FROM [SHOW CLUSTER SETTING version]"). withResults([][]string{{"true"}}), @@ -223,7 +231,9 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster, mkStmt(`SELECT * FROM foo LIMIT 1`). withResults([][]string{{"1", "bar"}}), mkStmt("SHOW CLUSTER SETTING version"). - withResults([][]string{{initialVersion}})) + withResults([][]string{{initialVersion}}). + withCustomResultsEqualFn(requireEqualVersionsIgnoreDevOffset), + ) // Upgrade the tenant created in the mixed version state to the final version. t.Status("migrating tenant 12 to the current version") @@ -295,10 +305,31 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster, withResults([][]string{{"true"}})) } +// EqualFn is implemented by both `require.Equal` and `requireEqualVersionsIgnoreDevOffset`. +type EqualFn = func(t require.TestingT, expected interface{}, actual interface{}, msgAndArgs ...interface{}) + +// TODO: replace this with require.Equal once #92608 is closed because +// asserting on specific cluster versions makes for a stronger +// test (and having or not having the version offset is part of that). +func requireEqualVersionsIgnoreDevOffset( + t require.TestingT, expected interface{}, actual interface{}, msgAndArgs ...interface{}, +) { + normalizeVersion := func(v roachpb.Version) roachpb.Version { + if v.Major > clusterversion.DevOffset { + v.Major -= clusterversion.DevOffset + } + return v + } + normalizedExpectedVersion := normalizeVersion(roachpb.MustParseVersion(expected.([][]string)[0][0])) + normalizedActualVersion := normalizeVersion(roachpb.MustParseVersion(actual.([][]string)[0][0])) + require.Equal(t, normalizedExpectedVersion, normalizedActualVersion, msgAndArgs...) +} + type sqlVerificationStmt struct { - stmt string - args []interface{} - optionalResults [][]string + stmt string + args []interface{} + optionalResults [][]string + optionalResultsEqualFn EqualFn } func (s sqlVerificationStmt) withResults(res [][]string) sqlVerificationStmt { @@ -306,8 +337,13 @@ func (s sqlVerificationStmt) withResults(res [][]string) sqlVerificationStmt { return s } +func (s sqlVerificationStmt) withCustomResultsEqualFn(fn EqualFn) sqlVerificationStmt { + s.optionalResultsEqualFn = fn + return s +} + func mkStmt(stmt string, args ...interface{}) sqlVerificationStmt { - return sqlVerificationStmt{stmt: stmt, args: args} + return sqlVerificationStmt{stmt: stmt, args: args, optionalResultsEqualFn: require.Equal} } func openDBAndMakeSQLRunner(t test.Test, url string) (*sqlutils.SQLRunner, func()) { @@ -328,7 +364,7 @@ func verifySQL(t test.Test, url string, stmts ...sqlVerificationStmt) { tdb.Exec(t, stmt.stmt, stmt.args...) } else { res := tdb.QueryStr(t, stmt.stmt, stmt.args...) - require.Equal(t, stmt.optionalResults, res) + stmt.optionalResultsEqualFn(t, stmt.optionalResults, res) } } } From 2428a4377d8ed6bdff81d7d820cf6d30f1060533 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Mon, 30 Jan 2023 16:42:40 -0500 Subject: [PATCH 4/9] schemachanger: Implement `DROP CONSTRAINT` This commit implements `DROP CONSTRAINT`, turn it on by default, and added tests. --- .../testdata/benchmark_expectations | 6 +- .../backup_base_generated_test.go | 15 + .../drop_database_multiregion_primary_region | 22 +- .../drop_database_multiregion_primary_region | 168 ++++--- .../explain_verbose/drop_table_multiregion | 18 +- .../drop_table_multiregion_primary_region | 12 +- pkg/sql/catalog/tabledesc/validate.go | 4 +- .../logictest/testdata/logic_test/alter_table | 122 +++++ pkg/sql/logictest/testdata/logic_test/fk | 2 +- .../scbuild/internal/scbuildstmt/BUILD.bazel | 1 + .../internal/scbuildstmt/alter_table.go | 1 + .../alter_table_drop_constraint.go | 76 +++ .../testdata/unimplemented_alter_table | 4 - .../scexec/scmutationexec/references.go | 2 +- .../internal/rules/current/assertions_test.go | 42 +- .../current/dep_add_index_and_constraint.go | 2 +- .../rules/current/dep_drop_constraint.go | 4 +- .../internal/rules/current/dep_drop_object.go | 98 +++- .../scplan/internal/rules/current/helpers.go | 32 +- .../scplan/internal/rules/current/op_drop.go | 2 +- .../internal/rules/current/testdata/deprules | 60 ++- .../scplan/testdata/alter_table_drop_column | 8 +- .../scplan/testdata/drop_database | 374 +++++++------- .../schemachanger/scplan/testdata/drop_index | 4 +- .../scplan/testdata/drop_owned_by | 44 +- .../schemachanger/scplan/testdata/drop_schema | 168 +++---- .../scplan/testdata/drop_sequence | 4 +- .../schemachanger/scplan/testdata/drop_table | 455 ++++-------------- .../schemachanger/scplan/testdata/drop_type | 16 +- .../schemachanger/scplan/testdata/drop_view | 24 +- .../schemachanger/sctest_generated_test.go | 75 +++ .../end_to_end/alter_table_add_foreign_key | 56 ++- .../alter_table_drop_constraint_check | 192 ++++++++ .../end_to_end/alter_table_drop_constraint_fk | 310 ++++++++++++ .../alter_table_drop_constraint_uwi | 184 +++++++ .../explain/alter_table_drop_constraint_check | 39 ++ .../explain/alter_table_drop_constraint_fk | 43 ++ .../explain/alter_table_drop_constraint_uwi | 41 ++ .../alter_table_drop_constraint_check | 129 +++++ .../alter_table_drop_constraint_fk | 145 ++++++ .../alter_table_drop_constraint_uwi | 131 +++++ .../create_function_in_txn.rollback_1_of_7 | 3 +- .../create_function_in_txn.rollback_2_of_7 | 3 +- .../create_function_in_txn.rollback_3_of_7 | 3 +- .../create_function_in_txn.rollback_4_of_7 | 3 +- .../create_function_in_txn.rollback_5_of_7 | 3 +- .../create_function_in_txn.rollback_6_of_7 | 3 +- .../create_function_in_txn.rollback_7_of_7 | 3 +- .../testdata/explain_verbose/drop_function | 6 +- .../drop_index_with_materialized_view_dep | 6 +- .../testdata/explain_verbose/drop_schema | 6 +- .../testdata/explain_verbose/drop_table | 12 +- 52 files changed, 2359 insertions(+), 827 deletions(-) create mode 100644 pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_constraint.go create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_check create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_fk create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_uwi create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_drop_constraint_check create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_drop_constraint_fk create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_drop_constraint_uwi create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_drop_constraint_check create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_drop_constraint_fk create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_drop_constraint_uwi diff --git a/pkg/bench/rttanalysis/testdata/benchmark_expectations b/pkg/bench/rttanalysis/testdata/benchmark_expectations index 7a4b603028c0..ce8b91c7fe35 100644 --- a/pkg/bench/rttanalysis/testdata/benchmark_expectations +++ b/pkg/bench/rttanalysis/testdata/benchmark_expectations @@ -18,9 +18,9 @@ exp,benchmark 15,AlterTableDropColumn/alter_table_drop_1_column 15,AlterTableDropColumn/alter_table_drop_2_columns 15,AlterTableDropColumn/alter_table_drop_3_columns -9,AlterTableDropConstraint/alter_table_drop_1_check_constraint -9,AlterTableDropConstraint/alter_table_drop_2_check_constraints -9,AlterTableDropConstraint/alter_table_drop_3_check_constraints +11,AlterTableDropConstraint/alter_table_drop_1_check_constraint +11,AlterTableDropConstraint/alter_table_drop_2_check_constraints +11,AlterTableDropConstraint/alter_table_drop_3_check_constraints 9,AlterTableSplit/alter_table_split_at_1_value 13,AlterTableSplit/alter_table_split_at_2_values 17,AlterTableSplit/alter_table_split_at_3_values diff --git a/pkg/ccl/schemachangerccl/backup_base_generated_test.go b/pkg/ccl/schemachangerccl/backup_base_generated_test.go index 011dff8f2395..8db271e972e9 100644 --- a/pkg/ccl/schemachangerccl/backup_base_generated_test.go +++ b/pkg/ccl/schemachangerccl/backup_base_generated_test.go @@ -78,6 +78,21 @@ func TestBackup_base_alter_table_alter_primary_key_vanilla(t *testing.T) { defer log.Scope(t).Close(t) sctest.Backup(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_vanilla", newCluster) } +func TestBackup_base_alter_table_drop_constraint_check(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Backup(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_check", newCluster) +} +func TestBackup_base_alter_table_drop_constraint_fk(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Backup(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_fk", newCluster) +} +func TestBackup_base_alter_table_drop_constraint_uwi(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Backup(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_uwi", newCluster) +} func TestBackup_base_create_function(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/ccl/schemachangerccl/testdata/explain/drop_database_multiregion_primary_region b/pkg/ccl/schemachangerccl/testdata/explain/drop_database_multiregion_primary_region index ed95d4f4555a..046a5bd7219b 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain/drop_database_multiregion_primary_region +++ b/pkg/ccl/schemachangerccl/testdata/explain/drop_database_multiregion_primary_region @@ -65,9 +65,6 @@ Schema change plan for DROP DATABASE ‹multi_region_test_db› CASCADE; │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} │ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} │ └── 15 Mutation operations - │ ├── MarkDescriptorAsDropped {"DescriptorID":104} - │ ├── MarkDescriptorAsDropped {"DescriptorID":105} - │ ├── RemoveSchemaParent {"Parent":{"ParentDatabaseID":104,"SchemaID":105}} │ ├── MarkDescriptorAsDropped {"DescriptorID":106} │ ├── RemoveObjectParent {"ObjectID":106,"ParentSchemaID":105} │ ├── MarkDescriptorAsDropped {"DescriptorID":107} @@ -75,11 +72,14 @@ Schema change plan for DROP DATABASE ‹multi_region_test_db› CASCADE; │ ├── MarkDescriptorAsDropped {"DescriptorID":108} │ ├── RemoveObjectParent {"ObjectID":108,"ParentSchemaID":105} │ ├── RemoveBackReferenceInTypes {"BackReferencedDescriptorID":108} - │ ├── DrainDescriptorName {"Namespace":{"DescriptorID":104,"Name":"multi_region_tes..."}} + │ ├── MarkDescriptorAsDropped {"DescriptorID":105} + │ ├── RemoveSchemaParent {"Parent":{"ParentDatabaseID":104,"SchemaID":105}} + │ ├── MarkDescriptorAsDropped {"DescriptorID":104} │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":105,"Name":"public"}} │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":106,"Name":"crdb_internal_re...","SchemaID":105}} │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":107,"Name":"_crdb_internal_r...","SchemaID":105}} - │ └── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":108,"Name":"table_regional_b...","SchemaID":105}} + │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":108,"Name":"table_regional_b...","SchemaID":105}} + │ └── DrainDescriptorName {"Namespace":{"DescriptorID":104,"Name":"multi_region_tes..."}} ├── PreCommitPhase │ ├── Stage 1 of 2 in PreCommitPhase │ │ ├── 53 elements transitioning toward ABSENT @@ -194,10 +194,6 @@ Schema change plan for DROP DATABASE ‹multi_region_test_db› CASCADE; │ │ ├── PUBLIC → ABSENT PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} │ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} │ └── 26 Mutation operations - │ ├── MarkDescriptorAsDropped {"DescriptorID":104} - │ ├── RemoveDatabaseRoleSettings {"DatabaseID":104} - │ ├── MarkDescriptorAsDropped {"DescriptorID":105} - │ ├── RemoveSchemaParent {"Parent":{"ParentDatabaseID":104,"SchemaID":105}} │ ├── MarkDescriptorAsDropped {"DescriptorID":106} │ ├── RemoveObjectParent {"ObjectID":106,"ParentSchemaID":105} │ ├── MarkDescriptorAsDropped {"DescriptorID":107} @@ -205,14 +201,18 @@ Schema change plan for DROP DATABASE ‹multi_region_test_db› CASCADE; │ ├── MarkDescriptorAsDropped {"DescriptorID":108} │ ├── RemoveObjectParent {"ObjectID":108,"ParentSchemaID":105} │ ├── RemoveBackReferenceInTypes {"BackReferencedDescriptorID":108} - │ ├── DrainDescriptorName {"Namespace":{"DescriptorID":104,"Name":"multi_region_tes..."}} + │ ├── MarkDescriptorAsDropped {"DescriptorID":105} + │ ├── RemoveSchemaParent {"Parent":{"ParentDatabaseID":104,"SchemaID":105}} + │ ├── RemoveColumnNotNull {"ColumnID":1,"TableID":108} + │ ├── MarkDescriptorAsDropped {"DescriptorID":104} + │ ├── RemoveDatabaseRoleSettings {"DatabaseID":104} │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":105,"Name":"public"}} │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":106,"Name":"crdb_internal_re...","SchemaID":105}} │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":107,"Name":"_crdb_internal_r...","SchemaID":105}} │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":108,"Name":"table_regional_b...","SchemaID":105}} - │ ├── RemoveColumnNotNull {"ColumnID":1,"TableID":108} │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":4294967295,"TableID":108} │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":4294967294,"TableID":108} + │ ├── DrainDescriptorName {"Namespace":{"DescriptorID":104,"Name":"multi_region_tes..."}} │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":1,"TableID":108} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ ├── SetJobStateOnDescriptor {"DescriptorID":105,"Initialize":true} diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_database_multiregion_primary_region b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_database_multiregion_primary_region index 5e2cc42a1d9f..bc01ded6ccc2 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_database_multiregion_primary_region +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_database_multiregion_primary_region @@ -58,7 +58,10 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Database:{DescID: 104} -│ │ │ PUBLIC → DROPPED +│ │ │ │ PUBLIC → DROPPED +│ │ │ │ +│ │ │ └── • Precedence dependency from ABSENT SchemaParent:{DescID: 105, ReferencedDescID: 104} +│ │ │ rule: "back-reference in parent descriptor is removed before parent descriptor is dropped" │ │ │ │ │ ├── • DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__} │ │ │ │ PUBLIC → ABSENT @@ -121,16 +124,23 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Schema:{DescID: 105} -│ │ │ PUBLIC → DROPPED +│ │ │ │ PUBLIC → DROPPED +│ │ │ │ +│ │ │ ├── • Precedence dependency from ABSENT ObjectParent:{DescID: 106, ReferencedDescID: 105} +│ │ │ │ rule: "back-reference in parent descriptor is removed before parent descriptor is dropped" +│ │ │ │ +│ │ │ ├── • Precedence dependency from ABSENT ObjectParent:{DescID: 107, ReferencedDescID: 105} +│ │ │ │ rule: "back-reference in parent descriptor is removed before parent descriptor is dropped" +│ │ │ │ +│ │ │ └── • Precedence dependency from ABSENT ObjectParent:{DescID: 108, ReferencedDescID: 105} +│ │ │ rule: "back-reference in parent descriptor is removed before parent descriptor is dropped" │ │ │ │ │ ├── • SchemaParent:{DescID: 105, ReferencedDescID: 104} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ ├── • SameStagePrecedence dependency from DROPPED Database:{DescID: 104} -│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" -│ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED Schema:{DescID: 105} +│ │ │ └── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 105} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • Namespace:{DescID: 106, Name: crdb_internal_region, ReferencedDescID: 104} │ │ │ │ PUBLIC → ABSENT @@ -210,11 +220,9 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ├── • ObjectParent:{DescID: 106, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ ├── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 105} -│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" -│ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ └── • SameStagePrecedence dependency from DROPPED EnumType:{DescID: 106} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • Namespace:{DescID: 107, Name: _crdb_internal_region, ReferencedDescID: 104} │ │ │ │ PUBLIC → ABSENT @@ -267,11 +275,9 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ├── • ObjectParent:{DescID: 107, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ ├── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 105} -│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" -│ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} +│ │ │ └── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} │ │ │ │ PUBLIC → ABSENT @@ -315,11 +321,9 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ ├── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 105} -│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" -│ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ └── • SameStagePrecedence dependency from DROPPED Table:{DescID: 108} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} │ │ │ │ PUBLIC → ABSENT @@ -372,6 +376,9 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ├── • ColumnNotNull:{DescID: 108, ColumnID: 1, IndexID: 0} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent constraint" +│ │ │ │ │ │ │ └── • skip PUBLIC → VALIDATED operations │ │ │ rule: "skip constraint removal ops on relation drop" │ │ │ @@ -475,17 +482,6 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ └── • 15 Mutation operations │ │ │ ├── • MarkDescriptorAsDropped -│ │ DescriptorID: 104 -│ │ -│ ├── • MarkDescriptorAsDropped -│ │ DescriptorID: 105 -│ │ -│ ├── • RemoveSchemaParent -│ │ Parent: -│ │ ParentDatabaseID: 104 -│ │ SchemaID: 105 -│ │ -│ ├── • MarkDescriptorAsDropped │ │ DescriptorID: 106 │ │ │ ├── • RemoveObjectParent @@ -511,10 +507,16 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ TypeIDs: │ │ - 106 │ │ -│ ├── • DrainDescriptorName -│ │ Namespace: -│ │ DescriptorID: 104 -│ │ Name: multi_region_test_db +│ ├── • MarkDescriptorAsDropped +│ │ DescriptorID: 105 +│ │ +│ ├── • RemoveSchemaParent +│ │ Parent: +│ │ ParentDatabaseID: 104 +│ │ SchemaID: 105 +│ │ +│ ├── • MarkDescriptorAsDropped +│ │ DescriptorID: 104 │ │ │ ├── • DrainDescriptorName │ │ Namespace: @@ -536,12 +538,17 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ Name: _crdb_internal_region │ │ SchemaID: 105 │ │ +│ ├── • DrainDescriptorName +│ │ Namespace: +│ │ DatabaseID: 104 +│ │ DescriptorID: 108 +│ │ Name: table_regional_by_table +│ │ SchemaID: 105 +│ │ │ └── • DrainDescriptorName │ Namespace: -│ DatabaseID: 104 -│ DescriptorID: 108 -│ Name: table_regional_by_table -│ SchemaID: 105 +│ DescriptorID: 104 +│ Name: multi_region_test_db │ ├── • PreCommitPhase │ │ @@ -760,7 +767,10 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Database:{DescID: 104} -│ │ │ PUBLIC → DROPPED +│ │ │ │ PUBLIC → DROPPED +│ │ │ │ +│ │ │ └── • Precedence dependency from ABSENT SchemaParent:{DescID: 105, ReferencedDescID: 104} +│ │ │ rule: "back-reference in parent descriptor is removed before parent descriptor is dropped" │ │ │ │ │ ├── • DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__} │ │ │ │ PUBLIC → ABSENT @@ -823,16 +833,23 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Schema:{DescID: 105} -│ │ │ PUBLIC → DROPPED +│ │ │ │ PUBLIC → DROPPED +│ │ │ │ +│ │ │ ├── • Precedence dependency from ABSENT ObjectParent:{DescID: 106, ReferencedDescID: 105} +│ │ │ │ rule: "back-reference in parent descriptor is removed before parent descriptor is dropped" +│ │ │ │ +│ │ │ ├── • Precedence dependency from ABSENT ObjectParent:{DescID: 107, ReferencedDescID: 105} +│ │ │ │ rule: "back-reference in parent descriptor is removed before parent descriptor is dropped" +│ │ │ │ +│ │ │ └── • Precedence dependency from ABSENT ObjectParent:{DescID: 108, ReferencedDescID: 105} +│ │ │ rule: "back-reference in parent descriptor is removed before parent descriptor is dropped" │ │ │ │ │ ├── • SchemaParent:{DescID: 105, ReferencedDescID: 104} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ ├── • SameStagePrecedence dependency from DROPPED Database:{DescID: 104} -│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" -│ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED Schema:{DescID: 105} +│ │ │ └── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 105} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • Namespace:{DescID: 106, Name: crdb_internal_region, ReferencedDescID: 104} │ │ │ │ PUBLIC → ABSENT @@ -912,11 +929,9 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ├── • ObjectParent:{DescID: 106, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ ├── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 105} -│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" -│ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ └── • SameStagePrecedence dependency from DROPPED EnumType:{DescID: 106} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • Namespace:{DescID: 107, Name: _crdb_internal_region, ReferencedDescID: 104} │ │ │ │ PUBLIC → ABSENT @@ -969,11 +984,9 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ├── • ObjectParent:{DescID: 107, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ ├── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 105} -│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" -│ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} +│ │ │ └── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} │ │ │ │ PUBLIC → ABSENT @@ -1017,11 +1030,9 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ ├── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 105} -│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" -│ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ └── • SameStagePrecedence dependency from DROPPED Table:{DescID: 108} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} │ │ │ │ PUBLIC → ABSENT @@ -1092,6 +1103,9 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ├── • ColumnNotNull:{DescID: 108, ColumnID: 1, IndexID: 0} │ │ │ │ PUBLIC → ABSENT │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent constraint" +│ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" │ │ │ │ @@ -1231,20 +1245,6 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ └── • 26 Mutation operations │ │ │ ├── • MarkDescriptorAsDropped -│ │ DescriptorID: 104 -│ │ -│ ├── • RemoveDatabaseRoleSettings -│ │ DatabaseID: 104 -│ │ -│ ├── • MarkDescriptorAsDropped -│ │ DescriptorID: 105 -│ │ -│ ├── • RemoveSchemaParent -│ │ Parent: -│ │ ParentDatabaseID: 104 -│ │ SchemaID: 105 -│ │ -│ ├── • MarkDescriptorAsDropped │ │ DescriptorID: 106 │ │ │ ├── • RemoveObjectParent @@ -1270,10 +1270,23 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ TypeIDs: │ │ - 106 │ │ -│ ├── • DrainDescriptorName -│ │ Namespace: -│ │ DescriptorID: 104 -│ │ Name: multi_region_test_db +│ ├── • MarkDescriptorAsDropped +│ │ DescriptorID: 105 +│ │ +│ ├── • RemoveSchemaParent +│ │ Parent: +│ │ ParentDatabaseID: 104 +│ │ SchemaID: 105 +│ │ +│ ├── • RemoveColumnNotNull +│ │ ColumnID: 1 +│ │ TableID: 108 +│ │ +│ ├── • MarkDescriptorAsDropped +│ │ DescriptorID: 104 +│ │ +│ ├── • RemoveDatabaseRoleSettings +│ │ DatabaseID: 104 │ │ │ ├── • DrainDescriptorName │ │ Namespace: @@ -1302,10 +1315,6 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ Name: table_regional_by_table │ │ SchemaID: 105 │ │ -│ ├── • RemoveColumnNotNull -│ │ ColumnID: 1 -│ │ TableID: 108 -│ │ │ ├── • MakeDeleteOnlyColumnAbsent │ │ ColumnID: 4294967295 │ │ TableID: 108 @@ -1314,6 +1323,11 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ColumnID: 4294967294 │ │ TableID: 108 │ │ +│ ├── • DrainDescriptorName +│ │ Namespace: +│ │ DescriptorID: 104 +│ │ Name: multi_region_test_db +│ │ │ ├── • MakeDeleteOnlyColumnAbsent │ │ ColumnID: 1 │ │ TableID: 108 diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion index 654a8fd7b66a..ba70a4bdaa63 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion @@ -54,8 +54,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ └── • SameStagePrecedence dependency from DROPPED Table:{DescID: 108} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • TablePartitioning:{DescID: 108} │ │ │ │ PUBLIC → ABSENT @@ -114,6 +115,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ ├── • ColumnNotNull:{DescID: 108, ColumnID: 1, IndexID: 0} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent constraint" +│ │ │ │ │ │ │ └── • skip PUBLIC → VALIDATED operations │ │ │ rule: "skip constraint removal ops on relation drop" │ │ │ @@ -153,6 +157,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ ├── • ColumnNotNull:{DescID: 108, ColumnID: 2, IndexID: 0} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent constraint" +│ │ │ │ │ │ │ └── • skip PUBLIC → VALIDATED operations │ │ │ rule: "skip constraint removal ops on relation drop" │ │ │ @@ -467,8 +474,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ └── • SameStagePrecedence dependency from DROPPED Table:{DescID: 108} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • TablePartitioning:{DescID: 108} │ │ │ │ PUBLIC → ABSENT @@ -545,6 +553,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ ├── • ColumnNotNull:{DescID: 108, ColumnID: 1, IndexID: 0} │ │ │ │ PUBLIC → ABSENT │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent constraint" +│ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" │ │ │ │ @@ -608,6 +619,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ ├── • ColumnNotNull:{DescID: 108, ColumnID: 2, IndexID: 0} │ │ │ │ PUBLIC → ABSENT │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent constraint" +│ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 2} │ │ │ │ rule: "column no longer public before dependents" │ │ │ │ diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion_primary_region b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion_primary_region index 1bcb840df84d..9287704c2182 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion_primary_region +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion_primary_region @@ -54,8 +54,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ └── • SameStagePrecedence dependency from DROPPED Table:{DescID: 108} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} │ │ │ │ PUBLIC → ABSENT @@ -105,6 +106,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ ├── • ColumnNotNull:{DescID: 108, ColumnID: 1, IndexID: 0} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent constraint" +│ │ │ │ │ │ │ └── • skip PUBLIC → VALIDATED operations │ │ │ rule: "skip constraint removal ops on relation drop" │ │ │ @@ -343,8 +347,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ └── • SameStagePrecedence dependency from DROPPED Table:{DescID: 108} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} │ │ │ │ PUBLIC → ABSENT @@ -412,6 +417,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ ├── • ColumnNotNull:{DescID: 108, ColumnID: 1, IndexID: 0} │ │ │ │ PUBLIC → ABSENT │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent constraint" +│ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" │ │ │ │ diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index 71d849600f43..c845b7f4c4e6 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -189,14 +189,14 @@ func (desc *wrapper) ValidateForwardReferences( // Row-level TTL is not compatible with foreign keys. // This check should be in ValidateSelf but interferes with AllocateIDs. if desc.HasRowLevelTTL() { - if len(desc.OutboundFKs) > 0 { + if len(desc.OutboundForeignKeys()) > 0 { vea.Report(unimplemented.NewWithIssuef( 76407, `foreign keys from table with TTL %q are not permitted`, desc.GetName(), )) } - if len(desc.InboundFKs) > 0 { + if len(desc.InboundForeignKeys()) > 0 { vea.Report(unimplemented.NewWithIssuef( 76407, `foreign keys to table with TTL %q are not permitted`, diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index 1755adcbb4a6..767e716e6a92 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -2781,3 +2781,125 @@ query IT SELECT c1, c3 FROM t_93398; ---- 0 f + +# The following subtest tests the case when there is a check constraint +# that references other descriptors (such as sequences and types): +# - drop referenced sequence cascade +# - drop the check itself +# - drop the referencing table +# Note that legacy schema changer won't support those cases correctly +# so we will skip them. +subtest cross_descriptor_checks_96115 + +statement ok +CREATE SEQUENCE seq_96115; + +statement ok +CREATE TYPE typ_96115 AS ENUM ('a', 'b'); + +statement ok +CREATE TABLE t_96115 (i INT PRIMARY KEY, j STRING); + +statement ok +ALTER TABLE t_96115 ADD CHECK (i > nextval('seq_96115') AND j::typ_96115 = 'a'::typ_96115); + +skipif config local-legacy-schema-changer +skipif config local-mixed-22.2-23.1 +query TTTB colnames +SELECT table_name, constraint_name, constraint_type, validated +FROM [SHOW CONSTRAINTS FROM t_96115] +ORDER BY constraint_type; +---- +table_name constraint_name constraint_type validated +t_96115 check_i_j CHECK true +t_96115 t_96115_pkey PRIMARY KEY true + +statement ok +ALTER TABLE t_96115 DROP CONSTRAINT check_i_j; + +skipif config local-legacy-schema-changer +skipif config local-mixed-22.2-23.1 +query TTTB colnames +SELECT table_name, constraint_name, constraint_type, validated +FROM [SHOW CONSTRAINTS FROM t_96115] +ORDER BY constraint_type; +---- +table_name constraint_name constraint_type validated +t_96115 t_96115_pkey PRIMARY KEY true + +statement ok +ALTER TABLE t_96115 ADD CHECK (i > nextval('seq_96115') AND j::typ_96115 = 'a'::typ_96115); + +skipif config local-legacy-schema-changer +skipif config local-mixed-22.2-23.1 +query TTTB colnames +SELECT table_name, constraint_name, constraint_type, validated +FROM [SHOW CONSTRAINTS FROM t_96115] +ORDER BY constraint_type; +---- +table_name constraint_name constraint_type validated +t_96115 check_i_j CHECK true +t_96115 t_96115_pkey PRIMARY KEY true + +statement ok +DROP SEQUENCE seq_96115 CASCADE; + +skipif config local-legacy-schema-changer +skipif config local-mixed-22.2-23.1 +query TTTB colnames +SELECT table_name, constraint_name, constraint_type, validated +FROM [SHOW CONSTRAINTS FROM t_96115] +ORDER BY constraint_type; +---- +table_name constraint_name constraint_type validated +t_96115 t_96115_pkey PRIMARY KEY true + +statement ok +CREATE SEQUENCE seq_96115; + +statement ok +ALTER TABLE t_96115 ADD CHECK (i > nextval('seq_96115') AND j::typ_96115 = 'a'::typ_96115); + +statement ok +DROP TABLE t_96115; + +statement ok +DROP TYPE typ_96115; + +statement ok +DROP SEQUENCE seq_96115 + +# This subtest ensures we can drop partial unique without index. +subtest drop_partial_unique_without_index_constraint_96115 + +statement ok +CREATE TABLE t_96115 (i INT PRIMARY KEY, j INT); + +statement ok +set experimental_enable_unique_without_index_constraints = true; + +statement ok +ALTER TABLE t_96115 ADD UNIQUE WITHOUT INDEX (j) WHERE j > 0; + +query TTTB colnames +SELECT table_name, constraint_name, constraint_type, validated +FROM [SHOW CONSTRAINTS FROM t_96115] +ORDER BY constraint_type; +---- +table_name constraint_name constraint_type validated +t_96115 t_96115_pkey PRIMARY KEY true +t_96115 unique_j UNIQUE true + +statement ok +ALTER TABLE t_96115 DROP CONSTRAINT unique_j; + +query TTTB colnames +SELECT table_name, constraint_name, constraint_type, validated +FROM [SHOW CONSTRAINTS FROM t_96115] +ORDER BY constraint_type; +---- +table_name constraint_name constraint_type validated +t_96115 t_96115_pkey PRIMARY KEY true + +statement ok +DROP TABLE t_96115; diff --git a/pkg/sql/logictest/testdata/logic_test/fk b/pkg/sql/logictest/testdata/logic_test/fk index 4e31c83ed372..bacccdd3941c 100644 --- a/pkg/sql/logictest/testdata/logic_test/fk +++ b/pkg/sql/logictest/testdata/logic_test/fk @@ -4119,7 +4119,7 @@ ALTER TABLE drop_fk_during_addition ADD CONSTRAINT fk FOREIGN KEY (name) REFERENCES drop_fk_during_addition_ref (name); DROP TABLE drop_fk_during_addition_ref; -statement error pgcode XXA00 pq: transaction committed but schema change aborted with error: \(42P01\): referenced relation \"drop_fk_during_addition_ref\" does not exist +statement error pgcode XXA00 pq: transaction committed but schema change aborted with error: .* referenced table "drop_fk_during_addition_ref" \(271\) is dropped.* COMMIT; # Regression test for #80828. Do not attempt a fast path cascading delete when diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel index a3dc07e18032..a17b35892f5a 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "alter_table_alter_column_set_not_null.go", "alter_table_alter_primary_key.go", "alter_table_drop_column.go", + "alter_table_drop_constraint.go", "comment_on.go", "create_function.go", "create_index.go", diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go index 866df485a1c2..bab311fe92f8 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go @@ -71,6 +71,7 @@ var supportedAlterTableStatements = map[reflect.Type]supportedAlterTableCommand{ return false }}, + reflect.TypeOf((*tree.AlterTableDropConstraint)(nil)): {fn: alterTableDropConstraint, on: true, minSupportedClusterVersion: clusterversion.V23_1}, } // alterTableAddConstraintMinSupportedClusterVersion tracks the minimal supported cluster version diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_constraint.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_constraint.go new file mode 100644 index 000000000000..ebe98c861ca0 --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_constraint.go @@ -0,0 +1,76 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scbuildstmt + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "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/util/errorutil/unimplemented" + "github.com/cockroachdb/errors" +) + +func alterTableDropConstraint( + b BuildCtx, tn *tree.TableName, tbl *scpb.Table, t *tree.AlterTableDropConstraint, +) { + constraintElems := b.ResolveConstraint(tbl.TableID, t.Constraint, ResolveParams{ + IsExistenceOptional: t.IfExists, + RequiredPrivilege: privilege.CREATE, + }) + if constraintElems == nil { + // Send a notice to user if constraint not found but `IF EXISTS` is set. + b.EvalCtx().ClientNoticeSender.BufferClientNotice(b, pgnotice.Newf( + "constraint %q of relation %q does not exist, skipping", t.Constraint, tn.Table())) + return + } + + // Dropping PK constraint: Fall back to legacy schema changer. + // CRDB only allows dropping PK constraint if it's immediately followed by a + // add PK constraint command in the same transaction. Declarative schema changer + // is not mature enough to deal with DDLs in transaction; we will fall back + // until it is. + fallBackIfDroppingPrimaryKey(constraintElems, t) + // Dropping UNIQUE constraint: error out as not implemented. + droppingUniqueConstraintNotImplemented(constraintElems, t) + + constraintElems.ForEachElementStatus(func( + _ scpb.Status, _ scpb.TargetStatus, e scpb.Element, + ) { + b.Drop(e) + }) +} + +func fallBackIfDroppingPrimaryKey( + constraintElems ElementResultSet, t *tree.AlterTableDropConstraint, +) { + _, _, pie := scpb.FindPrimaryIndex(constraintElems) + if pie != nil { + panic(scerrors.NotImplementedError(t)) + } +} + +func droppingUniqueConstraintNotImplemented( + constraintElems ElementResultSet, t *tree.AlterTableDropConstraint, +) { + _, _, sie := scpb.FindSecondaryIndex(constraintElems) + if sie != nil { + if sie.IsUnique { + panic(unimplemented.NewWithIssueDetailf(42840, "drop-constraint-unique", + "cannot drop UNIQUE constraint %q using ALTER TABLE DROP CONSTRAINT, use DROP INDEX CASCADE instead", + tree.ErrNameString(string(t.Constraint)))) + } else { + panic(errors.AssertionFailedf("dropping an index-backed constraint but the " + + "index is not unique")) + } + } +} diff --git a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table index b4330f0ae1fc..da5b164e5a14 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table +++ b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table @@ -48,10 +48,6 @@ unimplemented ALTER TABLE defaultdb.foo ADD COLUMN p INT NOT NULL UNIQUE ---- -unimplemented -ALTER TABLE defaultdb.foo DROP CONSTRAINT foobar ----- - unimplemented ALTER TABLE defaultdb.foo ALTER COLUMN i DROP NOT NULL ---- diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/references.go b/pkg/sql/schemachanger/scexec/scmutationexec/references.go index ececa4a92cf7..9fff00e5d619 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/references.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/references.go @@ -262,7 +262,7 @@ func updateBackReferencesInSequences( updated.Add(colID) } } else { - if !current.Contains(colID) { + if !current.Contains(colID) && colID != 0 { return nil } current.ForEach(func(id descpb.ColumnID) { diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/assertions_test.go b/pkg/sql/schemachanger/scplan/internal/rules/current/assertions_test.go index 6f5bfdcd7249..bd4d57f704ba 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/assertions_test.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/assertions_test.go @@ -36,6 +36,7 @@ func TestRuleAssertions(t *testing.T) { checkIsColumnDependent, checkIsIndexDependent, checkIsConstraintDependent, + checkConstraintPartitions, } { var fni interface{} = fn fullName := runtime.FuncForPC(reflect.ValueOf(fni).Pointer()).Name() @@ -150,7 +151,7 @@ func checkIsColumnDependent(e scpb.Element) error { // element. func checkIsIndexDependent(e scpb.Element) error { // Exclude indexes themselves and their data. - if isIndex(e) || isData(e) || isSupportedNonIndexBackedConstraint(e) { + if isIndex(e) || isData(e) || isNonIndexBackedConstraint(e) { return nil } // An index dependent should have an IndexID attribute. @@ -183,3 +184,42 @@ func checkIsConstraintDependent(e scpb.Element) error { } return nil } + +// Assert that any element with a "ConstraintID" attr is either a constraint +// or a constraintDependent. +// If it's a constraint, then +// - it's either a NonIndexBackedConstraint or isIndex +// - it's either a CrossDescriptorConstraint or it does not a referencedDescID|ReferencedSequenceIDs|ReferencedTypeIDs attr +func checkConstraintPartitions(e scpb.Element) error { + _, err := screl.Schema.GetAttribute(screl.ConstraintID, e) + if err != nil { + return nil //nolint:returnerrcheck + } + if !isConstraint(e) && !isConstraintDependent(e) { + return errors.New("has ConstraintID attr but is not a constraint nor a constraint dependent") + } + if isConstraintDependent(e) { + // The checks below only apply to constraints, so skip constraint dependents. + return nil + } + if isNonIndexBackedConstraint(e) == isIndex(e) { + if isIndex(e) { + return errors.New("verifies both isIndex and isNonIndexBackedConstraint") + } else { + return errors.New("doesn't verify isIndex nor isNonIndexBackedConstraint") + } + } + _, err1 := screl.Schema.GetAttribute(screl.ReferencedDescID, e) + _, err2 := screl.Schema.GetAttribute(screl.ReferencedSequenceIDs, e) + _, err3 := screl.Schema.GetAttribute(screl.ReferencedTypeIDs, e) + if isCrossDescriptorConstraint(e) { + if err1 != nil && err2 != nil && err3 != nil { + return errors.New("verifies isCrossDescriptorConstraint but does not have a Referenced.*ID attr") + } + } else { + if err1 == nil || err2 == nil || err3 == nil { + return errors.New("doesn't verify isCrossDescriptorConstraint but has a Referenced.*ID attr") + } + } + return nil +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_index_and_constraint.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_index_and_constraint.go index 7b34574a0b8b..2cae27f166bd 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_index_and_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_index_and_constraint.go @@ -28,7 +28,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.PrimaryIndex)(nil)), - to.TypeFilter(rulesVersionKey, isSupportedNonIndexBackedConstraint), + to.TypeFilter(rulesVersionKey, isNonIndexBackedConstraint), JoinOnDescID(from, to, "table-id"), JoinOn( from, screl.IndexID, diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_constraint.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_constraint.go index 3a8984b67508..c3a696572ee6 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_constraint.go @@ -31,7 +31,7 @@ func init() { scpb.Status_VALIDATED, scpb.Status_ABSENT, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(rulesVersionKey, isSupportedNonIndexBackedConstraint), + from.TypeFilter(rulesVersionKey, isNonIndexBackedConstraint), to.TypeFilter(rulesVersionKey, isConstraintDependent), JoinOnConstraintID(from, to, "table-id", "constraint-id"), } @@ -46,7 +46,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.TypeFilter(rulesVersionKey, isConstraintDependent), - to.TypeFilter(rulesVersionKey, isSupportedNonIndexBackedConstraint), + to.TypeFilter(rulesVersionKey, isNonIndexBackedConstraint), JoinOnConstraintID(from, to, "table-id", "constraint-id"), } }, diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_object.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_object.go index a38ade0ff73b..9865d7f33e69 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_object.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_object.go @@ -48,7 +48,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.TypeFilter(rulesVersionKey, isDescriptor), - to.TypeFilter(rulesVersionKey, isSimpleDependent), + to.TypeFilter(rulesVersionKey, isSimpleDependent, Not(isConstraintDependent)), JoinOnDescID(from, to, "desc-id"), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), } @@ -88,9 +88,9 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.Table)(nil)), - to.TypeFilter(rulesVersionKey, isSupportedNonIndexBackedConstraint), + to.TypeFilter(rulesVersionKey, isNonIndexBackedConstraint, Not(isCrossDescriptorConstraint)), JoinOnDescID(from, to, "desc-id"), - StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_WRITE_ONLY), + StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_VALIDATED), } }, ) @@ -115,7 +115,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.TypeFilter(rulesVersionKey, isDescriptor), - to.TypeFilter(rulesVersionKey, isSimpleDependent), + to.TypeFilter(rulesVersionKey, isSimpleDependent, Not(isDescriptorParentReference)), JoinReferencedDescID(to, from, "desc-id"), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), } @@ -154,3 +154,93 @@ func init() { }, ) } + +// These rules ensure that descriptor, back-reference in parent descriptor, +// and parent descriptor are dropped in appropriate order. +func init() { + + // We don't like those parent-descriptor-back-reference elements: in hindsight, + // we shouldn't have them in the first place because we cannot modify + // back-references in parent descriptor in isolation with the SQL syntax. + // This rule is to deal with this fact by tightly coupling them to the descriptor. + registerDepRule( + "descriptor dropped right before removing back-reference in its parent descriptor", + scgraph.SameStagePrecedence, + "descriptor", "back-reference-in-parent-descriptor", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(rulesVersionKey, isDescriptor), + to.TypeFilter(rulesVersionKey, isDescriptorParentReference), + JoinOnDescID(from, to, "desc-id"), + StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), + } + }) + + registerDepRule( + "back-reference in parent descriptor is removed before parent descriptor is dropped", + scgraph.Precedence, + "back-reference-in-parent-descriptor", "parent-descriptor", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(rulesVersionKey, isDescriptorParentReference), + to.TypeFilter(rulesVersionKey, isDescriptor), + JoinReferencedDescID(from, to, "desc-id"), + StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_DROPPED), + } + }, + ) +} + +// These rules ensures we drop cross-descriptor constraints before dropping +// descriptors, both the referencing and referenced. Namely, +// 1. cross-descriptor constraints are absent before referenced descriptor +// 2. cross-descriptor constraints are absent before referencing descriptor +// +// A canonical example is FKs: +// To illustrate why rule 1 is necessary, consider we have tables `t1` and `t2`, +// and `t1` has a FK to `t2` (call this schema `S1`). The statement is +// `DROP TABLE t2 CASCADE`. We will have to first transition the FK (dropped as +// a result of CASCADE) to an intermediate state and then (in a separate +// transaction) transition the table to the dropped state. Otherwise, if the FK +// transition to absent in the same transaction as the table becomes dropped +// (call this schema `S2`), it becomes unsafe for `S1` and `S2` to exist in the +// cluster at the same time, because allowed inserts under `S2` will violate `S1`. +// +// To illustrate why rule 2 is necessary, consider we have tables `t1`, `t2`, `t3`, +// and `t1` FKs to `t2` (call it `FK1`) and `t3` FKs to `t2` (call it `FK2`). +// The statement is `DROP TABLE t1, t2 CASCADE`. Without rule 2, rule 1 alone will +// ensure that `FK2` moves to an intermediate state first, and at the same stage, +// `t1` will be dropped together with `FK1`. Validation will then fail because +// `t2` will have an enforced FK constraint whose origin table (`t1`) is dropped. +// It's worth noting that relaxing validation in this case is safe but we choose +// not to do so because it requires other related changes and makes reasoning +// harder. +func init() { + registerDepRule( + "cross-descriptor constraint is absent before referenced descriptor is dropped", + scgraph.Precedence, + "cross-desc-constraint", "referenced-descriptor", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(rulesVersionKey, isCrossDescriptorConstraint), + to.TypeFilter(rulesVersionKey, isDescriptor), + JoinReferencedDescID(from, to, "desc-id"), + StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_DROPPED), + } + }, + ) + + registerDepRule( + "cross-descriptor constraint is absent before referencing descriptor is dropped", + scgraph.Precedence, + "cross-desc-constraint", "referencing-descriptor", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(rulesVersionKey, isCrossDescriptorConstraint), + to.TypeFilter(rulesVersionKey, isDescriptor), + JoinOnDescID(from, to, "desc-id"), + StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_DROPPED), + } + }, + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go index 4ee14cce54fb..7ee84882c1e2 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go @@ -65,7 +65,7 @@ func isSubjectTo2VersionInvariant(e scpb.Element) bool { // TODO(ajwerner): This should include constraints and enum values but it // currently does not because we do not support dropping them unless we're // dropping the descriptor and we do not support adding them. - return isIndex(e) || isColumn(e) || isSupportedNonIndexBackedConstraint(e) + return isIndex(e) || isColumn(e) || isNonIndexBackedConstraint(e) } func isIndex(e scpb.Element) bool { @@ -184,11 +184,8 @@ func isIndexDependent(e scpb.Element) bool { return false } -// isSupportedNonIndexBackedConstraint a non-index-backed constraint is one of {Check, FK, UniqueWithoutIndex}. We only -// support Check for now. -// TODO (xiang): Expand this predicate to include other non-index-backed constraints -// when we properly support adding/dropping them in the new schema changer. -func isSupportedNonIndexBackedConstraint(e scpb.Element) bool { +// isNonIndexBackedConstraint a non-index-backed constraint. +func isNonIndexBackedConstraint(e scpb.Element) bool { switch e.(type) { case *scpb.CheckConstraint, *scpb.ForeignKeyConstraint, *scpb.UniqueWithoutIndexConstraint, *scpb.ColumnNotNull: @@ -207,6 +204,21 @@ func isConstraint(e scpb.Element) bool { return false } +// isCrossDescriptorConstraint are constraints that might reference +// other descriptors: +// - FKs can reference other table +// - Checks can reference other sequences/types +// +// Those constraints need to be dropped first when we are dropping either +// the referencing or reference descriptor. +func isCrossDescriptorConstraint(e scpb.Element) bool { + switch e.(type) { + case *scpb.ForeignKeyConstraint, *scpb.CheckConstraint: + return true + } + return false +} + func isConstraintDependent(e scpb.Element) bool { switch e.(type) { case *scpb.ConstraintWithoutIndexName: @@ -228,3 +240,11 @@ func isData(e scpb.Element) bool { } return false } + +func isDescriptorParentReference(e scpb.Element) bool { + switch e.(type) { + case *scpb.ObjectParent, *scpb.SchemaParent: + return true + } + return false +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/op_drop.go b/pkg/sql/schemachanger/scplan/internal/rules/current/op_drop.go index 72b39a8d59e5..e10be4805636 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/op_drop.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/op_drop.go @@ -168,7 +168,7 @@ func init() { (*scpb.Table)(nil), (*scpb.View)(nil), ), - constraint.TypeFilter(rulesVersionKey, isSupportedNonIndexBackedConstraint), + constraint.TypeFilter(rulesVersionKey, isNonIndexBackedConstraint), JoinOnDescID(relation, constraint, relationID), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules index 2d60b746102d..fedf496f4f86 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules @@ -1629,6 +1629,19 @@ deprules - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) +- name: back-reference in parent descriptor is removed before parent descriptor is dropped + from: back-reference-in-parent-descriptor-Node + kind: Precedence + to: parent-descriptor-Node + query: + - $back-reference-in-parent-descriptor[Type] IN ['*scpb.SchemaParent', '*scpb.ObjectParent'] + - $parent-descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType', '*scpb.Function'] + - joinReferencedDescID($back-reference-in-parent-descriptor, $parent-descriptor, $desc-id) + - toAbsent($back-reference-in-parent-descriptor-Target, $parent-descriptor-Target) + - $back-reference-in-parent-descriptor-Node[CurrentStatus] = ABSENT + - $parent-descriptor-Node[CurrentStatus] = DROPPED + - joinTargetNode($back-reference-in-parent-descriptor, $back-reference-in-parent-descriptor-Target, $back-reference-in-parent-descriptor-Node) + - joinTargetNode($parent-descriptor, $parent-descriptor-Target, $parent-descriptor-Node) - name: column constraint removed right before column reaches delete only from: column-constraint-Node kind: SameStagePrecedence @@ -1925,6 +1938,32 @@ deprules - $dependent-Node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($constraint, $constraint-Target, $constraint-Node) - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: cross-descriptor constraint is absent before referenced descriptor is dropped + from: cross-desc-constraint-Node + kind: Precedence + to: referenced-descriptor-Node + query: + - $cross-desc-constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $referenced-descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType', '*scpb.Function'] + - joinReferencedDescID($cross-desc-constraint, $referenced-descriptor, $desc-id) + - toAbsent($cross-desc-constraint-Target, $referenced-descriptor-Target) + - $cross-desc-constraint-Node[CurrentStatus] = ABSENT + - $referenced-descriptor-Node[CurrentStatus] = DROPPED + - joinTargetNode($cross-desc-constraint, $cross-desc-constraint-Target, $cross-desc-constraint-Node) + - joinTargetNode($referenced-descriptor, $referenced-descriptor-Target, $referenced-descriptor-Node) +- name: cross-descriptor constraint is absent before referencing descriptor is dropped + from: cross-desc-constraint-Node + kind: Precedence + to: referencing-descriptor-Node + query: + - $cross-desc-constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $referencing-descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType', '*scpb.Function'] + - joinOnDescID($cross-desc-constraint, $referencing-descriptor, $desc-id) + - toAbsent($cross-desc-constraint-Target, $referencing-descriptor-Target) + - $cross-desc-constraint-Node[CurrentStatus] = ABSENT + - $referencing-descriptor-Node[CurrentStatus] = DROPPED + - joinTargetNode($cross-desc-constraint, $cross-desc-constraint-Target, $cross-desc-constraint-Node) + - joinTargetNode($referencing-descriptor, $referencing-descriptor-Target, $referencing-descriptor-Node) - name: dependent view absent before secondary index from: view-Node kind: Precedence @@ -2214,7 +2253,7 @@ deprules to: referencing-via-attr-Node query: - $referenced-descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType', '*scpb.Function'] - - $referencing-via-attr[Type] IN ['*scpb.ColumnFamily', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName', '*scpb.FunctionName', '*scpb.FunctionVolatility', '*scpb.FunctionLeakProof', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionBody', '*scpb.FunctionParamDefaultExpression'] + - $referencing-via-attr[Type] IN ['*scpb.ColumnFamily', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaComment', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName', '*scpb.FunctionName', '*scpb.FunctionVolatility', '*scpb.FunctionLeakProof', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionBody', '*scpb.FunctionParamDefaultExpression'] - joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id) - toAbsent($referenced-descriptor-Target, $referencing-via-attr-Target) - $referenced-descriptor-Node[CurrentStatus] = DROPPED @@ -2255,7 +2294,7 @@ deprules to: dependent-Node query: - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType', '*scpb.Function'] - - $dependent[Type] IN ['*scpb.ColumnFamily', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName', '*scpb.FunctionName', '*scpb.FunctionVolatility', '*scpb.FunctionLeakProof', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionBody', '*scpb.FunctionParamDefaultExpression'] + - $dependent[Type] IN ['*scpb.ColumnFamily', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName', '*scpb.FunctionName', '*scpb.FunctionVolatility', '*scpb.FunctionLeakProof', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionBody', '*scpb.FunctionParamDefaultExpression'] - joinOnDescID($descriptor, $dependent, $desc-id) - toAbsent($descriptor-Target, $dependent-Target) - $descriptor-Node[CurrentStatus] = DROPPED @@ -2275,6 +2314,19 @@ deprules - $absent-Node[CurrentStatus] = ABSENT - joinTargetNode($dropped, $dropped-Target, $dropped-Node) - joinTargetNode($absent, $absent-Target, $absent-Node) +- name: descriptor dropped right before removing back-reference in its parent descriptor + from: descriptor-Node + kind: SameStagePrecedence + to: back-reference-in-parent-descriptor-Node + query: + - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType', '*scpb.Function'] + - $back-reference-in-parent-descriptor[Type] IN ['*scpb.SchemaParent', '*scpb.ObjectParent'] + - joinOnDescID($descriptor, $back-reference-in-parent-descriptor, $desc-id) + - toAbsent($descriptor-Target, $back-reference-in-parent-descriptor-Target) + - $descriptor-Node[CurrentStatus] = DROPPED + - $back-reference-in-parent-descriptor-Node[CurrentStatus] = ABSENT + - joinTargetNode($descriptor, $descriptor-Target, $descriptor-Node) + - joinTargetNode($back-reference-in-parent-descriptor, $back-reference-in-parent-descriptor-Target, $back-reference-in-parent-descriptor-Node) - name: descriptor existence precedes dependents from: relation-Node kind: Precedence @@ -2822,11 +2874,11 @@ deprules to: constraint-Node query: - $descriptor[Type] = '*scpb.Table' - - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.ColumnNotNull'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.ColumnNotNull'] - joinOnDescID($descriptor, $constraint, $desc-id) - toAbsent($descriptor-Target, $constraint-Target) - $descriptor-Node[CurrentStatus] = DROPPED - - $constraint-Node[CurrentStatus] = WRITE_ONLY + - $constraint-Node[CurrentStatus] = VALIDATED - joinTargetNode($descriptor, $descriptor-Target, $descriptor-Node) - joinTargetNode($constraint, $constraint-Target, $constraint-Node) - name: relation dropped before dependent index diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column b/pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column index 13c7d869ec99..e09a5e56bfcb 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column @@ -981,8 +981,8 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 108}, DROPPED] to: [Owner:{DescID: 108}, ABSENT] kind: Precedence @@ -2040,8 +2040,8 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 108}, DROPPED] to: [Owner:{DescID: 108}, ABSENT] kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_database b/pkg/sql/schemachanger/scplan/testdata/drop_database index c95ee45643b5..67061eb6dac2 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_database +++ b/pkg/sql/schemachanger/scplan/testdata/drop_database @@ -220,24 +220,6 @@ StatementPhase stage 1 of 1 with 54 MutationType ops [[ColumnName:{DescID: 117, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsDropped - DescriptorID: 104 - *scop.RemoveDatabaseComment - DatabaseID: 104 - *scop.MarkDescriptorAsDropped - DescriptorID: 105 - *scop.RemoveSchemaParent - Parent: - ParentDatabaseID: 104 - SchemaID: 105 - *scop.MarkDescriptorAsDropped - DescriptorID: 106 - *scop.RemoveSchemaParent - Parent: - ParentDatabaseID: 104 - SchemaID: 106 - *scop.RemoveSchemaComment - SchemaID: 106 *scop.MarkDescriptorAsDropped DescriptorID: 107 *scop.RemoveObjectParent @@ -337,10 +319,24 @@ StatementPhase stage 1 of 1 with 54 MutationType ops *scop.RemoveObjectParent ObjectID: 117 ParentSchemaID: 106 - *scop.DrainDescriptorName - Namespace: - DescriptorID: 104 - Name: db1 + *scop.MarkDescriptorAsDropped + DescriptorID: 105 + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 104 + SchemaID: 105 + *scop.MarkDescriptorAsDropped + DescriptorID: 106 + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 104 + SchemaID: 106 + *scop.RemoveSchemaComment + SchemaID: 106 + *scop.MarkDescriptorAsDropped + DescriptorID: 104 + *scop.RemoveDatabaseComment + DatabaseID: 104 *scop.DrainDescriptorName Namespace: DatabaseID: 104 @@ -417,6 +413,10 @@ StatementPhase stage 1 of 1 with 54 MutationType ops DescriptorID: 117 Name: v5 SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DescriptorID: 104 + Name: db1 PreCommitPhase stage 1 of 2 with 1 MutationType op transitions: [[Namespace:{DescID: 104, Name: db1, ReferencedDescID: 0}, ABSENT], ABSENT] -> PUBLIC @@ -821,26 +821,6 @@ PreCommitPhase stage 2 of 2 with 102 MutationType ops [[ColumnName:{DescID: 117, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsDropped - DescriptorID: 104 - *scop.RemoveDatabaseRoleSettings - DatabaseID: 104 - *scop.RemoveDatabaseComment - DatabaseID: 104 - *scop.MarkDescriptorAsDropped - DescriptorID: 105 - *scop.RemoveSchemaParent - Parent: - ParentDatabaseID: 104 - SchemaID: 105 - *scop.MarkDescriptorAsDropped - DescriptorID: 106 - *scop.RemoveSchemaParent - Parent: - ParentDatabaseID: 104 - SchemaID: 106 - *scop.RemoveSchemaComment - SchemaID: 106 *scop.MarkDescriptorAsDropped DescriptorID: 107 *scop.RemoveObjectParent @@ -940,10 +920,32 @@ PreCommitPhase stage 2 of 2 with 102 MutationType ops *scop.RemoveObjectParent ObjectID: 117 ParentSchemaID: 106 - *scop.DrainDescriptorName - Namespace: - DescriptorID: 104 - Name: db1 + *scop.MarkDescriptorAsDropped + DescriptorID: 105 + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 104 + SchemaID: 105 + *scop.MarkDescriptorAsDropped + DescriptorID: 106 + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 104 + SchemaID: 106 + *scop.RemoveSchemaComment + SchemaID: 106 + *scop.RemoveColumnNotNull + ColumnID: 1 + TableID: 110 + *scop.RemoveColumnNotNull + ColumnID: 1 + TableID: 109 + *scop.MarkDescriptorAsDropped + DescriptorID: 104 + *scop.RemoveDatabaseRoleSettings + DatabaseID: 104 + *scop.RemoveDatabaseComment + DatabaseID: 104 *scop.DrainDescriptorName Namespace: DatabaseID: 104 @@ -966,8 +968,17 @@ PreCommitPhase stage 2 of 2 with 102 MutationType ops DescriptorID: 110 Name: t1 SchemaID: 105 - *scop.RemoveColumnNotNull - ColumnID: 1 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 2 + TableID: 110 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 3 + TableID: 110 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967295 + TableID: 110 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967294 TableID: 110 *scop.DrainDescriptorName Namespace: @@ -981,63 +992,6 @@ PreCommitPhase stage 2 of 2 with 102 MutationType ops DescriptorID: 109 Name: t1 SchemaID: 106 - *scop.RemoveColumnNotNull - ColumnID: 1 - TableID: 109 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 111 - Name: v1 - SchemaID: 106 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 112 - Name: v2 - SchemaID: 106 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 113 - Name: v3 - SchemaID: 106 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 114 - Name: v4 - SchemaID: 106 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 115 - Name: typ - SchemaID: 106 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 116 - Name: _typ - SchemaID: 106 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 117 - Name: v5 - SchemaID: 106 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 2 - TableID: 110 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 3 - TableID: 110 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967295 - TableID: 110 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967294 - TableID: 110 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 2 TableID: 109 @@ -1050,6 +1004,12 @@ PreCommitPhase stage 2 of 2 with 102 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 109 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 111 + Name: v1 + SchemaID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 111 @@ -1059,6 +1019,12 @@ PreCommitPhase stage 2 of 2 with 102 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 111 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 112 + Name: v2 + SchemaID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 112 @@ -1071,6 +1037,12 @@ PreCommitPhase stage 2 of 2 with 102 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 112 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 113 + Name: v3 + SchemaID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 113 @@ -1083,6 +1055,12 @@ PreCommitPhase stage 2 of 2 with 102 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 113 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 114 + Name: v4 + SchemaID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 114 @@ -1095,6 +1073,24 @@ PreCommitPhase stage 2 of 2 with 102 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 114 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 115 + Name: typ + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 116 + Name: _typ + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 117 + Name: v5 + SchemaID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 117 @@ -1110,6 +1106,10 @@ PreCommitPhase stage 2 of 2 with 102 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 117 + *scop.DrainDescriptorName + Namespace: + DescriptorID: 104 + Name: db1 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 110 @@ -1336,8 +1336,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [ObjectParent:{DescID: 116, ReferencedDescID: 106}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [Owner:{DescID: 116}, ABSENT] kind: Precedence @@ -1982,14 +1982,6 @@ DROP DATABASE db1 CASCADE to: [Owner:{DescID: 104}, ABSENT] kind: Precedence rule: descriptor dropped before dependent element removal -- from: [Database:{DescID: 104}, DROPPED] - to: [SchemaParent:{DescID: 105, ReferencedDescID: 104}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Database:{DescID: 104}, DROPPED] - to: [SchemaParent:{DescID: 106, ReferencedDescID: 104}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref - from: [Database:{DescID: 104}, DROPPED] to: [UserPrivileges:{DescID: 104, Name: admin}, ABSENT] kind: Precedence @@ -2016,8 +2008,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 115}, DROPPED] to: [ObjectParent:{DescID: 115, ReferencedDescID: 106}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [EnumType:{DescID: 115}, DROPPED] to: [Owner:{DescID: 115}, ABSENT] kind: Precedence @@ -2090,6 +2082,50 @@ DROP DATABASE db1 CASCADE to: [PrimaryIndex:{DescID: 110, IndexID: 1, ConstraintID: 1}, ABSENT] kind: Precedence rule: dependents removed before index +- from: [ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT] + to: [Schema:{DescID: 105}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 108, ReferencedDescID: 106}, ABSENT] + to: [Schema:{DescID: 106}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 109, ReferencedDescID: 106}, ABSENT] + to: [Schema:{DescID: 106}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 110, ReferencedDescID: 105}, ABSENT] + to: [Schema:{DescID: 105}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 111, ReferencedDescID: 106}, ABSENT] + to: [Schema:{DescID: 106}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 112, ReferencedDescID: 106}, ABSENT] + to: [Schema:{DescID: 106}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 113, ReferencedDescID: 106}, ABSENT] + to: [Schema:{DescID: 106}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 114, ReferencedDescID: 106}, ABSENT] + to: [Schema:{DescID: 106}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 115, ReferencedDescID: 106}, ABSENT] + to: [Schema:{DescID: 106}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 116, ReferencedDescID: 106}, ABSENT] + to: [Schema:{DescID: 106}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 117, ReferencedDescID: 106}, ABSENT] + to: [Schema:{DescID: 106}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped - from: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT] to: [IndexData:{DescID: 109, IndexID: 1}, DROPPED] kind: Precedence @@ -2134,14 +2170,6 @@ DROP DATABASE db1 CASCADE to: [Namespace:{DescID: 105, Name: public, ReferencedDescID: 104}, ABSENT] kind: Precedence rule: descriptor dropped before dependent element removal -- from: [Schema:{DescID: 105}, DROPPED] - to: [ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 105}, DROPPED] - to: [ObjectParent:{DescID: 110, ReferencedDescID: 105}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref - from: [Schema:{DescID: 105}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: Precedence @@ -2152,8 +2180,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped in transaction before removal - from: [Schema:{DescID: 105}, DROPPED] to: [SchemaParent:{DescID: 105, ReferencedDescID: 104}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [Schema:{DescID: 105}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: admin}, ABSENT] kind: Precedence @@ -2170,42 +2198,6 @@ DROP DATABASE db1 CASCADE to: [Namespace:{DescID: 106, Name: sc1, ReferencedDescID: 104}, ABSENT] kind: Precedence rule: descriptor dropped before dependent element removal -- from: [Schema:{DescID: 106}, DROPPED] - to: [ObjectParent:{DescID: 108, ReferencedDescID: 106}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 106}, DROPPED] - to: [ObjectParent:{DescID: 109, ReferencedDescID: 106}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 106}, DROPPED] - to: [ObjectParent:{DescID: 111, ReferencedDescID: 106}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 106}, DROPPED] - to: [ObjectParent:{DescID: 112, ReferencedDescID: 106}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 106}, DROPPED] - to: [ObjectParent:{DescID: 113, ReferencedDescID: 106}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 106}, DROPPED] - to: [ObjectParent:{DescID: 114, ReferencedDescID: 106}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 106}, DROPPED] - to: [ObjectParent:{DescID: 115, ReferencedDescID: 106}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 106}, DROPPED] - to: [ObjectParent:{DescID: 116, ReferencedDescID: 106}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 106}, DROPPED] - to: [ObjectParent:{DescID: 117, ReferencedDescID: 106}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref - from: [Schema:{DescID: 106}, DROPPED] to: [Owner:{DescID: 106}, ABSENT] kind: Precedence @@ -2220,8 +2212,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 106}, DROPPED] to: [SchemaParent:{DescID: 106, ReferencedDescID: 104}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [Schema:{DescID: 106}, DROPPED] to: [UserPrivileges:{DescID: 106, Name: admin}, ABSENT] kind: Precedence @@ -2230,6 +2222,14 @@ DROP DATABASE db1 CASCADE to: [UserPrivileges:{DescID: 106, Name: root}, ABSENT] kind: Precedence rule: descriptor dropped before dependent element removal +- from: [SchemaParent:{DescID: 105, ReferencedDescID: 104}, ABSENT] + to: [Database:{DescID: 104}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [SchemaParent:{DescID: 106, ReferencedDescID: 104}, ABSENT] + to: [Database:{DescID: 104}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped - from: [Sequence:{DescID: 107}, ABSENT] to: [TableData:{DescID: 107, ReferencedDescID: 104}, DROPPED] kind: SameStagePrecedence @@ -2244,8 +2244,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 107}, DROPPED] to: [ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [Sequence:{DescID: 107}, DROPPED] to: [Owner:{DescID: 107}, ABSENT] kind: Precedence @@ -2276,8 +2276,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 108}, DROPPED] to: [ObjectParent:{DescID: 108, ReferencedDescID: 106}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [Sequence:{DescID: 108}, DROPPED] to: [Owner:{DescID: 108}, ABSENT] kind: Precedence @@ -2346,6 +2346,10 @@ DROP DATABASE db1 CASCADE to: [ColumnName:{DescID: 109, Name: val, ColumnID: 3}, ABSENT] kind: Precedence rule: descriptor dropped before dependent element removal +- from: [Table:{DescID: 109}, DROPPED] + to: [ColumnNotNull:{DescID: 109, ColumnID: 1, IndexID: 0}, VALIDATED] + kind: Precedence + rule: relation dropped before dependent constraint - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence @@ -2388,8 +2392,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ObjectParent:{DescID: 109, ReferencedDescID: 106}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [Table:{DescID: 109}, DROPPED] to: [Owner:{DescID: 109}, ABSENT] kind: Precedence @@ -2466,6 +2470,10 @@ DROP DATABASE db1 CASCADE to: [ColumnName:{DescID: 110, Name: val, ColumnID: 3}, ABSENT] kind: Precedence rule: descriptor dropped before dependent element removal +- from: [Table:{DescID: 110}, DROPPED] + to: [ColumnNotNull:{DescID: 110, ColumnID: 1, IndexID: 0}, VALIDATED] + kind: Precedence + rule: relation dropped before dependent constraint - from: [Table:{DescID: 110}, DROPPED] to: [ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence @@ -2508,8 +2516,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ObjectParent:{DescID: 110, ReferencedDescID: 105}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [Table:{DescID: 110}, DROPPED] to: [Owner:{DescID: 110}, ABSENT] kind: Precedence @@ -2580,8 +2588,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ObjectParent:{DescID: 111, ReferencedDescID: 106}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 111}, DROPPED] to: [Owner:{DescID: 111}, ABSENT] kind: Precedence @@ -2652,8 +2660,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [ObjectParent:{DescID: 112, ReferencedDescID: 106}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 112}, DROPPED] to: [Owner:{DescID: 112}, ABSENT] kind: Precedence @@ -2724,8 +2732,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ObjectParent:{DescID: 113, ReferencedDescID: 106}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 113}, DROPPED] to: [Owner:{DescID: 113}, ABSENT] kind: Precedence @@ -2796,8 +2804,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [ObjectParent:{DescID: 114, ReferencedDescID: 106}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 114}, DROPPED] to: [Owner:{DescID: 114}, ABSENT] kind: Precedence @@ -2880,8 +2888,8 @@ DROP DATABASE db1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [ObjectParent:{DescID: 117, ReferencedDescID: 106}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 117}, DROPPED] to: [Owner:{DescID: 117}, ABSENT] kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_index b/pkg/sql/schemachanger/scplan/testdata/drop_index index 1612d12c64ee..73751a11a977 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_index +++ b/pkg/sql/schemachanger/scplan/testdata/drop_index @@ -971,8 +971,8 @@ DROP INDEX idx4 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 105}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_owned_by b/pkg/sql/schemachanger/scplan/testdata/drop_owned_by index 27265a87ed25..54223243e829 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_owned_by +++ b/pkg/sql/schemachanger/scplan/testdata/drop_owned_by @@ -149,12 +149,6 @@ StatementPhase stage 1 of 1 with 36 MutationType ops *scop.RemoveUserPrivileges DescriptorID: 100 User: r - *scop.MarkDescriptorAsDropped - DescriptorID: 105 - *scop.RemoveSchemaParent - Parent: - ParentDatabaseID: 100 - SchemaID: 105 *scop.RemoveUserPrivileges DescriptorID: 104 User: r @@ -227,11 +221,12 @@ StatementPhase stage 1 of 1 with 36 MutationType ops *scop.RemoveObjectParent ObjectID: 113 ParentSchemaID: 105 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 105 - Name: s + *scop.MarkDescriptorAsDropped + DescriptorID: 105 + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 100 + SchemaID: 105 *scop.DrainDescriptorName Namespace: DatabaseID: 100 @@ -280,6 +275,11 @@ StatementPhase stage 1 of 1 with 36 MutationType ops DescriptorID: 113 Name: v2 SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 105 + Name: s PreCommitPhase stage 1 of 2 with 1 MutationType op transitions: [[UserPrivileges:{DescID: 100, Name: r}, ABSENT], ABSENT] -> PUBLIC @@ -543,12 +543,6 @@ PreCommitPhase stage 2 of 2 with 67 MutationType ops *scop.RemoveUserPrivileges DescriptorID: 100 User: r - *scop.MarkDescriptorAsDropped - DescriptorID: 105 - *scop.RemoveSchemaParent - Parent: - ParentDatabaseID: 100 - SchemaID: 105 *scop.RemoveUserPrivileges DescriptorID: 104 User: r @@ -621,11 +615,12 @@ PreCommitPhase stage 2 of 2 with 67 MutationType ops *scop.RemoveObjectParent ObjectID: 113 ParentSchemaID: 105 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 105 - Name: s + *scop.MarkDescriptorAsDropped + DescriptorID: 105 + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 100 + SchemaID: 105 *scop.DrainDescriptorName Namespace: DatabaseID: 100 @@ -680,6 +675,11 @@ PreCommitPhase stage 2 of 2 with 67 MutationType ops DescriptorID: 113 Name: v2 SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 105 + Name: s *scop.MakeDeleteOnlyColumnAbsent ColumnID: 2 TableID: 109 diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_schema b/pkg/sql/schemachanger/scplan/testdata/drop_schema index 0a62c6c1d99a..7ee9e302f8f3 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_schema +++ b/pkg/sql/schemachanger/scplan/testdata/drop_schema @@ -25,8 +25,8 @@ DROP SCHEMA defaultdb.SC1 CASCADE rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [ObjectParent:{DescID: 112, ReferencedDescID: 104}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [Owner:{DescID: 112}, ABSENT] kind: Precedence @@ -493,8 +493,8 @@ DROP SCHEMA defaultdb.SC1 CASCADE rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 111}, DROPPED] to: [ObjectParent:{DescID: 111, ReferencedDescID: 104}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [EnumType:{DescID: 111}, DROPPED] to: [Owner:{DescID: 111}, ABSENT] kind: Precedence @@ -539,6 +539,42 @@ DROP SCHEMA defaultdb.SC1 CASCADE to: [PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1}, ABSENT] kind: Precedence rule: dependents removed before index +- from: [ObjectParent:{DescID: 105, ReferencedDescID: 104}, ABSENT] + to: [Schema:{DescID: 104}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 106, ReferencedDescID: 104}, ABSENT] + to: [Schema:{DescID: 104}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 107, ReferencedDescID: 104}, ABSENT] + to: [Schema:{DescID: 104}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 108, ReferencedDescID: 104}, ABSENT] + to: [Schema:{DescID: 104}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 109, ReferencedDescID: 104}, ABSENT] + to: [Schema:{DescID: 104}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 110, ReferencedDescID: 104}, ABSENT] + to: [Schema:{DescID: 104}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 111, ReferencedDescID: 104}, ABSENT] + to: [Schema:{DescID: 104}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 112, ReferencedDescID: 104}, ABSENT] + to: [Schema:{DescID: 104}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped +- from: [ObjectParent:{DescID: 113, ReferencedDescID: 104}, ABSENT] + to: [Schema:{DescID: 104}, DROPPED] + kind: Precedence + rule: back-reference in parent descriptor is removed before parent descriptor is dropped - from: [PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1}, ABSENT] to: [IndexData:{DescID: 106, IndexID: 1}, DROPPED] kind: Precedence @@ -563,42 +599,6 @@ DROP SCHEMA defaultdb.SC1 CASCADE to: [Namespace:{DescID: 104, Name: sc1, ReferencedDescID: 100}, ABSENT] kind: Precedence rule: descriptor dropped before dependent element removal -- from: [Schema:{DescID: 104}, DROPPED] - to: [ObjectParent:{DescID: 105, ReferencedDescID: 104}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 104}, DROPPED] - to: [ObjectParent:{DescID: 106, ReferencedDescID: 104}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 104}, DROPPED] - to: [ObjectParent:{DescID: 107, ReferencedDescID: 104}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 104}, DROPPED] - to: [ObjectParent:{DescID: 108, ReferencedDescID: 104}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 104}, DROPPED] - to: [ObjectParent:{DescID: 109, ReferencedDescID: 104}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 104}, DROPPED] - to: [ObjectParent:{DescID: 110, ReferencedDescID: 104}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 104}, DROPPED] - to: [ObjectParent:{DescID: 111, ReferencedDescID: 104}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 104}, DROPPED] - to: [ObjectParent:{DescID: 112, ReferencedDescID: 104}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref -- from: [Schema:{DescID: 104}, DROPPED] - to: [ObjectParent:{DescID: 113, ReferencedDescID: 104}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with attr ref - from: [Schema:{DescID: 104}, DROPPED] to: [Owner:{DescID: 104}, ABSENT] kind: Precedence @@ -613,8 +613,8 @@ DROP SCHEMA defaultdb.SC1 CASCADE rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 104}, DROPPED] to: [SchemaParent:{DescID: 104, ReferencedDescID: 100}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [Schema:{DescID: 104}, DROPPED] to: [UserPrivileges:{DescID: 104, Name: admin}, ABSENT] kind: Precedence @@ -637,8 +637,8 @@ DROP SCHEMA defaultdb.SC1 CASCADE rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 105}, DROPPED] to: [ObjectParent:{DescID: 105, ReferencedDescID: 104}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [Sequence:{DescID: 105}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: Precedence @@ -707,6 +707,10 @@ DROP SCHEMA defaultdb.SC1 CASCADE to: [ColumnName:{DescID: 106, Name: val, ColumnID: 3}, ABSENT] kind: Precedence rule: descriptor dropped before dependent element removal +- from: [Table:{DescID: 106}, DROPPED] + to: [ColumnNotNull:{DescID: 106, ColumnID: 1, IndexID: 0}, VALIDATED] + kind: Precedence + rule: relation dropped before dependent constraint - from: [Table:{DescID: 106}, DROPPED] to: [ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence @@ -749,8 +753,8 @@ DROP SCHEMA defaultdb.SC1 CASCADE rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ObjectParent:{DescID: 106, ReferencedDescID: 104}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [Table:{DescID: 106}, DROPPED] to: [Owner:{DescID: 106}, ABSENT] kind: Precedence @@ -821,8 +825,8 @@ DROP SCHEMA defaultdb.SC1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ObjectParent:{DescID: 107, ReferencedDescID: 104}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 107}, DROPPED] to: [Owner:{DescID: 107}, ABSENT] kind: Precedence @@ -893,8 +897,8 @@ DROP SCHEMA defaultdb.SC1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ObjectParent:{DescID: 108, ReferencedDescID: 104}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 108}, DROPPED] to: [Owner:{DescID: 108}, ABSENT] kind: Precedence @@ -965,8 +969,8 @@ DROP SCHEMA defaultdb.SC1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [ObjectParent:{DescID: 109, ReferencedDescID: 104}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 109}, DROPPED] to: [Owner:{DescID: 109}, ABSENT] kind: Precedence @@ -1037,8 +1041,8 @@ DROP SCHEMA defaultdb.SC1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [ObjectParent:{DescID: 110, ReferencedDescID: 104}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 110}, DROPPED] to: [Owner:{DescID: 110}, ABSENT] kind: Precedence @@ -1121,8 +1125,8 @@ DROP SCHEMA defaultdb.SC1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ObjectParent:{DescID: 113, ReferencedDescID: 104}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 113}, DROPPED] to: [Owner:{DescID: 113}, ABSENT] kind: Precedence @@ -1294,14 +1298,6 @@ StatementPhase stage 1 of 1 with 40 MutationType ops [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsDropped - DescriptorID: 104 - *scop.RemoveSchemaParent - Parent: - ParentDatabaseID: 100 - SchemaID: 104 - *scop.RemoveSchemaComment - SchemaID: 104 *scop.MarkDescriptorAsDropped DescriptorID: 105 *scop.RemoveObjectParent @@ -1383,11 +1379,14 @@ StatementPhase stage 1 of 1 with 40 MutationType ops *scop.RemoveObjectParent ObjectID: 113 ParentSchemaID: 104 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 104 - Name: sc1 + *scop.MarkDescriptorAsDropped + DescriptorID: 104 + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 100 + SchemaID: 104 + *scop.RemoveSchemaComment + SchemaID: 104 *scop.DrainDescriptorName Namespace: DatabaseID: 100 @@ -1442,6 +1441,11 @@ StatementPhase stage 1 of 1 with 40 MutationType ops DescriptorID: 113 Name: v5 SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 104 + Name: sc1 PreCommitPhase stage 1 of 2 with 1 MutationType op transitions: [[Namespace:{DescID: 104, Name: sc1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC @@ -1746,14 +1750,6 @@ PreCommitPhase stage 2 of 2 with 78 MutationType ops [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsDropped - DescriptorID: 104 - *scop.RemoveSchemaParent - Parent: - ParentDatabaseID: 100 - SchemaID: 104 - *scop.RemoveSchemaComment - SchemaID: 104 *scop.MarkDescriptorAsDropped DescriptorID: 105 *scop.RemoveObjectParent @@ -1835,11 +1831,14 @@ PreCommitPhase stage 2 of 2 with 78 MutationType ops *scop.RemoveObjectParent ObjectID: 113 ParentSchemaID: 104 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 104 - Name: sc1 + *scop.MarkDescriptorAsDropped + DescriptorID: 104 + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 100 + SchemaID: 104 + *scop.RemoveSchemaComment + SchemaID: 104 *scop.DrainDescriptorName Namespace: DatabaseID: 100 @@ -1897,6 +1896,11 @@ PreCommitPhase stage 2 of 2 with 78 MutationType ops DescriptorID: 113 Name: v5 SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 104 + Name: sc1 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 2 TableID: 106 diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_sequence b/pkg/sql/schemachanger/scplan/testdata/drop_sequence index cd38dd36180d..2c3d5c59d392 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_sequence +++ b/pkg/sql/schemachanger/scplan/testdata/drop_sequence @@ -259,8 +259,8 @@ DROP SEQUENCE defaultdb.SQ1 CASCADE rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 104}, DROPPED] to: [ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [Sequence:{DescID: 104}, DROPPED] to: [Owner:{DescID: 104}, ABSENT] kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_table b/pkg/sql/schemachanger/scplan/testdata/drop_table index 5ac833a2f2b2..775bede87414 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_table +++ b/pkg/sql/schemachanger/scplan/testdata/drop_table @@ -29,66 +29,13 @@ COMMENT ON CONSTRAINT fk_customers ON defaultdb.shipments IS 'customer is not go ops DROP TABLE defaultdb.shipments CASCADE; ---- -StatementPhase stage 1 of 1 with 22 MutationType ops +StatementPhase stage 1 of 1 with 5 MutationType ops transitions: - [[Namespace:{DescID: 109, Name: shipments, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT - [[Owner:{DescID: 109}, ABSENT], PUBLIC] -> ABSENT - [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], PUBLIC] -> ABSENT - [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Table:{DescID: 109}, ABSENT], PUBLIC] -> DROPPED - [[ObjectParent:{DescID: 109, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[TableComment:{DescID: 109, Comment: shipment is important}, ABSENT], PUBLIC] -> ABSENT - [[ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 109, Name: tracking_number, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[ColumnNotNull:{DescID: 109, ColumnID: 1, IndexID: 0}, ABSENT], PUBLIC] -> VALIDATED - [[ColumnDefaultExpression:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[ColumnComment:{DescID: 109, ColumnID: 1, Comment: tracking_number is a must}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 109, Name: carrier, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[SequenceOwner:{DescID: 109, ColumnID: 2, ReferencedDescID: 110}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 109, Name: status, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 4}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 109, Name: customer_id, ColumnID: 4}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 5}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 109, Name: randcol, ColumnID: 5}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 5}, ABSENT], PUBLIC] -> ABSENT - [[ColumnDefaultExpression:{DescID: 109, ColumnID: 5, ReferencedSequenceIDs: [106]}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 109, ColumnID: 5, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED - [[IndexName:{DescID: 109, Name: shipments_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[IndexComment:{DescID: 109, IndexID: 1, Comment: pkey is good}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[SecondaryIndexPartial:{DescID: 109, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, ABSENT], PUBLIC] -> VALIDATED [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT [[ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT], PUBLIC] -> ABSENT [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, ABSENT], PUBLIC] -> VALIDATED [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT], PUBLIC] -> ABSENT - [[Namespace:{DescID: 110, Name: sq1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT - [[Owner:{DescID: 110}, ABSENT], PUBLIC] -> ABSENT - [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], PUBLIC] -> ABSENT - [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Sequence:{DescID: 110}, ABSENT], PUBLIC] -> DROPPED - [[ObjectParent:{DescID: 110, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 111, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 111}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] -> ABSENT @@ -108,47 +55,9 @@ StatementPhase stage 1 of 1 with 22 MutationType ops [[ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsDropped - DescriptorID: 109 - *scop.RemoveObjectParent - ObjectID: 109 - ParentSchemaID: 101 - *scop.RemoveTableComment - TableID: 109 - *scop.RemoveColumnDefaultExpression - ColumnID: 1 - TableID: 109 - *scop.RemoveColumnComment - ColumnID: 1 - PgAttributeNum: 1 - TableID: 109 - *scop.RemoveDroppedColumnType - ColumnID: 3 - TableID: 109 - *scop.UpdateTableBackReferencesInTypes - BackReferencedTableID: 109 - TypeIDs: - - 107 - - 108 - *scop.RemoveColumnDefaultExpression - ColumnID: 5 - TableID: 109 - *scop.UpdateTableBackReferencesInSequences - BackReferencedColumnID: 5 - BackReferencedTableID: 109 - SequenceIDs: - - 106 - *scop.RemoveIndexComment - IndexID: 1 - TableID: 109 *scop.RemoveConstraintComment ConstraintID: 2 TableID: 109 - *scop.MarkDescriptorAsDropped - DescriptorID: 110 - *scop.RemoveObjectParent - ObjectID: 110 - ParentSchemaID: 101 *scop.MarkDescriptorAsDropped DescriptorID: 111 *scop.RemoveBackReferencesInRelations @@ -158,27 +67,6 @@ StatementPhase stage 1 of 1 with 22 MutationType ops *scop.RemoveObjectParent ObjectID: 111 ParentSchemaID: 101 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 109 - Name: shipments - SchemaID: 101 - *scop.RemoveSequenceOwner - ColumnID: 2 - OwnedSequenceID: 110 - TableID: 109 - *scop.RemoveOwnerBackReferenceInSequence - SequenceID: 110 - *scop.RemoveDroppedIndexPartialPredicate - IndexID: 2 - TableID: 109 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 110 - Name: sq1 - SchemaID: 101 *scop.DrainDescriptorName Namespace: DatabaseID: 100 @@ -187,64 +75,11 @@ StatementPhase stage 1 of 1 with 22 MutationType ops SchemaID: 101 PreCommitPhase stage 1 of 2 with 1 MutationType op transitions: - [[Namespace:{DescID: 109, Name: shipments, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC - [[Owner:{DescID: 109}, ABSENT], ABSENT] -> PUBLIC - [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], ABSENT] -> PUBLIC - [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], ABSENT] -> PUBLIC - [[Table:{DescID: 109}, ABSENT], DROPPED] -> PUBLIC - [[ObjectParent:{DescID: 109, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC - [[TableComment:{DescID: 109, Comment: shipment is important}, ABSENT], ABSENT] -> PUBLIC - [[ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT], ABSENT] -> PUBLIC - [[Column:{DescID: 109, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC - [[ColumnName:{DescID: 109, Name: tracking_number, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC - [[ColumnNotNull:{DescID: 109, ColumnID: 1, IndexID: 0}, ABSENT], VALIDATED] -> PUBLIC - [[ColumnDefaultExpression:{DescID: 109, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC - [[ColumnComment:{DescID: 109, ColumnID: 1, Comment: tracking_number is a must}, ABSENT], ABSENT] -> PUBLIC - [[Column:{DescID: 109, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC - [[ColumnName:{DescID: 109, Name: carrier, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC - [[SequenceOwner:{DescID: 109, ColumnID: 2, ReferencedDescID: 110}, ABSENT], ABSENT] -> PUBLIC - [[Column:{DescID: 109, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC - [[ColumnName:{DescID: 109, Name: status, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC - [[ColumnType:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC - [[Column:{DescID: 109, ColumnID: 4}, ABSENT], WRITE_ONLY] -> PUBLIC - [[ColumnName:{DescID: 109, Name: customer_id, ColumnID: 4}, ABSENT], ABSENT] -> PUBLIC - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4}, ABSENT], ABSENT] -> PUBLIC - [[Column:{DescID: 109, ColumnID: 5}, ABSENT], WRITE_ONLY] -> PUBLIC - [[ColumnName:{DescID: 109, Name: randcol, ColumnID: 5}, ABSENT], ABSENT] -> PUBLIC - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 5}, ABSENT], ABSENT] -> PUBLIC - [[ColumnDefaultExpression:{DescID: 109, ColumnID: 5, ReferencedSequenceIDs: [106]}, ABSENT], ABSENT] -> PUBLIC - [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC - [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC - [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC - [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 109, ColumnID: 5, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC - [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> PUBLIC - [[IndexName:{DescID: 109, Name: shipments_pkey, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC - [[IndexComment:{DescID: 109, IndexID: 1, Comment: pkey is good}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC - [[SecondaryIndexPartial:{DescID: 109, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC - [[SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, ABSENT], VALIDATED] -> PUBLIC [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, ABSENT], VALIDATED] -> PUBLIC [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT], ABSENT] -> PUBLIC [[ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT], ABSENT] -> PUBLIC [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, ABSENT], VALIDATED] -> PUBLIC [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT], ABSENT] -> PUBLIC - [[Namespace:{DescID: 110, Name: sq1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC - [[Owner:{DescID: 110}, ABSENT], ABSENT] -> PUBLIC - [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], ABSENT] -> PUBLIC - [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], ABSENT] -> PUBLIC - [[Sequence:{DescID: 110}, ABSENT], DROPPED] -> PUBLIC - [[ObjectParent:{DescID: 110, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC [[Namespace:{DescID: 111, Name: v1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC [[Owner:{DescID: 111}, ABSENT], ABSENT] -> PUBLIC [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], ABSENT] -> PUBLIC @@ -346,39 +181,6 @@ PreCommitPhase stage 2 of 2 with 47 MutationType ops [[ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsDropped - DescriptorID: 109 - *scop.RemoveObjectParent - ObjectID: 109 - ParentSchemaID: 101 - *scop.RemoveTableComment - TableID: 109 - *scop.RemoveColumnDefaultExpression - ColumnID: 1 - TableID: 109 - *scop.RemoveColumnComment - ColumnID: 1 - PgAttributeNum: 1 - TableID: 109 - *scop.RemoveDroppedColumnType - ColumnID: 3 - TableID: 109 - *scop.UpdateTableBackReferencesInTypes - BackReferencedTableID: 109 - TypeIDs: - - 107 - - 108 - *scop.RemoveColumnDefaultExpression - ColumnID: 5 - TableID: 109 - *scop.UpdateTableBackReferencesInSequences - BackReferencedColumnID: 5 - BackReferencedTableID: 109 - SequenceIDs: - - 106 - *scop.RemoveIndexComment - IndexID: 1 - TableID: 109 *scop.RemoveConstraintComment ConstraintID: 2 TableID: 109 @@ -396,24 +198,6 @@ PreCommitPhase stage 2 of 2 with 47 MutationType ops *scop.RemoveObjectParent ObjectID: 111 ParentSchemaID: 101 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 109 - Name: shipments - SchemaID: 101 - *scop.RemoveColumnNotNull - ColumnID: 1 - TableID: 109 - *scop.RemoveSequenceOwner - ColumnID: 2 - OwnedSequenceID: 110 - TableID: 109 - *scop.RemoveOwnerBackReferenceInSequence - SequenceID: 110 - *scop.RemoveDroppedIndexPartialPredicate - IndexID: 2 - TableID: 109 *scop.RemoveForeignKeyBackReference OriginConstraintID: 2 OriginTableID: 109 @@ -440,20 +224,44 @@ PreCommitPhase stage 2 of 2 with 47 MutationType ops DescriptorID: 111 Name: v1 SchemaID: 101 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 3 + *scop.MarkDescriptorAsDropped + DescriptorID: 109 + *scop.RemoveObjectParent + ObjectID: 109 + ParentSchemaID: 101 + *scop.RemoveTableComment TableID: 109 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4 + *scop.RemoveColumnDefaultExpression + ColumnID: 1 TableID: 109 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 5 + *scop.RemoveColumnComment + ColumnID: 1 + PgAttributeNum: 1 TableID: 109 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967295 + *scop.RemoveSequenceOwner + ColumnID: 2 + OwnedSequenceID: 110 TableID: 109 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967294 + *scop.RemoveOwnerBackReferenceInSequence + SequenceID: 110 + *scop.RemoveDroppedColumnType + ColumnID: 3 + TableID: 109 + *scop.UpdateTableBackReferencesInTypes + BackReferencedTableID: 109 + TypeIDs: + - 107 + - 108 + *scop.RemoveColumnDefaultExpression + ColumnID: 5 + TableID: 109 + *scop.UpdateTableBackReferencesInSequences + BackReferencedColumnID: 5 + BackReferencedTableID: 109 + SequenceIDs: + - 106 + *scop.RemoveIndexComment + IndexID: 1 TableID: 109 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 @@ -467,12 +275,39 @@ PreCommitPhase stage 2 of 2 with 47 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 111 - *scop.MakeDeleteOnlyColumnAbsent + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 109 + Name: shipments + SchemaID: 101 + *scop.RemoveColumnNotNull ColumnID: 1 TableID: 109 + *scop.RemoveDroppedIndexPartialPredicate + IndexID: 2 + TableID: 109 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 2 TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 3 + TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4 + TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 5 + TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967295 + TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967294 + TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 1 + TableID: 109 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -856,6 +691,10 @@ DROP TABLE defaultdb.shipments CASCADE; to: [ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, ABSENT] kind: Precedence rule: dependents removed before constraint +- from: [ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, ABSENT] + to: [Table:{DescID: 109}, DROPPED] + kind: Precedence + rule: cross-descriptor constraint is absent before referencing descriptor is dropped - from: [ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, VALIDATED] to: [ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT] kind: Precedence @@ -864,6 +703,10 @@ DROP TABLE defaultdb.shipments CASCADE; to: [ConstraintWithoutIndexName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT] kind: Precedence rule: constraint no longer public before dependents +- from: [ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, ABSENT] + to: [Table:{DescID: 109}, DROPPED] + kind: Precedence + rule: cross-descriptor constraint is absent before referencing descriptor is dropped - from: [ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, VALIDATED] to: [ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT] kind: Precedence @@ -1022,8 +865,8 @@ DROP TABLE defaultdb.shipments CASCADE; rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 110}, DROPPED] to: [ObjectParent:{DescID: 110, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [Sequence:{DescID: 110}, DROPPED] to: [Owner:{DescID: 110}, ABSENT] kind: Precedence @@ -1128,6 +971,10 @@ DROP TABLE defaultdb.shipments CASCADE; to: [ColumnName:{DescID: 109, Name: tracking_number, ColumnID: 1}, ABSENT] kind: Precedence rule: descriptor dropped before dependent element removal +- from: [Table:{DescID: 109}, DROPPED] + to: [ColumnNotNull:{DescID: 109, ColumnID: 1, IndexID: 0}, VALIDATED] + kind: Precedence + rule: relation dropped before dependent constraint - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence @@ -1156,18 +1003,6 @@ DROP TABLE defaultdb.shipments CASCADE; to: [ColumnType:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: Precedence rule: descriptor dropped before dependent element removal -- from: [Table:{DescID: 109}, DROPPED] - to: [ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal -- from: [Table:{DescID: 109}, DROPPED] - to: [ConstraintWithoutIndexName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal -- from: [Table:{DescID: 109}, DROPPED] - to: [ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT] kind: Precedence @@ -1218,8 +1053,8 @@ DROP TABLE defaultdb.shipments CASCADE; rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ObjectParent:{DescID: 109, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [Table:{DescID: 109}, DROPPED] to: [Owner:{DescID: 109}, ABSENT] kind: Precedence @@ -1318,8 +1153,8 @@ DROP TABLE defaultdb.shipments CASCADE; rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ObjectParent:{DescID: 111, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 111}, DROPPED] to: [Owner:{DescID: 111}, ABSENT] kind: Precedence @@ -1350,121 +1185,13 @@ CREATE TABLE defaultdb.greeter ( ops DROP TABLE defaultdb.greeter ---- -StatementPhase stage 1 of 1 with 10 MutationType ops +StatementPhase stage 1 of 1 with no ops transitions: - [[Namespace:{DescID: 114, Name: greeter, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT - [[Owner:{DescID: 114}, ABSENT], PUBLIC] -> ABSENT - [[UserPrivileges:{DescID: 114, Name: admin}, ABSENT], PUBLIC] -> ABSENT - [[UserPrivileges:{DescID: 114, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Table:{DescID: 114}, ABSENT], PUBLIC] -> DROPPED - [[ObjectParent:{DescID: 114, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[ColumnFamily:{DescID: 114, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 114, Name: x, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[ColumnDefaultExpression:{DescID: 114, ReferencedTypeIDs: [112 113], ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 114, Name: y, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 114, ReferencedTypeIDs: [112 113], ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[ColumnNotNull:{DescID: 114, ColumnID: 2, IndexID: 0}, ABSENT], PUBLIC] -> VALIDATED - [[Column:{DescID: 114, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 114, Name: rowid, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[ColumnNotNull:{DescID: 114, ColumnID: 3, IndexID: 0}, ABSENT], PUBLIC] -> VALIDATED - [[ColumnDefaultExpression:{DescID: 114, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 114, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 114, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 114, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 114, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 114, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[PrimaryIndex:{DescID: 114, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED - [[IndexName:{DescID: 114, Name: greeter_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 114, ColumnID: 2, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 114, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[SecondaryIndexPartial:{DescID: 114, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[SecondaryIndex:{DescID: 114, IndexID: 2, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED [[CheckConstraint:{DescID: 114, ReferencedTypeIDs: [112 113], IndexID: 0, ConstraintID: 2}, ABSENT], PUBLIC] -> VALIDATED [[ConstraintWithoutIndexName:{DescID: 114, Name: check, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT - ops: - *scop.MarkDescriptorAsDropped - DescriptorID: 114 - *scop.RemoveObjectParent - ObjectID: 114 - ParentSchemaID: 101 - *scop.RemoveColumnDefaultExpression - ColumnID: 1 - TableID: 114 - *scop.UpdateTableBackReferencesInTypes - BackReferencedTableID: 114 - TypeIDs: - - 112 - - 113 - *scop.RemoveDroppedColumnType - ColumnID: 2 - TableID: 114 - *scop.UpdateTableBackReferencesInTypes - BackReferencedTableID: 114 - TypeIDs: - - 112 - - 113 - *scop.RemoveColumnDefaultExpression - ColumnID: 3 - TableID: 114 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 114 - Name: greeter - SchemaID: 101 - *scop.RemoveDroppedIndexPartialPredicate - IndexID: 2 - TableID: 114 - *scop.UpdateTableBackReferencesInTypes - BackReferencedTableID: 114 - TypeIDs: - - 112 - - 113 + no ops PreCommitPhase stage 1 of 2 with 1 MutationType op transitions: - [[Namespace:{DescID: 114, Name: greeter, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC - [[Owner:{DescID: 114}, ABSENT], ABSENT] -> PUBLIC - [[UserPrivileges:{DescID: 114, Name: admin}, ABSENT], ABSENT] -> PUBLIC - [[UserPrivileges:{DescID: 114, Name: root}, ABSENT], ABSENT] -> PUBLIC - [[Table:{DescID: 114}, ABSENT], DROPPED] -> PUBLIC - [[ObjectParent:{DescID: 114, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC - [[ColumnFamily:{DescID: 114, Name: primary, ColumnFamilyID: 0}, ABSENT], ABSENT] -> PUBLIC - [[Column:{DescID: 114, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC - [[ColumnName:{DescID: 114, Name: x, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC - [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC - [[ColumnDefaultExpression:{DescID: 114, ReferencedTypeIDs: [112 113], ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC - [[Column:{DescID: 114, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC - [[ColumnName:{DescID: 114, Name: y, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC - [[ColumnType:{DescID: 114, ReferencedTypeIDs: [112 113], ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC - [[ColumnNotNull:{DescID: 114, ColumnID: 2, IndexID: 0}, ABSENT], VALIDATED] -> PUBLIC - [[Column:{DescID: 114, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC - [[ColumnName:{DescID: 114, Name: rowid, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC - [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC - [[ColumnNotNull:{DescID: 114, ColumnID: 3, IndexID: 0}, ABSENT], VALIDATED] -> PUBLIC - [[ColumnDefaultExpression:{DescID: 114, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC - [[Column:{DescID: 114, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC - [[ColumnName:{DescID: 114, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC - [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC - [[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC - [[ColumnName:{DescID: 114, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC - [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 114, ColumnID: 3, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 114, ColumnID: 1, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 114, ColumnID: 2, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC - [[PrimaryIndex:{DescID: 114, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> PUBLIC - [[IndexName:{DescID: 114, Name: greeter_pkey, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 114, ColumnID: 2, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 114, ColumnID: 3, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC - [[SecondaryIndexPartial:{DescID: 114, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC - [[SecondaryIndex:{DescID: 114, IndexID: 2, ConstraintID: 0}, ABSENT], VALIDATED] -> PUBLIC [[CheckConstraint:{DescID: 114, ReferencedTypeIDs: [112 113], IndexID: 0, ConstraintID: 2}, ABSENT], VALIDATED] -> PUBLIC [[ConstraintWithoutIndexName:{DescID: 114, Name: check, ConstraintID: 2}, ABSENT], ABSENT] -> PUBLIC ops: @@ -1511,6 +1238,14 @@ PreCommitPhase stage 2 of 2 with 23 MutationType ops [[CheckConstraint:{DescID: 114, ReferencedTypeIDs: [112 113], IndexID: 0, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT [[ConstraintWithoutIndexName:{DescID: 114, Name: check, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT ops: + *scop.RemoveCheckConstraint + ConstraintID: 2 + TableID: 114 + *scop.UpdateTableBackReferencesInTypes + BackReferencedTableID: 114 + TypeIDs: + - 112 + - 113 *scop.MarkDescriptorAsDropped DescriptorID: 114 *scop.RemoveObjectParent @@ -1555,14 +1290,6 @@ PreCommitPhase stage 2 of 2 with 23 MutationType ops TypeIDs: - 112 - 113 - *scop.RemoveCheckConstraint - ConstraintID: 2 - TableID: 114 - *scop.UpdateTableBackReferencesInTypes - BackReferencedTableID: 114 - TypeIDs: - - 112 - - 113 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 114 diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_type b/pkg/sql/schemachanger/scplan/testdata/drop_type index f8f530a52d3e..41dcef2e73cf 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_type +++ b/pkg/sql/schemachanger/scplan/testdata/drop_type @@ -160,8 +160,8 @@ DROP TYPE defaultdb.typ rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: Precedence @@ -192,8 +192,8 @@ DROP TYPE defaultdb.typ rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 104}, DROPPED] to: [ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [EnumType:{DescID: 104}, DROPPED] to: [Owner:{DescID: 104}, ABSENT] kind: Precedence @@ -382,8 +382,8 @@ DROP TYPE defaultdb.ctyp rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, DROPPED] to: [ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, DROPPED] to: [Owner:{DescID: 107}, ABSENT] kind: Precedence @@ -422,8 +422,8 @@ DROP TYPE defaultdb.ctyp rule: descriptor dropped before dependent element removal - from: [CompositeType:{DescID: 106}, DROPPED] to: [ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [CompositeType:{DescID: 106}, DROPPED] to: [Owner:{DescID: 106}, ABSENT] kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_view b/pkg/sql/schemachanger/scplan/testdata/drop_view index 2279637ed272..50746f4f7c37 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_view +++ b/pkg/sql/schemachanger/scplan/testdata/drop_view @@ -232,8 +232,8 @@ DROP VIEW defaultdb.v1 rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 105}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: Precedence @@ -1228,8 +1228,8 @@ DROP VIEW defaultdb.v1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 105}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: Precedence @@ -1300,8 +1300,8 @@ DROP VIEW defaultdb.v1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 106}, DROPPED] to: [Owner:{DescID: 106}, ABSENT] kind: Precedence @@ -1372,8 +1372,8 @@ DROP VIEW defaultdb.v1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 107}, DROPPED] to: [Owner:{DescID: 107}, ABSENT] kind: Precedence @@ -1444,8 +1444,8 @@ DROP VIEW defaultdb.v1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 108}, DROPPED] to: [Owner:{DescID: 108}, ABSENT] kind: Precedence @@ -1528,8 +1528,8 @@ DROP VIEW defaultdb.v1 CASCADE rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ObjectParent:{DescID: 111, ReferencedDescID: 101}, ABSENT] - kind: Precedence - rule: descriptor dropped before dependent element removal + kind: SameStagePrecedence + rules: [descriptor dropped before dependent element removal; descriptor dropped right before removing back-reference in its parent descriptor] - from: [View:{DescID: 111}, DROPPED] to: [Owner:{DescID: 111}, ABSENT] kind: Precedence diff --git a/pkg/sql/schemachanger/sctest_generated_test.go b/pkg/sql/schemachanger/sctest_generated_test.go index 16316dcc0148..3ee99c25c063 100644 --- a/pkg/sql/schemachanger/sctest_generated_test.go +++ b/pkg/sql/schemachanger/sctest_generated_test.go @@ -320,6 +320,81 @@ func TestRollback_alter_table_alter_primary_key_vanilla(t *testing.T) { defer log.Scope(t).Close(t) sctest.Rollback(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_vanilla", sctest.SingleNodeCluster) } +func TestEndToEndSideEffects_alter_table_drop_constraint_check(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.EndToEndSideEffects(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_check", sctest.SingleNodeCluster) +} +func TestExecuteWithDMLInjection_alter_table_drop_constraint_check(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.ExecuteWithDMLInjection(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_check", sctest.SingleNodeCluster) +} +func TestGenerateSchemaChangeCorpus_alter_table_drop_constraint_check(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.GenerateSchemaChangeCorpus(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_check", sctest.SingleNodeCluster) +} +func TestPause_alter_table_drop_constraint_check(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Pause(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_check", sctest.SingleNodeCluster) +} +func TestRollback_alter_table_drop_constraint_check(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Rollback(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_check", sctest.SingleNodeCluster) +} +func TestEndToEndSideEffects_alter_table_drop_constraint_fk(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.EndToEndSideEffects(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_fk", sctest.SingleNodeCluster) +} +func TestExecuteWithDMLInjection_alter_table_drop_constraint_fk(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.ExecuteWithDMLInjection(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_fk", sctest.SingleNodeCluster) +} +func TestGenerateSchemaChangeCorpus_alter_table_drop_constraint_fk(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.GenerateSchemaChangeCorpus(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_fk", sctest.SingleNodeCluster) +} +func TestPause_alter_table_drop_constraint_fk(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Pause(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_fk", sctest.SingleNodeCluster) +} +func TestRollback_alter_table_drop_constraint_fk(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Rollback(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_fk", sctest.SingleNodeCluster) +} +func TestEndToEndSideEffects_alter_table_drop_constraint_uwi(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.EndToEndSideEffects(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_uwi", sctest.SingleNodeCluster) +} +func TestExecuteWithDMLInjection_alter_table_drop_constraint_uwi(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.ExecuteWithDMLInjection(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_uwi", sctest.SingleNodeCluster) +} +func TestGenerateSchemaChangeCorpus_alter_table_drop_constraint_uwi(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.GenerateSchemaChangeCorpus(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_uwi", sctest.SingleNodeCluster) +} +func TestPause_alter_table_drop_constraint_uwi(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Pause(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_uwi", sctest.SingleNodeCluster) +} +func TestRollback_alter_table_drop_constraint_uwi(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Rollback(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_uwi", sctest.SingleNodeCluster) +} func TestEndToEndSideEffects_create_function(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_add_foreign_key b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key index 44c848bd0e21..454a593d8b72 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key @@ -58,6 +58,27 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "1" + version: "2" +upsert descriptor #105 + ... + formatVersion: 3 + id: 105 + + inboundFks: + + - constraintId: 2 + + name: crdb_internal_constraint_2_name_placeholder + + originColumnIds: + + - 1 + + originTableId: 104 + + referencedColumnIds: + + - 1 + + referencedTableId: 105 + + validity: Validating + modificationTime: {} + name: t2 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "1" + + version: "2" # end StatementPhase # begin PreCommitPhase ## PreCommitPhase stage 1 of 2 with 1 MutationType op @@ -128,6 +149,21 @@ upsert descriptor #105 + revertible: true families: - columnIds: + ... + formatVersion: 3 + id: 105 + + inboundFks: + + - constraintId: 2 + + name: crdb_internal_constraint_2_name_placeholder + + originColumnIds: + + - 1 + + originTableId: 104 + + referencedColumnIds: + + - 1 + + referencedTableId: 105 + + validity: Validating + modificationTime: {} + name: t2 ... time: {} unexposedParentSchemaId: 101 @@ -223,18 +259,16 @@ upsert descriptor #105 families: - columnIds: ... - formatVersion: 3 - id: 105 - + inboundFks: - + - constraintId: 2 + inboundFks: + - constraintId: 2 + - name: crdb_internal_constraint_2_name_placeholder + name: t1_i_fkey - + originColumnIds: - + - 1 - + originTableId: 104 - + referencedColumnIds: - + - 1 - + referencedTableId: 105 - + validity: Validating + originColumnIds: + - 1 + ... + - 1 + referencedTableId: 105 + - validity: Validating modificationTime: {} name: t2 ... diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_check b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_check new file mode 100644 index 000000000000..7df6e7e990f3 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_check @@ -0,0 +1,192 @@ +setup +CREATE TABLE t (i INT PRIMARY KEY CHECK (i > 0)); +---- +... ++object {100 101 t} -> 104 + +# Constraint is still enforced in this stage. +stage-exec phase=PostCommitNonRevertiblePhase stage=1 +INSERT INTO t VALUES (0); +---- +pq: failed to satisfy CHECK constraint \(i > 0:::INT8\) + +stage-query phase=PostCommitNonRevertiblePhase stage=1 +SELECT count(*) FROM t; +---- +0 + +test +ALTER TABLE t DROP CONSTRAINT check_i; +---- +begin transaction #1 +# begin StatementPhase +checking for feature: ALTER TABLE +increment telemetry for sql.schema.alter_table +increment telemetry for sql.schema.alter_table.drop_constraint +## StatementPhase stage 1 of 1 with 2 MutationType ops +upsert descriptor #104 + ... + constraintId: 2 + expr: i > 0:::INT8 + - name: check_i + + name: crdb_internal_constraint_2_name_placeholder + + validity: Dropping + columns: + - id: 1 + ... + id: 104 + modificationTime: {} + + mutations: + + - constraint: + + check: + + columnIds: + + - 1 + + constraintId: 2 + + expr: i > 0:::INT8 + + name: check_i + + validity: Dropping + + foreignKey: {} + + name: check_i + + uniqueWithoutIndexConstraint: {} + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 2 + ... + time: {} + 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 4 MutationType ops +upsert descriptor #104 + ... + constraintId: 2 + expr: i > 0:::INT8 + - name: check_i + + name: crdb_internal_constraint_2_name_placeholder + + validity: Dropping + columns: + - id: 1 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP CONSTRAINT ‹check_i› + + statement: ALTER TABLE t DROP CONSTRAINT check_i + + statementTag: ALTER TABLE + + targetRanks: + + targets: + families: + - columnIds: + ... + id: 104 + modificationTime: {} + + mutations: + + - constraint: + + check: + + columnIds: + + - 1 + + constraintId: 2 + + expr: i > 0:::INT8 + + name: check_i + + validity: Dropping + + foreignKey: {} + + name: check_i + + uniqueWithoutIndexConstraint: {} + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 2 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "1" + + version: "2" +persist all catalog changes to storage +create job #1 (non-cancelable: true): "ALTER TABLE defaultdb.public.t DROP CONSTRAINT check_i" + 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 +## PostCommitNonRevertiblePhase stage 1 of 1 with 3 MutationType ops +upsert descriptor #104 + table: + - checks: + - - columnIds: + - - 1 + - constraintId: 2 + - expr: i > 0:::INT8 + - name: crdb_internal_constraint_2_name_placeholder + - validity: Dropping + + checks: [] + columns: + - id: 1 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP CONSTRAINT ‹check_i› + - statement: ALTER TABLE t DROP CONSTRAINT check_i + - statementTag: ALTER TABLE + - targetRanks: + - targets: + families: + - columnIds: + ... + id: 104 + modificationTime: {} + - mutations: + - - constraint: + - check: + - columnIds: + - - 1 + - constraintId: 2 + - expr: i > 0:::INT8 + - name: check_i + - validity: Dropping + - foreignKey: {} + - name: check_i + - uniqueWithoutIndexConstraint: {} + - direction: DROP + - mutationId: 1 + - state: WRITE_ONLY + + mutations: [] + name: t + nextColumnId: 2 + ... + time: {} + 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_drop_constraint_fk b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_fk new file mode 100644 index 000000000000..c5929edc2df4 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_fk @@ -0,0 +1,310 @@ +setup +CREATE TABLE t2 (i INT PRIMARY KEY); +CREATE TABLE t1 (i INT PRIMARY KEY REFERENCES t2(i)); +---- +... ++object {100 101 t2} -> 104 ++object {100 101 t1} -> 105 + +# Constraint is still enforced in this stage, so the second +# insert will fail. +stage-exec phase=PostCommitNonRevertiblePhase stage=1 +INSERT INTO t1 VALUES (0); +---- +pq: insert on table "t1" violates foreign key constraint "crdb_internal_constraint_2_name_placeholder" + +stage-query phase=PostCommitNonRevertiblePhase stage=1 +SELECT count(*) FROM t1; +---- +0 + +# A dropping FK is only enforced for incoming writes, as demonstrated above, +# but not validated on existing rows. We hence use the following explain +# statement to ensure the optimizer effective ignores the FK constraint. +# This is proved by t1 using a *full scan*, rather than a limited scan. +stage-query phase=PostCommitNonRevertiblePhase stage=1 +EXPLAIN SELECT * FROM t1 JOIN t2 ON t1.i = t2.i LIMIT 1 +---- +---- +distribution: local +vectorized: true + +• limit +│ count: 1 +│ +└── • lookup join + │ table: t2@t2_pkey + │ equality: (i) = (i) + │ equality cols are key + │ + └── • scan + missing stats + table: t1@t1_pkey + spans: FULL SCAN (SOFT LIMIT) +---- +---- + +test +ALTER TABLE t1 DROP CONSTRAINT t1_i_fkey; +---- +begin transaction #1 +# begin StatementPhase +checking for feature: ALTER TABLE +increment telemetry for sql.schema.alter_table +increment telemetry for sql.schema.alter_table.drop_constraint +## StatementPhase stage 1 of 1 with 2 MutationType ops +upsert descriptor #104 + ... + inboundFks: + - constraintId: 2 + - name: t1_i_fkey + + name: crdb_internal_constraint_2_name_placeholder + originColumnIds: + - 1 + ... + - 1 + referencedTableId: 104 + + validity: Dropping + modificationTime: {} + name: t2 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "2" + + version: "3" +upsert descriptor #105 + ... + id: 105 + modificationTime: {} + + mutations: + + - constraint: + + check: {} + + constraintType: FOREIGN_KEY + + foreignKey: + + constraintId: 2 + + name: crdb_internal_constraint_2_name_placeholder + + originColumnIds: + + - 1 + + originTableId: 105 + + referencedColumnIds: + + - 1 + + referencedTableId: 104 + + validity: Dropping + + name: t1_i_fkey + + uniqueWithoutIndexConstraint: {} + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + name: t1 + nextColumnId: 2 + ... + nextIndexId: 2 + nextMutationId: 1 + - outboundFks: + - - constraintId: 2 + - name: t1_i_fkey + - originColumnIds: + - - 1 + - originTableId: 105 + - referencedColumnIds: + - - 1 + - referencedTableId: 104 + parentId: 100 + primaryIndex: + ... + time: {} + unexposedParentSchemaId: 101 + - version: "2" + + version: "3" +# 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 + + jobId: "1" + families: + - columnIds: + ... + inboundFks: + - constraintId: 2 + - name: t1_i_fkey + + name: crdb_internal_constraint_2_name_placeholder + originColumnIds: + - 1 + ... + - 1 + referencedTableId: 104 + + validity: Dropping + modificationTime: {} + name: t2 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "2" + + version: "3" +upsert descriptor #105 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t1› DROP CONSTRAINT ‹t1_i_fkey› + + statement: ALTER TABLE t1 DROP CONSTRAINT t1_i_fkey + + statementTag: ALTER TABLE + + targetRanks: + + targets: + families: + - columnIds: + ... + id: 105 + modificationTime: {} + + mutations: + + - constraint: + + check: {} + + constraintType: FOREIGN_KEY + + foreignKey: + + constraintId: 2 + + name: crdb_internal_constraint_2_name_placeholder + + originColumnIds: + + - 1 + + originTableId: 105 + + referencedColumnIds: + + - 1 + + referencedTableId: 104 + + validity: Dropping + + name: t1_i_fkey + + uniqueWithoutIndexConstraint: {} + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + name: t1 + nextColumnId: 2 + ... + nextIndexId: 2 + nextMutationId: 1 + - outboundFks: + - - constraintId: 2 + - name: t1_i_fkey + - originColumnIds: + - - 1 + - originTableId: 105 + - referencedColumnIds: + - - 1 + - referencedTableId: 104 + parentId: 100 + primaryIndex: + ... + time: {} + unexposedParentSchemaId: 101 + - version: "2" + + version: "3" +persist all catalog changes to storage +create job #1 (non-cancelable: true): "ALTER TABLE defaultdb.public.t1 DROP CONSTRAINT t1_i_fkey" + descriptor IDs: [104 105] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitNonRevertiblePhase stage 1 of 1 with 5 MutationType ops +upsert descriptor #104 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - jobId: "1" + families: + - columnIds: + ... + formatVersion: 3 + id: 104 + - inboundFks: + - - constraintId: 2 + - name: crdb_internal_constraint_2_name_placeholder + - originColumnIds: + - - 1 + - originTableId: 105 + - referencedColumnIds: + - - 1 + - referencedTableId: 104 + - validity: Dropping + modificationTime: {} + name: t2 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "3" + + version: "4" +upsert descriptor #105 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t1› DROP CONSTRAINT ‹t1_i_fkey› + - statement: ALTER TABLE t1 DROP CONSTRAINT t1_i_fkey + - statementTag: ALTER TABLE + - targetRanks: + - targets: + families: + - columnIds: + ... + id: 105 + modificationTime: {} + - mutations: + - - constraint: + - check: {} + - constraintType: FOREIGN_KEY + - foreignKey: + - constraintId: 2 + - name: crdb_internal_constraint_2_name_placeholder + - originColumnIds: + - - 1 + - originTableId: 105 + - referencedColumnIds: + - - 1 + - referencedTableId: 104 + - validity: Dropping + - name: t1_i_fkey + - uniqueWithoutIndexConstraint: {} + - direction: DROP + - mutationId: 1 + - state: WRITE_ONLY + name: t1 + nextColumnId: 2 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "3" + + version: "4" +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: 105 +commit transaction #3 +# end PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_uwi b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_uwi new file mode 100644 index 000000000000..f5ab34ac633b --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_drop_constraint_uwi @@ -0,0 +1,184 @@ +setup +SET experimental_enable_unique_without_index_constraints = true; +CREATE TABLE t (i INT PRIMARY KEY, j INT); +ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); +---- +... ++object {100 101 t} -> 104 + +# Constraint is still enforced in this stage, so the second +# insert will fail. +stage-exec phase=PostCommitNonRevertiblePhase stage=1 +INSERT INTO t VALUES ($stageKey, $stageKey); +INSERT INTO t VALUES ($stageKey+1, $stageKey); +---- +pq: duplicate key value violates unique constraint "crdb_internal_constraint_2_name_placeholder" + +stage-query phase=PostCommitNonRevertiblePhase stage=1 +SELECT count(*) FROM t; +---- +1 + + +test +ALTER TABLE t DROP CONSTRAINT unique_j; +---- +begin transaction #1 +# begin StatementPhase +checking for feature: ALTER TABLE +increment telemetry for sql.schema.alter_table +increment telemetry for sql.schema.alter_table.drop_constraint +## StatementPhase stage 1 of 1 with 2 MutationType ops +upsert descriptor #104 + ... + id: 104 + modificationTime: {} + + mutations: + + - constraint: + + check: {} + + constraintType: UNIQUE_WITHOUT_INDEX + + foreignKey: {} + + name: unique_j + + uniqueWithoutIndexConstraint: + + columnIds: + + - 2 + + constraintId: 2 + + name: crdb_internal_constraint_2_name_placeholder + + tableId: 104 + + validity: Dropping + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 101 + - uniqueWithoutIndexConstraints: + - - columnIds: + - - 2 + - constraintId: 2 + - name: unique_j + - tableId: 104 + - version: "3" + + version: "4" +# 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 4 MutationType ops +upsert descriptor #104 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP CONSTRAINT ‹unique_j› + + statement: ALTER TABLE t DROP CONSTRAINT unique_j + + statementTag: ALTER TABLE + + targetRanks: + + targets: + families: + - columnIds: + ... + id: 104 + modificationTime: {} + + mutations: + + - constraint: + + check: {} + + constraintType: UNIQUE_WITHOUT_INDEX + + foreignKey: {} + + name: unique_j + + uniqueWithoutIndexConstraint: + + columnIds: + + - 2 + + constraintId: 2 + + name: crdb_internal_constraint_2_name_placeholder + + tableId: 104 + + validity: Dropping + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 101 + - uniqueWithoutIndexConstraints: + - - columnIds: + - - 2 + - constraintId: 2 + - name: unique_j + - tableId: 104 + - version: "3" + + version: "4" +persist all catalog changes to storage +create job #1 (non-cancelable: true): "ALTER TABLE defaultdb.public.t DROP CONSTRAINT unique_j" + 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 +## PostCommitNonRevertiblePhase stage 1 of 1 with 3 MutationType ops +upsert descriptor #104 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP CONSTRAINT ‹unique_j› + - statement: ALTER TABLE t DROP CONSTRAINT unique_j + - statementTag: ALTER TABLE + - targetRanks: + - targets: + families: + - columnIds: + ... + id: 104 + modificationTime: {} + - mutations: + - - constraint: + - check: {} + - constraintType: UNIQUE_WITHOUT_INDEX + - foreignKey: {} + - name: unique_j + - uniqueWithoutIndexConstraint: + - columnIds: + - - 2 + - constraintId: 2 + - name: crdb_internal_constraint_2_name_placeholder + - tableId: 104 + - validity: Dropping + - direction: DROP + - mutationId: 1 + - state: WRITE_ONLY + name: t + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "4" + + version: "5" +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/explain/alter_table_drop_constraint_check b/pkg/sql/schemachanger/testdata/explain/alter_table_drop_constraint_check new file mode 100644 index 000000000000..eb3a1642db5b --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_drop_constraint_check @@ -0,0 +1,39 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY CHECK (i > 0)); + +/* test */ +EXPLAIN (ddl) ALTER TABLE t DROP CONSTRAINT check_i; +---- +Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP CONSTRAINT ‹check_i›; + ├── StatementPhase + │ └── Stage 1 of 1 in StatementPhase + │ ├── 2 elements transitioning toward ABSENT + │ │ ├── PUBLIC → VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ │ └── PUBLIC → ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2} + │ └── 2 Mutation operations + │ ├── MakePublicCheckConstraintValidated {"ConstraintID":2,"TableID":104} + │ └── SetConstraintName {"ConstraintID":2,"Name":"crdb_internal_co...","TableID":104} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 2 elements transitioning toward ABSENT + │ │ │ ├── VALIDATED → PUBLIC CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ │ │ └── ABSENT → PUBLIC ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 2 elements transitioning toward ABSENT + │ │ ├── PUBLIC → VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ │ └── PUBLIC → ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2} + │ └── 4 Mutation operations + │ ├── MakePublicCheckConstraintValidated {"ConstraintID":2,"TableID":104} + │ ├── SetConstraintName {"ConstraintID":2,"Name":"crdb_internal_co...","TableID":104} + │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} + │ └── CreateSchemaChangerJob {"NonCancelable":true,"RunningStatus":"PostCommitNonRev..."} + └── PostCommitNonRevertiblePhase + └── Stage 1 of 1 in PostCommitNonRevertiblePhase + ├── 1 element transitioning toward ABSENT + │ └── VALIDATED → ABSENT CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + └── 3 Mutation operations + ├── RemoveCheckConstraint {"ConstraintID":2,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_drop_constraint_fk b/pkg/sql/schemachanger/testdata/explain/alter_table_drop_constraint_fk new file mode 100644 index 000000000000..5dd39fe0cd84 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_drop_constraint_fk @@ -0,0 +1,43 @@ +/* setup */ +CREATE TABLE t2 (i INT PRIMARY KEY); +CREATE TABLE t1 (i INT PRIMARY KEY REFERENCES t2(i)); + +/* test */ +EXPLAIN (ddl) ALTER TABLE t1 DROP CONSTRAINT t1_i_fkey; +---- +Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t1› DROP CONSTRAINT ‹t1_i_fkey›; + ├── StatementPhase + │ └── Stage 1 of 1 in StatementPhase + │ ├── 2 elements transitioning toward ABSENT + │ │ ├── PUBLIC → VALIDATED ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} + │ │ └── PUBLIC → ABSENT ConstraintWithoutIndexName:{DescID: 105, Name: t1_i_fkey, ConstraintID: 2} + │ └── 2 Mutation operations + │ ├── MakePublicForeignKeyConstraintValidated {"ConstraintID":2,"TableID":105} + │ └── SetConstraintName {"ConstraintID":2,"Name":"crdb_internal_co...","TableID":105} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 2 elements transitioning toward ABSENT + │ │ │ ├── VALIDATED → PUBLIC ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} + │ │ │ └── ABSENT → PUBLIC ConstraintWithoutIndexName:{DescID: 105, Name: t1_i_fkey, ConstraintID: 2} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 2 elements transitioning toward ABSENT + │ │ ├── PUBLIC → VALIDATED ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} + │ │ └── PUBLIC → ABSENT ConstraintWithoutIndexName:{DescID: 105, Name: t1_i_fkey, ConstraintID: 2} + │ └── 5 Mutation operations + │ ├── MakePublicForeignKeyConstraintValidated {"ConstraintID":2,"TableID":105} + │ ├── SetConstraintName {"ConstraintID":2,"Name":"crdb_internal_co...","TableID":105} + │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} + │ ├── SetJobStateOnDescriptor {"DescriptorID":105,"Initialize":true} + │ └── CreateSchemaChangerJob {"NonCancelable":true,"RunningStatus":"PostCommitNonRev..."} + └── PostCommitNonRevertiblePhase + └── Stage 1 of 1 in PostCommitNonRevertiblePhase + ├── 1 element transitioning toward ABSENT + │ └── VALIDATED → ABSENT ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} + └── 5 Mutation operations + ├── RemoveForeignKeyBackReference {"OriginConstraintID":2,"OriginTableID":105,"ReferencedTableID":104} + ├── RemoveForeignKeyConstraint {"ConstraintID":2,"TableID":105} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":105} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_drop_constraint_uwi b/pkg/sql/schemachanger/testdata/explain/alter_table_drop_constraint_uwi new file mode 100644 index 000000000000..75337219064a --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_drop_constraint_uwi @@ -0,0 +1,41 @@ +/* setup */ +SET experimental_enable_unique_without_index_constraints = true; +CREATE TABLE t (i INT PRIMARY KEY, j INT); +ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); + +/* test */ +EXPLAIN (ddl) ALTER TABLE t DROP CONSTRAINT unique_j; +---- +Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP CONSTRAINT ‹unique_j›; + ├── StatementPhase + │ └── Stage 1 of 1 in StatementPhase + │ ├── 2 elements transitioning toward ABSENT + │ │ ├── PUBLIC → VALIDATED UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ └── PUBLIC → ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} + │ └── 2 Mutation operations + │ ├── MakePublicUniqueWithoutIndexConstraintValidated {"ConstraintID":2,"TableID":104} + │ └── SetConstraintName {"ConstraintID":2,"Name":"crdb_internal_co...","TableID":104} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 2 elements transitioning toward ABSENT + │ │ │ ├── VALIDATED → PUBLIC UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ │ └── ABSENT → PUBLIC ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 2 elements transitioning toward ABSENT + │ │ ├── PUBLIC → VALIDATED UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ └── PUBLIC → ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} + │ └── 4 Mutation operations + │ ├── MakePublicUniqueWithoutIndexConstraintValidated {"ConstraintID":2,"TableID":104} + │ ├── SetConstraintName {"ConstraintID":2,"Name":"crdb_internal_co...","TableID":104} + │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} + │ └── CreateSchemaChangerJob {"NonCancelable":true,"RunningStatus":"PostCommitNonRev..."} + └── PostCommitNonRevertiblePhase + └── Stage 1 of 1 in PostCommitNonRevertiblePhase + ├── 1 element transitioning toward ABSENT + │ └── VALIDATED → ABSENT UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + └── 3 Mutation operations + ├── RemoveUniqueWithoutIndexConstraint {"ConstraintID":2,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_drop_constraint_check b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_drop_constraint_check new file mode 100644 index 000000000000..8748983ea01f --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_drop_constraint_check @@ -0,0 +1,129 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY CHECK (i > 0)); + +/* test */ +EXPLAIN (ddl, verbose) ALTER TABLE t DROP CONSTRAINT check_i; +---- +• Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP CONSTRAINT ‹check_i›; +│ +├── • StatementPhase +│ │ +│ └── • Stage 1 of 1 in StatementPhase +│ │ +│ ├── • 2 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ │ rule: "CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" +│ │ │ +│ │ └── • ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ rule: "constraint no longer public before dependents" +│ │ +│ └── • 2 Mutation operations +│ │ +│ ├── • MakePublicCheckConstraintValidated +│ │ ConstraintID: 2 +│ │ TableID: 104 +│ │ +│ └── • SetConstraintName +│ ConstraintID: 2 +│ Name: crdb_internal_constraint_2_name_placeholder +│ TableID: 104 +│ +├── • PreCommitPhase +│ │ +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 2 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ └── • ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 2 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ │ rule: "CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" +│ │ │ +│ │ └── • ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ rule: "constraint no longer public before dependents" +│ │ +│ └── • 4 Mutation operations +│ │ +│ ├── • MakePublicCheckConstraintValidated +│ │ ConstraintID: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetConstraintName +│ │ ConstraintID: 2 +│ │ Name: crdb_internal_constraint_2_name_placeholder +│ │ TableID: 104 +│ │ +│ ├── • SetJobStateOnDescriptor +│ │ DescriptorID: 104 +│ │ Initialize: true +│ │ +│ └── • CreateSchemaChangerJob +│ Authorization: +│ UserName: root +│ DescriptorIDs: +│ - 104 +│ JobID: 1 +│ NonCancelable: true +│ RunningStatus: PostCommitNonRevertiblePhase stage 1 of 1 with 1 MutationType op pending +│ Statements: +│ - statement: ALTER TABLE t DROP CONSTRAINT check_i +│ redactedstatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP CONSTRAINT ‹check_i› +│ statementtag: ALTER TABLE +│ +└── • PostCommitNonRevertiblePhase + │ + └── • Stage 1 of 1 in PostCommitNonRevertiblePhase + │ + ├── • 1 element transitioning toward ABSENT + │ │ + │ └── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ │ VALIDATED → ABSENT + │ │ + │ ├── • PreviousStagePrecedence dependency from VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ │ rule: "CheckConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT" + │ │ + │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2} + │ rule: "dependents removed before constraint" + │ + └── • 3 Mutation operations + │ + ├── • RemoveCheckConstraint + │ ConstraintID: 2 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + DescriptorIDsToRemove: + - 104 + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_drop_constraint_fk b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_drop_constraint_fk new file mode 100644 index 000000000000..f8f619de15bd --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_drop_constraint_fk @@ -0,0 +1,145 @@ +/* setup */ +CREATE TABLE t2 (i INT PRIMARY KEY); +CREATE TABLE t1 (i INT PRIMARY KEY REFERENCES t2(i)); + +/* test */ +EXPLAIN (ddl, verbose) ALTER TABLE t1 DROP CONSTRAINT t1_i_fkey; +---- +• Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t1› DROP CONSTRAINT ‹t1_i_fkey›; +│ +├── • StatementPhase +│ │ +│ └── • Stage 1 of 1 in StatementPhase +│ │ +│ ├── • 2 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} +│ │ │ rule: "ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" +│ │ │ +│ │ └── • ConstraintWithoutIndexName:{DescID: 105, Name: t1_i_fkey, ConstraintID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from VALIDATED ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} +│ │ rule: "constraint no longer public before dependents" +│ │ +│ └── • 2 Mutation operations +│ │ +│ ├── • MakePublicForeignKeyConstraintValidated +│ │ ConstraintID: 2 +│ │ TableID: 105 +│ │ +│ └── • SetConstraintName +│ ConstraintID: 2 +│ Name: crdb_internal_constraint_2_name_placeholder +│ TableID: 105 +│ +├── • PreCommitPhase +│ │ +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 2 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ └── • ConstraintWithoutIndexName:{DescID: 105, Name: t1_i_fkey, ConstraintID: 2} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 2 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} +│ │ │ rule: "ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" +│ │ │ +│ │ └── • ConstraintWithoutIndexName:{DescID: 105, Name: t1_i_fkey, ConstraintID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from VALIDATED ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} +│ │ rule: "constraint no longer public before dependents" +│ │ +│ └── • 5 Mutation operations +│ │ +│ ├── • MakePublicForeignKeyConstraintValidated +│ │ ConstraintID: 2 +│ │ TableID: 105 +│ │ +│ ├── • SetConstraintName +│ │ ConstraintID: 2 +│ │ Name: crdb_internal_constraint_2_name_placeholder +│ │ TableID: 105 +│ │ +│ ├── • SetJobStateOnDescriptor +│ │ DescriptorID: 104 +│ │ Initialize: true +│ │ +│ ├── • SetJobStateOnDescriptor +│ │ DescriptorID: 105 +│ │ Initialize: true +│ │ +│ └── • CreateSchemaChangerJob +│ Authorization: +│ UserName: root +│ DescriptorIDs: +│ - 104 +│ - 105 +│ JobID: 1 +│ NonCancelable: true +│ RunningStatus: PostCommitNonRevertiblePhase stage 1 of 1 with 2 MutationType ops pending +│ Statements: +│ - statement: ALTER TABLE t1 DROP CONSTRAINT t1_i_fkey +│ redactedstatement: ALTER TABLE ‹defaultdb›.‹public›.‹t1› DROP CONSTRAINT ‹t1_i_fkey› +│ statementtag: ALTER TABLE +│ +└── • PostCommitNonRevertiblePhase + │ + └── • Stage 1 of 1 in PostCommitNonRevertiblePhase + │ + ├── • 1 element transitioning toward ABSENT + │ │ + │ └── • ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} + │ │ VALIDATED → ABSENT + │ │ + │ ├── • PreviousStagePrecedence dependency from VALIDATED ForeignKeyConstraint:{DescID: 105, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104} + │ │ rule: "ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT" + │ │ + │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 105, Name: t1_i_fkey, ConstraintID: 2} + │ rule: "dependents removed before constraint" + │ + └── • 5 Mutation operations + │ + ├── • RemoveForeignKeyBackReference + │ OriginConstraintID: 2 + │ OriginTableID: 105 + │ ReferencedTableID: 104 + │ + ├── • RemoveForeignKeyConstraint + │ ConstraintID: 2 + │ TableID: 105 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 105 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + DescriptorIDsToRemove: + - 104 + - 105 + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_drop_constraint_uwi b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_drop_constraint_uwi new file mode 100644 index 000000000000..a9992361cc03 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_drop_constraint_uwi @@ -0,0 +1,131 @@ +/* setup */ +SET experimental_enable_unique_without_index_constraints = true; +CREATE TABLE t (i INT PRIMARY KEY, j INT); +ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); + +/* test */ +EXPLAIN (ddl, verbose) ALTER TABLE t DROP CONSTRAINT unique_j; +---- +• Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP CONSTRAINT ‹unique_j›; +│ +├── • StatementPhase +│ │ +│ └── • Stage 1 of 1 in StatementPhase +│ │ +│ ├── • 2 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ │ rule: "UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" +│ │ │ +│ │ └── • ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from VALIDATED UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ rule: "constraint no longer public before dependents" +│ │ +│ └── • 2 Mutation operations +│ │ +│ ├── • MakePublicUniqueWithoutIndexConstraintValidated +│ │ ConstraintID: 2 +│ │ TableID: 104 +│ │ +│ └── • SetConstraintName +│ ConstraintID: 2 +│ Name: crdb_internal_constraint_2_name_placeholder +│ TableID: 104 +│ +├── • PreCommitPhase +│ │ +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 2 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ └── • ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 2 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ │ rule: "UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" +│ │ │ +│ │ └── • ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from VALIDATED UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ rule: "constraint no longer public before dependents" +│ │ +│ └── • 4 Mutation operations +│ │ +│ ├── • MakePublicUniqueWithoutIndexConstraintValidated +│ │ ConstraintID: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetConstraintName +│ │ ConstraintID: 2 +│ │ Name: crdb_internal_constraint_2_name_placeholder +│ │ TableID: 104 +│ │ +│ ├── • SetJobStateOnDescriptor +│ │ DescriptorID: 104 +│ │ Initialize: true +│ │ +│ └── • CreateSchemaChangerJob +│ Authorization: +│ UserName: root +│ DescriptorIDs: +│ - 104 +│ JobID: 1 +│ NonCancelable: true +│ RunningStatus: PostCommitNonRevertiblePhase stage 1 of 1 with 1 MutationType op pending +│ Statements: +│ - statement: ALTER TABLE t DROP CONSTRAINT unique_j +│ redactedstatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP CONSTRAINT ‹unique_j› +│ statementtag: ALTER TABLE +│ +└── • PostCommitNonRevertiblePhase + │ + └── • Stage 1 of 1 in PostCommitNonRevertiblePhase + │ + ├── • 1 element transitioning toward ABSENT + │ │ + │ └── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ VALIDATED → ABSENT + │ │ + │ ├── • PreviousStagePrecedence dependency from VALIDATED UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ rule: "UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT" + │ │ + │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} + │ rule: "dependents removed before constraint" + │ + └── • 3 Mutation operations + │ + ├── • RemoveUniqueWithoutIndexConstraint + │ ConstraintID: 2 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + DescriptorIDsToRemove: + - 104 + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_1_of_7 index 2b76677d3048..eb3c285923d5 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_1_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_1_of_7 @@ -50,8 +50,9 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ │ ├── • ObjectParent:{DescID: 105, ReferencedDescID: 101} │ │ │ │ PUBLIC → ABSENT │ │ │ │ - │ │ │ └── • Precedence dependency from DROPPED Function:{DescID: 105} + │ │ │ └── • SameStagePrecedence dependency from DROPPED Function:{DescID: 105} │ │ │ rule: "descriptor dropped before dependent element removal" + │ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • FunctionName:{DescID: 105} │ │ │ │ PUBLIC → ABSENT diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_2_of_7 index cc1b81cb841d..60a46e2087fe 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_2_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_2_of_7 @@ -50,8 +50,9 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • ObjectParent:{DescID: 105, ReferencedDescID: 101} │ │ │ │ PUBLIC → ABSENT │ │ │ │ - │ │ │ └── • Precedence dependency from DROPPED Function:{DescID: 105} + │ │ │ └── • SameStagePrecedence dependency from DROPPED Function:{DescID: 105} │ │ │ rule: "descriptor dropped before dependent element removal" + │ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • FunctionName:{DescID: 105} │ │ │ │ PUBLIC → ABSENT diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_3_of_7 index 4de07cc6043b..c86292f1336e 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_3_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_3_of_7 @@ -50,8 +50,9 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • ObjectParent:{DescID: 105, ReferencedDescID: 101} │ │ │ │ PUBLIC → ABSENT │ │ │ │ - │ │ │ └── • Precedence dependency from DROPPED Function:{DescID: 105} + │ │ │ └── • SameStagePrecedence dependency from DROPPED Function:{DescID: 105} │ │ │ rule: "descriptor dropped before dependent element removal" + │ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • FunctionName:{DescID: 105} │ │ │ │ PUBLIC → ABSENT diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_4_of_7 index 8a65be7f6cff..23e79aebd86a 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_4_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_4_of_7 @@ -50,8 +50,9 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • ObjectParent:{DescID: 105, ReferencedDescID: 101} │ │ │ │ PUBLIC → ABSENT │ │ │ │ - │ │ │ └── • Precedence dependency from DROPPED Function:{DescID: 105} + │ │ │ └── • SameStagePrecedence dependency from DROPPED Function:{DescID: 105} │ │ │ rule: "descriptor dropped before dependent element removal" + │ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • FunctionName:{DescID: 105} │ │ │ │ PUBLIC → ABSENT diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_5_of_7 index 73e22c22e72d..808468ac70dc 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_5_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_5_of_7 @@ -50,8 +50,9 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • ObjectParent:{DescID: 105, ReferencedDescID: 101} │ │ │ │ PUBLIC → ABSENT │ │ │ │ - │ │ │ └── • Precedence dependency from DROPPED Function:{DescID: 105} + │ │ │ └── • SameStagePrecedence dependency from DROPPED Function:{DescID: 105} │ │ │ rule: "descriptor dropped before dependent element removal" + │ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • FunctionName:{DescID: 105} │ │ │ │ PUBLIC → ABSENT diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_6_of_7 index 2ecc29de674a..26b9333bedff 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_6_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_6_of_7 @@ -50,8 +50,9 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • ObjectParent:{DescID: 105, ReferencedDescID: 101} │ │ │ │ PUBLIC → ABSENT │ │ │ │ - │ │ │ └── • Precedence dependency from DROPPED Function:{DescID: 105} + │ │ │ └── • SameStagePrecedence dependency from DROPPED Function:{DescID: 105} │ │ │ rule: "descriptor dropped before dependent element removal" + │ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • FunctionName:{DescID: 105} │ │ │ │ PUBLIC → ABSENT diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_7_of_7 index d12354b5ae77..e7b77c18ce8a 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_7_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_function_in_txn.rollback_7_of_7 @@ -50,8 +50,9 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • ObjectParent:{DescID: 105, ReferencedDescID: 101} │ │ │ │ PUBLIC → ABSENT │ │ │ │ - │ │ │ └── • Precedence dependency from DROPPED Function:{DescID: 105} + │ │ │ └── • SameStagePrecedence dependency from DROPPED Function:{DescID: 105} │ │ │ rule: "descriptor dropped before dependent element removal" + │ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • FunctionName:{DescID: 105} │ │ │ │ PUBLIC → ABSENT diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_function b/pkg/sql/schemachanger/testdata/explain_verbose/drop_function index 27dc014ea258..2bee9fce9d14 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_function +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_function @@ -62,8 +62,9 @@ EXPLAIN (ddl, verbose) DROP FUNCTION f; │ │ ├── • ObjectParent:{DescID: 109, ReferencedDescID: 101} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED Function:{DescID: 109} +│ │ │ └── • SameStagePrecedence dependency from DROPPED Function:{DescID: 109} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • FunctionName:{DescID: 109} │ │ │ │ PUBLIC → ABSENT @@ -209,8 +210,9 @@ EXPLAIN (ddl, verbose) DROP FUNCTION f; │ │ ├── • ObjectParent:{DescID: 109, ReferencedDescID: 101} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED Function:{DescID: 109} +│ │ │ └── • SameStagePrecedence dependency from DROPPED Function:{DescID: 109} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • FunctionName:{DescID: 109} │ │ │ │ PUBLIC → ABSENT diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_with_materialized_view_dep b/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_with_materialized_view_dep index bf30d0ba08f6..b6e974f73f0d 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_with_materialized_view_dep +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_with_materialized_view_dep @@ -81,8 +81,9 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ ├── • ObjectParent:{DescID: 106, ReferencedDescID: 101} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ └── • SameStagePrecedence dependency from DROPPED View:{DescID: 106} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0} │ │ │ │ PUBLIC → ABSENT @@ -478,8 +479,9 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ ├── • ObjectParent:{DescID: 106, ReferencedDescID: 101} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ └── • SameStagePrecedence dependency from DROPPED View:{DescID: 106} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0} │ │ │ │ PUBLIC → ABSENT diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_schema b/pkg/sql/schemachanger/testdata/explain_verbose/drop_schema index eb08a80d03e9..b1bd2b06a9b0 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_schema +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_schema @@ -52,8 +52,9 @@ EXPLAIN (ddl, verbose) DROP SCHEMA db.sc; │ │ └── • SchemaParent:{DescID: 106, ReferencedDescID: 104} │ │ │ PUBLIC → ABSENT │ │ │ -│ │ └── • Precedence dependency from DROPPED Schema:{DescID: 106} +│ │ └── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 106} │ │ rule: "descriptor dropped before dependent element removal" +│ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ └── • 3 Mutation operations │ │ @@ -143,8 +144,9 @@ EXPLAIN (ddl, verbose) DROP SCHEMA db.sc; │ │ └── • SchemaParent:{DescID: 106, ReferencedDescID: 104} │ │ │ PUBLIC → ABSENT │ │ │ -│ │ └── • Precedence dependency from DROPPED Schema:{DescID: 106} +│ │ └── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 106} │ │ rule: "descriptor dropped before dependent element removal" +│ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ └── • 6 Mutation operations │ │ diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_table b/pkg/sql/schemachanger/testdata/explain_verbose/drop_table index 9ccecf64128f..91a065ba5237 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_table +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_table @@ -55,8 +55,9 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ ├── • ObjectParent:{DescID: 107, ReferencedDescID: 106} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ └── • SameStagePrecedence dependency from DROPPED Table:{DescID: 107} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • TableComment:{DescID: 107, Comment: t has a comment} │ │ │ │ PUBLIC → ABSENT @@ -169,6 +170,9 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ ├── • ColumnNotNull:{DescID: 107, ColumnID: 3, IndexID: 0} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent constraint" +│ │ │ │ │ │ │ └── • skip PUBLIC → VALIDATED operations │ │ │ rule: "skip constraint removal ops on relation drop" │ │ │ @@ -477,8 +481,9 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ ├── • ObjectParent:{DescID: 107, ReferencedDescID: 106} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ └── • SameStagePrecedence dependency from DROPPED Table:{DescID: 107} │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ rule: "descriptor dropped right before removing back-reference in its parent descriptor" │ │ │ │ │ ├── • TableComment:{DescID: 107, Comment: t has a comment} │ │ │ │ PUBLIC → ABSENT @@ -636,6 +641,9 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ ├── • ColumnNotNull:{DescID: 107, ColumnID: 3, IndexID: 0} │ │ │ │ PUBLIC → ABSENT │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent constraint" +│ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 3} │ │ │ │ rule: "column no longer public before dependents" │ │ │ │ From 38299ef04bef2f1cc500e2980385d4f853ae73b1 Mon Sep 17 00:00:00 2001 From: Evan Wall Date: Mon, 30 Jan 2023 16:55:53 -0500 Subject: [PATCH 5/9] sql: use RLock in connExecutor.CancelQuery and connExecutor.CancelActiveQueries Fixes #95994 `connExecutor.CancelQuery` and `connExecutor.CancelActiveQueries` do not modify `mu.ActiveQueries` or the `*queryMetas` inside so they can safely use `RLock` instead of `Lock`. Release note: None --- pkg/server/status.go | 4 ++-- pkg/sql/conn_executor.go | 27 ++++++++++++++------------- pkg/sql/exec_util.go | 11 ++--------- 3 files changed, 18 insertions(+), 24 deletions(-) diff --git a/pkg/server/status.go b/pkg/server/status.go index 60fe7639b103..7decb5f99cac 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -3056,7 +3056,7 @@ func (s *statusServer) CancelSession( }, nil } - if err := s.checkCancelPrivilege(ctx, reqUsername, session.BaseSessionUser()); err != nil { + if err := s.checkCancelPrivilege(ctx, reqUsername, session.SessionUser()); err != nil { // NB: not using serverError() here since the priv checker // already returns a proper gRPC error status. return nil, err @@ -3109,7 +3109,7 @@ func (s *statusServer) CancelQuery( }, nil } - if err := s.checkCancelPrivilege(ctx, reqUsername, session.BaseSessionUser()); err != nil { + if err := s.checkCancelPrivilege(ctx, reqUsername, session.SessionUser()); err != nil { // NB: not using serverError() here since the priv checker // already returns a proper gRPC error status. return nil, err diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 8cc4ed2c0350..b593acf1f863 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -3144,10 +3144,12 @@ func (ex *connExecutor) hasQuery(queryID clusterunique.ID) bool { // CancelQuery is part of the RegistrySession interface. func (ex *connExecutor) CancelQuery(queryID clusterunique.ID) bool { - ex.mu.Lock() - defer ex.mu.Unlock() + // RLock can be used because map deletion happens in + // connExecutor.removeActiveQuery. + ex.mu.RLock() + defer ex.mu.RUnlock() if queryMeta, exists := ex.mu.ActiveQueries[queryID]; exists { - queryMeta.cancel() + queryMeta.cancelQuery() return true } return false @@ -3155,11 +3157,13 @@ func (ex *connExecutor) CancelQuery(queryID clusterunique.ID) bool { // CancelActiveQueries is part of the RegistrySession interface. func (ex *connExecutor) CancelActiveQueries() bool { - ex.mu.Lock() - defer ex.mu.Unlock() + // RLock can be used because map deletion happens in + // connExecutor.removeActiveQuery. + ex.mu.RLock() + defer ex.mu.RUnlock() canceled := false for _, queryMeta := range ex.mu.ActiveQueries { - queryMeta.cancel() + queryMeta.cancelQuery() canceled = true } return canceled @@ -3174,13 +3178,10 @@ func (ex *connExecutor) CancelSession() { ex.onCancelSession() } -// user is part of the RegistrySession interface. -func (ex *connExecutor) user() username.SQLUsername { - return ex.sessionData().User() -} - -// BaseSessionUser is part of the RegistrySession interface. -func (ex *connExecutor) BaseSessionUser() username.SQLUsername { +// SessionUser is part of the RegistrySession interface. +func (ex *connExecutor) SessionUser() username.SQLUsername { + // SessionUser is the same for all elements in the stack so use Base() + // to avoid needing a lock and race conditions. return ex.sessionDataStack.Base().SessionUser() } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 948c264180f6..5a5e3df2ba1d 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -2034,12 +2034,6 @@ type queryMeta struct { database string } -// cancel cancels the query associated with this queryMeta, by closing the -// associated stmt context. -func (q *queryMeta) cancel() { - q.cancelQuery() -} - // SessionDefaults mirrors fields in Session, for restoring default // configuration values in SET ... TO DEFAULT (or RESET ...) statements. type SessionDefaults map[string]string @@ -2138,9 +2132,8 @@ func (r *SessionRegistry) deregister( } type RegistrySession interface { - user() username.SQLUsername - // BaseSessionUser returns the base session's username. - BaseSessionUser() username.SQLUsername + // SessionUser returns the session user's username. + SessionUser() username.SQLUsername hasQuery(queryID clusterunique.ID) bool // CancelQuery cancels the query specified by queryID if it exists. CancelQuery(queryID clusterunique.ID) bool From 68fd6477fc2f0fa15918cb0188eca31f56ddd010 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Tue, 10 Jan 2023 21:41:48 -0500 Subject: [PATCH 6/9] kv: check txn push marker on commit, not txn record creation This commit moves the point when a transaction checks the timestamp cache for its minimum commit timestamp from transaction record creation time back to commit time. This allows us to use the timestamp cache to communicate successful PushTxn(TIMESTAMP) to a pushee with an existing record without re-writing its transaction record. See the changes to the state machine diagram attached to `Replica.CanCreateTxnRecord` for a visual depiction of this change. --- .../kvserver/batcheval/cmd_end_transaction.go | 21 ++ .../batcheval/cmd_end_transaction_test.go | 229 +++++++++++++----- .../kvserver/batcheval/cmd_heartbeat_txn.go | 11 + pkg/kv/kvserver/batcheval/eval_context.go | 57 +++-- pkg/kv/kvserver/batcheval/transaction.go | 22 +- pkg/kv/kvserver/batcheval/transaction_test.go | 3 - pkg/kv/kvserver/replica_eval_context_span.go | 14 +- pkg/kv/kvserver/replica_tscache.go | 114 +++++---- pkg/kv/kvserver/replica_write.go | 7 +- 9 files changed, 332 insertions(+), 146 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index a367724e51f8..61f908994f97 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -338,6 +338,27 @@ func EndTxn( // Attempt to commit or abort the transaction per the args.Commit parameter. if args.Commit { + // Bump the transaction's provisional commit timestamp to account for any + // transaction pushes, if necessary. See the state machine diagram in + // Replica.CanCreateTxnRecord for details. + switch { + case !recordAlreadyExisted, existingTxn.Status == roachpb.PENDING: + BumpToMinTxnCommitTS(ctx, cArgs.EvalCtx, reply.Txn) + case existingTxn.Status == roachpb.STAGING: + // Don't check timestamp cache. The transaction could not have been pushed + // while its record was in the STAGING state so checking is unnecessary. + // Furthermore, checking the timestamp cache and increasing the commit + // timestamp at this point would be incorrect, because the transaction may + // have entered the implicit commit state. + default: + panic("unreachable") + } + + // Determine whether the transaction's commit is successful or should + // trigger a retry error. + // NOTE: if the transaction is in the implicit commit state and this EndTxn + // request is marking the commit as explicit, this check must succeed. We + // assert this in txnCommitter.makeTxnCommitExplicitAsync. if retry, reason, extraMsg := IsEndTxnTriggeringRetryError(reply.Txn, args); retry { return result.Result{}, roachpb.NewTransactionRetryError(reason, extraMsg) } diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go index 36f282bc6e0e..4808c90ca221 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go @@ -93,8 +93,9 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { testCases := []struct { name string // Replica state. - existingTxn *roachpb.TransactionRecord - canCreateTxn func() (can bool, minTS hlc.Timestamp) + existingTxn *roachpb.TransactionRecord + canCreateTxn bool + minTxnCommitTS hlc.Timestamp // Request state. headerTxn *roachpb.Transaction commit bool @@ -110,8 +111,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { // there are intents to clean up. name: "record missing, try rollback", // Replica state. - existingTxn: nil, - canCreateTxn: nil, // not needed + existingTxn: nil, // Request state. headerTxn: headerTxn, commit: false, @@ -125,8 +125,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { // when all intents are on the transaction record's range. name: "record missing, try rollback without intents", // Replica state. - existingTxn: nil, - canCreateTxn: nil, // not needed + existingTxn: nil, // Request state. headerTxn: headerTxn, commit: false, @@ -143,7 +142,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { name: "record missing, can't create, try stage", // Replica state. existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return false, hlc.Timestamp{} }, + canCreateTxn: false, // Request state. headerTxn: headerTxn, commit: true, @@ -158,7 +157,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { name: "record missing, can't create, try commit", // Replica state. existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return false, hlc.Timestamp{} }, + canCreateTxn: false, // Request state. headerTxn: headerTxn, commit: true, @@ -171,7 +170,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { name: "record missing, can create, try stage", // Replica state. existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, hlc.Timestamp{} }, + canCreateTxn: true, // Request state. headerTxn: headerTxn, commit: true, @@ -185,7 +184,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { name: "record missing, can create, try commit", // Replica state. existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, hlc.Timestamp{} }, + canCreateTxn: true, // Request state. headerTxn: headerTxn, commit: true, @@ -198,7 +197,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { name: "record missing, can create, try stage without intents", // Replica state. existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, hlc.Timestamp{} }, + canCreateTxn: true, // Request state. headerTxn: headerTxn, commit: true, @@ -219,7 +218,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { name: "record missing, can create, try commit without intents", // Replica state. existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, hlc.Timestamp{} }, + canCreateTxn: true, // Request state. headerTxn: headerTxn, commit: true, @@ -236,7 +235,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { name: "record missing, can create, try stage at pushed timestamp", // Replica state. existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, hlc.Timestamp{} }, + canCreateTxn: true, // Request state. headerTxn: pushedHeaderTxn, commit: true, @@ -251,7 +250,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { name: "record missing, can create, try commit at pushed timestamp", // Replica state. existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, hlc.Timestamp{} }, + canCreateTxn: true, // Request state. headerTxn: pushedHeaderTxn, commit: true, @@ -265,7 +264,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { name: "record missing, can create, try stage at pushed timestamp after refresh", // Replica state. existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, hlc.Timestamp{} }, + canCreateTxn: true, // Request state. headerTxn: refreshedHeaderTxn, commit: true, @@ -284,7 +283,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { name: "record missing, can create, try commit at pushed timestamp after refresh", // Replica state. existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, hlc.Timestamp{} }, + canCreateTxn: true, // Request state. headerTxn: refreshedHeaderTxn, commit: true, @@ -297,11 +296,12 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { }, { // A PushTxn(TIMESTAMP) request bumped the minimum timestamp that the - // transaction can be created with. This will trigger a retry error. - name: "record missing, can create with min timestamp, try stage", + // transaction can be committed with. This will trigger a retry error. + name: "record missing, can commit with min timestamp, try stage", // Replica state. - existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, ts2 }, + existingTxn: nil, + canCreateTxn: true, + minTxnCommitTS: ts2, // Request state. headerTxn: headerTxn, commit: true, @@ -311,11 +311,12 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { }, { // A PushTxn(TIMESTAMP) request bumped the minimum timestamp that the - // transaction can be created with. This will trigger a retry error. - name: "record missing, can create with min timestamp, try commit", + // transaction can be committed with. This will trigger a retry error. + name: "record missing, can commit with min timestamp, try commit", // Replica state. - existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, ts2 }, + existingTxn: nil, + canCreateTxn: true, + minTxnCommitTS: ts2, // Request state. headerTxn: headerTxn, commit: true, @@ -324,12 +325,13 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { }, { // A PushTxn(TIMESTAMP) request bumped the minimum timestamp that - // the transaction can be created with. Luckily, the transaction has + // the transaction can be committed with. Luckily, the transaction has // already refreshed above this time, so it can avoid a retry error. - name: "record missing, can create with min timestamp, try stage at pushed timestamp after refresh", + name: "record missing, can commit with min timestamp, try stage at pushed timestamp after refresh", // Replica state. - existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, ts2 }, + existingTxn: nil, + canCreateTxn: true, + minTxnCommitTS: ts2, // Request state. headerTxn: refreshedHeaderTxn, commit: true, @@ -343,12 +345,13 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { }, { // A PushTxn(TIMESTAMP) request bumped the minimum timestamp that - // the transaction can be created with. Luckily, the transaction has + // the transaction can be committed with. Luckily, the transaction has // already refreshed above this time, so it can avoid a retry error. - name: "record missing, can create with min timestamp, try commit at pushed timestamp after refresh", + name: "record missing, can commit with min timestamp, try commit at pushed timestamp after refresh", // Replica state. - existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, ts2 }, + existingTxn: nil, + canCreateTxn: true, + minTxnCommitTS: ts2, // Request state. headerTxn: refreshedHeaderTxn, commit: true, @@ -365,7 +368,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { name: "record missing, can create, try stage after write too old", // Replica state. existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, hlc.Timestamp{} }, + canCreateTxn: true, // Request state. headerTxn: func() *roachpb.Transaction { clone := txn.Clone() @@ -383,7 +386,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { name: "record missing, can create, try commit after write too old", // Replica state. existingTxn: nil, - canCreateTxn: func() (bool, hlc.Timestamp) { return true, hlc.Timestamp{} }, + canCreateTxn: true, // Request state. headerTxn: func() *roachpb.Transaction { clone := txn.Clone() @@ -407,9 +410,8 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { expTxn: abortedRecord, }, { - // Standard case where a transaction record is created during a - // parallel commit. The record already exists because it has been - // heartbeated. + // Standard case where a transaction record is staged by a parallel + // commit. The record already exists because it has been heartbeated. name: "record pending, try stage", // Replica state. existingTxn: pendingRecord, @@ -421,9 +423,9 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { expTxn: stagingRecord, }, { - // Standard case where a transaction record is created during a - // non-parallel commit. The record already exists because it has - // been heartbeated. + // Standard case where a transaction record is committed during a + // non-parallel commit. The record already exists because it has been + // heartbeated. name: "record pending, try commit", // Replica state. existingTxn: pendingRecord, @@ -495,6 +497,74 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { return &record }(), }, + { + // A PushTxn(TIMESTAMP) request bumped the minimum timestamp that the + // transaction can be committed with. The record already exists because + // it has been heartbeated. This will trigger a retry error. + name: "record pending, can commit with min timestamp, try stage", + // Replica state. + existingTxn: pendingRecord, + minTxnCommitTS: ts2, + // Request state. + headerTxn: headerTxn, + commit: true, + inFlightWrites: writes, + // Expected result. + expError: "TransactionRetryError: retry txn (RETRY_SERIALIZABLE)", + }, + { + // A PushTxn(TIMESTAMP) request bumped the minimum timestamp that the + // transaction can be committed with. The record already exists because + // it has been heartbeated. This will trigger a retry error. + name: "record pending, can commit with min timestamp, try commit", + // Replica state. + existingTxn: pendingRecord, + minTxnCommitTS: ts2, + // Request state. + headerTxn: headerTxn, + commit: true, + // Expected result. + expError: "TransactionRetryError: retry txn (RETRY_SERIALIZABLE)", + }, + { + // A PushTxn(TIMESTAMP) request bumped the minimum timestamp that + // the transaction can be committed with. The record already exists + // because it has been heartbeated. Luckily, the transaction has + // already refreshed above this time, so it can avoid a retry error. + name: "record pending, can commit with min timestamp, try stage at pushed timestamp after refresh", + // Replica state. + existingTxn: pendingRecord, + minTxnCommitTS: ts2, + // Request state. + headerTxn: refreshedHeaderTxn, + commit: true, + inFlightWrites: writes, + // Expected result. + expTxn: func() *roachpb.TransactionRecord { + record := *stagingRecord + record.WriteTimestamp.Forward(ts2) + return &record + }(), + }, + { + // A PushTxn(TIMESTAMP) request bumped the minimum timestamp that + // the transaction can be committed with. The record already exists + // because it has been heartbeated. Luckily, the transaction has + // already refreshed above this time, so it can avoid a retry error. + name: "record pending, can commit with min timestamp, try commit at pushed timestamp after refresh", + // Replica state. + existingTxn: pendingRecord, + minTxnCommitTS: ts2, + // Request state. + headerTxn: refreshedHeaderTxn, + commit: true, + // Expected result. + expTxn: func() *roachpb.TransactionRecord { + record := *committedRecord + record.WriteTimestamp.Forward(ts2) + return &record + }(), + }, { // The transaction has run into a WriteTooOld error during its // lifetime. The stage will be rejected. @@ -547,7 +617,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { }(), }, { - // Standard case where a transaction record is created during a + // Standard case where a transaction record is staged during a // parallel commit after it has written a record at a lower epoch. // The existing record is upgraded. name: "record pending, try stage at higher epoch", @@ -566,7 +636,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { }(), }, { - // Standard case where a transaction record is created during a + // Standard case where a transaction record is committed during a // non-parallel commit after it has written a record at a lower // epoch. The existing record is upgraded. name: "record pending, try commit at higher epoch", @@ -628,7 +698,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { expError: "found txn in indeterminate STAGING state", }, { - // Standard case where a transaction record is created during a + // Standard case where a transaction record is re-staged during a // parallel commit. The record already exists because of a failed // parallel commit attempt. name: "record staging, try re-stage", @@ -642,9 +712,9 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { expTxn: stagingRecord, }, { - // Standard case where a transaction record is created during a - // non-parallel commit. The record already exists because of a - // failed parallel commit attempt. + // Standard case where a transaction record is explicitly committed. + // The record already exists and is staging because it was previously + // implicitly committed. name: "record staging, try commit", // Replica state. existingTxn: stagingRecord, @@ -655,8 +725,39 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { expTxn: committedRecord, }, { - // Non-standard case where a transaction record is created during a + // Standard case where a transaction record is re-staged during a // parallel commit. The record already exists because of a failed + // parallel commit attempt. The timestamp cache's minimum commit + // timestamp is ignored when the transaction record is staging. + name: "record staging, can commit with min timestamp, try re-stage", + // Replica state. + existingTxn: stagingRecord, + // Request state. + headerTxn: headerTxn, + minTxnCommitTS: ts2, + commit: true, + inFlightWrites: writes, + // Expected result. + expTxn: stagingRecord, + }, + { + // Non-standard case where a transaction record is explicitly committed. + // The record already exists and is staging because it was previously + // implicitly committed. The timestamp cache's minimum commit timestamp is + // ignored when the transaction record is staging. + name: "record staging, can commit with min timestamp, try commit", + // Replica state. + existingTxn: stagingRecord, + minTxnCommitTS: ts2, + // Request state. + headerTxn: headerTxn, + commit: true, + // Expected result. + expTxn: committedRecord, + }, + { + // Non-standard case where a transaction record is re-staged during + // a parallel commit. The record already exists because of a failed // parallel commit attempt. The re-stage will fail because of the // pushed timestamp. name: "record staging, try re-stage at pushed timestamp", @@ -670,7 +771,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { expError: "TransactionRetryError: retry txn (RETRY_SERIALIZABLE)", }, { - // Non-standard case where a transaction record is created during + // Non-standard case where a transaction record is committed during // a non-parallel commit. The record already exists because of a // failed parallel commit attempt. The commit will fail because of // the pushed timestamp. @@ -702,8 +803,8 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { }(), }, { - // Non-standard case where a transaction record is created during a - // parallel commit. The record already exists because of a failed + // Non-standard case where a transaction record is re-staged during + // a parallel commit. The record already exists because of a failed // parallel commit attempt in a prior epoch. name: "record staging, try re-stage at higher epoch", // Replica state. @@ -721,7 +822,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { }(), }, { - // Non-standard case where a transaction record is created during + // Non-standard case where a transaction record is committed during // a non-parallel commit. The record already exists because of a // failed parallel commit attempt in a prior epoch. name: "record staging, try commit at higher epoch", @@ -739,8 +840,8 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { }(), }, { - // Non-standard case where a transaction record is created during a - // parallel commit. The record already exists because of a failed + // Non-standard case where a transaction record is re-staged during + // a parallel commit. The record already exists because of a failed // parallel commit attempt in a prior epoch. The re-stage will fail // because of the pushed timestamp. name: "record staging, try re-stage at higher epoch and pushed timestamp", @@ -754,8 +855,8 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { expError: "TransactionRetryError: retry txn (RETRY_SERIALIZABLE)", }, { - // Non-standard case where a transaction record is created during a - // non-parallel commit. The record already exists because of a + // Non-standard case where a transaction record is committed during + // a non-parallel commit. The record already exists because of a // failed parallel commit attempt in a prior epoch. The commit will // fail because of the pushed timestamp. name: "record staging, try commit at higher epoch and pushed timestamp", @@ -936,12 +1037,14 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { EvalCtx: (&MockEvalCtx{ Desc: &desc, AbortSpan: as, - CanCreateTxn: func() (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) { - require.NotNil(t, c.canCreateTxn, "CanCreateTxnRecord unexpectedly called") - if can, minTS := c.canCreateTxn(); can { - return true, minTS, 0 + CanCreateTxnRecordFn: func() (bool, roachpb.TransactionAbortedReason) { + if c.canCreateTxn { + return true, 0 } - return false, hlc.Timestamp{}, roachpb.ABORT_REASON_ABORTED_RECORD_FOUND + return false, roachpb.ABORT_REASON_ABORTED_RECORD_FOUND + }, + MinTxnCommitTSFn: func() hlc.Timestamp { + return c.minTxnCommitTS }, }).EvalContext(), Args: &req, @@ -1047,9 +1150,6 @@ func TestPartialRollbackOnEndTransaction(t *testing.T) { if _, err := EndTxn(ctx, batch, CommandArgs{ EvalCtx: (&MockEvalCtx{ Desc: &desc, - CanCreateTxn: func() (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) { - return true, ts, 0 - }, }).EvalContext(), Args: &req, Header: roachpb.Header{ @@ -1162,9 +1262,6 @@ func TestCommitWaitBeforeIntentResolutionIfCommitTrigger(t *testing.T) { EvalCtx: (&MockEvalCtx{ Desc: &desc, Clock: clock, - CanCreateTxn: func() (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) { - return true, hlc.Timestamp{}, 0 - }, }).EvalContext(), Args: &req, Header: roachpb.Header{ diff --git a/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go b/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go index 52032f7fdfa6..434ea1233dec 100644 --- a/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go @@ -73,6 +73,17 @@ func HeartbeatTxn( } } + // If the transaction is pending, take the opportunity to determine the + // minimum timestamp that it will be allowed to commit at to account for any + // transaction pushes. This can help inform the transaction coordinator of + // pushes earlier than commit time, but is entirely best-effort. + // + // NOTE: we cannot do this if the transaction record is STAGING because it may + // already be implicitly committed. + if txn.Status == roachpb.PENDING { + BumpToMinTxnCommitTS(ctx, cArgs.EvalCtx, &txn) + } + if !txn.Status.IsFinalized() { // NOTE: this only updates the LastHeartbeat. It doesn't update any other // field from h.Txn, even if it could. Whether that's a good thing or not diff --git a/pkg/kv/kvserver/batcheval/eval_context.go b/pkg/kv/kvserver/batcheval/eval_context.go index 49799483353f..3831691a1a73 100644 --- a/pkg/kv/kvserver/batcheval/eval_context.go +++ b/pkg/kv/kvserver/batcheval/eval_context.go @@ -73,7 +73,12 @@ type EvalContext interface { // for details about its arguments, return values, and preconditions. CanCreateTxnRecord( ctx context.Context, txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp, - ) (ok bool, minCommitTS hlc.Timestamp, reason roachpb.TransactionAbortedReason) + ) (ok bool, reason roachpb.TransactionAbortedReason) + + // MinTxnCommitTS determines the minimum timestamp at which a transaction with + // the provided ID and key can commit. See Replica.MinTxnCommitTS for details + // about its arguments, return values, and preconditions. + MinTxnCommitTS(ctx context.Context, txnID uuid.UUID, txnKey []byte) hlc.Timestamp // GetMVCCStats returns a snapshot of the MVCC stats for the range. // If called from a command that declares a read/write span on the @@ -157,23 +162,24 @@ type ImmutableEvalContext interface { // MockEvalCtx is a dummy implementation of EvalContext for testing purposes. // For technical reasons, the interface is implemented by a wrapper .EvalContext(). type MockEvalCtx struct { - ClusterSettings *cluster.Settings - Desc *roachpb.RangeDescriptor - StoreID roachpb.StoreID - NodeID roachpb.NodeID - Clock *hlc.Clock - Stats enginepb.MVCCStats - QPS float64 - AbortSpan *abortspan.AbortSpan - GCThreshold hlc.Timestamp - Term, FirstIndex uint64 - CanCreateTxn func() (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) - Lease roachpb.Lease - CurrentReadSummary rspb.ReadSummary - ClosedTimestamp hlc.Timestamp - RevokedLeaseSeq roachpb.LeaseSequence - MaxBytes int64 - ApproxDiskBytes uint64 + ClusterSettings *cluster.Settings + Desc *roachpb.RangeDescriptor + StoreID roachpb.StoreID + NodeID roachpb.NodeID + Clock *hlc.Clock + Stats enginepb.MVCCStats + QPS float64 + AbortSpan *abortspan.AbortSpan + GCThreshold hlc.Timestamp + Term, FirstIndex uint64 + CanCreateTxnRecordFn func() (bool, roachpb.TransactionAbortedReason) + MinTxnCommitTSFn func() hlc.Timestamp + Lease roachpb.Lease + CurrentReadSummary rspb.ReadSummary + ClosedTimestamp hlc.Timestamp + RevokedLeaseSeq roachpb.LeaseSequence + MaxBytes int64 + ApproxDiskBytes uint64 } // EvalContext returns the MockEvalCtx as an EvalContext. It will reflect future @@ -247,8 +253,19 @@ func (m *mockEvalCtxImpl) GetLastSplitQPS(context.Context) float64 { } func (m *mockEvalCtxImpl) CanCreateTxnRecord( context.Context, uuid.UUID, []byte, hlc.Timestamp, -) (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) { - return m.CanCreateTxn() +) (bool, roachpb.TransactionAbortedReason) { + if m.CanCreateTxnRecordFn == nil { + return true, 0 + } + return m.CanCreateTxnRecordFn() +} +func (m *mockEvalCtxImpl) MinTxnCommitTS( + ctx context.Context, txnID uuid.UUID, txnKey []byte, +) hlc.Timestamp { + if m.MinTxnCommitTSFn == nil { + return hlc.Timestamp{} + } + return m.MinTxnCommitTSFn() } func (m *mockEvalCtxImpl) GetGCThreshold() hlc.Timestamp { return m.GCThreshold diff --git a/pkg/kv/kvserver/batcheval/transaction.go b/pkg/kv/kvserver/batcheval/transaction.go index a4ce70367dfe..b300843165c0 100644 --- a/pkg/kv/kvserver/batcheval/transaction.go +++ b/pkg/kv/kvserver/batcheval/transaction.go @@ -119,20 +119,29 @@ func UpdateAbortSpan( } // CanCreateTxnRecord determines whether a transaction record can be created for -// the provided transaction. If not, the function will return an error. If so, -// the function may modify the provided transaction. +// the provided transaction. If not, the function will return an error. func CanCreateTxnRecord(ctx context.Context, rec EvalContext, txn *roachpb.Transaction) error { // The transaction could not have written a transaction record previously // with a timestamp below txn.MinTimestamp. - ok, minCommitTS, reason := rec.CanCreateTxnRecord(ctx, txn.ID, txn.Key, txn.MinTimestamp) + ok, reason := rec.CanCreateTxnRecord(ctx, txn.ID, txn.Key, txn.MinTimestamp) if !ok { log.VEventf(ctx, 2, "txn tombstone present; transaction has been aborted") return roachpb.NewTransactionAbortedError(reason) } + return nil +} + +// BumpToMinTxnCommitTS increases the provided transaction's write timestamp to +// the minimum timestamp at which it is allowed to commit. The transaction must +// be PENDING. +func BumpToMinTxnCommitTS(ctx context.Context, rec EvalContext, txn *roachpb.Transaction) { + if txn.Status != roachpb.PENDING { + log.Fatalf(ctx, "non-pending txn passed to BumpToMinTxnCommitTS: %v", txn) + } + minCommitTS := rec.MinTxnCommitTS(ctx, txn.ID, txn.Key) if bumped := txn.WriteTimestamp.Forward(minCommitTS); bumped { log.VEventf(ctx, 2, "write timestamp bumped by txn tombstone to: %s", txn.WriteTimestamp) } - return nil } // SynthesizeTxnFromMeta creates a synthetic transaction object from @@ -178,10 +187,11 @@ func SynthesizeTxnFromMeta( // Determine whether the record could ever be allowed to be written in the // future. The transaction could not have written a transaction record // previously with a timestamp below txn.MinTimestamp. - ok, minCommitTS, _ := rec.CanCreateTxnRecord(ctx, txn.ID, txn.Key, txn.MinTimestamp) + ok, _ := rec.CanCreateTxnRecord(ctx, txn.ID, txn.Key, txn.MinTimestamp) if ok { // Forward the provisional commit timestamp by the minimum timestamp that - // the transaction would be able to create a transaction record at. + // the transaction would be able to commit at. + minCommitTS := rec.MinTxnCommitTS(ctx, txn.ID, txn.Key) synth.WriteTimestamp.Forward(minCommitTS) } else { // Mark the transaction as ABORTED because it is uncommittable. diff --git a/pkg/kv/kvserver/batcheval/transaction_test.go b/pkg/kv/kvserver/batcheval/transaction_test.go index 2b63eab6f401..be32ce54636f 100644 --- a/pkg/kv/kvserver/batcheval/transaction_test.go +++ b/pkg/kv/kvserver/batcheval/transaction_test.go @@ -739,9 +739,6 @@ func TestUpdateAbortSpan(t *testing.T) { ClusterSettings: st, Desc: &desc, AbortSpan: as, - CanCreateTxn: func() (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) { - return true, hlc.Timestamp{}, 0 - }, } ms := enginepb.MVCCStats{} if c.before != nil { diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go index ea103a9ec97f..a2525072e35b 100644 --- a/pkg/kv/kvserver/replica_eval_context_span.go +++ b/pkg/kv/kvserver/replica_eval_context_span.go @@ -146,13 +146,25 @@ func (rec SpanSetReplicaEvalContext) GetLastSplitQPS(ctx context.Context) float6 // for details about its arguments, return values, and preconditions. func (rec SpanSetReplicaEvalContext) CanCreateTxnRecord( ctx context.Context, txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp, -) (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) { +) (bool, roachpb.TransactionAbortedReason) { rec.ss.AssertAllowed(spanset.SpanReadOnly, roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)}, ) return rec.i.CanCreateTxnRecord(ctx, txnID, txnKey, txnMinTS) } +// MinTxnCommitTS determines the minimum timestamp at which a transaction with +// the provided ID and key can commit. See Replica.MinTxnCommitTS for details +// about its arguments, return values, and preconditions. +func (rec SpanSetReplicaEvalContext) MinTxnCommitTS( + ctx context.Context, txnID uuid.UUID, txnKey []byte, +) hlc.Timestamp { + rec.ss.AssertAllowed(spanset.SpanReadOnly, + roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)}, + ) + return rec.i.MinTxnCommitTS(ctx, txnID, txnKey) +} + // GetGCThreshold returns the GC threshold of the Range, typically updated when // keys are garbage collected. Reads and writes at timestamps <= this time will // not be served. diff --git a/pkg/kv/kvserver/replica_tscache.go b/pkg/kv/kvserver/replica_tscache.go index 49014a5a5789..9023ad570e55 100644 --- a/pkg/kv/kvserver/replica_tscache.go +++ b/pkg/kv/kvserver/replica_tscache.go @@ -390,11 +390,9 @@ func (r *Replica) applyTimestampCache( // the provided transaction information. Callers must provide the transaction's // minimum timestamp across all epochs, along with its ID and its key. // -// If the method return true, it also returns the minimum provisional commit -// timestamp that the record can be created with. If the method returns false, -// it returns the reason that transaction record was rejected. If the method -// ever determines that a transaction record must be rejected, it will continue -// to reject that transaction going forwards. +// If the method returns false, it returns the reason that transaction record +// was rejected. If the method ever determines that a transaction record must be +// rejected, it will continue to reject that transaction going forwards. // // The method performs two critical roles: // @@ -403,14 +401,17 @@ func (r *Replica) applyTimestampCache( // to be created after the transaction has already been finalized and its // record cleaned up. // -// 2. It serves as the mechanism by which successful push requests convey -// information to transactions who have not yet written their transaction -// record. In doing so, it ensures that transaction records are created -// with a sufficiently high timestamp after a successful PushTxn(TIMESTAMP) -// and ensures that transactions records are never created at all after a -// successful PushTxn(ABORT). As a result of this mechanism, a transaction -// never needs to explicitly create the transaction record for contending -// transactions. +// 2. It serves as the mechanism through which successful PushTxn(ABORT) +// requests convey information to pushee transactions who have not yet +// written their transaction record. In doing so, it ensures that +// transactions records are never created for the pushee after a successful +// PushTxn(ABORT). As a result of this mechanism, a pusher transaction never +// needs to create the transaction record for contending transactions that +// it forms write-write conflicts with. +// +// NOTE: write-read conflicts result in PushTxn(TIMESTAMP) requests, which +// coordinate with pushee transactions through the MinTxnCommitTS mechanism +// (see below). // // In addition, it is used when considering 1PC evaluation, to avoid checking // for a transaction record on disk. @@ -431,10 +432,10 @@ func (r *Replica) applyTimestampCache( // HeartbeatTxn // PushTxn(TIMESTAMP) then: update record // then: v1 -> push.ts v2 -> txn.ts -// +------+ HeartbeatTxn +------+ -// PushTxn(ABORT) | | if: v2 < txn.orig | | PushTxn(TIMESTAMP) -// then: v2 -> txn.ts | v then: txn.ts -> v1 | v then: update record -// +-----------------+ v2 -> txn.ts +--------------------+ +// +------+ +------+ +// PushTxn(ABORT) | | HeartbeatTxn | | PushTxn(TIMESTAMP) +// then: v2 -> txn.ts | v if: v2 < txn.orig | v then: v1 -> push.ts +// +-----------------+ then: v2 -> txn.ts +--------------------+ // +----| | else: fail | |----+ // | | |------------------------->| | | // | | no txn record | | txn record written | | @@ -442,22 +443,23 @@ func (r *Replica) applyTimestampCache( // | |__ if: v2 < txn.orig | | // +-----------------+ \__ then: txn.ts -> v1 +--------------------+ // | ^ \__ else: fail _/ | ^ -// | | \__ _/ | | -// EndTxn(!STAGING) | | \__ _/ | EndTxn(STAGING) -// if: v2 < txn.orig | Eager GC | \____ _/______ | | -// then: v2 -> txn.ts | or | _/ \ | | HeartbeatTxn +// EndTxn(!STAGING) | | \__ _/ | EndTxn(STAGING) +// if: v2 < txn.orig | | \__ _/ | then: txn.ts -> v1 +// then: txn.ts -> v1 | Eager GC | \____ _/______ | | +// v2 -> txn.ts | or | _/ \ | | HeartbeatTxn // else: fail | GC queue | /----------------/ | | | if: epoch update // v | v EndTxn(!STAGING) v v | // +--------------------+ or PushTxn(ABORT) +--------------------+ -// | | then: v2 -> txn.ts | | -// +--->| |<-----------------------| |----+ +// | | then: txn.ts -> v1 | | +// | | v2 -> txn.ts | | +// +--->| | | |----+ // | | txn record written | | txn record written | | // | | [finalized] | | [staging] | | // +----| | | |<---+ -// PushTxn(*) +--------------------+ +--------------------+ -// then: no-op ^ PushTxn(*) + RecoverTxn | EndTxn(STAGING) -// | then: v2 -> txn.ts | or HeartbeatTxn -// +------------------------------+ then: update record +// PushTxn(*) +--------------------+ EndTxn(!STAGING) +--------------------+ +// then: no-op ^ or PushTxn(*) + RecoverTxn | EndTxn(STAGING) +// | then: v2 -> txn.ts | or HeartbeatTxn +// +----------------------------------+ then: update record // // In the diagram, CanCreateTxnRecord is consulted in all three of the // state transitions that move away from the "no txn record" state. @@ -518,27 +520,17 @@ func (r *Replica) applyTimestampCache( // system. func (r *Replica) CanCreateTxnRecord( ctx context.Context, txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp, -) (ok bool, minCommitTS hlc.Timestamp, reason roachpb.TransactionAbortedReason) { - // Consult the timestamp cache with the transaction's key. The timestamp - // cache is used in two ways for transactions without transaction records. - // The timestamp cache is used to push the timestamp of transactions - // that don't have transaction records. The timestamp cache is used - // to abort transactions entirely that don't have transaction records. +) (ok bool, reason roachpb.TransactionAbortedReason) { + // Consult the timestamp cache with the transaction's key. The timestamp cache + // is used to abort transactions that don't have transaction records. // // Using this strategy, we enforce the invariant that only requests sent // from a transaction's own coordinator can create its transaction record. // However, once a transaction record is written, other concurrent actors // can modify it. This is reflected in the diagram above. tombstoneKey := transactionTombstoneMarker(txnKey, txnID) - pushKey := transactionPushMarker(txnKey, txnID) - - // Look in the timestamp cache to see if there is an entry for this - // transaction, which indicates the minimum timestamp that the transaction - // can commit at. This is used by pushers to push the timestamp of a - // transaction that hasn't yet written its transaction record. - minCommitTS, _ = r.store.tsCache.GetMax(pushKey, nil /* end */) - // Also look in the timestamp cache to see if there is a tombstone entry for + // Look in the timestamp cache to see if there is a tombstone entry for // this transaction, which indicates that this transaction has already written // a transaction record. If there is an entry, then we return a retriable // error: if this is a re-evaluation, then the error will be transformed into @@ -561,26 +553,54 @@ func (r *Replica) CanCreateTxnRecord( // If there were other requests in the EndTxn batch, then the client would // still have trouble reconstructing the result, but at least it could // provide a non-ambiguous error to the application. - return false, hlc.Timestamp{}, - roachpb.ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY + return false, roachpb.ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY case uuid.Nil: lease, _ /* nextLease */ := r.GetLease() // Recognize the case where a lease started recently. Lease transfers bump // the ts cache low water mark. if tombstoneTimestamp == lease.Start.ToTimestamp() { - return false, hlc.Timestamp{}, roachpb.ABORT_REASON_NEW_LEASE_PREVENTS_TXN + return false, roachpb.ABORT_REASON_NEW_LEASE_PREVENTS_TXN } - return false, hlc.Timestamp{}, roachpb.ABORT_REASON_TIMESTAMP_CACHE_REJECTED + return false, roachpb.ABORT_REASON_TIMESTAMP_CACHE_REJECTED default: // If we find another transaction's ID then that transaction has // aborted us before our transaction record was written. It obeyed // the restriction that it couldn't create a transaction record for // us, so it recorded a tombstone cache instead to prevent us // from ever creating a transaction record. - return false, hlc.Timestamp{}, roachpb.ABORT_REASON_ABORTED_RECORD_FOUND + return false, roachpb.ABORT_REASON_ABORTED_RECORD_FOUND } } - return true, minCommitTS, 0 + + return true, 0 +} + +// MinTxnCommitTS determines the minimum timestamp at which a transaction with +// the provided ID and key can commit. +// +// The method serves as a mechanism through which successful PushTxn(TIMESTAMP) +// requests convey information to pushee transactions. In doing so, it ensures +// that transaction records are committed with a sufficiently high timestamp +// after a successful PushTxn(TIMESTAMP). As a result of this mechanism, a +// transaction never needs to write to the transaction record for contending +// transactions that it forms write-read conflicts with. +// +// NOTE: write-write conflicts result in PushTxn(ABORT) requests, which +// coordinate with pushee transactions through the CanCreateTxnRecord mechanism +// (see above). +// +// The mechanism is detailed in the transaction record state machine above on +// CanCreateTxnRecord. +func (r *Replica) MinTxnCommitTS( + ctx context.Context, txnID uuid.UUID, txnKey []byte, +) hlc.Timestamp { + // Look in the timestamp cache to see if there is a push marker entry for this + // transaction, which contains the minimum timestamp that the transaction can + // commit at. This is used by pushers to push the timestamp of a transaction + // without writing to the pushee's transaction record. + pushKey := transactionPushMarker(txnKey, txnID) + minCommitTS, _ := r.store.tsCache.GetMax(pushKey, nil /* end */) + return minCommitTS } // Pseudo range local key suffixes used to construct "marker" keys for use diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index ad969cd1d0e8..b975b7b439bb 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -357,18 +357,19 @@ func (r *Replica) canAttempt1PCEvaluation( ba.Timestamp, ba.Txn.WriteTimestamp) } - // The EndTxn checks whether the txn record can be created, but we're - // eliding the EndTxn. So, we'll do the check instead. + // The EndTxn checks whether the txn record can be created and, if so, at what + // timestamp. We're eliding the EndTxn, so, we'll do the checks instead. // // Note that the returned reason does not distinguish between an existing // record (which should fall back to non-1PC EndTxn evaluation) and a // finalized record (which should return an error), so we ignore it here and // let EndTxn return an error as appropriate. This lets us avoid a disk read // to check for an existing record. - ok, minCommitTS, _ := r.CanCreateTxnRecord(ctx, ba.Txn.ID, ba.Txn.Key, ba.Txn.MinTimestamp) + ok, _ := r.CanCreateTxnRecord(ctx, ba.Txn.ID, ba.Txn.Key, ba.Txn.MinTimestamp) if !ok { return false } + minCommitTS := r.MinTxnCommitTS(ctx, ba.Txn.ID, ba.Txn.Key) if ba.Timestamp.Less(minCommitTS) { ba.Txn.WriteTimestamp = minCommitTS // We can only evaluate at the new timestamp if we manage to bump the read From 2abca5cb42c6d76c72a479a81fcec3071b045ce6 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 11 Jan 2023 01:35:08 -0500 Subject: [PATCH 7/9] kv: always promote PUSH_TIMESTAMP to PUSH_ABORT on failed staging record This commit simplifies logic in PushTxnRequest that promoted a PUSH_TIMESTAMP to a PUSH_ABORT when it found a STAGING transaction record which it knew to be part of a failed parallel commit attempt. The logic was trying to be smart and minimize the cases where it needed to promote a PUSH_TIMESTAMP to a PUSH_ABORT. It was avoiding doing so if it had previously found an intent with a higher epoch. In practice, this optimization doesn't seem to matter. It was also making logic in a following commit harder to write because it was preserving cases where a PUSH_TIMESTAMP would succeed against a STAGING transaction record. We don't want to support such state transitions, so eliminate them. --- pkg/kv/kvserver/batcheval/cmd_push_txn.go | 20 ++++++++++--------- pkg/kv/kvserver/replica_test.go | 4 ++-- .../kvserver/txn_recovery_integration_test.go | 20 +++---------------- 3 files changed, 16 insertions(+), 28 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_push_txn.go b/pkg/kv/kvserver/batcheval/cmd_push_txn.go index cc99f4c72281..8b3d7c11bd70 100644 --- a/pkg/kv/kvserver/batcheval/cmd_push_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_push_txn.go @@ -219,14 +219,12 @@ func PushTxn( reply.PusheeTxn.Status = roachpb.PENDING reply.PusheeTxn.InFlightWrites = nil // If the pusher is aware that the pushee's currently recorded attempt - // at a parallel commit failed but the transaction's epoch has not yet - // been incremented, upgrade PUSH_TIMESTAMPs to PUSH_ABORTs. We don't - // want to move the transaction back to PENDING in the same epoch, as - // this is not (currently) allowed by the recovery protocol. We also - // don't want to move the transaction to a new timestamp while retaining - // the STAGING status, as this could allow the transaction to enter an - // implicit commit state without its knowledge, leading to atomicity - // violations. + // at a parallel commit failed, upgrade PUSH_TIMESTAMPs to PUSH_ABORTs. + // We don't want to move the transaction back to PENDING, as this is not + // (currently) allowed by the recovery protocol. We also don't want to + // move the transaction to a new timestamp while retaining the STAGING + // status, as this could allow the transaction to enter an implicit + // commit state without its knowledge, leading to atomicity violations. // // This has no effect on pushes that fail with a TransactionPushError. // Such pushes will still wait on the pushee to retry its commit and @@ -236,7 +234,7 @@ func PushTxn( // cases, the push acts the same as a short-circuited transaction // recovery process, because the transaction recovery procedure always // finalizes target transactions, even if initiated by a PUSH_TIMESTAMP. - if !knownHigherEpoch && pushType == roachpb.PUSH_TIMESTAMP { + if pushType == roachpb.PUSH_TIMESTAMP { pushType = roachpb.PUSH_ABORT } } @@ -306,6 +304,10 @@ func PushTxn( reply.PusheeTxn.WriteTimestamp.Forward(reply.PusheeTxn.LastActive()) } case roachpb.PUSH_TIMESTAMP: + if existTxn.Status != roachpb.PENDING { + return result.Result{}, errors.AssertionFailedf( + "PUSH_TIMESTAMP succeeded against non-PENDING txn: %v", existTxn) + } // Otherwise, update timestamp to be one greater than the request's // timestamp. This new timestamp will be use to update the read timestamp // cache. If the transaction record was not already present then we rely on diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index bdd2cf0d3557..7ce2cd5c1ad7 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -12290,9 +12290,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(roachpb.Header{}, &pt) }, expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { - record := txn.AsRecord() + record := txnWithStatus(roachpb.ABORTED)(txn, pushTs) record.Epoch = txn.Epoch + 1 - record.WriteTimestamp.Forward(pushTs) + record.WriteTimestamp = record.WriteTimestamp.Add(0, 1) record.Priority = pusher.Priority - 1 return record }, diff --git a/pkg/kv/kvserver/txn_recovery_integration_test.go b/pkg/kv/kvserver/txn_recovery_integration_test.go index 5bd3a3154ada..a4bba0cd7dbf 100644 --- a/pkg/kv/kvserver/txn_recovery_integration_test.go +++ b/pkg/kv/kvserver/txn_recovery_integration_test.go @@ -296,23 +296,9 @@ func TestTxnRecoveryFromStagingWithHighPriority(t *testing.T) { require.Equal(t, roachpb.COMMITTED, qtTxn.Status) require.Equal(t, txn.Epoch, qtTxn.Epoch) require.Equal(t, txn.WriteTimestamp, qtTxn.WriteTimestamp) - } else if newEpoch { - // The transaction is aborted if that's what the high-priority - // request wants. Otherwise, the transaction's record is bumped to - // the new epoch pulled from its intent and pushed above the - // high-priority request's timestamp. - if pushAbort { - require.Equal(t, roachpb.ABORTED, qtTxn.Status) - } else /* pushTimestamp */ { - require.Equal(t, roachpb.PENDING, qtTxn.Status) - require.Equal(t, txn.Epoch+1, qtTxn.Epoch) - require.Equal(t, conflictH.Timestamp.Next(), qtTxn.WriteTimestamp) - } - } else /* if newTimestamp */ { - // The transaction is aborted, even if the high-priority request - // only needed it to be pushed to a higher timestamp. This is - // because we don't allow a STAGING transaction record to move back - // to PENDING in the same epoch. + } else { + // The pusher knows that the transaction is not implicitly committed, so + // it is aborted. require.Equal(t, roachpb.ABORTED, qtTxn.Status) } } From 3dbb321e1e21aa0746e51c48569e54fa0edea19c Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 11 Jan 2023 01:48:04 -0500 Subject: [PATCH 8/9] kv: don't rewrite txn record on PushTxn(TIMESTAMP) Fixes #94728. With the previous commit, transactions will check the timestamp cache before committing to determine whether they have had their commit timestamp pushed. This commit exploits this to avoid ever rewriting a transaction's record on a timestamp push. Instead, the timestamp cache is used, regardless of whether the record already existed or not. Doing so avoids consensus. --- pkg/kv/kvserver/batcheval/cmd_push_txn.go | 50 +++++++++++++---------- pkg/kv/kvserver/replica_test.go | 11 +++-- 2 files changed, 34 insertions(+), 27 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_push_txn.go b/pkg/kv/kvserver/batcheval/cmd_push_txn.go index 8b3d7c11bd70..3fec1fc7bca4 100644 --- a/pkg/kv/kvserver/batcheval/cmd_push_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_push_txn.go @@ -15,6 +15,7 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -298,10 +299,21 @@ func PushTxn( case roachpb.PUSH_ABORT: // If aborting the transaction, set the new status. reply.PusheeTxn.Status = roachpb.ABORTED - // If the transaction record was already present, forward the timestamp - // to accommodate AbortSpan GC. See method comment for details. + // Forward the timestamp to accommodate AbortSpan GC. See method comment for + // details. + reply.PusheeTxn.WriteTimestamp.Forward(reply.PusheeTxn.LastActive()) + // If the transaction record was already present, persist the updates to it. + // If not, then we don't want to create it. This could allow for finalized + // transactions to be revived. Instead, we obey the invariant that only the + // transaction's own coordinator can issue requests that create its + // transaction record. To ensure that a timestamp push or an abort is + // respected for transactions without transaction records, we rely on markers + // in the timestamp cache. if ok { - reply.PusheeTxn.WriteTimestamp.Forward(reply.PusheeTxn.LastActive()) + txnRecord := reply.PusheeTxn.AsRecord() + if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { + return result.Result{}, err + } } case roachpb.PUSH_TIMESTAMP: if existTxn.Status != roachpb.PENDING { @@ -309,29 +321,25 @@ func PushTxn( "PUSH_TIMESTAMP succeeded against non-PENDING txn: %v", existTxn) } // Otherwise, update timestamp to be one greater than the request's - // timestamp. This new timestamp will be use to update the read timestamp - // cache. If the transaction record was not already present then we rely on - // the timestamp cache to prevent the record from ever being written with a - // timestamp beneath this timestamp. + // timestamp. This new timestamp will be used to update the read timestamp + // cache. We rely on the timestamp cache to prevent the record from ever + // being committed with a timestamp beneath this timestamp. reply.PusheeTxn.WriteTimestamp.Forward(args.PushTo) + // If the transaction record was already present, continue to update the + // transaction record until all nodes are running v23.1. v22.2 nodes won't + // know to check the timestamp cache again on commit to learn about any + // successful timestamp pushes. + // TODO(nvanbenschoten): remove this logic in v23.2. + if ok && !cArgs.EvalCtx.ClusterSettings().Version.IsActive(ctx, clusterversion.V23_1) { + txnRecord := reply.PusheeTxn.AsRecord() + if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { + return result.Result{}, err + } + } default: return result.Result{}, errors.AssertionFailedf("unexpected push type: %v", pushType) } - // If the transaction record was already present, persist the updates to it. - // If not, then we don't want to create it. This could allow for finalized - // transactions to be revived. Instead, we obey the invariant that only the - // transaction's own coordinator can issue requests that create its - // transaction record. To ensure that a timestamp push or an abort is - // respected for transactions without transaction records, we rely on markers - // in the timestamp cache. - if ok { - txnRecord := reply.PusheeTxn.AsRecord() - if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { - return result.Result{}, err - } - } - result := result.Result{} result.Local.UpdatedTxns = []*roachpb.Transaction{&reply.PusheeTxn} return result, nil diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 7ce2cd5c1ad7..5cc5370cc04f 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -11991,12 +11991,11 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { pt.PushTo = now return sendWrappedWithErr(roachpb.Header{}, &pt) }, - expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { - record := txn.AsRecord() - record.WriteTimestamp.Forward(pushTs) - record.Priority = pusher.Priority - 1 - return record - }, + // The transaction record **is not** updated in this case. Instead, the + // push is communicated through the timestamp cache. When the pushee goes + // to commit, it will consult the timestamp cache and find that it must + // commit above the push timestamp. + expTxn: txnWithoutChanges, }, { name: "push transaction (abort) after heartbeat transaction", From d7a6974f276e3bdb26109524dd3bc2c136adb68c Mon Sep 17 00:00:00 2001 From: adityamaru Date: Tue, 31 Jan 2023 15:25:42 -0500 Subject: [PATCH 9/9] backupccl: add missing context cancel checks to restore In #95257 we saw a restore grind to a halt 2 hours into a 5 hour roachtest. The stacks indicated that we may have seen a context cancellation that was not being respected by the goroutine running `generateAndSendImportSpans`. This resulted in the `generative_split_and_scatter_processor` getting stuck writing to a channel nobody was reading from (https://github.com/cockroachdb/cockroach/blob/master/pkg/ccl/backupccl/restore_span_covering.go#L516) since the other goroutines in the processor had seen the ctx cancellation and exited. A side effect of the generative processor not shutting down was that the downstream restore data processors would also hang on their call to `input.Next()` as they would not receive a row or a meta from the generative processor signalling them to shutdown. This fix adds a ctx cancellation check to the goroutine described above, thereby allowing a graceful teardown of the flow. This fix also adds the JobID to the generative processor spec so that logs on remote nodes are correctly tagged with the JobID making for easier debugging. Informs: #95257 Release note (bug fix): fixes a bug where a restore flow could hang indefinitely in the face of a context cancellation, manifesting as a stuck restore job. --- pkg/ccl/backupccl/restore_data_processor.go | 2 +- pkg/ccl/backupccl/restore_job.go | 14 ++++++++++---- pkg/ccl/backupccl/restore_processor_planning.go | 1 + pkg/ccl/backupccl/restore_span_covering.go | 17 ++++++++++++----- 4 files changed, 24 insertions(+), 10 deletions(-) diff --git a/pkg/ccl/backupccl/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go index a099d68c7a07..9cb148ad4dd1 100644 --- a/pkg/ccl/backupccl/restore_data_processor.go +++ b/pkg/ccl/backupccl/restore_data_processor.go @@ -172,7 +172,7 @@ func (rd *restoreDataProcessor) Start(ctx context.Context) { _ = rd.phaseGroup.Wait() } rd.phaseGroup = ctxgroup.WithContext(ctx) - log.Infof(ctx, "starting restore data") + log.Infof(ctx, "starting restore data processor") entries := make(chan execinfrapb.RestoreSpanEntry, rd.numWorkers) rd.sstCh = make(chan mergedSST, rd.numWorkers) diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 9188fe697186..8aebb9576135 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -318,7 +318,7 @@ func restore( genSpan := func(ctx context.Context, spanCh chan execinfrapb.RestoreSpanEntry) error { defer close(spanCh) return generateAndSendImportSpans( - restoreCtx, + ctx, dataToRestore.getSpans(), backupManifests, layerToBackupManifestFileIterFactory, @@ -334,7 +334,6 @@ func restore( // Count number of import spans. var numImportSpans int var countTasks []func(ctx context.Context) error - log.Infof(restoreCtx, "rh_debug: starting count task") spanCountTask := func(ctx context.Context) error { for range countSpansCh { numImportSpans++ @@ -397,7 +396,12 @@ func restore( if idx >= mu.ceiling { for i := mu.ceiling; i <= idx; i++ { - importSpan := <-importSpanCh + importSpan, ok := <-importSpanCh + if !ok { + // The channel has been closed, there is nothing left to do. + log.Infof(ctx, "exiting restore checkpoint loop as the import span channel has been closed") + return nil + } mu.inFlightImportSpans[i] = importSpan.Span } mu.ceiling = idx + 1 @@ -416,7 +420,6 @@ func restore( for j := mu.highWaterMark + 1; j < mu.ceiling && mu.requestsCompleted[j]; j++ { mu.highWaterMark = j } - for j := prevHighWater; j < mu.highWaterMark; j++ { delete(mu.requestsCompleted, j) delete(mu.inFlightImportSpans, j) @@ -1714,6 +1717,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro return err } } + log.Infof(ctx, "finished restoring the pre-data bundle") } if !preValidateData.isEmpty() { @@ -1734,6 +1738,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro } resTotal.Add(res) + log.Infof(ctx, "finished restoring the validate data bundle") } { // Restore the main data bundle. We notably only restore the system tables @@ -1755,6 +1760,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro } resTotal.Add(res) + log.Infof(ctx, "finished restoring the main data bundle") } if err := insertStats(ctx, r.job, p.ExecCfg(), remappedStats); err != nil { diff --git a/pkg/ccl/backupccl/restore_processor_planning.go b/pkg/ccl/backupccl/restore_processor_planning.go index 08ac4f069f45..fd68d1d5f477 100644 --- a/pkg/ccl/backupccl/restore_processor_planning.go +++ b/pkg/ccl/backupccl/restore_processor_planning.go @@ -188,6 +188,7 @@ func distRestore( NumEntries: int64(numImportSpans), NumNodes: int64(numNodes), UseSimpleImportSpans: useSimpleImportSpans, + JobID: jobID, } proc := physicalplan.Processor{ diff --git a/pkg/ccl/backupccl/restore_span_covering.go b/pkg/ccl/backupccl/restore_span_covering.go index b4abf18fa247..3d451eec28c4 100644 --- a/pkg/ccl/backupccl/restore_span_covering.go +++ b/pkg/ccl/backupccl/restore_span_covering.go @@ -497,7 +497,7 @@ func generateAndSendImportSpans( var covFilesByLayer [][]backuppb.BackupManifest_File var firstInSpan bool - flush := func() { + flush := func(ctx context.Context) error { entry := execinfrapb.RestoreSpanEntry{ Span: lastCovSpan, } @@ -513,8 +513,14 @@ func generateAndSendImportSpans( } if len(entry.Files) > 0 { - spanCh <- entry + select { + case <-ctx.Done(): + return ctx.Err() + case spanCh <- entry: + } } + + return nil } for _, span := range requiredSpans { @@ -630,7 +636,9 @@ func generateAndSendImportSpans( lastCovSpan.EndKey = coverSpan.EndKey lastCovSpanSize = lastCovSpanSize + newCovFilesSize } else { - flush() + if err := flush(ctx); err != nil { + return err + } lastCovSpan = coverSpan covFilesByLayer = filesByLayer lastCovSpanSize = covSize @@ -646,8 +654,7 @@ func generateAndSendImportSpans( } } - flush() - return nil + return flush(ctx) } // fileSpanStartAndEndKeyIterator yields (almost) all of the start and end keys