From 7c8accd255ab23b53d1a2e7a4443959b45fcba48 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Sun, 10 Jul 2022 12:11:55 -0400 Subject: [PATCH 1/9] sql/schemachanger/scbuild: rework ADD COLUMN in support of DROP COLUMN This refactors the code a tad and extracts some helpers. Release note: None --- .../scbuildstmt/alter_table_add_column.go | 233 ++++++++++-------- .../scbuild/internal/scbuildstmt/helpers.go | 36 +++ 2 files changed, 172 insertions(+), 97 deletions(-) diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go index a7fce2d000ec..1f7abc1b2c9d 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go @@ -300,68 +300,13 @@ func addColumn(b BuildCtx, spec addColumnSpec) (backing *scpb.PrimaryIndex) { } // Check whether a target to add a new primary index already exists. If so, // simply add the new column to its storing columns. - var existing, freshlyAdded *scpb.PrimaryIndex - allTargets := b.QueryByID(spec.tbl.TableID) - publicTargets := allTargets.Filter( - func(_ scpb.Status, target scpb.TargetStatus, _ scpb.Element) bool { - return target == scpb.ToPublic - }, - ) - scpb.ForEachPrimaryIndex(publicTargets, func(status scpb.Status, _ scpb.TargetStatus, idx *scpb.PrimaryIndex) { - existing = idx - if status == scpb.Status_ABSENT || status == scpb.Status_BACKFILL_ONLY { - // TODO(postamar): does it matter that there could be more than one? - freshlyAdded = idx - } - }) + tableID := spec.tbl.TableID + existing, freshlyAdded := getPrimaryIndexes(b, tableID) if freshlyAdded != nil { - var tempIndex *scpb.TemporaryIndex - scpb.ForEachTemporaryIndex(b.QueryByID(spec.tbl.TableID), func( - status scpb.Status, ts scpb.TargetStatus, e *scpb.TemporaryIndex, - ) { - if ts != scpb.Transient { - return - } - if e.IndexID == freshlyAdded.TemporaryIndexID { - if tempIndex != nil { - panic(errors.AssertionFailedf( - "multiple temporary index elements exist with index id %d for table %d", - freshlyAdded.TemporaryIndexID, e.TableID, - )) - } - tempIndex = e - } - }) - if tempIndex == nil { - panic(errors.AssertionFailedf( - "failed to find temporary index element for new primary index id %d for table %d", - freshlyAdded.IndexID, freshlyAdded.TableID, - )) - } - // Exceptionally, we can edit the element directly here, by virtue of it - // currently being in the ABSENT state we know that it was introduced as a - // PUBLIC target by the current statement. - - // We want to just add a new index column to the index and to its temp index. - ic := &scpb.IndexColumn{ - TableID: spec.tbl.TableID, - IndexID: freshlyAdded.IndexID, - ColumnID: spec.col.ColumnID, - OrdinalInKind: getNextStoredIndexColumnOrdinal(allTargets, freshlyAdded), - Kind: scpb.IndexColumn_STORED, - } - b.Add(ic) - tempIC := protoutil.Clone(ic).(*scpb.IndexColumn) - tempIC.IndexID = tempIndex.IndexID - b.Add(tempIC) + handleAddColumnFreshlyAddedPrimaryIndex(b, spec, freshlyAdded) return freshlyAdded } - // Otherwise, create a new primary index target and swap it with the existing - // primary index. - if existing == nil { - // TODO(postamar): can this even be possible? - panic(pgerror.Newf(pgcode.NoPrimaryKey, "missing active primary key")) - } + // As a special case, if we have a new column which has no computed // expression and no default value, then we can just add it to the // current primary index; there's no need to build a new index as @@ -377,7 +322,7 @@ func addColumn(b BuildCtx, spec addColumnSpec) (backing *scpb.PrimaryIndex) { // making a new column public is not revertible). // // If ever we were to change how we encoded NULLs, perhaps so that we could - // intepret a missing value as an arbitrary default expression, we'd need + // interpret a missing value as an arbitrary default expression, we'd need // to revisit this optimization. // // TODO(ajwerner): The above comment is incorrect in that we don't mark @@ -388,6 +333,7 @@ func addColumn(b BuildCtx, spec addColumnSpec) (backing *scpb.PrimaryIndex) { // to public before the column is published as public. We'll need to figure // out how to make sure that that happens atomically. Leaving that for a // follow-up change in order to get this in. + allTargets := b.QueryByID(spec.tbl.TableID) if spec.def == nil && spec.colType.ComputeExpr == nil { b.Add(&scpb.IndexColumn{ TableID: spec.tbl.TableID, @@ -398,50 +344,108 @@ func addColumn(b BuildCtx, spec addColumnSpec) (backing *scpb.PrimaryIndex) { }) return existing } - // Drop all existing primary index elements. - b.Drop(existing) - var existingName *scpb.IndexName - var existingPartitioning *scpb.IndexPartitioning - scpb.ForEachIndexName(publicTargets, func(_ scpb.Status, _ scpb.TargetStatus, name *scpb.IndexName) { - if name.IndexID == existing.IndexID { - existingName = name + + // Otherwise, create a new primary index target and swap it with the existing + // primary index. + return createNewPrimaryIndex(b, spec.tbl, existing, func( + b BuildCtx, newIndex *scpb.PrimaryIndex, existingColumns []*scpb.IndexColumn, + ) (newColumns []*scpb.IndexColumn) { + for _, ec := range existingColumns { + cloned := protoutil.Clone(ec).(*scpb.IndexColumn) + cloned.IndexID = newIndex.IndexID + newColumns = append(newColumns, cloned) + b.Add(cloned) } - }) - scpb.ForEachIndexPartitioning(publicTargets, func(_ scpb.Status, _ scpb.TargetStatus, part *scpb.IndexPartitioning) { - if part.IndexID == existing.IndexID { - existingPartitioning = part + ic := &scpb.IndexColumn{ + TableID: newIndex.TableID, + IndexID: newIndex.IndexID, + ColumnID: spec.col.ColumnID, + OrdinalInKind: getNextStoredIndexColumnOrdinal(allTargets, newIndex), + Kind: scpb.IndexColumn_STORED, } + newColumns = append(newColumns, ic) + b.Add(ic) + return newColumns }) - var existingColumns []*scpb.IndexColumn - scpb.ForEachIndexColumn(publicTargets, func(_ scpb.Status, _ scpb.TargetStatus, e *scpb.IndexColumn) { - if e.IndexID == existing.IndexID { - existingColumns = append(existingColumns, e) - b.Drop(e) +} + +// handleAddColumnFreshlyAddedPrimaryIndex is used when adding a column to a +// table and a previous command has already created a new primary index. In +// this situation, we need to add the new column to this new primary index. +func handleAddColumnFreshlyAddedPrimaryIndex( + b BuildCtx, spec addColumnSpec, freshlyAdded *scpb.PrimaryIndex, +) { + // TODO(ajwerner): Make sure we aren't removing any columns from this index. + // If we are, it means that this transaction is both adding and removing + // physical columns from the table, and we need an intermediate, transient + // primary index. + + var tempIndex *scpb.TemporaryIndex + scpb.ForEachTemporaryIndex(b.QueryByID(spec.tbl.TableID), func( + status scpb.Status, ts scpb.TargetStatus, e *scpb.TemporaryIndex, + ) { + if ts != scpb.Transient { + return + } + if e.IndexID == freshlyAdded.TemporaryIndexID { + if tempIndex != nil { + panic(errors.AssertionFailedf( + "multiple temporary index elements exist with index id %d for table %d", + freshlyAdded.TemporaryIndexID, e.TableID, + )) + } + tempIndex = e } }) - if existingPartitioning != nil { - b.Drop(existingPartitioning) + if tempIndex == nil { + panic(errors.AssertionFailedf( + "failed to find temporary index element for new primary index id %d for table %d", + freshlyAdded.IndexID, freshlyAdded.TableID, + )) } - if existingName != nil { - b.Drop(existingName) + // Add the new index column to the index and to its temp index. + ic := &scpb.IndexColumn{ + TableID: spec.tbl.TableID, + IndexID: freshlyAdded.IndexID, + ColumnID: spec.col.ColumnID, + OrdinalInKind: getNextStoredIndexColumnOrdinal( + b.QueryByID(spec.tbl.TableID), freshlyAdded, + ), + Kind: scpb.IndexColumn_STORED, } + b.Add(ic) + tempIC := protoutil.Clone(ic).(*scpb.IndexColumn) + tempIC.IndexID = tempIndex.IndexID + b.Add(tempIC) +} + +// newPrimaryIndexColumnFn is a callback which is supplied the old primary +// index and its columns and is responsible for constructing the set of +// columns for use in the new primary index. It is used by +// createNewPrimaryIndex. +type newPrimaryIndexColumnsFn = func( + b BuildCtx, + newIndex *scpb.PrimaryIndex, + existingColumns []*scpb.IndexColumn, +) (newColumns []*scpb.IndexColumn) + +// createNewPrimaryIndex creates a replacement primary index by dropping the +// existing primary index and adding a newly synthesized primary index. The +// makeColumnsFn is passed the existing columns and the new index and must +// synthesize the new IndexColumn elements to be added. +func createNewPrimaryIndex( + b BuildCtx, tbl *scpb.Table, existing *scpb.PrimaryIndex, makeColumnsFn newPrimaryIndexColumnsFn, +) *scpb.PrimaryIndex { + + // Drop all existing primary index elements. + existingName, existingPartitioning, + existingColumns := dropExistingPrimaryIndex(b, existing) + // Create the new primary index element and its dependents. replacement := protoutil.Clone(existing).(*scpb.PrimaryIndex) - replacement.IndexID = b.NextTableIndexID(spec.tbl) + replacement.IndexID = b.NextTableIndexID(tbl) replacement.SourceIndexID = existing.IndexID - for _, ec := range existingColumns { - cloned := protoutil.Clone(ec).(*scpb.IndexColumn) - cloned.IndexID = replacement.IndexID - b.Add(cloned) - } - ic := &scpb.IndexColumn{ - TableID: spec.tbl.TableID, - IndexID: replacement.IndexID, - ColumnID: spec.col.ColumnID, - OrdinalInKind: getNextStoredIndexColumnOrdinal(allTargets, replacement), - Kind: scpb.IndexColumn_STORED, - } - b.Add(ic) + replacementColumns := makeColumnsFn(b, replacement, existingColumns) replacement.TemporaryIndexID = replacement.IndexID + 1 b.Add(replacement) if existingName != nil { @@ -454,20 +458,19 @@ func addColumn(b BuildCtx, spec addColumnSpec) (backing *scpb.PrimaryIndex) { updatedPartitioning.IndexID = replacement.IndexID b.Add(updatedPartitioning) } - temp := &scpb.TemporaryIndex{ Index: protoutil.Clone(replacement).(*scpb.PrimaryIndex).Index, IsUsingSecondaryEncoding: false, } temp.TemporaryIndexID = 0 - temp.IndexID = b.NextTableIndexID(spec.tbl) + temp.IndexID = b.NextTableIndexID(tbl) b.AddTransient(temp) if existingPartitioning != nil { updatedPartitioning := protoutil.Clone(existingPartitioning).(*scpb.IndexPartitioning) updatedPartitioning.IndexID = temp.IndexID b.Add(updatedPartitioning) } - for _, ec := range append(existingColumns, ic) { + for _, ec := range replacementColumns { cloned := protoutil.Clone(ec).(*scpb.IndexColumn) cloned.IndexID = temp.IndexID b.Add(cloned) @@ -475,6 +478,42 @@ func addColumn(b BuildCtx, spec addColumnSpec) (backing *scpb.PrimaryIndex) { return replacement } +// dropExistingPrimaryIndex drops a primary index which existed before this +// schema change. +func dropExistingPrimaryIndex( + b BuildCtx, existing *scpb.PrimaryIndex, +) ( + existingName *scpb.IndexName, + existingPartitioning *scpb.IndexPartitioning, + existingColumns []*scpb.IndexColumn, +) { + b.Drop(existing) + publicTargets := b.QueryByID(existing.TableID).Filter(publicTargetFilter) + scpb.ForEachIndexName(publicTargets, func(_ scpb.Status, _ scpb.TargetStatus, name *scpb.IndexName) { + if name.IndexID == existing.IndexID { + existingName = name + } + }) + scpb.ForEachIndexPartitioning(publicTargets, func(_ scpb.Status, _ scpb.TargetStatus, part *scpb.IndexPartitioning) { + if part.IndexID == existing.IndexID { + existingPartitioning = part + } + }) + scpb.ForEachIndexColumn(publicTargets, func(_ scpb.Status, _ scpb.TargetStatus, e *scpb.IndexColumn) { + if e.IndexID == existing.IndexID { + existingColumns = append(existingColumns, e) + b.Drop(e) + } + }) + if existingPartitioning != nil { + b.Drop(existingPartitioning) + } + if existingName != nil { + b.Drop(existingName) + } + return existingName, existingPartitioning, existingColumns +} + func getNextStoredIndexColumnOrdinal(allTargets ElementResultSet, idx *scpb.PrimaryIndex) uint32 { max := -1 scpb.ForEachIndexColumn(allTargets, func( diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go index 914022383096..c27db5acdb60 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go @@ -299,3 +299,39 @@ func getSortedColumnIDsInIndex( } return keyColumnIDs, keySuffixColumnIDs, storingColumnIDs } + +func publicTargetFilter(_ scpb.Status, target scpb.TargetStatus, _ scpb.Element) bool { + return target == scpb.ToPublic +} + +func statusAbsentOrBackfillOnly(status scpb.Status, _ scpb.TargetStatus, _ scpb.Element) bool { + return status == scpb.Status_ABSENT || status == scpb.Status_BACKFILL_ONLY +} + +func statusPublic(status scpb.Status, _ scpb.TargetStatus, _ scpb.Element) bool { + return status == scpb.Status_PUBLIC +} + +// getPrimaryIndexes returns the primary indexes of the current table. +// Note that it assumes that there are at most two primary indexes and at +// least one. The existing primary index is the primary index which is +// currently public. The freshlyAdded primary index is one which is targeting +// public. +// +// TODO(ajwerner): This will not be true at some point in the near future when +// we need an intermediate primary index to support adding and dropping columns +// in the same transaction. +func getPrimaryIndexes( + b BuildCtx, tableID catid.DescID, +) (existing, freshlyAdded *scpb.PrimaryIndex) { + allTargets := b.QueryByID(tableID) + _, _, freshlyAdded = scpb.FindPrimaryIndex(allTargets. + Filter(publicTargetFilter). + Filter(statusAbsentOrBackfillOnly)) + _, _, existing = scpb.FindPrimaryIndex(allTargets.Filter(statusPublic)) + if existing == nil { + // TODO(postamar): can this even be possible? + panic(pgerror.Newf(pgcode.NoPrimaryKey, "missing active primary key")) + } + return existing, freshlyAdded +} From 11feecd55c7a1afca2435a5694efd37af790cf2f Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Sun, 10 Jul 2022 12:23:17 -0400 Subject: [PATCH 2/9] sql/schemachanger/scbuild: rework DROP INDEX in support of DROP COLUMN Extracted a few helpers for code reuse. Release note: None --- .../internal/scbuildstmt/drop_index.go | 41 +++++++++++-------- .../internal/scbuildstmt/drop_sequence.go | 2 +- .../scbuild/internal/scbuildstmt/helpers.go | 14 ++++--- 3 files changed, 35 insertions(+), 22 deletions(-) diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go index 83801a5af465..f0363479f874 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go @@ -61,7 +61,21 @@ func DropIndex(b BuildCtx, n *tree.DropIndex) { for _, index := range n.IndexList { dropAnIndex(b, index, n.IfExists, n.DropBehavior) + + // Increment subwork ID so we know exactly which portion in + // a `DROP INDEX index1, index2, ...` statement is responsible + // for the creation of the targets. + b.IncrementSubWorkID() + b.IncrementSchemaChangeDropCounter("index") + } + b.EvalCtx().ClientNoticeSender.BufferClientNotice( + b, + errors.WithHint( + pgnotice.Newf("the data for dropped indexes is reclaimed asynchronously"), + "The reclamation delay can be customized in the zone configuration for the table.", + ), + ) } // dropAnIndex resolves `index` and mark its constituent elements as ToAbsent @@ -90,12 +104,10 @@ func dropAnIndex( } // TODO (Xiang): Check if requires CCL binary for eventual zone config removal. - _, _, sie := scpb.FindSecondaryIndex(toBeDroppedIndexElms) if sie == nil { panic(errors.AssertionFailedf("programming error: cannot find secondary index element.")) } - // Cannot drop the index if not CASCADE and a unique constraint depends on it. if dropBehavior != tree.DropCascade && sie.IsUnique && !sie.IsCreatedExplicitly { panic(errors.WithHint( @@ -104,7 +116,18 @@ func dropAnIndex( "use CASCADE if you really want to drop it.", )) } + dropSecondaryIndex(b, index, dropBehavior, sie, toBeDroppedIndexElms) +} +// dropSecondaryIndex is a helper to drop a secondary index which may be used +// both in DROP INDEX and as a cascade from another operation. +func dropSecondaryIndex( + b BuildCtx, + index *tree.TableIndexName, + dropBehavior tree.DropBehavior, + sie *scpb.SecondaryIndex, + toBeDroppedIndexElms ElementResultSet, +) { // Maybe drop dependent views. // If CASCADE and there are "dependent" views (i.e. views that use this // to-be-dropped index), then we will drop all dependent views and their @@ -134,20 +157,6 @@ func dropAnIndex( b.Drop(e) } }) - - b.EvalCtx().ClientNoticeSender.BufferClientNotice( - b, - errors.WithHint( - pgnotice.Newf("the data for dropped indexes is reclaimed asynchronously"), - "The reclamation delay can be customized in the zone configuration for the table.", - ), - ) - - // Increment subwork ID so we know exactly which portion in - // a `DROP INDEX index1, index2, ...` statement is responsible - // for the creation of the targets. - b.IncrementSubWorkID() - b.IncrementSchemaChangeDropCounter("index") } // maybeDropDependentViews attempts to drop all views that depend diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_sequence.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_sequence.go index 1d089f35844b..961111db4af9 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_sequence.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_sequence.go @@ -48,7 +48,7 @@ func DropSequence(b BuildCtx, n *tree.DropSequence) { scpb.ForEachSequenceOwner( undroppedBackrefs(b, seq.SequenceID), func(_ scpb.Status, _ scpb.TargetStatus, so *scpb.SequenceOwner) { - dropElement(b, so) + dropElementWhenDroppingDescriptor(b, so) }, ) toCheckBackrefs = append(toCheckBackrefs, seq.SequenceID) diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go index c27db5acdb60..1e5694069a83 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go @@ -52,7 +52,7 @@ func dropRestrictDescriptor(b BuildCtx, id catid.DescID) (hasChanged bool) { } undropped.ForEachElementStatus(func(_ scpb.Status, target scpb.TargetStatus, e scpb.Element) { b.CheckPrivilege(e, privilege.DROP) - dropElement(b, e) + dropElementWhenDroppingDescriptor(b, e) }) return true } @@ -125,8 +125,12 @@ func errMsgPrefix(b BuildCtx, id catid.DescID) string { return fmt.Sprintf("%s %q", typ, name) } -func dropElement(b BuildCtx, e scpb.Element) { - // TODO(postamar): remove this dirty hack ASAP, see column/index dep rules. +// dropElementWhenDroppingDescriptor is a helper to drop an element when +// dropping a descriptor which sets the bit to indicate that the descriptor +// is being dropped. +// +// TODO(postamar): remove this dirty hack ASAP, see column/index dep rules. +func dropElementWhenDroppingDescriptor(b BuildCtx, e scpb.Element) { switch t := e.(type) { case *scpb.ColumnType: t.IsRelationBeingDropped = true @@ -183,7 +187,7 @@ func dropCascadeDescriptor(b BuildCtx, id catid.DescID) { // Don't actually drop any elements of virtual schemas. return } - dropElement(b, e) + dropElementWhenDroppingDescriptor(b, e) switch t := e.(type) { case *scpb.EnumType: dropCascadeDescriptor(next, t.ArrayTypeID) @@ -217,7 +221,7 @@ func dropCascadeDescriptor(b BuildCtx, id catid.DescID) { *scpb.ForeignKeyConstraint, *scpb.SequenceOwner, *scpb.DatabaseRegionConfig: - dropElement(b, e) + dropElementWhenDroppingDescriptor(b, e) } }) } From 563e7ed5821e980d1687e1bbda67951d0eb30518 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Sun, 10 Jul 2022 13:44:27 -0400 Subject: [PATCH 3/9] sql/schemachanger/scplan/internal/rules: adjust some rules One rule was too strict, and another would only match the secondary index and not the temp index. Also, the variable names were swapped. Release note: None --- .../testdata/logic_test/new_schema_changer | 24 +-- .../internal/rules/dep_index_and_column.go | 76 +++++++-- .../scplan/internal/rules/testdata/deprules | 73 ++++++-- .../schemachanger/scplan/testdata/drop_index | 160 +++++++++++------- .../schemachanger/scplan/testdata/drop_table | 64 ++++--- .../testdata/drop_index_hash_sharded_index | 24 ++- .../drop_index_partial_expression_index | 14 +- .../testdata/drop_index_vanilla_index | 19 ++- 8 files changed, 302 insertions(+), 152 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/new_schema_changer b/pkg/sql/logictest/testdata/logic_test/new_schema_changer index daea9e58865f..02d3ed1cfc1f 100644 --- a/pkg/sql/logictest/testdata/logic_test/new_schema_changer +++ b/pkg/sql/logictest/testdata/logic_test/new_schema_changer @@ -35,8 +35,8 @@ EXPLAIN (DDL, VERBOSE) ALTER TABLE foo ADD COLUMN j INT │ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} │ │ │ │ ABSENT → PUBLIC │ │ │ │ -│ │ │ └── • SameStagePrecedence dependency from PUBLIC ColumnName:{DescID: 106, Name: j, ColumnID: 2} -│ │ │ rule: "column named right before column type becomes public" +│ │ │ └── • Precedence dependency from PUBLIC ColumnName:{DescID: 106, Name: j, ColumnID: 2} +│ │ │ rule: "column named before column type becomes public" │ │ │ │ │ └── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} │ │ │ ABSENT → PUBLIC @@ -173,6 +173,7 @@ EXPLAIN (DDL, VERBOSE) ALTER TABLE foo ADD COLUMN j INT JobID: 1 RunningStatus: all stages completed + statement ok ALTER TABLE foo ADD COLUMN j INT @@ -1923,21 +1924,17 @@ EXPLAIN (DDL) DROP INDEX parent@idx CASCADE; Schema change plan for DROP INDEX ‹test›.public.‹parent›@‹idx› CASCADE; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 8 elements transitioning toward ABSENT - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 265, ColumnID: 2, IndexID: 2} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 265, ColumnID: 1, IndexID: 2} + │ ├── 6 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED SecondaryIndex:{DescID: 265, IndexID: 2, ConstraintID: 2} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 265, Name: idx, IndexID: 2} │ │ ├── PUBLIC → OFFLINE View:{DescID: 266} │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 266, ColumnID: 1} │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 266, ColumnID: 4294967295} │ │ └── PUBLIC → WRITE_ONLY Column:{DescID: 266, ColumnID: 4294967294} - │ └── 5 Mutation operations + │ └── 3 Mutation operations │ ├── MakeDroppedNonPrimaryIndexDeleteAndWriteOnly {"IndexID":2,"TableID":265} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":265} - │ ├── MarkDescriptorAsOffline {"DescID":266,"Reason":"DROP INDEX test...."} - │ ├── scop.RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"TableID":265} - │ └── scop.RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"Kind":1,"TableID":265} + │ └── MarkDescriptorAsOffline {"DescID":266,"Reason":"DROP INDEX test...."} ├── PreCommitPhase │ └── Stage 1 of 1 in PreCommitPhase │ └── 3 Mutation operations @@ -1946,7 +1943,9 @@ Schema change plan for DROP INDEX ‹test›.public.‹parent›@‹idx› CASCA │ └── CreateSchemaChangerJob {"NonCancelable":true,"RunningStatus":"PostCommitNonRev..."} └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 17 elements transitioning toward ABSENT + │ ├── 19 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 265, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 265, ColumnID: 1, IndexID: 2} │ │ ├── VALIDATED → DELETE_ONLY SecondaryIndex:{DescID: 265, IndexID: 2, ConstraintID: 2} │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 266, Name: child, ReferencedDescID: 104} │ │ ├── PUBLIC → ABSENT Owner:{DescID: 266} @@ -1964,12 +1963,14 @@ Schema change plan for DROP INDEX ‹test›.public.‹parent›@‹idx› CASCA │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 266, ColumnID: 4294967294} │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 266, Name: tableoid, ColumnID: 4294967294} │ │ └── PUBLIC → ABSENT ColumnType:{DescID: 266, ColumnFamilyID: 0, ColumnID: 4294967294} - │ └── 8 Mutation operations + │ └── 10 Mutation operations │ ├── MarkDescriptorAsDropped {"DescID":266} │ ├── RemoveViewBackReferencesInRelations {"BackReferencedViewID":266} │ ├── RemoveAllTableComments {"TableID":266} │ ├── MakeDroppedIndexDeleteOnly {"IndexID":2,"TableID":265} │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":266,"Name":"child","SchemaID":105}} + │ ├── scop.RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"TableID":265} + │ ├── scop.RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"Kind":1,"TableID":265} │ ├── SetJobStateOnDescriptor {"DescriptorID":265} │ ├── SetJobStateOnDescriptor {"DescriptorID":266} │ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"PostCommitNonRev..."} @@ -1993,6 +1994,7 @@ Schema change plan for DROP INDEX ‹test›.public.‹parent›@‹idx› CASCA ├── RemoveJobStateFromDescriptor {"DescriptorID":266} └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} + statement ok SET use_declarative_schema_changer = 'on' diff --git a/pkg/sql/schemachanger/scplan/internal/rules/dep_index_and_column.go b/pkg/sql/schemachanger/scplan/internal/rules/dep_index_and_column.go index 810306dfdf65..e1af0bf1786f 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/dep_index_and_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/dep_index_and_column.go @@ -92,12 +92,11 @@ func init() { } }) - // It's very important that columns are added to the descriptor's - // IndexDescriptor in the same mutation stage which adds that index - // to the TableDescriptor. + // This rule pairs with the rule which ensures that columns are added to + // the index before it receives writes. registerDepRule( - "temp index exists right before columns, partitioning, and partial", - scgraph.SameStagePrecedence, + "temp index exists before columns, partitioning, and partial", + scgraph.Precedence, "temp-index", "index-partitioning", func(from, to nodeVars) rel.Clauses { return rel.Clauses{ @@ -160,20 +159,68 @@ func init() { // do it once the index is definitely being dropped. The reason for // this is roundabout: dropping a column from an index which is itself // being dropped is treated as a no-op by the execution layer. + // + // TODO(ajwerner): This pair of rules really feels like it ought to be a + // same stage precedence sort of rule where we remove the columns from the + // index when we remove the index, but for some reason, that overconstrains + // the graph when dropping the table. Because of that, we allow the column + // to be removed from the index in DELETE_ONLY, and we no-op the removal. registerDepRule( - "temp and secondary index columns removed just before removing the index", - scgraph.SameStagePrecedence, + "secondary index columns removed before removing the index", + scgraph.Precedence, + "index-column", "index", + func(from, to nodeVars) rel.Clauses { + return rel.Clauses{ + from.el.Type((*scpb.IndexColumn)(nil)), + to.el.Type((*scpb.SecondaryIndex)(nil)), + joinOnIndexID(from.el, to.el, "table-id", "index-id"), + toAbsent(from.target, to.target), + currentStatus(from.node, scpb.Status_ABSENT), + currentStatus(to.node, scpb.Status_ABSENT), + } + }, + ) + registerDepRule( + "secondary index in DELETE_ONLY before removing columns", + scgraph.Precedence, "index", "index-column", + func(from, to nodeVars) rel.Clauses { + return rel.Clauses{ + from.el.Type((*scpb.SecondaryIndex)(nil)), + to.el.Type((*scpb.IndexColumn)(nil)), + joinOnIndexID(from.el, to.el, "table-id", "index-id"), + toAbsent(from.target, to.target), + currentStatus(from.node, scpb.Status_DELETE_ONLY), + currentStatus(to.node, scpb.Status_ABSENT), + } + }, + ) + registerDepRule( + "temp index columns removed before removing the index", + scgraph.Precedence, + "index-column", "index", func(from, to nodeVars) rel.Clauses { return rel.Clauses{ from.el.Type((*scpb.IndexColumn)(nil)), - to.el.Type( - (*scpb.SecondaryIndex)(nil), - (*scpb.TemporaryIndex)(nil), - ), + to.el.Type((*scpb.TemporaryIndex)(nil)), joinOnIndexID(from.el, to.el, "table-id", "index-id"), toAbsent(from.target, to.target), currentStatus(from.node, scpb.Status_ABSENT), + currentStatus(to.node, scpb.Status_TRANSIENT_ABSENT), + } + }, + ) + registerDepRule( + "temp index in DELETE_ONLY before removing columns", + scgraph.Precedence, + "index", "index-column", + func(from, to nodeVars) rel.Clauses { + return rel.Clauses{ + from.el.Type((*scpb.TemporaryIndex)(nil)), + to.el.Type((*scpb.IndexColumn)(nil)), + joinOnIndexID(from.el, to.el, "table-id", "index-id"), + toAbsent(from.target, to.target), + currentStatus(from.node, scpb.Status_TRANSIENT_DELETE_ONLY), currentStatus(to.node, scpb.Status_ABSENT), } }, @@ -316,9 +363,10 @@ func init() { }, ) + // TODO(ajwerner): Understand this rule and why it needs to exist. registerDepRule( - "column named right before column type becomes public", - scgraph.SameStagePrecedence, + "column named before column type becomes public", + scgraph.Precedence, "column-name", "column-type", func(from, to nodeVars) rel.Clauses { return rel.Clauses{ @@ -624,7 +672,7 @@ func init() { registerDepRule( "column name and type to public after all index column to public", scgraph.Precedence, - "index-column", "column-name", + "column-name-or-type", "index-column", func(from, to nodeVars) rel.Clauses { return rel.Clauses{ from.el.Type((*scpb.ColumnName)(nil), (*scpb.ColumnType)(nil)), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules index 2a6c060746fa..c7938168fc2e 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules @@ -158,9 +158,9 @@ deprules - $index-dependent-node[CurrentStatus] = PUBLIC - joinTargetNode($index, $index-target, $index-node) - joinTargetNode($index-dependent, $index-dependent-target, $index-dependent-node) -- name: temp index exists right before columns, partitioning, and partial +- name: temp index exists before columns, partitioning, and partial from: temp-index-node - kind: SameStagePrecedence + kind: Precedence to: index-partitioning-node query: - $temp-index[Type] = '*scpb.TemporaryIndex' @@ -200,16 +200,55 @@ deprules - joinOnIndexID($index-name, $index, $table-id, $index-id) - joinTargetNode($index-name, $index-name-target, $index-name-node) - joinTargetNode($index, $index-target, $index-node) -- name: temp and secondary index columns removed just before removing the index +- name: secondary index columns removed before removing the index + from: index-column-node + kind: Precedence + to: index-node + query: + - $index-column[Type] = '*scpb.IndexColumn' + - $index[Type] = '*scpb.SecondaryIndex' + - joinOnIndexID($index-column, $index, $table-id, $index-id) + - toAbsent($index-column-target, $index-target) + - $index-column-node[CurrentStatus] = ABSENT + - $index-node[CurrentStatus] = ABSENT + - joinTargetNode($index-column, $index-column-target, $index-column-node) + - joinTargetNode($index, $index-target, $index-node) +- name: secondary index in DELETE_ONLY before removing columns from: index-node - kind: SameStagePrecedence + kind: Precedence to: index-column-node query: - - $index[Type] = '*scpb.IndexColumn' - - $index-column[Type] IN ['*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - $index[Type] = '*scpb.SecondaryIndex' + - $index-column[Type] = '*scpb.IndexColumn' - joinOnIndexID($index, $index-column, $table-id, $index-id) - toAbsent($index-target, $index-column-target) - - $index-node[CurrentStatus] = ABSENT + - $index-node[CurrentStatus] = DELETE_ONLY + - $index-column-node[CurrentStatus] = ABSENT + - joinTargetNode($index, $index-target, $index-node) + - joinTargetNode($index-column, $index-column-target, $index-column-node) +- name: temp index columns removed before removing the index + from: index-column-node + kind: Precedence + to: index-node + query: + - $index-column[Type] = '*scpb.IndexColumn' + - $index[Type] = '*scpb.TemporaryIndex' + - joinOnIndexID($index-column, $index, $table-id, $index-id) + - toAbsent($index-column-target, $index-target) + - $index-column-node[CurrentStatus] = ABSENT + - $index-node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($index-column, $index-column-target, $index-column-node) + - joinTargetNode($index, $index-target, $index-node) +- name: temp index in DELETE_ONLY before removing columns + from: index-node + kind: Precedence + to: index-column-node + query: + - $index[Type] = '*scpb.TemporaryIndex' + - $index-column[Type] = '*scpb.IndexColumn' + - joinOnIndexID($index, $index-column, $table-id, $index-id) + - toAbsent($index-target, $index-column-target) + - $index-node[CurrentStatus] = TRANSIENT_DELETE_ONLY - $index-column-node[CurrentStatus] = ABSENT - joinTargetNode($index, $index-target, $index-node) - joinTargetNode($index-column, $index-column-target, $index-column-node) @@ -294,9 +333,9 @@ deprules - $column-node[CurrentStatus] = WRITE_ONLY - joinTargetNode($expr, $expr-target, $expr-node) - joinTargetNode($column, $column-target, $column-node) -- name: column named right before column type becomes public +- name: column named before column type becomes public from: column-name-node - kind: SameStagePrecedence + kind: Precedence to: column-type-node query: - $column-name[Type] = '*scpb.ColumnName' @@ -484,19 +523,19 @@ deprules - joinTargetNode($primary-index, $primary-index-target, $primary-index-node) - joinTargetNode($second-index, $second-index-target, $second-index-node) - name: column name and type to public after all index column to public - from: index-column-node + from: column-name-or-type-node kind: Precedence - to: column-name-node + to: index-column-node query: - - $index-column[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType'] - - $column-name[Type] = '*scpb.IndexColumn' - - joinOnColumnID($index-column, $column-name, $table-id, $column-id) + - $column-name-or-type[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType'] + - $index-column[Type] = '*scpb.IndexColumn' + - joinOnColumnID($column-name-or-type, $index-column, $table-id, $column-id) + - $column-name-or-type-target[TargetStatus] = PUBLIC - $index-column-target[TargetStatus] = PUBLIC - - $column-name-target[TargetStatus] = PUBLIC + - $column-name-or-type-node[CurrentStatus] = PUBLIC - $index-column-node[CurrentStatus] = PUBLIC - - $column-name-node[CurrentStatus] = PUBLIC + - joinTargetNode($column-name-or-type, $column-name-or-type-target, $column-name-or-type-node) - joinTargetNode($index-column, $index-column-target, $index-column-node) - - joinTargetNode($column-name, $column-name-target, $column-name-node) - name: index-column added to index after index exists from: index-node kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_index b/pkg/sql/schemachanger/scplan/testdata/drop_index index 4108c47da2b8..6a400f2fd3f0 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_index +++ b/pkg/sql/schemachanger/scplan/testdata/drop_index @@ -15,10 +15,8 @@ CREATE VIEW v AS SELECT count(j) FROM t1@idx4 ops DROP INDEX idx1 CASCADE ---- -StatementPhase stage 1 of 1 with 4 MutationType ops +StatementPhase stage 1 of 1 with 2 MutationType ops transitions: - [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED [[IndexName:{DescID: 104, Name: idx1, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT ops: @@ -29,15 +27,6 @@ StatementPhase stage 1 of 1 with 4 MutationType ops IndexID: 2 Name: crdb_internal_index_2_name_placeholder TableID: 104 - scop.RemoveColumnFromIndex - ColumnID: 1 - IndexID: 2 - TableID: 104 - scop.RemoveColumnFromIndex - ColumnID: 3 - IndexID: 2 - Kind: 1 - TableID: 104 PreCommitPhase stage 1 of 1 with 2 MutationType ops transitions: ops: @@ -51,18 +40,29 @@ PreCommitPhase stage 1 of 1 with 2 MutationType ops - 104 JobID: 1 NonCancelable: true - RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 1 MutationType op pending + RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 3 MutationType ops pending Statements: - statement: DROP INDEX idx1 CASCADE redactedstatement: DROP INDEX ‹defaultdb›.public.‹t1›@‹idx1› CASCADE statementtag: DROP INDEX -PostCommitNonRevertiblePhase stage 1 of 2 with 3 MutationType ops +PostCommitNonRevertiblePhase stage 1 of 2 with 5 MutationType ops transitions: + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, ABSENT], VALIDATED] -> DELETE_ONLY ops: *scop.MakeDroppedIndexDeleteOnly IndexID: 2 TableID: 104 + scop.RemoveColumnFromIndex + ColumnID: 1 + IndexID: 2 + TableID: 104 + scop.RemoveColumnFromIndex + ColumnID: 3 + IndexID: 2 + Kind: 1 + TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 *scop.UpdateSchemaChangerJob @@ -118,6 +118,14 @@ DROP INDEX idx1 CASCADE to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, ABSENT] kind: Precedence rule: dependents removed before index +- from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, DELETE_ONLY] + to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, ABSENT] + kind: Precedence + rule: secondary index in DELETE_ONLY before removing columns +- from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, DELETE_ONLY] + to: [IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, ABSENT] + kind: Precedence + rule: secondary index in DELETE_ONLY before removing columns - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, VALIDATED] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, ABSENT] kind: Precedence @@ -134,12 +142,10 @@ DROP INDEX idx1 CASCADE ops DROP INDEX idx2 CASCADE ---- -StatementPhase stage 1 of 1 with 7 MutationType ops +StatementPhase stage 1 of 1 with 5 MutationType ops transitions: [[Column:{DescID: 104, ColumnID: 4}, ABSENT], PUBLIC] -> WRITE_ONLY [[ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 4}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED [[IndexName:{DescID: 104, Name: idx2, IndexID: 4}, ABSENT], PUBLIC] -> ABSENT ops: @@ -173,15 +179,6 @@ StatementPhase stage 1 of 1 with 7 MutationType ops IndexID: 4 Name: crdb_internal_index_4_name_placeholder TableID: 104 - scop.RemoveColumnFromIndex - ColumnID: 4 - IndexID: 4 - TableID: 104 - scop.RemoveColumnFromIndex - ColumnID: 3 - IndexID: 4 - Kind: 1 - TableID: 104 PreCommitPhase stage 1 of 1 with 2 MutationType ops transitions: ops: @@ -195,14 +192,16 @@ PreCommitPhase stage 1 of 1 with 2 MutationType ops - 104 JobID: 1 NonCancelable: true - RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 2 MutationType ops pending + RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 4 MutationType ops pending Statements: - statement: DROP INDEX idx2 CASCADE redactedstatement: DROP INDEX ‹defaultdb›.public.‹t1›@‹idx2› CASCADE statementtag: DROP INDEX -PostCommitNonRevertiblePhase stage 1 of 2 with 4 MutationType ops +PostCommitNonRevertiblePhase stage 1 of 2 with 6 MutationType ops transitions: [[Column:{DescID: 104, ColumnID: 4}, ABSENT], WRITE_ONLY] -> DELETE_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 4}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, ABSENT], VALIDATED] -> DELETE_ONLY ops: *scop.MakeDroppedColumnDeleteOnly @@ -211,6 +210,15 @@ PostCommitNonRevertiblePhase stage 1 of 2 with 4 MutationType ops *scop.MakeDroppedIndexDeleteOnly IndexID: 4 TableID: 104 + scop.RemoveColumnFromIndex + ColumnID: 4 + IndexID: 4 + TableID: 104 + scop.RemoveColumnFromIndex + ColumnID: 3 + IndexID: 4 + Kind: 1 + TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 *scop.UpdateSchemaChangerJob @@ -299,6 +307,14 @@ DROP INDEX idx2 CASCADE to: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, ABSENT] kind: Precedence rule: dependents removed before index +- from: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, DELETE_ONLY] + to: [IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4}, ABSENT] + kind: Precedence + rule: secondary index in DELETE_ONLY before removing columns +- from: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, DELETE_ONLY] + to: [IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 4}, ABSENT] + kind: Precedence + rule: secondary index in DELETE_ONLY before removing columns - from: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, VALIDATED] to: [IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4}, ABSENT] kind: Precedence @@ -323,13 +339,10 @@ DROP INDEX idx2 CASCADE ops DROP INDEX idx3 CASCADE ---- -StatementPhase stage 1 of 1 with 10 MutationType ops +StatementPhase stage 1 of 1 with 7 MutationType ops transitions: [[Column:{DescID: 104, ColumnID: 5}, ABSENT], PUBLIC] -> WRITE_ONLY [[ColumnName:{DescID: 104, Name: crdb_internal_i_shard_16, ColumnID: 5}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 104, ColumnID: 5, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED [[IndexName:{DescID: 104, Name: idx3, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT [[CheckConstraint:{DescID: 104, ConstraintID: 4}, ABSENT], PUBLIC] -> ABSENT @@ -370,20 +383,6 @@ StatementPhase stage 1 of 1 with 10 MutationType ops TableID: 104 *scop.NotImplemented ElementType: scpb.ConstraintName - scop.RemoveColumnFromIndex - ColumnID: 5 - IndexID: 6 - TableID: 104 - scop.RemoveColumnFromIndex - ColumnID: 1 - IndexID: 6 - Ordinal: 1 - TableID: 104 - scop.RemoveColumnFromIndex - ColumnID: 3 - IndexID: 6 - Kind: 1 - TableID: 104 PreCommitPhase stage 1 of 1 with 2 MutationType ops transitions: ops: @@ -397,14 +396,17 @@ PreCommitPhase stage 1 of 1 with 2 MutationType ops - 104 JobID: 1 NonCancelable: true - RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 2 MutationType ops pending + RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 5 MutationType ops pending Statements: - statement: DROP INDEX idx3 CASCADE redactedstatement: DROP INDEX ‹defaultdb›.public.‹t1›@‹idx3› CASCADE statementtag: DROP INDEX -PostCommitNonRevertiblePhase stage 1 of 2 with 4 MutationType ops +PostCommitNonRevertiblePhase stage 1 of 2 with 7 MutationType ops transitions: [[Column:{DescID: 104, ColumnID: 5}, ABSENT], WRITE_ONLY] -> DELETE_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 5, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT], VALIDATED] -> DELETE_ONLY ops: *scop.MakeDroppedColumnDeleteOnly @@ -413,6 +415,20 @@ PostCommitNonRevertiblePhase stage 1 of 2 with 4 MutationType ops *scop.MakeDroppedIndexDeleteOnly IndexID: 6 TableID: 104 + scop.RemoveColumnFromIndex + ColumnID: 5 + IndexID: 6 + TableID: 104 + scop.RemoveColumnFromIndex + ColumnID: 1 + IndexID: 6 + Ordinal: 1 + TableID: 104 + scop.RemoveColumnFromIndex + ColumnID: 3 + IndexID: 6 + Kind: 1 + TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 *scop.UpdateSchemaChangerJob @@ -507,6 +523,18 @@ DROP INDEX idx3 CASCADE to: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT] kind: Precedence rule: dependents removed before index +- from: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, DELETE_ONLY] + to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 6}, ABSENT] + kind: Precedence + rule: secondary index in DELETE_ONLY before removing columns +- from: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, DELETE_ONLY] + to: [IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6}, ABSENT] + kind: Precedence + rule: secondary index in DELETE_ONLY before removing columns +- from: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, DELETE_ONLY] + to: [IndexColumn:{DescID: 104, ColumnID: 5, IndexID: 6}, ABSENT] + kind: Precedence + rule: secondary index in DELETE_ONLY before removing columns - from: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, VALIDATED] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 6}, ABSENT] kind: Precedence @@ -527,10 +555,8 @@ DROP INDEX idx3 CASCADE ops DROP INDEX idx4 CASCADE ---- -StatementPhase stage 1 of 1 with 5 MutationType ops +StatementPhase stage 1 of 1 with 3 MutationType ops transitions: - [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 8}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 8}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 9}, ABSENT], PUBLIC] -> VALIDATED [[IndexName:{DescID: 104, Name: idx4, IndexID: 8}, ABSENT], PUBLIC] -> ABSENT [[View:{DescID: 105}, ABSENT], PUBLIC] -> OFFLINE @@ -548,15 +574,6 @@ StatementPhase stage 1 of 1 with 5 MutationType ops *scop.MarkDescriptorAsOffline DescID: 105 Reason: DROP INDEX defaultdb.public.t1@idx4 CASCADE - scop.RemoveColumnFromIndex - ColumnID: 2 - IndexID: 8 - TableID: 104 - scop.RemoveColumnFromIndex - ColumnID: 3 - IndexID: 8 - Kind: 1 - TableID: 104 PreCommitPhase stage 1 of 1 with 3 MutationType ops transitions: ops: @@ -574,13 +591,15 @@ PreCommitPhase stage 1 of 1 with 3 MutationType ops - 105 JobID: 1 NonCancelable: true - RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 5 MutationType ops pending + RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 7 MutationType ops pending Statements: - statement: DROP INDEX idx4 CASCADE redactedstatement: DROP INDEX ‹defaultdb›.public.‹t1›@‹idx4› CASCADE statementtag: DROP INDEX -PostCommitNonRevertiblePhase stage 1 of 2 with 8 MutationType ops +PostCommitNonRevertiblePhase stage 1 of 2 with 10 MutationType ops transitions: + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 8}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 8}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 9}, ABSENT], VALIDATED] -> DELETE_ONLY [[Namespace:{DescID: 105, Name: v, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT @@ -615,6 +634,15 @@ PostCommitNonRevertiblePhase stage 1 of 2 with 8 MutationType ops DescriptorID: 105 Name: v SchemaID: 101 + scop.RemoveColumnFromIndex + ColumnID: 2 + IndexID: 8 + TableID: 104 + scop.RemoveColumnFromIndex + ColumnID: 3 + IndexID: 8 + Kind: 1 + TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 *scop.SetJobStateOnDescriptor @@ -784,6 +812,14 @@ DROP INDEX idx4 CASCADE to: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 9}, ABSENT] kind: Precedence rule: dependents removed before index +- from: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 9}, DELETE_ONLY] + to: [IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 8}, ABSENT] + kind: Precedence + rule: secondary index in DELETE_ONLY before removing columns +- from: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 9}, DELETE_ONLY] + to: [IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 8}, ABSENT] + kind: Precedence + rule: secondary index in DELETE_ONLY before removing columns - from: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 9}, VALIDATED] to: [IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 8}, ABSENT] kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_table b/pkg/sql/schemachanger/scplan/testdata/drop_table index e85867427960..89a86a0fc00c 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_table +++ b/pkg/sql/schemachanger/scplan/testdata/drop_table @@ -183,6 +183,24 @@ PostCommitNonRevertiblePhase stage 1 of 2 with 32 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 + DescID: 111 + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 111 + RelationIDs: + - 109 + *scop.RemoveAllTableComments + TableID: 111 + *scop.MarkDescriptorAsDropped + DescID: 110 + *scop.RemoveAllTableComments + TableID: 110 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 111 + Name: v1 + SchemaID: 101 *scop.MarkDescriptorAsDropped DescID: 109 *scop.RemoveAllTableComments @@ -190,6 +208,12 @@ PostCommitNonRevertiblePhase stage 1 of 2 with 32 MutationType ops *scop.RemoveColumnDefaultExpression ColumnID: 1 TableID: 109 + *scop.RemoveSequenceOwner + ColumnID: 2 + OwnedSequenceID: 110 + TableID: 109 + *scop.RemoveOwnerBackReferenceInSequence + SequenceID: 110 *scop.RemoveDroppedColumnType ColumnID: 3 TableID: 109 @@ -228,30 +252,6 @@ PostCommitNonRevertiblePhase stage 1 of 2 with 32 MutationType ops *scop.RemoveForeignKeyConstraint ConstraintID: 3 TableID: 109 - *scop.MarkDescriptorAsDropped - DescID: 110 - *scop.RemoveAllTableComments - TableID: 110 - *scop.MarkDescriptorAsDropped - DescID: 111 - *scop.RemoveViewBackReferencesInRelations - BackReferencedViewID: 111 - RelationIDs: - - 109 - *scop.RemoveAllTableComments - TableID: 111 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 109 - Name: shipments - SchemaID: 101 - *scop.RemoveSequenceOwner - ColumnID: 2 - OwnedSequenceID: 110 - TableID: 109 - *scop.RemoveOwnerBackReferenceInSequence - SequenceID: 110 *scop.DrainDescriptorName Namespace: DatabaseID: 100 @@ -261,8 +261,8 @@ PostCommitNonRevertiblePhase stage 1 of 2 with 32 MutationType ops *scop.DrainDescriptorName Namespace: DatabaseID: 100 - DescriptorID: 111 - Name: v1 + DescriptorID: 109 + Name: shipments SchemaID: 101 *scop.SetJobStateOnDescriptor DescriptorID: 104 @@ -774,6 +774,18 @@ DROP TABLE defaultdb.shipments CASCADE; to: [IndexName:{DescID: 109, Name: shipments_pkey, IndexID: 1}, ABSENT] kind: Precedence rule: index no longer public before dependents removed +- from: [SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, DELETE_ONLY] + to: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 2}, ABSENT] + kind: Precedence + rule: secondary index in DELETE_ONLY before removing columns +- from: [SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, DELETE_ONLY] + to: [IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 2}, ABSENT] + kind: Precedence + rule: secondary index in DELETE_ONLY before removing columns +- from: [SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, DELETE_ONLY] + to: [IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 2}, ABSENT] + kind: Precedence + rule: secondary index in DELETE_ONLY before removing columns - from: [SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, VALIDATED] to: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 2}, ABSENT] kind: Precedence diff --git a/pkg/sql/schemachanger/testdata/drop_index_hash_sharded_index b/pkg/sql/schemachanger/testdata/drop_index_hash_sharded_index index d57aa3d7d8eb..32163a241a77 100644 --- a/pkg/sql/schemachanger/testdata/drop_index_hash_sharded_index +++ b/pkg/sql/schemachanger/testdata/drop_index_hash_sharded_index @@ -12,7 +12,7 @@ begin transaction #1 # begin StatementPhase checking for feature: DROP INDEX increment telemetry for sql.schema.drop_index -## StatementPhase stage 1 of 1 with 10 MutationType ops +## StatementPhase stage 1 of 1 with 7 MutationType ops upsert descriptor #104 table: - checks: @@ -140,9 +140,9 @@ upsert descriptor #104 + - WRITE_ONLY + - ABSENT + - PUBLIC - + - ABSENT - + - ABSENT - + - ABSENT + + - PUBLIC + + - PUBLIC + + - PUBLIC + - VALIDATED + - ABSENT + - ABSENT @@ -294,7 +294,7 @@ notified job registry to adopt jobs: [1] begin transaction #2 commit transaction #2 begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 2 with 4 MutationType ops +## PostCommitNonRevertiblePhase stage 1 of 2 with 7 MutationType ops upsert descriptor #104 ... userName: root @@ -303,13 +303,19 @@ upsert descriptor #104 + - DELETE_ONLY - ABSENT - PUBLIC - ... - - ABSENT - - ABSENT + - - PUBLIC + - - PUBLIC + - - PUBLIC - - VALIDATED - + - DELETE_ONLY - ABSENT - ABSENT + - ABSENT + + - DELETE_ONLY + + - ABSENT + + - ABSENT + + - ABSENT + jobId: "1" + relevantStatements: ... id: 104 indexes: [] diff --git a/pkg/sql/schemachanger/testdata/drop_index_partial_expression_index b/pkg/sql/schemachanger/testdata/drop_index_partial_expression_index index c3aede6929cf..952ea0c61618 100644 --- a/pkg/sql/schemachanger/testdata/drop_index_partial_expression_index +++ b/pkg/sql/schemachanger/testdata/drop_index_partial_expression_index @@ -12,7 +12,7 @@ begin transaction #1 # begin StatementPhase checking for feature: DROP INDEX increment telemetry for sql.schema.drop_index -## StatementPhase stage 1 of 1 with 7 MutationType ops +## StatementPhase stage 1 of 1 with 5 MutationType ops upsert descriptor #104 ... family: StringFamily @@ -114,8 +114,8 @@ upsert descriptor #104 + - WRITE_ONLY + - ABSENT + - PUBLIC - + - ABSENT - + - ABSENT + + - PUBLIC + + - PUBLIC + - PUBLIC + - VALIDATED + - ABSENT @@ -235,7 +235,7 @@ notified job registry to adopt jobs: [1] begin transaction #2 commit transaction #2 begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 2 with 4 MutationType ops +## PostCommitNonRevertiblePhase stage 1 of 2 with 6 MutationType ops upsert descriptor #104 ... userName: root @@ -244,9 +244,11 @@ upsert descriptor #104 + - DELETE_ONLY - ABSENT - PUBLIC - ... - - ABSENT + + - ABSENT + + - ABSENT - PUBLIC + - - PUBLIC + - - PUBLIC - - VALIDATED + - DELETE_ONLY - ABSENT diff --git a/pkg/sql/schemachanger/testdata/drop_index_vanilla_index b/pkg/sql/schemachanger/testdata/drop_index_vanilla_index index 081457cc13df..b86cf55bead4 100644 --- a/pkg/sql/schemachanger/testdata/drop_index_vanilla_index +++ b/pkg/sql/schemachanger/testdata/drop_index_vanilla_index @@ -12,7 +12,7 @@ begin transaction #1 # begin StatementPhase checking for feature: DROP INDEX increment telemetry for sql.schema.drop_index -## StatementPhase stage 1 of 1 with 4 MutationType ops +## StatementPhase stage 1 of 1 with 2 MutationType ops upsert descriptor #104 ... formatVersion: 3 @@ -81,8 +81,8 @@ upsert descriptor #104 + authorization: + userName: root + currentStatuses: - + - ABSENT - + - ABSENT + + - PUBLIC + + - PUBLIC + - VALIDATED + - ABSENT + jobId: "1" @@ -154,15 +154,20 @@ notified job registry to adopt jobs: [1] begin transaction #2 commit transaction #2 begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 2 with 3 MutationType ops +## PostCommitNonRevertiblePhase stage 1 of 2 with 5 MutationType ops upsert descriptor #104 ... - - ABSENT - - ABSENT + userName: root + currentStatuses: + - - PUBLIC + - - PUBLIC - - VALIDATED - + - DELETE_ONLY - ABSENT + + - ABSENT + + - DELETE_ONLY + + - ABSENT jobId: "1" + relevantStatements: ... id: 104 indexes: [] From a298133adc280e357012afc302d87d67b7b6d618 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Sun, 10 Jul 2022 14:45:46 -0400 Subject: [PATCH 4/9] sql/schemachanger/scplan/internal/scgraph: allow multiple dep rules to apply Before this change, due to how the depEdgeTree works, we'd arbitrarily throw away matching depEdges for the purpose of planning. This was extremely confusing in some cases. Release note: None --- .../testdata/decomp/multiregion | 3 ++ .../testdata/logic_test/new_schema_changer | 4 +- .../scplan/internal/scgraph/dep_edge_tree.go | 11 +++++ .../scplan/internal/scgraph/edge.go | 49 ++++++++++++++++--- .../scplan/internal/scgraph/graph.go | 15 ++++-- .../scplan/internal/scgraphviz/graphviz.go | 2 +- .../scplan/internal/scstage/build.go | 2 +- pkg/sql/schemachanger/scplan/plan_explain.go | 4 +- pkg/sql/schemachanger/scplan/plan_test.go | 6 ++- .../scplan/testdata/create_index | 12 ++--- .../schemachanger/scplan/testdata/drop_index | 24 ++++----- .../schemachanger/scplan/testdata/drop_table | 6 +-- 12 files changed, 102 insertions(+), 36 deletions(-) diff --git a/pkg/ccl/schemachangerccl/testdata/decomp/multiregion b/pkg/ccl/schemachangerccl/testdata/decomp/multiregion index dc3dd9924188..049136c97404 100644 --- a/pkg/ccl/schemachangerccl/testdata/decomp/multiregion +++ b/pkg/ccl/schemachangerccl/testdata/decomp/multiregion @@ -257,6 +257,7 @@ ElementState: - ColumnDefaultExpression: columnId: 2 expr: unique_rowid() + referencedColumnIds: [] tableId: 110 usesSequenceIds: [] usesTypeIds: [] @@ -505,6 +506,7 @@ ElementState: - ColumnDefaultExpression: columnId: 2 expr: unique_rowid() + referencedColumnIds: [] tableId: 109 usesSequenceIds: [] usesTypeIds: [] @@ -800,6 +802,7 @@ ElementState: - ColumnDefaultExpression: columnId: 3 expr: default_to_database_primary_region(gateway_region())::@100105 + referencedColumnIds: [] tableId: 108 usesSequenceIds: [] usesTypeIds: diff --git a/pkg/sql/logictest/testdata/logic_test/new_schema_changer b/pkg/sql/logictest/testdata/logic_test/new_schema_changer index 02d3ed1cfc1f..fd59d000e2fe 100644 --- a/pkg/sql/logictest/testdata/logic_test/new_schema_changer +++ b/pkg/sql/logictest/testdata/logic_test/new_schema_changer @@ -29,7 +29,8 @@ EXPLAIN (DDL, VERBOSE) ALTER TABLE foo ADD COLUMN j INT │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} │ │ │ │ ABSENT → PUBLIC │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ └── • SameStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column name set right after column existence" │ │ │ rule: "column existence precedes column dependents" │ │ │ │ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} @@ -173,7 +174,6 @@ EXPLAIN (DDL, VERBOSE) ALTER TABLE foo ADD COLUMN j INT JobID: 1 RunningStatus: all stages completed - statement ok ALTER TABLE foo ADD COLUMN j INT diff --git a/pkg/sql/schemachanger/scplan/internal/scgraph/dep_edge_tree.go b/pkg/sql/schemachanger/scplan/internal/scgraph/dep_edge_tree.go index 76975ff10d7e..0d61753276b2 100644 --- a/pkg/sql/schemachanger/scplan/internal/scgraph/dep_edge_tree.go +++ b/pkg/sql/schemachanger/scplan/internal/scgraph/dep_edge_tree.go @@ -70,6 +70,17 @@ func (et *depEdgeTree) insert(e *DepEdge) { }) } +func (et *depEdgeTree) get(e *DepEdge) *DepEdge { + got, ok := et.t.Get(&edgeTreeEntry{ + t: et, + edge: e, + }).(*edgeTreeEntry) + if !ok { + return nil + } + return got.edge +} + func (et *depEdgeTree) iterateSourceNode(n *screl.Node, it DepEdgeIterator) (err error) { e := &edgeTreeEntry{t: et, edge: &DepEdge{}} if et.order == fromTo { diff --git a/pkg/sql/schemachanger/scplan/internal/scgraph/edge.go b/pkg/sql/schemachanger/scplan/internal/scgraph/edge.go index 9f3c9e7a3a59..86724458a73f 100644 --- a/pkg/sql/schemachanger/scplan/internal/scgraph/edge.go +++ b/pkg/sql/schemachanger/scplan/internal/scgraph/edge.go @@ -12,6 +12,7 @@ package scgraph import ( "fmt" + "strings" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" @@ -91,9 +92,34 @@ type DepEdge struct { from, to *screl.Node kind DepEdgeKind - // TODO(ajwerner): Deal with the possibility that multiple rules could - // generate the same edge. - rule RuleName + rules []Rule +} + +// Rule describes a reason for a DepEdge to exist. +type Rule struct { + Name RuleName + Kind DepEdgeKind +} + +// RuleNames is a slice of RuleName. +type RuleNames []RuleName + +// String makes RuleNames a fmt.Stringer. +func (rn RuleNames) String() string { + var sb strings.Builder + if len(rn) == 1 { + sb.WriteString(string(rn[0])) + } else { + sb.WriteString("[") + for i, r := range rn { + if i > 0 { + sb.WriteString("; ") + } + sb.WriteString(string(r)) + } + sb.WriteString("]") + } + return sb.String() } // From implements the Edge interface. @@ -102,10 +128,21 @@ func (de *DepEdge) From() *screl.Node { return de.from } // To implements the Edge interface. func (de *DepEdge) To() *screl.Node { return de.to } -// Name returns the name of the rule which generated this edge. -func (de *DepEdge) Name() RuleName { return de.rule } +// RuleNames returns the names of the rules which generated this edge. +func (de *DepEdge) RuleNames() RuleNames { + ret := make(RuleNames, len(de.rules)) + for i, r := range de.rules { + ret[i] = r.Name + } + return ret +} + +// Rules returns the metadata about the rules which generated this edge. +func (de *DepEdge) Rules() []Rule { return de.rules } -// Kind returns the kind of the DepEdge. +// Kind returns the kind of the DepEdge. Note that it returns the strongest +// kind implied by a rule; if one rule which created this edge is Precedence, +// and another is SameStagePrecedence, this will return SameStagePrecedence. func (de *DepEdge) Kind() DepEdgeKind { return de.kind } // String returns a string representation of this edge diff --git a/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go b/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go index f9746b6077f5..41a920f36d37 100644 --- a/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go +++ b/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go @@ -247,20 +247,29 @@ func (g *Graph) GetOpEdgeFromOp(op scop.Op) *OpEdge { // AddDepEdge adds a dep edge connecting two nodes (specified by their targets // and statuses). func (g *Graph) AddDepEdge( - rule RuleName, + ruleName RuleName, kind DepEdgeKind, fromTarget *scpb.Target, fromStatus scpb.Status, toTarget *scpb.Target, toStatus scpb.Status, ) (err error) { - de := &DepEdge{rule: rule, kind: kind} + de := &DepEdge{kind: kind} + rule := Rule{Name: ruleName, Kind: kind} if de.from, err = g.getOrCreateNode(fromTarget, fromStatus); err != nil { return err } if de.to, err = g.getOrCreateNode(toTarget, toStatus); err != nil { return err } + if got := g.depEdgesFrom.get(de); got != nil { + if got.kind == Precedence && kind == SameStagePrecedence { + got.kind = SameStagePrecedence + } + got.rules = append(got.rules, rule) + return + } + de.rules = []Rule{rule} g.edges = append(g.edges, de) g.depEdgesFrom.insert(de) g.depEdgesTo.insert(de) @@ -363,7 +372,7 @@ func cycleErrorDetail(target *screl.Node, edge Edge, pred map[*screl.Node]Edge) sb.WriteString(screl.NodeString(e.From())) sb.WriteString(" --> ") if de, ok := e.(*DepEdge); ok { - sb.WriteString(string(de.rule)) + sb.WriteString(de.RuleNames().String()) } else { sb.WriteString("op edge") } diff --git a/pkg/sql/schemachanger/scplan/internal/scgraphviz/graphviz.go b/pkg/sql/schemachanger/scplan/internal/scgraphviz/graphviz.go index 02ead31ca1c3..150a811f776e 100644 --- a/pkg/sql/schemachanger/scplan/internal/scgraphviz/graphviz.go +++ b/pkg/sql/schemachanger/scplan/internal/scgraphviz/graphviz.go @@ -228,7 +228,7 @@ func drawDeps(cs scpb.CurrentState, g *scgraph.Graph) (*dot.Graph, error) { ge.Attr("fontsize", "9") case *scgraph.DepEdge: ge.Attr("color", "red") - ge.Attr("label", e.Name()) + ge.Attr("label", e.RuleNames()) if e.Kind() == scgraph.SameStagePrecedence { ge.Attr("arrowhead", "diamond") } diff --git a/pkg/sql/schemachanger/scplan/internal/scstage/build.go b/pkg/sql/schemachanger/scplan/internal/scstage/build.go index 2be650938bb7..c38c7786e00c 100644 --- a/pkg/sql/schemachanger/scplan/internal/scstage/build.go +++ b/pkg/sql/schemachanger/scplan/internal/scstage/build.go @@ -353,7 +353,7 @@ func (sb *stageBuilder) isUnmetInboundDep(de *scgraph.DepEdge) bool { } // The dependency constraint is somehow unsatisfiable. panic(errors.AssertionFailedf("failed to satisfy %s rule %q", - de.String(), de.Name())) + de.String(), de.RuleNames())) } // hasUnmeetableOutboundDeps returns true iff the candidate node has outbound diff --git a/pkg/sql/schemachanger/scplan/plan_explain.go b/pkg/sql/schemachanger/scplan/plan_explain.go index 674c2ec91aa6..74cc47d34b1f 100644 --- a/pkg/sql/schemachanger/scplan/plan_explain.go +++ b/pkg/sql/schemachanger/scplan/plan_explain.go @@ -225,7 +225,9 @@ func (p Plan) explainTargets(s scstage.Stage, sn treeprinter.Node, style treepri for _, de := range depEdges { rn := en.Childf("%s dependency from %s %s", de.Kind(), de.From().CurrentStatus, screl.ElementString(de.From().Element())) - rn.AddLine(fmt.Sprintf("rule: %q", de.Name())) + for _, r := range de.Rules() { + rn.AddLine(fmt.Sprintf("rule: %q", r.Name)) + } } noOpEdges := noOpByElement[t.Element()] for _, oe := range noOpEdges { diff --git a/pkg/sql/schemachanger/scplan/plan_test.go b/pkg/sql/schemachanger/scplan/plan_test.go index fc3cfdbc6028..d83bf244dc73 100644 --- a/pkg/sql/schemachanger/scplan/plan_test.go +++ b/pkg/sql/schemachanger/scplan/plan_test.go @@ -173,7 +173,11 @@ func marshalDeps(t *testing.T, plan *scplan.Plan) string { fmt.Fprintf(&deps, " to: [%s, %s]\n", screl.ElementString(de.To().Element()), de.To().CurrentStatus) fmt.Fprintf(&deps, " kind: %s\n", de.Kind()) - fmt.Fprintf(&deps, " rule: %s\n", de.Name()) + if rn := de.RuleNames(); len(rn) == 1 { + fmt.Fprintf(&deps, " rule: %s\n", rn) + } else { + fmt.Fprintf(&deps, " rules: %s\n", rn) + } sortedDeps = append(sortedDeps, deps.String()) return nil }) diff --git a/pkg/sql/schemachanger/scplan/testdata/create_index b/pkg/sql/schemachanger/scplan/testdata/create_index index 9d84a8e0c333..71f5a7255fea 100644 --- a/pkg/sql/schemachanger/scplan/testdata/create_index +++ b/pkg/sql/schemachanger/scplan/testdata/create_index @@ -227,15 +227,15 @@ CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) - from: [TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, PUBLIC] kind: Precedence - rule: index-column added to index after temp index exists + rules: [temp index exists before columns, partitioning, and partial; index-column added to index after temp index exists] - from: [TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, PUBLIC] kind: Precedence - rule: index-column added to index after temp index exists + rules: [temp index exists before columns, partitioning, and partial; index-column added to index after temp index exists] - from: [TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, PUBLIC] kind: Precedence - rule: index-column added to index after temp index exists + rules: [temp index exists before columns, partitioning, and partial; index-column added to index after temp index exists] - from: [TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1}, WRITE_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILLED] kind: Precedence @@ -471,15 +471,15 @@ CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) - from: [TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, PUBLIC] kind: Precedence - rule: index-column added to index after temp index exists + rules: [temp index exists before columns, partitioning, and partial; index-column added to index after temp index exists] - from: [TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, PUBLIC] kind: Precedence - rule: index-column added to index after temp index exists + rules: [temp index exists before columns, partitioning, and partial; index-column added to index after temp index exists] - from: [TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, PUBLIC] kind: Precedence - rule: index-column added to index after temp index exists + rules: [temp index exists before columns, partitioning, and partial; index-column added to index after temp index exists] - from: [TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1}, WRITE_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILLED] kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_index b/pkg/sql/schemachanger/scplan/testdata/drop_index index 6a400f2fd3f0..0e62c339ed5d 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_index +++ b/pkg/sql/schemachanger/scplan/testdata/drop_index @@ -109,11 +109,11 @@ DROP INDEX idx1 CASCADE - from: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, ABSENT] kind: Precedence - rule: dependents removed before index + rules: [secondary index columns removed before removing the index; dependents removed before index] - from: [IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, ABSENT] kind: Precedence - rule: dependents removed before index + rules: [secondary index columns removed before removing the index; dependents removed before index] - from: [IndexName:{DescID: 104, Name: idx1, IndexID: 2}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, ABSENT] kind: Precedence @@ -294,15 +294,15 @@ DROP INDEX idx2 CASCADE - from: [ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4}, ABSENT] to: [Column:{DescID: 104, ColumnID: 4}, ABSENT] kind: SameStagePrecedence - rule: column type removed right before column when not dropping relation + rules: [dependents removed before column; column type removed right before column when not dropping relation] - from: [IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, ABSENT] kind: Precedence - rule: dependents removed before index + rules: [secondary index columns removed before removing the index; dependents removed before index] - from: [IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 4}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, ABSENT] kind: Precedence - rule: dependents removed before index + rules: [secondary index columns removed before removing the index; dependents removed before index] - from: [IndexName:{DescID: 104, Name: idx2, IndexID: 4}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, ABSENT] kind: Precedence @@ -334,7 +334,7 @@ DROP INDEX idx2 CASCADE - from: [SecondaryIndexPartial:{DescID: 104, IndexID: 4}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, ABSENT] kind: SameStagePrecedence - rule: partial predicate removed right before secondary index when not dropping relation + rules: [dependents removed before index; partial predicate removed right before secondary index when not dropping relation] ops DROP INDEX idx3 CASCADE @@ -506,19 +506,19 @@ DROP INDEX idx3 CASCADE - from: [ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 5}, ABSENT] to: [Column:{DescID: 104, ColumnID: 5}, ABSENT] kind: SameStagePrecedence - rule: column type removed right before column when not dropping relation + rules: [dependents removed before column; column type removed right before column when not dropping relation] - from: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 6}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT] kind: Precedence - rule: dependents removed before index + rules: [secondary index columns removed before removing the index; dependents removed before index] - from: [IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT] kind: Precedence - rule: dependents removed before index + rules: [secondary index columns removed before removing the index; dependents removed before index] - from: [IndexColumn:{DescID: 104, ColumnID: 5, IndexID: 6}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT] kind: Precedence - rule: dependents removed before index + rules: [secondary index columns removed before removing the index; dependents removed before index] - from: [IndexName:{DescID: 104, Name: idx3, IndexID: 6}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT] kind: Precedence @@ -803,11 +803,11 @@ DROP INDEX idx4 CASCADE - from: [IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 8}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 9}, ABSENT] kind: Precedence - rule: dependents removed before index + rules: [secondary index columns removed before removing the index; dependents removed before index] - from: [IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 8}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 9}, ABSENT] kind: Precedence - rule: dependents removed before index + rules: [secondary index columns removed before removing the index; dependents removed before index] - from: [IndexName:{DescID: 104, Name: idx4, IndexID: 8}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 9}, ABSENT] kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_table b/pkg/sql/schemachanger/scplan/testdata/drop_table index 89a86a0fc00c..ac529b561dcd 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_table +++ b/pkg/sql/schemachanger/scplan/testdata/drop_table @@ -709,7 +709,7 @@ DROP TABLE defaultdb.shipments CASCADE; - from: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 2}, ABSENT] to: [SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, ABSENT] kind: Precedence - rule: dependents removed before index + rules: [secondary index columns removed before removing the index; dependents removed before index] - from: [IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT] to: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT] kind: Precedence @@ -721,7 +721,7 @@ DROP TABLE defaultdb.shipments CASCADE; - from: [IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 2}, ABSENT] to: [SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, ABSENT] kind: Precedence - rule: dependents removed before index + rules: [secondary index columns removed before removing the index; dependents removed before index] - from: [IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 1}, ABSENT] to: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT] kind: Precedence @@ -729,7 +729,7 @@ DROP TABLE defaultdb.shipments CASCADE; - from: [IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 2}, ABSENT] to: [SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, ABSENT] kind: Precedence - rule: dependents removed before index + rules: [secondary index columns removed before removing the index; dependents removed before index] - from: [IndexColumn:{DescID: 109, ColumnID: 5, IndexID: 1}, ABSENT] to: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT] kind: Precedence From f533d7380d55d92252f0a5af349bb7f691362338 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Sun, 10 Jul 2022 14:48:47 -0400 Subject: [PATCH 5/9] sql/schemachanger/scdecomp,scpb: track referenced columns in expressions This is needed to determine whether an expression uses a column. Release note: None --- pkg/sql/schemachanger/scbuild/testdata/drop_index | 8 ++++---- pkg/sql/schemachanger/scdecomp/BUILD.bazel | 1 + pkg/sql/schemachanger/scdecomp/helpers.go | 15 ++++++++++++--- pkg/sql/schemachanger/scdecomp/testdata/sequence | 2 ++ pkg/sql/schemachanger/scdecomp/testdata/table | 4 ++++ pkg/sql/schemachanger/scdecomp/testdata/type | 7 +++++++ pkg/sql/schemachanger/scpb/elements.proto | 3 +++ .../testdata/drop_index_hash_sharded_index | 8 ++++++++ .../testdata/drop_index_partial_expression_index | 8 ++++++++ 9 files changed, 49 insertions(+), 7 deletions(-) diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_index b/pkg/sql/schemachanger/scbuild/testdata/drop_index index 70c2874cdfca..49adf974d010 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_index +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_index @@ -30,13 +30,13 @@ DROP INDEX idx2 CASCADE - [[ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4}, ABSENT], PUBLIC] {columnId: 4, name: crdb_internal_idx_expr, tableId: 104} - [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4}, ABSENT], PUBLIC] - {columnId: 4, computeExpr: {expr: lower(j)}, isNullable: true, isVirtual: true, tableId: 104, type: {family: StringFamily, oid: 25}} + {columnId: 4, computeExpr: {expr: lower(j), referencedColumnIds: [2]}, isNullable: true, isVirtual: true, tableId: 104, type: {family: StringFamily, oid: 25}} - [[IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 4}, ABSENT], PUBLIC] {columnId: 4, indexId: 4, tableId: 104} - [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4}, ABSENT], PUBLIC] {columnId: 3, indexId: 4, kind: KEY_SUFFIX, tableId: 104} - [[SecondaryIndexPartial:{DescID: 104, IndexID: 4}, ABSENT], PUBLIC] - {expr: 'i > 0:::INT8', indexId: 4, tableId: 104} + {expr: 'i > 0:::INT8', indexId: 4, referencedColumnIds: [1], tableId: 104} - [[SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, ABSENT], PUBLIC] {indexId: 4, isCreatedExplicitly: true, tableId: 104} - [[IndexName:{DescID: 104, Name: idx2, IndexID: 4}, ABSENT], PUBLIC] @@ -50,7 +50,7 @@ DROP INDEX idx3 CASCADE - [[ColumnName:{DescID: 104, Name: crdb_internal_i_shard_16, ColumnID: 5}, ABSENT], PUBLIC] {columnId: 5, name: crdb_internal_i_shard_16, tableId: 104} - [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 5}, ABSENT], PUBLIC] - {columnId: 5, computeExpr: {expr: 'mod(fnv32(crdb_internal.datums_to_bytes(i)), 16:::INT8)'}, isVirtual: true, tableId: 104, type: {family: IntFamily, oid: 20, width: 64}} + {columnId: 5, computeExpr: {expr: 'mod(fnv32(crdb_internal.datums_to_bytes(i)), 16:::INT8)', referencedColumnIds: [1]}, isVirtual: true, tableId: 104, type: {family: IntFamily, oid: 20, width: 64}} - [[IndexColumn:{DescID: 104, ColumnID: 5, IndexID: 6}, ABSENT], PUBLIC] {columnId: 5, indexId: 6, tableId: 104} - [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 6}, ABSENT], PUBLIC] @@ -62,6 +62,6 @@ DROP INDEX idx3 CASCADE - [[IndexName:{DescID: 104, Name: idx3, IndexID: 6}, ABSENT], PUBLIC] {indexId: 6, name: idx3, tableId: 104} - [[CheckConstraint:{DescID: 104, ConstraintID: 4}, ABSENT], PUBLIC] - {columnIds: [5], constraintId: 4, expr: 'crdb_internal_i_shard_16 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8)', tableId: 104} + {columnIds: [5], constraintId: 4, expr: 'crdb_internal_i_shard_16 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8)', referencedColumnIds: [5], tableId: 104} - [[ConstraintName:{DescID: 104, Name: check_crdb_internal_i_shard_16, ConstraintID: 4}, ABSENT], PUBLIC] {constraintId: 4, name: check_crdb_internal_i_shard_16, tableId: 104} diff --git a/pkg/sql/schemachanger/scdecomp/BUILD.bazel b/pkg/sql/schemachanger/scdecomp/BUILD.bazel index aff0f56b8de0..136b7937be53 100644 --- a/pkg/sql/schemachanger/scdecomp/BUILD.bazel +++ b/pkg/sql/schemachanger/scdecomp/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/sql/catalog", "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/schemaexpr", "//pkg/sql/catalog/seqexpr", "//pkg/sql/catalog/typedesc", "//pkg/sql/parser", diff --git a/pkg/sql/schemachanger/scdecomp/helpers.go b/pkg/sql/schemachanger/scdecomp/helpers.go index 18584946ace8..c29eeed4ecb7 100644 --- a/pkg/sql/schemachanger/scdecomp/helpers.go +++ b/pkg/sql/schemachanger/scdecomp/helpers.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/catalog/seqexpr" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -108,10 +109,18 @@ func (w *walkCtx) newExpression(expr string) (*scpb.Expression, error) { } } } + + referencedColumns, err := schemaexpr.ExtractColumnIDs( + w.desc.(catalog.TableDescriptor), e, + ) + if err != nil { + return nil, err + } return &scpb.Expression{ - Expr: catpb.Expression(expr), - UsesTypeIDs: typIDs.Ordered(), - UsesSequenceIDs: seqIDs.Ordered(), + Expr: catpb.Expression(expr), + UsesTypeIDs: typIDs.Ordered(), + UsesSequenceIDs: seqIDs.Ordered(), + ReferencedColumnIDs: referencedColumns.Ordered(), }, nil } diff --git a/pkg/sql/schemachanger/scdecomp/testdata/sequence b/pkg/sql/schemachanger/scdecomp/testdata/sequence index eb43945bba31..ae15de5b4d20 100644 --- a/pkg/sql/schemachanger/scdecomp/testdata/sequence +++ b/pkg/sql/schemachanger/scdecomp/testdata/sequence @@ -280,6 +280,7 @@ ElementState: - ColumnDefaultExpression: columnId: 2 expr: nextval(104:::REGCLASS) + referencedColumnIds: [] tableId: 105 usesSequenceIds: - 104 @@ -288,6 +289,7 @@ ElementState: - ColumnOnUpdateExpression: columnId: 2 expr: 123:::INT8 + referencedColumnIds: [] tableId: 105 usesSequenceIds: [] usesTypeIds: [] diff --git a/pkg/sql/schemachanger/scdecomp/testdata/table b/pkg/sql/schemachanger/scdecomp/testdata/table index 52804139203a..a678864a6b87 100644 --- a/pkg/sql/schemachanger/scdecomp/testdata/table +++ b/pkg/sql/schemachanger/scdecomp/testdata/table @@ -477,6 +477,8 @@ ElementState: expr: id > 0:::INT8 indexId: 2 isRelationBeingDropped: false + referencedColumnIds: + - 1 tableId: 105 usesSequenceIds: [] usesTypeIds: [] @@ -911,6 +913,7 @@ ElementState: columnId: 1 computeExpr: expr: x'80':::@100106::STRING + referencedColumnIds: [] usesSequenceIds: [] usesTypeIds: - 106 @@ -1018,6 +1021,7 @@ ElementState: - ColumnDefaultExpression: columnId: 2 expr: unique_rowid() + referencedColumnIds: [] tableId: 108 usesSequenceIds: [] usesTypeIds: [] diff --git a/pkg/sql/schemachanger/scdecomp/testdata/type b/pkg/sql/schemachanger/scdecomp/testdata/type index a0018fd7396b..50de3979c8d2 100644 --- a/pkg/sql/schemachanger/scdecomp/testdata/type +++ b/pkg/sql/schemachanger/scdecomp/testdata/type @@ -177,6 +177,9 @@ ElementState: - 5 constraintId: 2 expr: s::STRING = name + referencedColumnIds: + - 3 + - 5 tableId: 108 usesSequenceIds: [] usesTypeIds: [] @@ -223,6 +226,7 @@ ElementState: columnId: 2 computeExpr: expr: x'80':::@100104 + referencedColumnIds: [] usesSequenceIds: [] usesTypeIds: - 104 @@ -301,6 +305,7 @@ ElementState: columnId: 3 computeExpr: expr: x'80':::@100106 + referencedColumnIds: [] usesSequenceIds: [] usesTypeIds: - 106 @@ -446,6 +451,8 @@ ElementState: expr: g::STRING = 'hi':::STRING indexId: 2 isRelationBeingDropped: false + referencedColumnIds: + - 2 tableId: 108 usesSequenceIds: [] usesTypeIds: [] diff --git a/pkg/sql/schemachanger/scpb/elements.proto b/pkg/sql/schemachanger/scpb/elements.proto index 12e784505130..caf9a199ae39 100644 --- a/pkg/sql/schemachanger/scpb/elements.proto +++ b/pkg/sql/schemachanger/scpb/elements.proto @@ -135,6 +135,9 @@ message Expression { string expr = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb.Expression"]; repeated uint32 uses_type_ids = 2 [(gogoproto.customname) = "UsesTypeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; repeated uint32 uses_sequence_ids = 3 [(gogoproto.customname) = "UsesSequenceIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; + + // ReferencedColumnIDs stores the IDs of the columns referenced by the expression. + repeated uint32 referenced_column_ids = 4 [(gogoproto.customname) = "ReferencedColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.ColumnID"]; } message Column { diff --git a/pkg/sql/schemachanger/testdata/drop_index_hash_sharded_index b/pkg/sql/schemachanger/testdata/drop_index_hash_sharded_index index 32163a241a77..20b3fa619128 100644 --- a/pkg/sql/schemachanger/testdata/drop_index_hash_sharded_index +++ b/pkg/sql/schemachanger/testdata/drop_index_hash_sharded_index @@ -189,6 +189,8 @@ upsert descriptor #104 + columnId: 3 + computeExpr: + expr: mod(fnv32(crdb_internal.datums_to_bytes(j)), 16:::INT8) + + referencedColumnIds: + + - 2 + isVirtual: true + tableId: 104 + type: @@ -260,6 +262,8 @@ upsert descriptor #104 + expr: crdb_internal_j_shard_16 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, + 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, + 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8) + + referencedColumnIds: + + - 3 + tableId: 104 + metadata: + sourceElementId: 1 @@ -407,6 +411,8 @@ upsert descriptor #104 - columnId: 3 - computeExpr: - expr: mod(fnv32(crdb_internal.datums_to_bytes(j)), 16:::INT8) + - referencedColumnIds: + - - 2 - isVirtual: true - tableId: 104 - type: @@ -478,6 +484,8 @@ upsert descriptor #104 - expr: crdb_internal_j_shard_16 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, - 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, - 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8) + - referencedColumnIds: + - - 3 - tableId: 104 - metadata: - sourceElementId: 1 diff --git a/pkg/sql/schemachanger/testdata/drop_index_partial_expression_index b/pkg/sql/schemachanger/testdata/drop_index_partial_expression_index index 952ea0c61618..3387c5a7aa37 100644 --- a/pkg/sql/schemachanger/testdata/drop_index_partial_expression_index +++ b/pkg/sql/schemachanger/testdata/drop_index_partial_expression_index @@ -159,6 +159,8 @@ upsert descriptor #104 + columnId: 3 + computeExpr: + expr: lower(j) + + referencedColumnIds: + + - 2 + isNullable: true + isVirtual: true + tableId: 104 @@ -192,6 +194,8 @@ upsert descriptor #104 + secondaryIndexPartial: + expr: i > 0:::INT8 + indexId: 2 + + referencedColumnIds: + + - 1 + tableId: 104 + metadata: + sourceElementId: 1 @@ -340,6 +344,8 @@ upsert descriptor #104 - columnId: 3 - computeExpr: - expr: lower(j) + - referencedColumnIds: + - - 2 - isNullable: true - isVirtual: true - tableId: 104 @@ -373,6 +379,8 @@ upsert descriptor #104 - secondaryIndexPartial: - expr: i > 0:::INT8 - indexId: 2 + - referencedColumnIds: + - - 1 - tableId: 104 - metadata: - sourceElementId: 1 From dc3ba94e4e840c11e3488da1784607415afe224a Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Sun, 10 Jul 2022 14:52:01 -0400 Subject: [PATCH 6/9] sql/sem/catid: add IndexSet Release note: None --- pkg/sql/sem/catid/BUILD.bazel | 6 ++- pkg/sql/sem/catid/index_id_set.go | 87 +++++++++++++++++++++++++++++++ 2 files changed, 92 insertions(+), 1 deletion(-) create mode 100644 pkg/sql/sem/catid/index_id_set.go diff --git a/pkg/sql/sem/catid/BUILD.bazel b/pkg/sql/sem/catid/BUILD.bazel index 8612711790dc..a46d15068968 100644 --- a/pkg/sql/sem/catid/BUILD.bazel +++ b/pkg/sql/sem/catid/BUILD.bazel @@ -3,11 +3,15 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "catid", - srcs = ["ids.go"], + srcs = [ + "ids.go", + "index_id_set.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid", visibility = ["//visibility:public"], deps = [ "//pkg/sql/oidext", + "//pkg/util", "@com_github_lib_pq//oid", ], ) diff --git a/pkg/sql/sem/catid/index_id_set.go b/pkg/sql/sem/catid/index_id_set.go new file mode 100644 index 000000000000..a3c140130b63 --- /dev/null +++ b/pkg/sql/sem/catid/index_id_set.go @@ -0,0 +1,87 @@ +// 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 catid + +import "github.com/cockroachdb/cockroach/pkg/util" + +// IndexSet efficiently stores an unordered set of index ids. +type IndexSet struct { + set util.FastIntSet +} + +// MakeIndexIDSet returns a set initialized with the given values. +func MakeIndexIDSet(vals ...IndexID) IndexSet { + var res IndexSet + for _, v := range vals { + res.Add(v) + } + return res +} + +// Add adds an index to the set. No-op if the index is already in the set. +func (s *IndexSet) Add(col IndexID) { s.set.Add(int(col)) } + +// Contains returns true if the set contains the index. +func (s IndexSet) Contains(col IndexID) bool { return s.set.Contains(int(col)) } + +// Empty returns true if the set is empty. +func (s IndexSet) Empty() bool { return s.set.Empty() } + +// Len returns the number of the indexes in the set. +func (s IndexSet) Len() int { return s.set.Len() } + +// Next returns the first value in the set which is >= startVal. If there is no +// value, the second return value is false. +func (s IndexSet) Next(startVal IndexID) (IndexID, bool) { + c, ok := s.set.Next(int(startVal)) + return IndexID(c), ok +} + +// ForEach calls a function for each index in the set (in increasing order). +func (s IndexSet) ForEach(f func(col IndexID)) { + s.set.ForEach(func(i int) { f(IndexID(i)) }) +} + +// SubsetOf returns true if s is a subset of other. +func (s IndexSet) SubsetOf(other IndexSet) bool { + return s.set.SubsetOf(other.set) +} + +// Intersection returns the intersection between s and other. +func (s IndexSet) Intersection(other IndexSet) IndexSet { + return IndexSet{set: s.set.Intersection(other.set)} +} + +// Difference returns the index IDs in s which are not in other. +func (s IndexSet) Difference(other IndexSet) IndexSet { + return IndexSet{set: s.set.Difference(other.set)} +} + +// Ordered returns a slice with all the IndexIDs in the set, in +// increasing order. +func (s IndexSet) Ordered() []IndexID { + if s.Empty() { + return nil + } + result := make([]IndexID, 0, s.Len()) + s.ForEach(func(i IndexID) { + result = append(result, i) + }) + return result +} + +// UnionWith adds all the indexes from rhs to this set. +func (s *IndexSet) UnionWith(rhs IndexSet) { s.set.UnionWith(rhs.set) } + +// String returns a list representation of elements. Sequential runs of positive +// numbers are shown as ranges. For example, for the set {1, 2, 3 5, 6, 10}, +// the output is "(1-3,5,6,10)". +func (s IndexSet) String() string { return s.set.String() } From 5cf66fa1c23ca9b36577e09dc78dbaf284e71093 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Sun, 10 Jul 2022 15:25:33 -0400 Subject: [PATCH 7/9] sql/schemachanger/scexec/scmutationexec: fix bug rolling back dropped index Release note: None --- pkg/sql/schemachanger/scexec/scmutationexec/index.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/index.go b/pkg/sql/schemachanger/scexec/scmutationexec/index.go index db1936fe32cf..537c94c678ce 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/index.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/index.go @@ -178,6 +178,9 @@ func (m *visitor) MakeAddedSecondaryIndexPublic( for idx, idxMutation := range tbl.GetMutations() { if idxMutation.GetIndex() != nil && idxMutation.GetIndex().ID == op.IndexID { + // If this is a rollback of a drop, we are trying to add the index back, + // so swap the direction before making it complete. + idxMutation.Direction = descpb.DescriptorMutation_ADD err := tbl.MakeMutationComplete(idxMutation) if err != nil { return err From ea759fec7d40d2a43771c6bed360be0c87997237 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 11 Jul 2022 16:43:56 -0400 Subject: [PATCH 8/9] sql/catalog,tabledesc: make MutationPublicationFilter more flexible I wanted to extend this and making it internally a set of filters works better than a hodge-podge of policies. Release note: None --- pkg/sql/backfill.go | 4 ++- pkg/sql/backfill/backfill.go | 2 +- pkg/sql/catalog/descriptor.go | 11 +++----- pkg/sql/catalog/tabledesc/structured.go | 37 ++++++++++++++++++++++--- pkg/sql/rowexec/indexbackfiller.go | 2 +- 5 files changed, 42 insertions(+), 14 deletions(-) diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 31453d079be2..da720e24eb3b 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -1800,7 +1800,9 @@ func populateExpectedCounts( // added earlier in the same mutation. Make the mutations public in an // in-memory copy of the descriptor and add it to the Collection's synthetic // descriptors, so that we can use SQL below to perform the validation. - fakeDesc, err := tableDesc.MakeFirstMutationPublic(catalog.IgnoreConstraintsAndPKSwaps) + fakeDesc, err := tableDesc.MakeFirstMutationPublic( + catalog.IgnoreConstraints, catalog.IgnorePKSwaps, + ) if err != nil { return 0, err } diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 6e9fcab80720..94d4e9eef7ec 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -425,7 +425,7 @@ func ConvertBackfillError( // information useful in printing a sensible error. However // ConvertBatchError() will only work correctly if the schema elements // are "live" in the tableDesc. - desc, err := tableDesc.MakeFirstMutationPublic(catalog.IncludeConstraints) + desc, err := tableDesc.MakeFirstMutationPublic() if err != nil { return err } diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 3b885cb3e08e..b72ec0fce798 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -56,13 +56,10 @@ const ( // table descriptor returned should not include newly added constraints, which // is useful when passing the returned table descriptor to be used in // validating constraints to be added. - IgnoreConstraints MutationPublicationFilter = 1 - // IgnoreConstraintsAndPKSwaps is used in MakeFirstMutationPublic to indicate that the + IgnoreConstraints MutationPublicationFilter = iota + // IgnorePKSwaps is used in MakeFirstMutationPublic to indicate that the // table descriptor returned should include newly added constraints. - IgnoreConstraintsAndPKSwaps = 2 - // IncludeConstraints is used in MakeFirstMutationPublic to indicate that the - // table descriptor returned should include newly added constraints. - IncludeConstraints = 0 + IgnorePKSwaps ) // DescriptorBuilder interfaces are used to build catalog.Descriptor @@ -589,7 +586,7 @@ type TableDescriptor interface { // This is super valuable when trying to run SQL over data associated // with a schema mutation that is still not yet public: Data validation, // error reporting. - MakeFirstMutationPublic(includeConstraints MutationPublicationFilter) (TableDescriptor, error) + MakeFirstMutationPublic(...MutationPublicationFilter) (TableDescriptor, error) // MakePublic creates a Mutable from the immutable by making the it public. MakePublic() TableDescriptor // AllMutations returns all of the table descriptor's mutations. diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 0189ba93bfef..2912af1c33a2 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" @@ -2208,12 +2209,13 @@ func (desc *Mutable) addMutationWithNextID(m descpb.DescriptorMutation) { // MakeFirstMutationPublic implements the TableDescriptor interface. func (desc *wrapper) MakeFirstMutationPublic( - includeConstraints catalog.MutationPublicationFilter, + filters ...catalog.MutationPublicationFilter, ) (catalog.TableDescriptor, error) { // Clone the ImmutableTable descriptor because we want to create an ImmutableCopy one. table := desc.NewBuilder().(TableDescriptorBuilder).BuildExistingMutableTable() mutationID := desc.Mutations[0].MutationID i := 0 + policy := makeMutationPublicationPolicy(filters...) for _, mutation := range desc.Mutations { if mutation.MutationID != mutationID { // Mutations are applied in a FIFO order. Only apply the first set @@ -2221,9 +2223,7 @@ func (desc *wrapper) MakeFirstMutationPublic( break } i++ - if mutation.GetPrimaryKeySwap() != nil && includeConstraints == catalog.IgnoreConstraintsAndPKSwaps { - continue - } else if mutation.GetConstraint() != nil && includeConstraints > catalog.IncludeConstraints { + if policy.shouldSkip(&mutation) { continue } if err := table.MakeMutationComplete(mutation); err != nil { @@ -2235,6 +2235,35 @@ func (desc *wrapper) MakeFirstMutationPublic( return table, nil } +type mutationPublicationPolicy struct { + policy util.FastIntSet +} + +func makeMutationPublicationPolicy( + filters ...catalog.MutationPublicationFilter, +) mutationPublicationPolicy { + var p mutationPublicationPolicy + for _, f := range filters { + p.policy.Add(int(f)) + } + return p +} + +func (p mutationPublicationPolicy) includes(f catalog.MutationPublicationFilter) bool { + return p.policy.Contains(int(f)) +} + +func (p mutationPublicationPolicy) shouldSkip(m *descpb.DescriptorMutation) bool { + switch { + case m.GetPrimaryKeySwap() != nil: + return p.includes(catalog.IgnorePKSwaps) + case m.GetConstraint() != nil: + return p.includes(catalog.IgnoreConstraints) + default: + return false + } +} + // MakePublic implements the TableDescriptor interface. func (desc *wrapper) MakePublic() catalog.TableDescriptor { // Clone the ImmutableTable descriptor because we want to create an ImmutableCopy one. diff --git a/pkg/sql/rowexec/indexbackfiller.go b/pkg/sql/rowexec/indexbackfiller.go index d8f92e0f56ff..fbf6672d829f 100644 --- a/pkg/sql/rowexec/indexbackfiller.go +++ b/pkg/sql/rowexec/indexbackfiller.go @@ -389,7 +389,7 @@ func (ib *indexBackfiller) wrapDupError(ctx context.Context, orig error) error { return orig } - desc, err := ib.desc.MakeFirstMutationPublic(catalog.IncludeConstraints) + desc, err := ib.desc.MakeFirstMutationPublic() if err != nil { return err } From 3c0c52bb35945d52e8a96d7e91f6ec369580e104 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Thu, 14 Jul 2022 00:19:43 -0400 Subject: [PATCH 9/9] sql/schemachanger/scplan: ensure revertibility applies correctly Without this patch, it would not apply at Statement or PreCommit Release note: None --- .../scplan/internal/scstage/build.go | 4 +- .../schemachanger/scplan/testdata/drop_index | 20 ++++----- .../scplan/testdata/drop_owned_by | 22 ++++----- .../testdata/drop_index_hash_sharded_index | 45 ++++++++++++------- 4 files changed, 53 insertions(+), 38 deletions(-) diff --git a/pkg/sql/schemachanger/scplan/internal/scstage/build.go b/pkg/sql/schemachanger/scplan/internal/scstage/build.go index c38c7786e00c..67942c109b9f 100644 --- a/pkg/sql/schemachanger/scplan/internal/scstage/build.go +++ b/pkg/sql/schemachanger/scplan/internal/scstage/build.go @@ -283,7 +283,9 @@ func (sb stageBuilder) isOutgoingOpEdgeAllowed(e *scgraph.OpEdge) bool { if !e.IsPhaseSatisfied(sb.bs.phase) { return false } - if !sb.bc.isRevertibilityIgnored && sb.bs.phase == scop.PostCommitPhase && !e.Revertible() { + if !sb.bc.isRevertibilityIgnored && + sb.bs.phase < scop.PostCommitNonRevertiblePhase && + !e.Revertible() { return false } return true diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_index b/pkg/sql/schemachanger/scplan/testdata/drop_index index 0e62c339ed5d..ec190d9eb33f 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_index +++ b/pkg/sql/schemachanger/scplan/testdata/drop_index @@ -339,14 +339,12 @@ DROP INDEX idx2 CASCADE ops DROP INDEX idx3 CASCADE ---- -StatementPhase stage 1 of 1 with 7 MutationType ops +StatementPhase stage 1 of 1 with 5 MutationType ops transitions: [[Column:{DescID: 104, ColumnID: 5}, ABSENT], PUBLIC] -> WRITE_ONLY [[ColumnName:{DescID: 104, Name: crdb_internal_i_shard_16, ColumnID: 5}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED [[IndexName:{DescID: 104, Name: idx3, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT - [[CheckConstraint:{DescID: 104, ConstraintID: 4}, ABSENT], PUBLIC] -> ABSENT - [[ConstraintName:{DescID: 104, Name: check_crdb_internal_i_shard_16, ConstraintID: 4}, ABSENT], PUBLIC] -> ABSENT ops: *scop.MakeDroppedColumnDeleteAndWriteOnly ColumnID: 5 @@ -378,11 +376,6 @@ StatementPhase stage 1 of 1 with 7 MutationType ops IndexID: 6 Name: crdb_internal_index_6_name_placeholder TableID: 104 - *scop.RemoveCheckConstraint - ConstraintID: 4 - TableID: 104 - *scop.NotImplemented - ElementType: scpb.ConstraintName PreCommitPhase stage 1 of 1 with 2 MutationType ops transitions: ops: @@ -396,22 +389,29 @@ PreCommitPhase stage 1 of 1 with 2 MutationType ops - 104 JobID: 1 NonCancelable: true - RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 5 MutationType ops pending + RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 7 MutationType ops pending Statements: - statement: DROP INDEX idx3 CASCADE redactedstatement: DROP INDEX ‹defaultdb›.public.‹t1›@‹idx3› CASCADE statementtag: DROP INDEX -PostCommitNonRevertiblePhase stage 1 of 2 with 7 MutationType ops +PostCommitNonRevertiblePhase stage 1 of 2 with 9 MutationType ops transitions: [[Column:{DescID: 104, ColumnID: 5}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[IndexColumn:{DescID: 104, ColumnID: 5, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT], VALIDATED] -> DELETE_ONLY + [[CheckConstraint:{DescID: 104, ConstraintID: 4}, ABSENT], PUBLIC] -> ABSENT + [[ConstraintName:{DescID: 104, Name: check_crdb_internal_i_shard_16, ConstraintID: 4}, ABSENT], PUBLIC] -> ABSENT ops: *scop.MakeDroppedColumnDeleteOnly ColumnID: 5 TableID: 104 + *scop.RemoveCheckConstraint + ConstraintID: 4 + TableID: 104 + *scop.NotImplemented + ElementType: scpb.ConstraintName *scop.MakeDroppedIndexDeleteOnly IndexID: 6 TableID: 104 diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_owned_by b/pkg/sql/schemachanger/scplan/testdata/drop_owned_by index 358eb8ba5aa3..b8bc786134c5 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_owned_by +++ b/pkg/sql/schemachanger/scplan/testdata/drop_owned_by @@ -17,11 +17,9 @@ CREATE VIEW s.v2 AS (SELECT 'a'::s.typ::string AS k, name FROM s.v1); ops DROP OWNED BY r ---- -StatementPhase stage 1 of 1 with 11 MutationType ops +StatementPhase stage 1 of 1 with 9 MutationType ops transitions: - [[UserPrivileges:{DescID: 100, Name: r}, ABSENT], PUBLIC] -> ABSENT [[Schema:{DescID: 105}, ABSENT], PUBLIC] -> OFFLINE - [[UserPrivileges:{DescID: 104, Name: r}, ABSENT], PUBLIC] -> ABSENT [[Sequence:{DescID: 106}, ABSENT], PUBLIC] -> OFFLINE [[Table:{DescID: 109}, ABSENT], PUBLIC] -> OFFLINE [[Column:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY @@ -50,15 +48,9 @@ StatementPhase stage 1 of 1 with 11 MutationType ops [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY ops: - *scop.RemoveUserPrivileges - DescID: 100 - User: r *scop.MarkDescriptorAsOffline DescID: 105 Reason: DROP OWNED BY r - *scop.RemoveUserPrivileges - DescID: 104 - User: r *scop.MarkDescriptorAsOffline DescID: 106 Reason: DROP OWNED BY r @@ -136,14 +128,15 @@ PreCommitPhase stage 1 of 1 with 12 MutationType ops - 113 JobID: 1 NonCancelable: true - RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 32 MutationType ops + RunningStatus: PostCommitNonRevertiblePhase stage 1 of 2 with 34 MutationType ops pending Statements: - statement: DROP OWNED BY r redactedstatement: DROP OWNED BY r statementtag: DROP OWNED BY -PostCommitNonRevertiblePhase stage 1 of 2 with 44 MutationType ops +PostCommitNonRevertiblePhase stage 1 of 2 with 46 MutationType ops transitions: + [[UserPrivileges:{DescID: 100, Name: r}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 105, Name: s, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT @@ -151,6 +144,7 @@ PostCommitNonRevertiblePhase stage 1 of 2 with 44 MutationType ops [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT [[Schema:{DescID: 105}, ABSENT], OFFLINE] -> DROPPED [[SchemaParent:{DescID: 105, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: r}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 106, Name: sq, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] -> ABSENT @@ -276,12 +270,18 @@ PostCommitNonRevertiblePhase stage 1 of 2 with 44 MutationType ops [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: + *scop.RemoveUserPrivileges + DescID: 100 + User: r *scop.MarkDescriptorAsDropped DescID: 105 *scop.RemoveSchemaParent Parent: ParentDatabaseID: 100 SchemaID: 105 + *scop.RemoveUserPrivileges + DescID: 104 + User: r *scop.MarkDescriptorAsDropped DescID: 106 *scop.RemoveAllTableComments diff --git a/pkg/sql/schemachanger/testdata/drop_index_hash_sharded_index b/pkg/sql/schemachanger/testdata/drop_index_hash_sharded_index index 20b3fa619128..5796bce9d0f1 100644 --- a/pkg/sql/schemachanger/testdata/drop_index_hash_sharded_index +++ b/pkg/sql/schemachanger/testdata/drop_index_hash_sharded_index @@ -12,21 +12,19 @@ begin transaction #1 # begin StatementPhase checking for feature: DROP INDEX increment telemetry for sql.schema.drop_index -## StatementPhase stage 1 of 1 with 7 MutationType ops +## StatementPhase stage 1 of 1 with 5 MutationType ops upsert descriptor #104 - table: - - checks: - - - columnIds: - - - 3 - - constraintId: 4 + ... + - 3 + constraintId: 4 - expr: crdb_internal_j_shard_16 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, - 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, 11:::INT8, 12:::INT8, - 13:::INT8, 14:::INT8, 15:::INT8) - - hidden: true - - name: check_crdb_internal_j_shard_16 - + checks: [] - columns: - - id: 1 + + expr: crdb_internal_column_3_name_placeholder IN (0:::INT8, 1:::INT8, 2:::INT8, + + 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, + + 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8) + hidden: true + name: check_crdb_internal_j_shard_16 ... oid: 20 width: 64 @@ -145,8 +143,8 @@ upsert descriptor #104 + - PUBLIC + - VALIDATED + - ABSENT - + - ABSENT - + - ABSENT + + - PUBLIC + + - PUBLIC + jobId: "1" + relevantStatements: + - statement: @@ -298,8 +296,21 @@ notified job registry to adopt jobs: [1] begin transaction #2 commit transaction #2 begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 2 with 7 MutationType ops +## PostCommitNonRevertiblePhase stage 1 of 2 with 9 MutationType ops upsert descriptor #104 + table: + - checks: + - - columnIds: + - - 3 + - constraintId: 4 + - expr: crdb_internal_column_3_name_placeholder IN (0:::INT8, 1:::INT8, 2:::INT8, + - 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, + - 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8) + - hidden: true + - name: check_crdb_internal_j_shard_16 + + checks: [] + columns: + - id: 1 ... userName: root currentStatuses: @@ -312,8 +323,10 @@ upsert descriptor #104 - - PUBLIC - - VALIDATED - ABSENT - - ABSENT - - ABSENT + - - PUBLIC + - - PUBLIC + + - ABSENT + + - ABSENT + - DELETE_ONLY + - ABSENT + - ABSENT