Skip to content

Commit

Permalink
Merge #84157
Browse files Browse the repository at this point in the history
84157: sql/schemachanger: prerequisite changes for DROP COLUMN r=ajwerner a=ajwerner

This PR comes in a number of commits and is best reviewed commit-by-commit. This PR contains cleanup needed to implement `DROP COLUMN` (#84072). The actual support for that feature will follow in a separate PR.

* Two commits do some internal refactoring of the builder.
* The next commit adjust the planning rules which over-constrained the graph.
* The next fixes a rather thorny bug related to multiple dep edges between the same two nodes not being handled correctly.
* The next commit augments descriptor decomposition to include columns referenced in expressions.
* The next commit adds an `IndexSet` to `catid`.
* The next commit fixes a bug which prevents adding a dropping index back to a table.
* The next commit reworks `MakeFirstMutationPublic` in order to make introducing a new filter easier.
* Fixes a bug in scstage which lead to ignoring revertibility in the Statement and PreCommit phases

Co-authored-by: Andrew Werner <[email protected]>
  • Loading branch information
craig[bot] and ajwerner committed Jul 14, 2022
2 parents 840d146 + 3c0c52b commit 6ea03b9
Show file tree
Hide file tree
Showing 37 changed files with 843 additions and 360 deletions.
3 changes: 3 additions & 0 deletions pkg/ccl/schemachangerccl/testdata/decomp/multiregion
Original file line number Diff line number Diff line change
Expand Up @@ -257,6 +257,7 @@ ElementState:
- ColumnDefaultExpression:
columnId: 2
expr: unique_rowid()
referencedColumnIds: []
tableId: 110
usesSequenceIds: []
usesTypeIds: []
Expand Down Expand Up @@ -505,6 +506,7 @@ ElementState:
- ColumnDefaultExpression:
columnId: 2
expr: unique_rowid()
referencedColumnIds: []
tableId: 109
usesSequenceIds: []
usesTypeIds: []
Expand Down Expand Up @@ -800,6 +802,7 @@ ElementState:
- ColumnDefaultExpression:
columnId: 3
expr: default_to_database_primary_region(gateway_region())::@100105
referencedColumnIds: []
tableId: 108
usesSequenceIds: []
usesTypeIds:
Expand Down
4 changes: 3 additions & 1 deletion pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
11 changes: 4 additions & 7 deletions pkg/sql/catalog/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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.
Expand Down
37 changes: 33 additions & 4 deletions pkg/sql/catalog/tabledesc/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -2208,22 +2209,21 @@ 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
// of mutations if they have the mutation ID we're looking for.
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 {
Expand All @@ -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.
Expand Down
26 changes: 14 additions & 12 deletions pkg/sql/logictest/testdata/logic_test/new_schema_changer
Original file line number Diff line number Diff line change
Expand Up @@ -29,14 +29,15 @@ 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}
│ │ │ │ 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
Expand Down Expand Up @@ -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
Expand All @@ -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}
Expand All @@ -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..."}
Expand All @@ -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'

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/rowexec/indexbackfiller.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
Loading

0 comments on commit 6ea03b9

Please sign in to comment.