diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 9990679ca202..8eb7d58c059a 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -1691,11 +1691,19 @@ func ValidateForwardIndexes( invalid <- idx.GetID() return nil } + // Resolve the table index descriptor name. + indexName, err := tableDesc.GetIndexNameByID(idx.GetID()) + if err != nil { + log.Warningf(ctx, + "unable to find index by ID for ValidateForwardIndexes: %d", + idx.GetID()) + indexName = idx.GetName() + } // TODO(vivek): find the offending row and include it in the error. return pgerror.WithConstraintName(pgerror.Newf(pgcode.UniqueViolation, "duplicate key value violates unique constraint %q", - idx.GetName()), - idx.GetName()) + indexName), + indexName) } case <-ctx.Done(): diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index e4f7e79e68f1..215eb3987d34 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -710,6 +710,12 @@ type TableDescriptor interface { // GetAutoStatsSettings returns the table settings related to automatic // statistics collection. May return nil if none are set. GetAutoStatsSettings() *catpb.AutoStatsSettings + // GetIndexNameByID returns the name of an index based on an ID, taking into + // account any ongoing declarative schema changes. Declarative schema changes + // do not propagate the index name into the mutations until changes are fully + // validated and swap operations are complete (to avoid having two constraints + // with the same name). + GetIndexNameByID(indexID descpb.IndexID) (name string, err error) } // MutableTableDescriptor is both a MutableDescriptor and a TableDescriptor. diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index d9d3e32b835a..7e633c9fec8f 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -714,7 +714,6 @@ func (desc *Mutable) allocateIndexIDs(columnNames map[string]descpb.ColumnID) er if err != nil { return err } - var compositeColIDs catalog.TableColSet for i := range desc.Columns { col := &desc.Columns[i] diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index fa7fe98c8afa..a15b05724bdc 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/errors" ) @@ -591,3 +592,24 @@ func (desc *wrapper) getExistingOrNewMutationCache() *mutationCache { func (desc *wrapper) AllMutations() []catalog.Mutation { return desc.getExistingOrNewMutationCache().all } + +func (desc *wrapper) GetIndexNameByID(indexID descpb.IndexID) (string, error) { + // Check if there are any ongoing schema changes and prefer the name from + // them. + if scState := desc.GetDeclarativeSchemaChangerState(); scState != nil { + for _, target := range scState.Targets { + if target.IndexName != nil && + target.TargetStatus == scpb.Status_PUBLIC && + target.IndexName.TableID == desc.GetID() && + target.IndexName.IndexID == indexID { + return target.IndexName.Name, nil + } + } + } + // Otherwise, try fetching the name from the index descriptor. + index, err := desc.FindIndexWithID(indexID) + if err != nil { + return "", err + } + return index.GetName(), err +} diff --git a/pkg/sql/logictest/testdata/logic_test/new_schema_changer b/pkg/sql/logictest/testdata/logic_test/new_schema_changer index 801ab21990f1..3bcaa4cb3a49 100644 --- a/pkg/sql/logictest/testdata/logic_test/new_schema_changer +++ b/pkg/sql/logictest/testdata/logic_test/new_schema_changer @@ -257,13 +257,13 @@ EXPLAIN (DDL, VERBOSE) ALTER TABLE foo ADD COLUMN j INT │ ├── • 2 elements transitioning toward ABSENT │ │ │ │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} -│ │ │ PUBLIC → VALIDATED +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from ABSENT IndexName:{DescID: 106, Name: foo_pkey, IndexID: 1} +│ │ │ rule: "dependents removed after index no longer public" │ │ │ │ │ └── • IndexName:{DescID: 106, Name: foo_pkey, IndexID: 1} -│ │ │ PUBLIC → ABSENT -│ │ │ -│ │ └── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} -│ │ rule: "dependents removed after index no longer public" +│ │ PUBLIC → ABSENT │ │ │ ├── • 3 elements transitioning toward PUBLIC │ │ │ @@ -290,10 +290,6 @@ EXPLAIN (DDL, VERBOSE) ALTER TABLE foo ADD COLUMN j INT │ │ │ └── • 8 Mutation operations │ │ -│ ├── • MakeDroppedPrimaryIndexDeleteAndWriteOnly -│ │ IndexID: 1 -│ │ TableID: 106 -│ │ │ ├── • SetIndexName │ │ IndexID: 1 │ │ Name: crdb_internal_index_1_name_placeholder @@ -304,6 +300,10 @@ EXPLAIN (DDL, VERBOSE) ALTER TABLE foo ADD COLUMN j INT │ │ Name: foo_pkey │ │ TableID: 106 │ │ +│ ├── • MakeDroppedPrimaryIndexDeleteAndWriteOnly +│ │ IndexID: 1 +│ │ TableID: 106 +│ │ │ ├── • MakeAddedPrimaryIndexPublic │ │ EventBase: │ │ Authorization: @@ -430,6 +430,7 @@ EXPLAIN (DDL, VERBOSE) ALTER TABLE foo ADD COLUMN j INT + statement ok ALTER TABLE foo ADD COLUMN j INT diff --git a/pkg/sql/row/errors.go b/pkg/sql/row/errors.go index 082734b042f7..49a0ca1bf0a0 100644 --- a/pkg/sql/row/errors.go +++ b/pkg/sql/row/errors.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -129,14 +130,21 @@ func NewUniquenessConstraintViolationError( return pgerror.Wrap(err, pgcode.UniqueViolation, "duplicate key value got decoding error") } - + // Resolve the table index descriptor name. + indexName, err := tableDesc.GetIndexNameByID(index.GetID()) + if err != nil { + log.Warningf(ctx, + "unable to find index by ID for NewUniquenessConstraintViolationError: %d", + index.GetID()) + indexName = index.GetName() + } // Exclude implicit partitioning columns and hash sharded index columns from // the error message. skipCols := index.ExplicitColumnStartIdx() return errors.WithDetail( pgerror.WithConstraintName(pgerror.Newf(pgcode.UniqueViolation, - "duplicate key value violates unique constraint %q", index.GetName(), - ), index.GetName()), + "duplicate key value violates unique constraint %q", indexName, + ), indexName), fmt.Sprintf( "Key (%s)=(%s) already exists.", strings.Join(names[skipCols:], ","), diff --git a/pkg/sql/schemachanger/scbuild/builder_state.go b/pkg/sql/schemachanger/scbuild/builder_state.go index f03b568ff081..ad0d1dcbef80 100644 --- a/pkg/sql/schemachanger/scbuild/builder_state.go +++ b/pkg/sql/schemachanger/scbuild/builder_state.go @@ -237,6 +237,11 @@ func (b *builderState) nextIndexID(id catid.DescID) (ret catid.IndexID) { ret = index.IndexID + 1 } }) + scpb.ForEachTemporaryIndex(b, func(_ scpb.Status, _ scpb.TargetStatus, index *scpb.TemporaryIndex) { + if index.TableID == id && index.IndexID >= ret { + ret = index.IndexID + 1 + } + }) scpb.ForEachSecondaryIndex(b, func(_ scpb.Status, _ scpb.TargetStatus, index *scpb.SecondaryIndex) { if index.TableID == id && index.IndexID >= ret { ret = index.IndexID + 1 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 e699f367da6d..94be4de8dbec 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 @@ -11,6 +11,10 @@ package scbuildstmt import ( + "fmt" + "strings" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -22,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" - "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -62,6 +65,19 @@ func alterTableAddColumn( if d.GeneratedIdentity.IsGeneratedAsIdentity { panic(scerrors.NotImplementedErrorf(d, "contains generated identity type")) } + // Unique without an index is unsupported. + if d.Unique.WithoutIndex { + // TODO(rytaft): add support for this in the future if we want to expose + // UNIQUE WITHOUT INDEX to users. + panic(errors.WithHint( + pgerror.New( + pgcode.FeatureNotSupported, + "adding a column marked as UNIQUE WITHOUT INDEX is unsupported", + ), + "add the column first, then run ALTER TABLE ... ADD CONSTRAINT to add a "+ + "UNIQUE WITHOUT INDEX constraint on the column", + )) + } if d.IsComputed() { d.Computed.Expr = schemaexpr.MaybeRewriteComputedColumn(d.Computed.Expr, b.SessionData()) } @@ -72,11 +88,6 @@ func alterTableAddColumn( "regional by row partitioning is not supported")) } } - // Some of the building for the index exists below but end-to-end support is - // not complete, so return an error for new unique columns. - if d.Unique.IsUnique { - panic(scerrors.NotImplementedErrorf(d, "contains unique constraint")) - } cdd, err := tabledesc.MakeColumnDefDescs(b, d, b.SemaCtx(), b.EvalCtx()) if err != nil { panic(err) @@ -188,11 +199,33 @@ func alterTableAddColumn( b.IncrementSchemaChangeAddColumnQualificationCounter("on_update") } // Add secondary indexes for this column. + var primaryIdx *scpb.PrimaryIndex + if newPrimary := addColumn(b, spec); newPrimary != nil { - if idx := cdd.PrimaryKeyOrUniqueIndexDescriptor; idx != nil { - idx.ID = b.NextTableIndexID(tbl) - addSecondaryIndexTargetsForAddColumn(b, tbl, idx, newPrimary.SourceIndexID) + primaryIdx = newPrimary + } else { + publicTargets := b.QueryByID(tbl.TableID).Filter( + func(_ scpb.Status, target scpb.TargetStatus, _ scpb.Element) bool { + return target == scpb.ToPublic + }, + ) + _, _, primaryIdx = scpb.FindPrimaryIndex(publicTargets) + } + if idx := cdd.PrimaryKeyOrUniqueIndexDescriptor; idx != nil { + idx.ID = b.NextTableIndexID(tbl) + { + tableElts := b.QueryByID(tbl.TableID) + namesToIDs := make(map[string]descpb.ColumnID) + scpb.ForEachColumnName(tableElts, func(current scpb.Status, target scpb.TargetStatus, e *scpb.ColumnName) { + if target == scpb.ToPublic { + namesToIDs[e.Name] = e.ColumnID + } + }) + for _, colName := range cdd.PrimaryKeyOrUniqueIndexDescriptor.KeyColumnNames { + cdd.PrimaryKeyOrUniqueIndexDescriptor.KeyColumnIDs = append(cdd.PrimaryKeyOrUniqueIndexDescriptor.KeyColumnIDs, namesToIDs[colName]) + } } + addSecondaryIndexTargetsForAddColumn(b, tbl, idx, primaryIdx) } switch spec.colType.Type.Family() { case types.EnumFamily: @@ -348,9 +381,97 @@ func addColumn(b BuildCtx, spec addColumnSpec) (backing *scpb.PrimaryIndex) { return replacement } +// getImplicitSecondaryIndexName determines the implicit name for a secondary +// index, this logic matches tabledesc.BuildIndexName. +func getImplicitSecondaryIndexName( + b BuildCtx, tbl *scpb.Table, id descpb.IndexID, numImplicitColumns int, +) string { + elts := b.QueryByID(tbl.TableID) + var idx *scpb.Index + scpb.ForEachSecondaryIndex(elts, func(current scpb.Status, target scpb.TargetStatus, e *scpb.SecondaryIndex) { + if e.IndexID == id { + idx = &e.Index + } + }) + if idx == nil { + panic(errors.AssertionFailedf("unable to find secondary index.")) + } + // An index name has a segment for the table name, each key column, and a + // final word (either "idx" or "key"). + segments := make([]string, 0, len(idx.KeyColumnIDs)+2) + // Add the table name segment. + var tblName *scpb.Namespace + scpb.ForEachNamespace(b, func(current scpb.Status, target scpb.TargetStatus, e *scpb.Namespace) { + if e.DescriptorID == tbl.TableID { + tblName = e + } + }) + if tblName == nil { + panic(errors.AssertionFailedf("unable to find table name.")) + } + segments = append(segments, tblName.Name) + findColumnNameByID := func(colID descpb.ColumnID) ElementResultSet { + var columnName *scpb.ColumnName + scpb.ForEachColumnName(b, func(current scpb.Status, target scpb.TargetStatus, e *scpb.ColumnName) { + if e.ColumnID == colID { + columnName = e + } + }) + if columnName == nil { + panic(errors.AssertionFailedf("unable to find column name.")) + } + return b.ResolveColumn(tbl.TableID, tree.Name(columnName.Name), ResolveParams{}) + } + // Add the key column segments. For inaccessible columns, use "expr" as the + // segment. If there are multiple inaccessible columns, add an incrementing + // integer suffix. + exprCount := 0 + for i, n := numImplicitColumns, len(idx.KeyColumnIDs); i < n; i++ { + var segmentName string + colElts := findColumnNameByID(idx.KeyColumnIDs[i]) + _, _, col := scpb.FindColumnType(colElts) + if col.ComputeExpr != nil { + if exprCount == 0 { + segmentName = "expr" + } else { + segmentName = fmt.Sprintf("expr%d", exprCount) + } + exprCount++ + } else { + _, _, colName := scpb.FindColumnName(colElts) + segmentName = colName.Name + } + segments = append(segments, segmentName) + } + + // Add the final segment. + if idx.IsUnique { + segments = append(segments, "key") + } else { + segments = append(segments, "idx") + } + // Append digits to the index name to make it unique, if necessary. + baseName := strings.Join(segments, "_") + name := baseName + for i := 1; ; i++ { + foundIndex := false + scpb.ForEachIndexName(elts, func(current scpb.Status, target scpb.TargetStatus, e *scpb.IndexName) { + if e.Name == name { + foundIndex = true + } + }) + if !foundIndex { + break + } + name = fmt.Sprintf("%s%d", baseName, i) + } + return name +} + func addSecondaryIndexTargetsForAddColumn( - b BuildCtx, tbl *scpb.Table, desc *descpb.IndexDescriptor, sourceID catid.IndexID, + b BuildCtx, tbl *scpb.Table, desc *descpb.IndexDescriptor, newPrimaryIdx *scpb.PrimaryIndex, ) { + var partitioning *catpb.PartitioningDescriptor index := scpb.Index{ TableID: tbl.TableID, IndexID: desc.ID, @@ -361,8 +482,10 @@ func addSecondaryIndexTargetsForAddColumn( CompositeColumnIDs: desc.CompositeColumnIDs, IsUnique: desc.Unique, IsInverted: desc.Type == descpb.IndexDescriptor_INVERTED, - SourceIndexID: sourceID, + SourceIndexID: newPrimaryIdx.IndexID, } + tempIndexID := index.IndexID + 1 // this is enforced below + index.TemporaryIndexID = tempIndexID for i, dir := range desc.KeyColumnDirections { if dir == descpb.IndexDescriptor_DESC { index.KeyColumnDirections[i] = scpb.Index_DESC @@ -371,17 +494,108 @@ func addSecondaryIndexTargetsForAddColumn( if desc.Sharded.IsSharded { index.Sharding = &desc.Sharded } - b.Add(&scpb.SecondaryIndex{Index: index}) + // If necessary add suffix columns, this would normally be done inside + // allocateIndexIDs, but we are going to do it explicitly for the declarative + // schema changer. + { + publicTargets := b.QueryByID(tbl.TableID).Filter( + func(_ scpb.Status, target scpb.TargetStatus, _ scpb.Element) bool { + return target == scpb.ToPublic + }, + ) + // Apply any implicit partitioning columns first, if they are missing. + scpb.ForEachIndexPartitioning(b, func(current scpb.Status, target scpb.TargetStatus, e *scpb.IndexPartitioning) { + if e.IndexID == newPrimaryIdx.IndexID && + e.TableID == newPrimaryIdx.TableID { + partitioning = &e.PartitioningDescriptor + } + }) + keyColSet := catalog.TableColSet{} + extraSuffixColumns := catalog.TableColSet{} + for _, colID := range index.KeyColumnIDs { + keyColSet.Add(colID) + } + if partitioning != nil && + len(desc.Partitioning.Range) == 0 && len(desc.Partitioning.List) == 0 && + partitioning.NumImplicitColumns > 0 { + keyColumns := make([]descpb.ColumnID, 0, len(index.KeyColumnIDs)+int(partitioning.NumImplicitColumns)) + for _, colID := range newPrimaryIdx.KeyColumnIDs[0:partitioning.NumImplicitColumns] { + if !keyColSet.Contains(colID) { + keyColumns = append(keyColumns, colID) + keyColSet.Add(colID) + } + } + index.KeyColumnIDs = append(keyColumns, index.KeyColumnIDs...) + } else if len(desc.Partitioning.Range) != 0 || len(desc.Partitioning.List) != 0 { + partitioning = &desc.Partitioning + } + for _, colID := range newPrimaryIdx.KeyColumnIDs { + if !keyColSet.Contains(colID) { + extraSuffixColumns.Add(colID) + } + } + if !extraSuffixColumns.Empty() { + index.KeySuffixColumnIDs = append(index.KeySuffixColumnIDs, extraSuffixColumns.Ordered()...) + } + // Add in any composite columns at the same time. + // CompositeColumnIDs is defined as the subset of columns in the index key + // or in the primary key whose type has a composite encoding, like DECIMAL + // for instance. + compositeColIDs := catalog.TableColSet{} + scpb.ForEachColumnType(publicTargets, func(current scpb.Status, target scpb.TargetStatus, e *scpb.ColumnType) { + if colinfo.CanHaveCompositeKeyEncoding(e.Type) { + compositeColIDs.Add(e.ColumnID) + } + }) + for _, colID := range index.KeyColumnIDs { + if compositeColIDs.Contains(colID) { + index.CompositeColumnIDs = append(index.CompositeColumnIDs, colID) + } + } + for _, colID := range index.KeySuffixColumnIDs { + if compositeColIDs.Contains(colID) { + index.CompositeColumnIDs = append(index.CompositeColumnIDs, colID) + } + } + } + sec := &scpb.SecondaryIndex{Index: index} + b.Add(sec) + indexName := desc.Name + numImplicitColumns := 0 + if partitioning != nil { + numImplicitColumns = int(partitioning.NumImplicitColumns) + } + if indexName == "" { + indexName = getImplicitSecondaryIndexName(b, tbl, index.IndexID, numImplicitColumns) + } b.Add(&scpb.IndexName{ TableID: tbl.TableID, IndexID: index.IndexID, - Name: desc.Name, + Name: indexName, }) - if p := &desc.Partitioning; len(p.List)+len(p.Range) > 0 { + temp := &scpb.TemporaryIndex{ + Index: protoutil.Clone(sec).(*scpb.SecondaryIndex).Index, + IsUsingSecondaryEncoding: true, + } + temp.TemporaryIndexID = 0 + temp.IndexID = nextRelationIndexID(b, tbl) + if temp.IndexID != tempIndexID { + panic(errors.AssertionFailedf( + "assumed temporary index ID %d != %d", tempIndexID, temp.IndexID, + )) + } + b.AddTransient(temp) + // Add in the partitioning descriptor for the final and temporary index. + if partitioning != nil { b.Add(&scpb.IndexPartitioning{ TableID: tbl.TableID, IndexID: index.IndexID, - PartitioningDescriptor: *protoutil.Clone(p).(*catpb.PartitioningDescriptor), + PartitioningDescriptor: *protoutil.Clone(partitioning).(*catpb.PartitioningDescriptor), + }) + b.Add(&scpb.IndexPartitioning{ + TableID: tbl.TableID, + IndexID: temp.IndexID, + PartitioningDescriptor: *protoutil.Clone(partitioning).(*catpb.PartitioningDescriptor), }) } } diff --git a/pkg/sql/schemachanger/scbuild/testdata/create_index b/pkg/sql/schemachanger/scbuild/testdata/create_index index 5504ad7e236c..5e8003335ef0 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/create_index +++ b/pkg/sql/schemachanger/scbuild/testdata/create_index @@ -165,9 +165,9 @@ CREATE INDEX id4 oid: 23 width: 32 tableId: 104 -- [[IndexName:{DescID: 104, Name: id4, IndexID: 3}, PUBLIC], ABSENT] +- [[IndexName:{DescID: 104, Name: id4, IndexID: 4}, PUBLIC], ABSENT] details: - indexId: 3 + indexId: 4 name: id4 tableId: 104 - [[IndexName:{DescID: 104, Name: t1_pkey, IndexID: 1}, ABSENT], PUBLIC] @@ -211,10 +211,10 @@ CREATE INDEX id4 - 4 tableId: 104 temporaryIndexId: 3 -- [[SecondaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 0, TemporaryIndexID: 4}, PUBLIC], ABSENT] +- [[SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5}, PUBLIC], ABSENT] details: embeddedIndex: - indexId: 3 + indexId: 4 keyColumnDirections: - ASC - ASC @@ -232,7 +232,7 @@ CREATE INDEX id4 storingColumnIds: - 3 tableId: 104 - temporaryIndexId: 4 + temporaryIndexId: 5 - [[TemporaryIndex:{DescID: 104, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] details: embeddedIndex: @@ -249,10 +249,10 @@ CREATE INDEX id4 - 3 - 4 tableId: 104 -- [[TemporaryIndex:{DescID: 104, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] +- [[TemporaryIndex:{DescID: 104, IndexID: 5}, TRANSIENT_ABSENT], ABSENT] details: embeddedIndex: - indexId: 4 + indexId: 5 keyColumnDirections: - ASC - ASC diff --git a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table index 5d3f0740ab7c..48ff65ea7ba0 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table +++ b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table @@ -2,10 +2,6 @@ setup CREATE TABLE defaultdb.foo (i INT PRIMARY KEY) ---- -unimplemented -ALTER TABLE defaultdb.foo ADD COLUMN j INT UNIQUE ----- - unimplemented ALTER TABLE defaultdb.foo ADD COLUMN j INT CHECK (j > 0) ---- diff --git a/pkg/sql/schemachanger/scexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/BUILD.bazel index 84c91c747e95..362d530f42e4 100644 --- a/pkg/sql/schemachanger/scexec/BUILD.bazel +++ b/pkg/sql/schemachanger/scexec/BUILD.bazel @@ -24,6 +24,8 @@ go_library( "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/nstree", + "//pkg/sql/pgwire/pgcode", + "//pkg/sql/pgwire/pgerror", "//pkg/sql/schemachanger/scerrors", "//pkg/sql/schemachanger/scexec/scmutationexec", "//pkg/sql/schemachanger/scop", diff --git a/pkg/sql/schemachanger/scexec/exec_backfill.go b/pkg/sql/schemachanger/scexec/exec_backfill.go index 8c9a56764680..18fd065507c9 100644 --- a/pkg/sql/schemachanger/scexec/exec_backfill.go +++ b/pkg/sql/schemachanger/scexec/exec_backfill.go @@ -16,6 +16,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" @@ -329,8 +331,18 @@ func runBackfiller( return im.MergeIndexes(ctx, *p, tracker, tables[p.TableID]) } if err := forEachProgressConcurrently(ctx, op, backfillProgresses, mergeProgresses, bf, mf); err != nil { - // We ran into an uncategorized schema change error. - deps.Telemetry().IncrementSchemaChangeErrorType("uncategorized") + pgCode := pgerror.GetPGCode(err) + // Determine the type of error we encountered. + if pgCode == pgcode.CheckViolation || + pgCode == pgcode.UniqueViolation || + pgCode == pgcode.ForeignKeyViolation || + pgCode == pgcode.NotNullViolation || + pgCode == pgcode.IntegrityConstraintViolation { + deps.Telemetry().IncrementSchemaChangeErrorType("constraint_violation") + } else { + // We ran into an uncategorized schema change error. + deps.Telemetry().IncrementSchemaChangeErrorType("uncategorized") + } return scerrors.SchemaChangerUserError(err) } if err := stop(); err != nil { diff --git a/pkg/sql/schemachanger/scexec/exec_validation.go b/pkg/sql/schemachanger/scexec/exec_validation.go index a5091a7d3167..31d468dac001 100644 --- a/pkg/sql/schemachanger/scexec/exec_validation.go +++ b/pkg/sql/schemachanger/scexec/exec_validation.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/errors" @@ -46,7 +47,10 @@ func executeValidateUniqueIndex( } else { err = deps.IndexValidator().ValidateInvertedIndexes(ctx, table, []catalog.Index{index}, execOverride) } - return err + if err != nil { + return scerrors.SchemaChangerUserError(err) + } + return nil } func executeValidateCheckConstraint( 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 4aba1466e077..d9ebdb98de10 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 @@ -43,6 +43,28 @@ func init() { } }, ) + + registerDepRule( + "reverting primary index swap", + scgraph.SameStagePrecedence, + "new-index", "old-index", + func(from, fromTarget, fromNode, to, toTarget, toNode rel.Var) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.PrimaryIndex)(nil)), + to.Type((*scpb.PrimaryIndex)(nil)), + join(from, to, screl.DescID, "table-id"), + targetStatus(fromTarget, scpb.ToAbsent), + targetStatus(toTarget, scpb.ToPublic), + currentStatus(fromNode, scpb.Status_VALIDATED), + currentStatus(toNode, scpb.Status_PUBLIC), + rel.Filter( + "new-primary-index-depends-on-old", from, to, + )(func(add, drop *scpb.PrimaryIndex) bool { + return add.SourceIndexID == drop.IndexID + }), + } + }, + ) } // These rules ensure that index-dependent elements, like an index's name, its @@ -145,24 +167,25 @@ func init() { registerDepRule( "dependents removed after index no longer public", - scgraph.Precedence, - "index", "child", + scgraph.SameStagePrecedence, + "child", "index", func(from, fromTarget, fromNode, to, toTarget, toNode rel.Var) rel.Clauses { return rel.Clauses{ from.Type( - (*scpb.PrimaryIndex)(nil), - (*scpb.SecondaryIndex)(nil), - ), - to.Type( (*scpb.IndexName)(nil), (*scpb.IndexPartitioning)(nil), (*scpb.SecondaryIndexPartial)(nil), (*scpb.IndexComment)(nil), ), + to.Type( + (*scpb.PrimaryIndex)(nil), + (*scpb.SecondaryIndex)(nil), + ), + joinOnIndexID(from, to), targetStatusEq(fromTarget, toTarget, scpb.ToAbsent), - currentStatus(fromNode, scpb.Status_VALIDATED), - currentStatus(toNode, scpb.Status_ABSENT), + currentStatus(fromNode, scpb.Status_ABSENT), + currentStatus(toNode, scpb.Status_VALIDATED), } }, ) @@ -232,7 +255,6 @@ func init() { } }, ) - registerDepRule( "column existence precedes column dependents", scgraph.Precedence, @@ -455,6 +477,9 @@ func init() { columnInList(from.ColumnID, idx.StoringColumnIDs) || columnInList(from.ColumnID, idx.KeySuffixColumnIDs) } + columnInPrimaryIndexSwap := func(from *scpb.Column, to *scpb.PrimaryIndex) bool { + return columnInIndex(from, to) && to.SourceIndexID != 0 + } registerDepRule( "column depends on primary index", @@ -473,7 +498,22 @@ func init() { } }, ) - + registerDepRule( + "primary index should be cleaned up before newly added column when reverting", + scgraph.Precedence, + "index", "column", + func(from, fromTarget, fromNode, to, toTarget, toNode rel.Var) rel.Clauses { + var status rel.Var = "status" + return rel.Clauses{ + from.Type((*scpb.PrimaryIndex)(nil)), + to.Type((*scpb.Column)(nil)), + targetStatusEq(fromTarget, toTarget, scpb.ToAbsent), + join(from, to, screl.DescID, "table-id"), + rel.Filter("columnFeaturedInIndex", to, from)(columnInPrimaryIndexSwap), + status.Eq(scpb.Status_WRITE_ONLY), + status.Entities(screl.CurrentStatus, fromNode, toNode), + } + }) registerDepRule( "column existence precedes index existence", scgraph.Precedence, @@ -507,6 +547,89 @@ func init() { } }, ) + primaryIndexHasSecondaryColumns := func(from *scpb.PrimaryIndex, to scpb.Element) bool { + switch to := to.(type) { + case *scpb.SecondaryIndex: + for _, colID := range from.StoringColumnIDs { + if columnInList(colID, to.KeyColumnIDs) || + columnInList(colID, to.StoringColumnIDs) || + columnInList(colID, to.KeySuffixColumnIDs) { + return true + } + } + case *scpb.TemporaryIndex: + if !to.IsUsingSecondaryEncoding { + return false + } + for _, colID := range from.StoringColumnIDs { + if columnInList(colID, to.KeyColumnIDs) || + columnInList(colID, to.StoringColumnIDs) || + columnInList(colID, to.KeySuffixColumnIDs) { + return true + } + } + } + return false + } + secondaryIndexHasPrimarySwapColumns := func(to scpb.Element, from *scpb.PrimaryIndex) bool { + if from.SourceIndexID == 0 { + return false + } + switch to := to.(type) { + case *scpb.SecondaryIndex: + for _, colID := range from.StoringColumnIDs { + if columnInList(colID, to.KeyColumnIDs) || + columnInList(colID, to.StoringColumnIDs) || + columnInList(colID, to.KeySuffixColumnIDs) { + return true + } + } + case *scpb.TemporaryIndex: + if !to.IsUsingSecondaryEncoding { + return false + } + for _, colID := range from.StoringColumnIDs { + if columnInList(colID, to.KeyColumnIDs) || + columnInList(colID, to.StoringColumnIDs) || + columnInList(colID, to.KeySuffixColumnIDs) { + return true + } + } + } + return false + } + registerDepRule( + "primary index with new columns should exist before secondary/temp indexes", + scgraph.Precedence, + "primary-index", "second-index", + func(from, fromTarget, fromNode, to, toTarget, toNode rel.Var) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.PrimaryIndex)(nil)), + to.Type((*scpb.SecondaryIndex)(nil), (*scpb.TemporaryIndex)(nil)), + join(from, to, screl.DescID, "table-id"), + targetStatus(fromTarget, scpb.ToPublic), + targetStatus(toTarget, scpb.ToPublic), + currentStatus(fromNode, scpb.Status_VALIDATED), + currentStatus(toNode, scpb.Status_BACKFILL_ONLY), + rel.Filter("newColumnFeaturedInIndex", from, to)(primaryIndexHasSecondaryColumns), + } + }) + registerDepRule( + "secondary indexes should be cleaned up before any primary index with columns when reverting", + scgraph.Precedence, + "second-index", "primary-index", + func(from, fromTarget, fromNode, to, toTarget, toNode rel.Var) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.SecondaryIndex)(nil), (*scpb.TemporaryIndex)(nil)), + to.Type((*scpb.PrimaryIndex)(nil)), + join(from, to, screl.DescID, "table-id"), + targetStatus(fromTarget, scpb.ToAbsent), + targetStatus(toTarget, scpb.ToAbsent), + currentStatus(fromNode, scpb.Status_ABSENT), + currentStatus(toNode, scpb.Status_VALIDATED), + rel.Filter("newColumnFeaturedInIndex", from, to)(secondaryIndexHasPrimarySwapColumns), + } + }) } // This rule ensures that columns depend on each other in increasing order. diff --git a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules index 64b2821f1134..b9c27b82b781 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules @@ -407,6 +407,30 @@ deprules - $new-index-node[Target] = $new-index-target - $old-index[DescID] = $var-to-tell-rel-from-is-an-element - $new-index[DescID] = $var-to-tell-rel-to-is-an-element +- name: reverting primary index swap + from: new-index-node + kind: SameStagePrecedence + to: old-index-node + query: + - $new-index[Type] = '*scpb.PrimaryIndex' + - $old-index[Type] = '*scpb.PrimaryIndex' + - $new-index[DescID] = $table-id + - $old-index[DescID] = $table-id + - $new-index-target[TargetStatus] = ABSENT + - $old-index-target[TargetStatus] = PUBLIC + - $new-index-node[CurrentStatus] = VALIDATED + - $old-index-node[CurrentStatus] = PUBLIC + - new-primary-index-depends-on-old(*scpb.PrimaryIndex, *scpb.PrimaryIndex)($new-index, $old-index) + - $new-index-target[Type] = '*scpb.Target' + - $new-index-target[Element] = $new-index + - $new-index-node[Type] = '*screl.Node' + - $new-index-node[Target] = $new-index-target + - $old-index-target[Type] = '*scpb.Target' + - $old-index-target[Element] = $old-index + - $old-index-node[Type] = '*screl.Node' + - $old-index-node[Target] = $old-index-target + - $new-index[DescID] = $var-to-tell-rel-from-is-an-element + - $old-index[DescID] = $var-to-tell-rel-to-is-an-element - name: index existence precedes index dependents from: index-node kind: Precedence @@ -537,30 +561,30 @@ deprules - $index-name[DescID] = $var-to-tell-rel-from-is-an-element - $index[DescID] = $var-to-tell-rel-to-is-an-element - name: dependents removed after index no longer public - from: index-node - kind: Precedence - to: child-node + from: child-node + kind: SameStagePrecedence + to: index-node query: - - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] - $child[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment'] - - $index[DescID] = $desc-id + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] - $child[DescID] = $desc-id - - $index[IndexID] = $index-id + - $index[DescID] = $desc-id - $child[IndexID] = $index-id - - $index-target[TargetStatus] = ABSENT + - $index[IndexID] = $index-id - $child-target[TargetStatus] = ABSENT - - $index-node[CurrentStatus] = VALIDATED + - $index-target[TargetStatus] = ABSENT - $child-node[CurrentStatus] = ABSENT - - $index-target[Type] = '*scpb.Target' - - $index-target[Element] = $index - - $index-node[Type] = '*screl.Node' - - $index-node[Target] = $index-target + - $index-node[CurrentStatus] = VALIDATED - $child-target[Type] = '*scpb.Target' - $child-target[Element] = $child - $child-node[Type] = '*screl.Node' - $child-node[Target] = $child-target - - $index[DescID] = $var-to-tell-rel-from-is-an-element - - $child[DescID] = $var-to-tell-rel-to-is-an-element + - $index-target[Type] = '*scpb.Target' + - $index-target[Element] = $index + - $index-node[Type] = '*screl.Node' + - $index-node[Target] = $index-target + - $child[DescID] = $var-to-tell-rel-from-is-an-element + - $index[DescID] = $var-to-tell-rel-to-is-an-element - name: dependents removed before index from: dependent-node kind: Precedence @@ -888,6 +912,31 @@ deprules - $column-node[Target] = $column-target - $index[DescID] = $var-to-tell-rel-from-is-an-element - $column[DescID] = $var-to-tell-rel-to-is-an-element +- name: primary index should be cleaned up before newly added column when reverting + from: index-node + kind: Precedence + to: column-node + query: + - $index[Type] = '*scpb.PrimaryIndex' + - $column[Type] = '*scpb.Column' + - $index-target[TargetStatus] = ABSENT + - $column-target[TargetStatus] = ABSENT + - $index[DescID] = $table-id + - $column[DescID] = $table-id + - columnFeaturedInIndex(*scpb.Column, *scpb.PrimaryIndex)($column, $index) + - $status = WRITE_ONLY + - $index-node[CurrentStatus] = $status + - $column-node[CurrentStatus] = $status + - $index-target[Type] = '*scpb.Target' + - $index-target[Element] = $index + - $index-node[Type] = '*screl.Node' + - $index-node[Target] = $index-target + - $column-target[Type] = '*scpb.Target' + - $column-target[Element] = $column + - $column-node[Type] = '*screl.Node' + - $column-node[Target] = $column-target + - $index[DescID] = $var-to-tell-rel-from-is-an-element + - $column[DescID] = $var-to-tell-rel-to-is-an-element - name: column existence precedes index existence from: column-node kind: Precedence @@ -936,6 +985,54 @@ deprules - $temp-index-node[Target] = $temp-index-target - $column[DescID] = $var-to-tell-rel-from-is-an-element - $temp-index[DescID] = $var-to-tell-rel-to-is-an-element +- name: primary index with new columns should exist before secondary/temp indexes + from: primary-index-node + kind: Precedence + to: second-index-node + query: + - $primary-index[Type] = '*scpb.PrimaryIndex' + - $second-index[Type] IN ['*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - $primary-index[DescID] = $table-id + - $second-index[DescID] = $table-id + - $primary-index-target[TargetStatus] = PUBLIC + - $second-index-target[TargetStatus] = PUBLIC + - $primary-index-node[CurrentStatus] = VALIDATED + - $second-index-node[CurrentStatus] = BACKFILL_ONLY + - newColumnFeaturedInIndex(*scpb.PrimaryIndex, scpb.Element)($primary-index, $second-index) + - $primary-index-target[Type] = '*scpb.Target' + - $primary-index-target[Element] = $primary-index + - $primary-index-node[Type] = '*screl.Node' + - $primary-index-node[Target] = $primary-index-target + - $second-index-target[Type] = '*scpb.Target' + - $second-index-target[Element] = $second-index + - $second-index-node[Type] = '*screl.Node' + - $second-index-node[Target] = $second-index-target + - $primary-index[DescID] = $var-to-tell-rel-from-is-an-element + - $second-index[DescID] = $var-to-tell-rel-to-is-an-element +- name: secondary indexes should be cleaned up before any primary index with columns when reverting + from: second-index-node + kind: Precedence + to: primary-index-node + query: + - $second-index[Type] IN ['*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - $primary-index[Type] = '*scpb.PrimaryIndex' + - $second-index[DescID] = $table-id + - $primary-index[DescID] = $table-id + - $second-index-target[TargetStatus] = ABSENT + - $primary-index-target[TargetStatus] = ABSENT + - $second-index-node[CurrentStatus] = ABSENT + - $primary-index-node[CurrentStatus] = VALIDATED + - newColumnFeaturedInIndex(scpb.Element, *scpb.PrimaryIndex)($second-index, $primary-index) + - $second-index-target[Type] = '*scpb.Target' + - $second-index-target[Element] = $second-index + - $second-index-node[Type] = '*screl.Node' + - $second-index-node[Target] = $second-index-target + - $primary-index-target[Type] = '*scpb.Target' + - $primary-index-target[Element] = $primary-index + - $primary-index-node[Type] = '*screl.Node' + - $primary-index-node[Target] = $primary-index-target + - $second-index[DescID] = $var-to-tell-rel-from-is-an-element + - $primary-index[DescID] = $var-to-tell-rel-to-is-an-element - name: ensure columns are in increasing order from: later-column-node kind: SameStagePrecedence diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table b/pkg/sql/schemachanger/scplan/testdata/alter_table index 907d4a4bf2fd..d2c88d291056 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table @@ -186,9 +186,6 @@ PostCommitPhase stage 7 of 7 with 10 MutationType ops [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3}, PUBLIC], VALIDATED] -> PUBLIC [[IndexName:{DescID: 104, Name: foo_pkey, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC ops: - *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly - IndexID: 1 - TableID: 104 *scop.SetIndexName IndexID: 1 Name: crdb_internal_index_1_name_placeholder @@ -197,6 +194,9 @@ PostCommitPhase stage 7 of 7 with 10 MutationType ops IndexID: 2 Name: foo_pkey TableID: 104 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + IndexID: 1 + TableID: 104 *scop.MakeAddedPrimaryIndexPublic EventBase: Authorization: @@ -452,9 +452,6 @@ PostCommitPhase stage 7 of 7 with 8 MutationType ops [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3}, PUBLIC], VALIDATED] -> PUBLIC [[IndexName:{DescID: 104, Name: foo_pkey, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC ops: - *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly - IndexID: 1 - TableID: 104 *scop.SetIndexName IndexID: 1 Name: crdb_internal_index_1_name_placeholder @@ -463,6 +460,9 @@ PostCommitPhase stage 7 of 7 with 8 MutationType ops IndexID: 2 Name: foo_pkey TableID: 104 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + IndexID: 1 + TableID: 104 *scop.MakeAddedPrimaryIndexPublic EventBase: Authorization: @@ -761,9 +761,6 @@ PostCommitPhase stage 7 of 7 with 10 MutationType ops [[IndexName:{DescID: 104, Name: foo_pkey, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC [[Column:{DescID: 104, ColumnID: 3}, PUBLIC], WRITE_ONLY] -> PUBLIC ops: - *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly - IndexID: 1 - TableID: 104 *scop.SetIndexName IndexID: 1 Name: crdb_internal_index_1_name_placeholder @@ -772,6 +769,9 @@ PostCommitPhase stage 7 of 7 with 10 MutationType ops IndexID: 2 Name: foo_pkey TableID: 104 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + IndexID: 1 + TableID: 104 *scop.MakeAddedPrimaryIndexPublic EventBase: Authorization: @@ -1023,9 +1023,6 @@ PostCommitPhase stage 7 of 7 with 8 MutationType ops [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3}, PUBLIC], VALIDATED] -> PUBLIC [[IndexName:{DescID: 104, Name: foo_pkey, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC ops: - *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly - IndexID: 1 - TableID: 104 *scop.SetIndexName IndexID: 1 Name: crdb_internal_index_1_name_placeholder @@ -1034,6 +1031,9 @@ PostCommitPhase stage 7 of 7 with 8 MutationType ops IndexID: 2 Name: foo_pkey TableID: 104 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + IndexID: 1 + TableID: 104 *scop.MakeAddedPrimaryIndexPublic EventBase: Authorization: @@ -1393,9 +1393,6 @@ PostCommitPhase stage 7 of 7 with 15 MutationType ops [[PrimaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3}, PUBLIC], VALIDATED] -> PUBLIC [[IndexName:{DescID: 107, Name: bar_pkey, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC ops: - *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly - IndexID: 1 - TableID: 104 *scop.SetIndexName IndexID: 1 Name: crdb_internal_index_1_name_placeholder @@ -1404,9 +1401,6 @@ PostCommitPhase stage 7 of 7 with 15 MutationType ops IndexID: 2 Name: foo_pkey TableID: 104 - *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly - IndexID: 1 - TableID: 107 *scop.SetIndexName IndexID: 1 Name: crdb_internal_index_1_name_placeholder @@ -1415,6 +1409,9 @@ PostCommitPhase stage 7 of 7 with 15 MutationType ops IndexID: 2 Name: bar_pkey TableID: 107 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + IndexID: 1 + TableID: 104 *scop.MakeAddedPrimaryIndexPublic EventBase: Authorization: @@ -1426,6 +1423,9 @@ PostCommitPhase stage 7 of 7 with 15 MutationType ops SubWorkID: 1 IndexID: 2 TableID: 104 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + IndexID: 1 + TableID: 107 *scop.MakeAddedPrimaryIndexPublic EventBase: Authorization: diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_database b/pkg/sql/schemachanger/scplan/testdata/drop_database index 659d15f33077..8de927e903be 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_database +++ b/pkg/sql/schemachanger/scplan/testdata/drop_database @@ -1937,6 +1937,10 @@ DROP DATABASE db1 CASCADE to: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT] kind: Precedence rule: dependents removed before index +- from: [IndexName:{DescID: 109, Name: t1_pkey, IndexID: 1}, ABSENT] + to: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, VALIDATED] + kind: SameStagePrecedence + rule: dependents removed after index no longer public - from: [IndexName:{DescID: 109, Name: t1_pkey, IndexID: 1}, ABSENT] to: [Table:{DescID: 109}, DROPPED] kind: Precedence @@ -1945,6 +1949,10 @@ DROP DATABASE db1 CASCADE to: [PrimaryIndex:{DescID: 110, IndexID: 1, ConstraintID: 1}, ABSENT] kind: Precedence rule: dependents removed before index +- from: [IndexName:{DescID: 110, Name: t1_pkey, IndexID: 1}, ABSENT] + to: [PrimaryIndex:{DescID: 110, IndexID: 1, ConstraintID: 1}, VALIDATED] + kind: SameStagePrecedence + rule: dependents removed after index no longer public - from: [IndexName:{DescID: 110, Name: t1_pkey, IndexID: 1}, ABSENT] to: [Table:{DescID: 110}, DROPPED] kind: Precedence @@ -2149,14 +2157,6 @@ DROP DATABASE db1 CASCADE to: [View:{DescID: 117}, DROPPED] kind: Precedence rule: dependent element removal before descriptor drop -- from: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, VALIDATED] - to: [IndexName:{DescID: 109, Name: t1_pkey, IndexID: 1}, ABSENT] - kind: Precedence - rule: dependents removed after index no longer public -- from: [PrimaryIndex:{DescID: 110, IndexID: 1, ConstraintID: 1}, VALIDATED] - to: [IndexName:{DescID: 110, Name: t1_pkey, IndexID: 1}, ABSENT] - kind: Precedence - rule: dependents removed after index no longer public - from: [SchemaComment:{DescID: 106, Comment: sc1 is good}, ABSENT] to: [Schema:{DescID: 106}, DROPPED] kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_schema b/pkg/sql/schemachanger/scplan/testdata/drop_schema index 0c75dfec651c..ecd68421ffbd 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_schema +++ b/pkg/sql/schemachanger/scplan/testdata/drop_schema @@ -539,6 +539,10 @@ DROP SCHEMA defaultdb.SC1 CASCADE to: [PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1}, ABSENT] kind: Precedence rule: dependents removed before index +- from: [IndexName:{DescID: 106, Name: t1_pkey, IndexID: 1}, ABSENT] + to: [PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1}, VALIDATED] + kind: SameStagePrecedence + rule: dependents removed after index no longer public - from: [IndexName:{DescID: 106, Name: t1_pkey, IndexID: 1}, ABSENT] to: [Table:{DescID: 106}, DROPPED] kind: Precedence @@ -695,10 +699,6 @@ DROP SCHEMA defaultdb.SC1 CASCADE to: [View:{DescID: 113}, DROPPED] kind: Precedence rule: dependent element removal before descriptor drop -- from: [PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1}, VALIDATED] - to: [IndexName:{DescID: 106, Name: t1_pkey, IndexID: 1}, ABSENT] - kind: Precedence - rule: dependents removed after index no longer public - from: [SchemaComment:{DescID: 104, Comment: sc1 is good schema}, ABSENT] to: [Schema:{DescID: 104}, DROPPED] kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_table b/pkg/sql/schemachanger/scplan/testdata/drop_table index aa9a4a4967eb..f538f57345b7 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_table +++ b/pkg/sql/schemachanger/scplan/testdata/drop_table @@ -732,6 +732,10 @@ DROP TABLE defaultdb.shipments CASCADE; to: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1}, ABSENT] kind: Precedence rule: dependents removed before index +- from: [IndexComment:{DescID: 107, IndexID: 1, Comment: pkey is good}, ABSENT] + to: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1}, VALIDATED] + kind: SameStagePrecedence + rule: dependents removed after index no longer public - from: [IndexComment:{DescID: 107, IndexID: 1, Comment: pkey is good}, ABSENT] to: [Table:{DescID: 107}, DROPPED] kind: Precedence @@ -740,6 +744,10 @@ DROP TABLE defaultdb.shipments CASCADE; to: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1}, ABSENT] kind: Precedence rule: dependents removed before index +- from: [IndexName:{DescID: 107, Name: shipments_pkey, IndexID: 1}, ABSENT] + to: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1}, VALIDATED] + kind: SameStagePrecedence + rule: dependents removed after index no longer public - from: [IndexName:{DescID: 107, Name: shipments_pkey, IndexID: 1}, ABSENT] to: [Table:{DescID: 107}, DROPPED] kind: Precedence @@ -780,14 +788,6 @@ DROP TABLE defaultdb.shipments CASCADE; to: [View:{DescID: 109}, DROPPED] kind: Precedence rule: dependent element removal before descriptor drop -- from: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1}, VALIDATED] - to: [IndexComment:{DescID: 107, IndexID: 1, Comment: pkey is good}, ABSENT] - kind: Precedence - rule: dependents removed after index no longer public -- from: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1}, VALIDATED] - to: [IndexName:{DescID: 107, Name: shipments_pkey, IndexID: 1}, ABSENT] - kind: Precedence - rule: dependents removed after index no longer public - from: [SequenceOwner:{DescID: 107, ColumnID: 2, ReferencedDescID: 108}, ABSENT] to: [ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: SameStagePrecedence @@ -965,6 +965,14 @@ PreCommitPhase stage 1 of 1 with 20 MutationType ops *scop.RemoveDroppedColumnType ColumnID: 4294967294 TableID: 112 + *scop.RemoveDroppedIndexPartialPredicate + IndexID: 2 + TableID: 112 + *scop.UpdateTableBackReferencesInTypes + BackReferencedTableID: 112 + TypeIDs: + - 110 + - 111 *scop.RemoveCheckConstraint ConstraintID: 2 TableID: 112 @@ -983,14 +991,6 @@ PreCommitPhase stage 1 of 1 with 20 MutationType ops *scop.RemoveDroppedColumnType ColumnID: 3 TableID: 112 - *scop.RemoveDroppedIndexPartialPredicate - IndexID: 2 - TableID: 112 - *scop.UpdateTableBackReferencesInTypes - BackReferencedTableID: 112 - TypeIDs: - - 110 - - 111 *scop.SetJobStateOnDescriptor DescriptorID: 110 Initialize: true diff --git a/pkg/sql/schemachanger/testdata/alter_table_add_column b/pkg/sql/schemachanger/testdata/alter_table_add_column index 9fcd33aff2f5..b8f94726dcc9 100644 --- a/pkg/sql/schemachanger/testdata/alter_table_add_column +++ b/pkg/sql/schemachanger/testdata/alter_table_add_column @@ -467,7 +467,7 @@ upsert descriptor #106 keyColumnNames: - i - name: crdb_internal_index_3_name_placeholder - + name: tbl_pkey + + name: crdb_internal_index_1_name_placeholder partitioning: {} sharded: {} - storeColumnIds: @@ -744,7 +744,7 @@ upsert descriptor #106 - - 1 - keyColumnNames: - - i - - name: tbl_pkey + - name: crdb_internal_index_1_name_placeholder - partitioning: {} - sharded: {} - unique: true @@ -1236,7 +1236,7 @@ upsert descriptor #106 keyColumnNames: - i - name: crdb_internal_index_5_name_placeholder - + name: tbl_pkey + + name: crdb_internal_index_2_name_placeholder partitioning: {} sharded: {} storeColumnIds: @@ -1523,7 +1523,7 @@ upsert descriptor #106 - - 1 - keyColumnNames: - - i - - name: tbl_pkey + - name: crdb_internal_index_2_name_placeholder - partitioning: {} - sharded: {} - storeColumnIds: @@ -2076,7 +2076,7 @@ upsert descriptor #106 keyColumnNames: - i - name: crdb_internal_index_7_name_placeholder - + name: tbl_pkey + + name: crdb_internal_index_4_name_placeholder partitioning: {} sharded: {} ... @@ -2391,7 +2391,7 @@ upsert descriptor #106 - - 1 - keyColumnNames: - - i - - name: tbl_pkey + - name: crdb_internal_index_4_name_placeholder - partitioning: {} - sharded: {} - storeColumnIds: @@ -2434,3 +2434,1302 @@ update progress of schema change job #1: "all stages completed" commit transaction #11 notified job registry to adopt jobs: [2] # end PostCommitPhase + +test +ALTER TABLE db.public.tbl ADD COLUMN m INT NOT NULL UNIQUE DEFAULT nextval('db.public.sq1') +---- +checking for feature: ALTER TABLE +increment telemetry for sql.schema.alter_table +increment telemetry for sql.schema.alter_table.add_column +increment telemetry for sql.schema.qualifcation.default_expr +increment telemetry for sql.schema.new_column_type.int8 +begin transaction #1 +# begin StatementPhase +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 1 with 12 MutationType ops +upsert descriptor #106 + ... + createAsOfTime: + wallTime: "1" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + - PUBLIC + + - PUBLIC + + - DELETE_ONLY + + - PUBLIC + + - PUBLIC + + - PUBLIC + + - BACKFILL_ONLY + + - ABSENT + + - DELETE_ONLY + + - ABSENT + + - ABSENT + + - DELETE_ONLY + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹m› INT8 NOT + + NULL UNIQUE DEFAULT nextval(‹'db.public.sq1'›) + + statement: ALTER TABLE db.public.tbl ADD COLUMN m INT8 NOT NULL UNIQUE DEFAULT + + nextval('db.public.sq1') + + statementTag: ALTER TABLE + + revertible: true + + targetRanks: + + - 0 + + - 1 + + - 2 + + - 3 + + - 4 + + - 5 + + - 6 + + - 7 + + - 8 + + - 9 + + - 10 + + - 11 + + targets: + + - elementProto: + + primaryIndex: + + embeddedIndex: + + constraintId: 6 + + indexId: 6 + + isCreatedExplicitly: true + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + storingColumnIds: + + - 2 + + - 3 + + - 4 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexName: + + indexId: 6 + + name: tbl_pkey + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + column: + + columnId: 5 + + pgAttributeNum: 5 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + columnName: + + columnId: 5 + + name: m + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + columnType: + + columnId: 5 + + embeddedTypeT: + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + columnDefaultExpression: + + columnId: 5 + + embeddedExpr: + + expr: nextval(107:::REGCLASS) + + usesSequenceIds: + + - 107 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + primaryIndex: + + embeddedIndex: + + constraintId: 6 + + indexId: 8 + + isCreatedExplicitly: true + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + sourceIndexId: 6 + + storingColumnIds: + + - 2 + + - 3 + + - 4 + + - 5 + + tableId: 106 + + temporaryIndexId: 9 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + indexName: + + indexId: 8 + + name: tbl_pkey + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + temporaryIndex: + + embeddedIndex: + + constraintId: 6 + + indexId: 9 + + isCreatedExplicitly: true + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + sourceIndexId: 6 + + storingColumnIds: + + - 2 + + - 3 + + - 4 + + - 5 + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: TRANSIENT_ABSENT + + - elementProto: + + secondaryIndex: + + embeddedIndex: + + indexId: 10 + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 5 + + keySuffixColumnIds: + + - 1 + + sourceIndexId: 8 + + tableId: 106 + + temporaryIndexId: 11 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + indexName: + + indexId: 10 + + name: tbl_m_key + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + temporaryIndex: + + embeddedIndex: + + indexId: 11 + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 5 + + keySuffixColumnIds: + + - 1 + + sourceIndexId: 8 + + tableId: 106 + + isUsingSecondaryEncoding: true + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: TRANSIENT_ABSENT + families: + - columnIds: + ... + - 3 + - 4 + + - 5 + columnNames: + - i + ... + - k + - l + + - m + defaultColumnId: 2 + name: primary + ... + id: 106 + modificationTime: {} + + mutations: + + - column: + + defaultExpr: nextval(107:::REGCLASS) + + id: 5 + + name: m + + pgAttributeNum: 5 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + usesSequenceIds: + + - 107 + + direction: ADD + + mutationId: 1 + + state: DELETE_ONLY + + - direction: ADD + + index: + + constraintId: 8 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 8 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_8_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + - 3 + + - 4 + + - 5 + + storeColumnNames: + + - j + + - k + + - l + + - m + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 9 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 9 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_9_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + - 3 + + - 4 + + - 5 + + storeColumnNames: + + - j + + - k + + - l + + - m + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + + - direction: ADD + + index: + + constraintId: 10 + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 11 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 5 + + keyColumnNames: + + - m + + keySuffixColumnIds: + + - 1 + + name: crdb_internal_index_11_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: tbl + - nextColumnId: 5 + - nextConstraintId: 8 + + nextColumnId: 6 + + nextConstraintId: 11 + nextFamilyId: 1 + - nextIndexId: 8 + + nextIndexId: 12 + nextMutationId: 1 + parentId: 104 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "22" + + version: "23" +upsert descriptor #107 + ... + createAsOfTime: + wallTime: "1" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + jobId: "1" + + revertible: true + dependedOnBy: + - byId: true + columnIds: + - 4 + + - 5 + id: 106 + families: + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "8" + + version: "9" +write *eventpb.AlterTable to event log for descriptor #106: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹m› INT8 NOT NULL UNIQUE DEFAULT nextval(‹'db.public.sq1'›) +create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN m INT8 NOT NULL UNIQUE DEFAULT nextval('db.public.sq1')" + descriptor IDs: [106 107] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitPhase stage 1 of 13 with 6 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - PUBLIC + - - DELETE_ONLY + + - WRITE_ONLY + - PUBLIC + - PUBLIC + ... + - BACKFILL_ONLY + - ABSENT + - - DELETE_ONLY + + - WRITE_ONLY + - ABSENT + - ABSENT + - - DELETE_ONLY + + - WRITE_ONLY + jobId: "1" + relevantStatements: + ... + direction: ADD + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + ... + version: 4 + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + ... + version: 4 + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + name: tbl + nextColumnId: 6 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "23" + + version: "24" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "9" + + version: "10" +update progress of schema change job #1: "PostCommitPhase stage 2 of 13 with 1 BackfillType op pending" +commit transaction #3 +begin transaction #4 +## PostCommitPhase stage 2 of 13 with 1 BackfillType op +backfill indexes [8] from index #6 in table #106 +commit transaction #4 +begin transaction #5 +## PostCommitPhase stage 3 of 13 with 4 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - PUBLIC + - - BACKFILL_ONLY + + - DELETE_ONLY + - ABSENT + - WRITE_ONLY + ... + version: 4 + mutationId: 1 + - state: BACKFILLING + + state: DELETE_ONLY + - direction: ADD + index: + ... + time: {} + unexposedParentSchemaId: 105 + - version: "24" + + version: "25" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "10" + + version: "11" +update progress of schema change job #1: "PostCommitPhase stage 4 of 13 with 1 MutationType op pending" +commit transaction #5 +begin transaction #6 +## PostCommitPhase stage 4 of 13 with 4 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - PUBLIC + - - DELETE_ONLY + + - MERGE_ONLY + - ABSENT + - WRITE_ONLY + ... + version: 4 + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + ... + time: {} + unexposedParentSchemaId: 105 + - version: "25" + + version: "26" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "11" + + version: "12" +update progress of schema change job #1: "PostCommitPhase stage 5 of 13 with 1 BackfillType op pending" +commit transaction #6 +begin transaction #7 +## PostCommitPhase stage 5 of 13 with 1 BackfillType op +merge temporary indexes [9] into backfilled indexes [8] in table #106 +commit transaction #7 +begin transaction #8 +## PostCommitPhase stage 6 of 13 with 1 ValidationType op +validate forward indexes [8] in table #106 +commit transaction #8 +begin transaction #9 +## PostCommitPhase stage 7 of 13 with 10 MutationType ops +upsert descriptor #106 + ... + usesSequenceIds: + - 107 + + - defaultExpr: nextval(107:::REGCLASS) + + id: 5 + + name: m + + pgAttributeNum: 5 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + usesSequenceIds: + + - 107 + createAsOfTime: + wallTime: "1" + ... + userName: root + currentStatuses: + + - VALIDATED + + - ABSENT + - PUBLIC + - PUBLIC + - - WRITE_ONLY + - PUBLIC + - PUBLIC + - PUBLIC + - - MERGE_ONLY + - - ABSENT + + - PUBLIC + - WRITE_ONLY + + - BACKFILL_ONLY + - ABSENT + - - ABSENT + - WRITE_ONLY + jobId: "1" + ... + modificationTime: {} + mutations: + - - column: + - defaultExpr: nextval(107:::REGCLASS) + - id: 5 + - name: m + - pgAttributeNum: 5 + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - usesSequenceIds: + - - 107 + - direction: ADD + - mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + - constraintId: 8 + + constraintId: 9 + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 8 + + id: 9 + interleave: {} + keyColumnDirections: + ... + keyColumnNames: + - i + - name: crdb_internal_index_8_name_placeholder + + name: crdb_internal_index_9_name_placeholder + partitioning: {} + sharded: {} + ... + - m + unique: true + + useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + ... + - direction: ADD + index: + - constraintId: 9 + + constraintId: 10 + createdExplicitly: true + - encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 9 + + id: 11 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - - 1 + + - 5 + keyColumnNames: + - - i + - name: crdb_internal_index_9_name_placeholder + + - m + + keySuffixColumnIds: + + - 1 + + name: crdb_internal_index_11_name_placeholder + partitioning: {} + sharded: {} + - storeColumnIds: + - - 2 + - - 3 + - - 4 + - - 5 + - storeColumnNames: + - - j + - - k + - - l + - - m + + storeColumnNames: [] + unique: true + useDeletePreservingEncoding: true + ... + - direction: ADD + index: + - constraintId: 10 + + constraintId: 11 + createdExplicitly: true + foreignKey: {} + geoConfig: {} + - id: 11 + + id: 10 + interleave: {} + keyColumnDirections: + ... + keySuffixColumnIds: + - 1 + - name: crdb_internal_index_11_name_placeholder + + name: crdb_internal_index_10_name_placeholder + partitioning: {} + sharded: {} + storeColumnNames: [] + unique: true + - useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + + state: BACKFILLING + + - direction: DROP + + index: + + constraintId: 6 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 6 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_6_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + - 3 + + - 4 + + storeColumnNames: + + - j + + - k + + - l + + unique: true + + version: 4 + + mutationId: 1 + state: DELETE_AND_WRITE_ONLY + name: tbl + nextColumnId: 6 + - nextConstraintId: 11 + + nextConstraintId: 12 + nextFamilyId: 1 + nextIndexId: 12 + ... + parentId: 104 + primaryIndex: + - constraintId: 6 + + constraintId: 8 + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 6 + + id: 8 + interleave: {} + keyColumnDirections: + ... + - 3 + - 4 + + - 5 + storeColumnNames: + - j + - k + - l + + - m + unique: true + version: 4 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "26" + + version: "27" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "12" + + version: "13" +adding table for stats refresh: 106 +update progress of schema change job #1: "PostCommitPhase stage 8 of 13 with 1 BackfillType op pending" +commit transaction #9 +begin transaction #10 +## PostCommitPhase stage 8 of 13 with 1 BackfillType op +backfill indexes [10] from index #8 in table #106 +commit transaction #10 +begin transaction #11 +## PostCommitPhase stage 9 of 13 with 4 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - WRITE_ONLY + - - BACKFILL_ONLY + + - DELETE_ONLY + - ABSENT + - WRITE_ONLY + ... + version: 4 + mutationId: 1 + - state: BACKFILLING + + state: DELETE_ONLY + - direction: DROP + index: + ... + time: {} + unexposedParentSchemaId: 105 + - version: "27" + + version: "28" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "13" + + version: "14" +update progress of schema change job #1: "PostCommitPhase stage 10 of 13 with 1 MutationType op pending" +commit transaction #11 +begin transaction #12 +## PostCommitPhase stage 10 of 13 with 4 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - WRITE_ONLY + - - DELETE_ONLY + + - MERGE_ONLY + - ABSENT + - WRITE_ONLY + ... + version: 4 + mutationId: 1 + - state: DELETE_ONLY + + state: DELETE_AND_WRITE_ONLY + - direction: DROP + index: + ... + time: {} + unexposedParentSchemaId: 105 + - version: "28" + + version: "29" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "14" + + version: "15" +update progress of schema change job #1: "PostCommitPhase stage 11 of 13 with 1 BackfillType op pending" +commit transaction #12 +begin transaction #13 +## PostCommitPhase stage 11 of 13 with 1 BackfillType op +merge temporary indexes [11] into backfilled indexes [10] in table #106 +commit transaction #13 +begin transaction #14 +## PostCommitPhase stage 12 of 13 with 1 ValidationType op +validate forward indexes [10] in table #106 +commit transaction #14 +begin transaction #15 +## PostCommitPhase stage 13 of 13 with 5 MutationType ops +upsert descriptor #106 + ... + - PUBLIC + - WRITE_ONLY + - - MERGE_ONLY + - - ABSENT + + - PUBLIC + + - PUBLIC + - WRITE_ONLY + jobId: "1" + ... + nextval('db.public.sq1') + statementTag: ALTER TABLE + - revertible: true + targetRanks: + - 0 + ... + formatVersion: 3 + id: 106 + + indexes: + + - constraintId: 11 + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 10 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 5 + + keyColumnNames: + + - m + + keySuffixColumnIds: + + - 1 + + name: tbl_m_key + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + modificationTime: {} + mutations: + ... + mutationId: 1 + state: DELETE_AND_WRITE_ONLY + - - direction: ADD + - index: + - constraintId: 11 + - createdExplicitly: true + - foreignKey: {} + - geoConfig: {} + - id: 10 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 5 + - keyColumnNames: + - - m + - keySuffixColumnIds: + - - 1 + - name: crdb_internal_index_10_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnNames: [] + - unique: true + - version: 4 + - mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + - direction: DROP + index: + ... + time: {} + unexposedParentSchemaId: 105 + - version: "29" + + version: "30" +upsert descriptor #107 + ... + userName: root + jobId: "1" + - revertible: true + dependedOnBy: + - byId: true + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "15" + + version: "16" +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 1 of 2 with 3 MutationType ops pending" +set schema change job #1 to non-cancellable +commit transaction #15 +begin transaction #16 +## PostCommitNonRevertiblePhase stage 1 of 2 with 6 MutationType ops +upsert descriptor #106 + ... + userName: root + currentStatuses: + - - VALIDATED + + - DELETE_ONLY + - ABSENT + - PUBLIC + ... + - PUBLIC + - PUBLIC + - - WRITE_ONLY + + - TRANSIENT_DELETE_ONLY + - PUBLIC + - PUBLIC + - - WRITE_ONLY + + - TRANSIENT_DELETE_ONLY + jobId: "1" + relevantStatements: + ... + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + + state: DELETE_ONLY + - direction: ADD + index: + ... + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + + state: DELETE_ONLY + - direction: DROP + index: + ... + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + + state: DELETE_ONLY + name: tbl + nextColumnId: 6 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "30" + + version: "31" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "16" + + version: "17" +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 6 MutationType ops pending" +commit transaction #16 +begin transaction #17 +## PostCommitNonRevertiblePhase stage 2 of 2 with 9 MutationType ops +upsert descriptor #106 + ... + createAsOfTime: + wallTime: "1" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - - DELETE_ONLY + - - ABSENT + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - PUBLIC + - - TRANSIENT_DELETE_ONLY + - - PUBLIC + - - PUBLIC + - - TRANSIENT_DELETE_ONLY + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹m› INT8 NOT + - NULL UNIQUE DEFAULT nextval(‹'db.public.sq1'›) + - statement: ALTER TABLE db.public.tbl ADD COLUMN m INT8 NOT NULL UNIQUE DEFAULT + - nextval('db.public.sq1') + - statementTag: ALTER TABLE + - targetRanks: + - - 0 + - - 1 + - - 2 + - - 3 + - - 4 + - - 5 + - - 6 + - - 7 + - - 8 + - - 9 + - - 10 + - - 11 + - targets: + - - elementProto: + - primaryIndex: + - embeddedIndex: + - constraintId: 6 + - indexId: 6 + - isCreatedExplicitly: true + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - storingColumnIds: + - - 2 + - - 3 + - - 4 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexName: + - indexId: 6 + - name: tbl_pkey + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 5 + - pgAttributeNum: 5 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - columnName: + - columnId: 5 + - name: m + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - columnType: + - columnId: 5 + - embeddedTypeT: + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - columnDefaultExpression: + - columnId: 5 + - embeddedExpr: + - expr: nextval(107:::REGCLASS) + - usesSequenceIds: + - - 107 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - primaryIndex: + - embeddedIndex: + - constraintId: 6 + - indexId: 8 + - isCreatedExplicitly: true + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - sourceIndexId: 6 + - storingColumnIds: + - - 2 + - - 3 + - - 4 + - - 5 + - tableId: 106 + - temporaryIndexId: 9 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - indexName: + - indexId: 8 + - name: tbl_pkey + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - temporaryIndex: + - embeddedIndex: + - constraintId: 6 + - indexId: 9 + - isCreatedExplicitly: true + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - sourceIndexId: 6 + - storingColumnIds: + - - 2 + - - 3 + - - 4 + - - 5 + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: TRANSIENT_ABSENT + - - elementProto: + - secondaryIndex: + - embeddedIndex: + - indexId: 10 + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 5 + - keySuffixColumnIds: + - - 1 + - sourceIndexId: 8 + - tableId: 106 + - temporaryIndexId: 11 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - indexName: + - indexId: 10 + - name: tbl_m_key + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - temporaryIndex: + - embeddedIndex: + - indexId: 11 + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 5 + - keySuffixColumnIds: + - - 1 + - sourceIndexId: 8 + - tableId: 106 + - isUsingSecondaryEncoding: true + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: TRANSIENT_ABSENT + families: + - columnIds: + ... + version: 4 + modificationTime: {} + - mutations: + - - direction: ADD + - index: + - constraintId: 9 + - createdExplicitly: true + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 9 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - keyColumnNames: + - - i + - name: crdb_internal_index_9_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnIds: + - - 2 + - - 3 + - - 4 + - - 5 + - storeColumnNames: + - - j + - - k + - - l + - - m + - unique: true + - useDeletePreservingEncoding: true + - version: 4 + - mutationId: 1 + - state: DELETE_ONLY + - - direction: ADD + - index: + - constraintId: 10 + - createdExplicitly: true + - foreignKey: {} + - geoConfig: {} + - id: 11 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 5 + - keyColumnNames: + - - m + - keySuffixColumnIds: + - - 1 + - name: crdb_internal_index_11_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnNames: [] + - unique: true + - useDeletePreservingEncoding: true + - version: 4 + - mutationId: 1 + - state: DELETE_ONLY + - - direction: DROP + - index: + - constraintId: 6 + - createdExplicitly: true + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 6 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - keyColumnNames: + - - i + - name: crdb_internal_index_6_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnIds: + - - 2 + - - 3 + - - 4 + - storeColumnNames: + - - j + - - k + - - l + - unique: true + - version: 4 + - mutationId: 1 + - state: DELETE_ONLY + + mutations: [] + name: tbl + nextColumnId: 6 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "31" + + version: "32" +upsert descriptor #107 + ... + createAsOfTime: + wallTime: "1" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - jobId: "1" + dependedOnBy: + - byId: true + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "17" + + version: "18" +write *eventpb.FinishSchemaChange to event log for descriptor 106 +create job #2 (non-cancelable: true): "GC for ALTER TABLE db.public.tbl ADD COLUMN m INT8 NOT NULL UNIQUE DEFAULT nextval('db.public.sq1')" + descriptor IDs: [106] +update progress of schema change job #1: "all stages completed" +commit transaction #17 +notified job registry to adopt jobs: [2] +# end PostCommitPhase