From 44092778a48e118f628ed7afaaaababf38cef298 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Wed, 25 May 2022 13:34:05 -0400 Subject: [PATCH] sql: use declarative schema changer for add column with unique Previously, the declarative schema changer was disabled when adding a column with unique constraint, since we didn't have a complete set of elements / logic for adding secondary indexes. This was inadequate because operations would not ordered correctly and rollbacks could potentially break. To address this, this patch enables support and addresses the missing pieces to setup the secondary indexes correctly, such as adding suffix columns, ordering operations correctly, and returning appropriate errors. Release note: None --- pkg/ccl/backupccl/restore_job.go | 2 +- pkg/sql/backfill.go | 12 +- pkg/sql/catalog/descriptor.go | 6 + pkg/sql/catalog/rewrite/rewrite.go | 4 +- pkg/sql/catalog/tabledesc/structured.go | 1 - pkg/sql/catalog/tabledesc/table_desc.go | 22 + pkg/sql/row/errors.go | 14 +- .../schemachanger/scbuild/builder_state.go | 5 + .../scbuildstmt/alter_table_add_column.go | 244 ++++++- .../scbuild/testdata/create_index | 14 +- .../testdata/unimplemented_alter_table | 4 - pkg/sql/schemachanger/scexec/BUILD.bazel | 2 + pkg/sql/schemachanger/scexec/exec_backfill.go | 16 +- .../schemachanger/scexec/exec_validation.go | 6 +- .../internal/rules/dep_index_and_column.go | 191 +++++- .../scplan/internal/rules/testdata/deprules | 94 +++ .../testdata/alter_table_add_column | 593 +++++++++++++++--- 17 files changed, 1058 insertions(+), 172 deletions(-) diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 10dfc944d29a..da9e1b62cc1f 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -821,7 +821,7 @@ func createImportingDescriptors( // Finally, clean up / update any schema changer state inside descriptors // globally. - if err := rewrite.SchemaChangerStateInDescs(allMutableDescs); err != nil { + if err := rewrite.CleanSchemaChangerStateInDescs(allMutableDescs); err != nil { return nil, nil, err } 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/rewrite/rewrite.go b/pkg/sql/catalog/rewrite/rewrite.go index 68fdd13c9e02..d1665eb2f987 100644 --- a/pkg/sql/catalog/rewrite/rewrite.go +++ b/pkg/sql/catalog/rewrite/rewrite.go @@ -394,9 +394,9 @@ func rewriteIDsInTypesT(typ *types.T, descriptorRewrites jobspb.DescRewriteMap) return nil } -// SchemaChangerStateInDescs goes over all mutable descriptors and cleans any +// CleanSchemaChangerStateInDescs goes over all mutable descriptors and cleans any // empty state information. -func SchemaChangerStateInDescs(descriptors []catalog.MutableDescriptor) error { +func CleanSchemaChangerStateInDescs(descriptors []catalog.MutableDescriptor) error { nonEmptyJobs := make(map[jobspb.JobID]struct{}) // Track all the schema changer states that have a non-empty job associated // withe them. diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 7a9825823f99..f7441c6c2826 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/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 bb159e2d5e76..851a5cc8e973 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 @@ -22,34 +22,67 @@ import ( // old primary index starts getting removed, effectively swapping one for the // other. func init() { - newIndex, newIndexTarget, newIndexNode := targetNodeVars("new-index") - oldIndex, oldIndexTarget, oldIndexNode := targetNodeVars("old-index") - var tableID rel.Var = "table-id" - - registerDepRule( - "primary index swap", - scgraph.SameStagePrecedence, - oldIndexNode, newIndexNode, - screl.MustQuery( - newIndex.Type((*scpb.PrimaryIndex)(nil)), - oldIndex.Type((*scpb.PrimaryIndex)(nil)), - tableID.Entities(screl.DescID, newIndex, oldIndex), - - rel.Filter( - "new-primary-index-depends-on-old", newIndex, oldIndex, - )(func(add, drop *scpb.PrimaryIndex) bool { - return add.SourceIndexID == drop.IndexID - }), - - screl.JoinTargetNode(newIndex, newIndexTarget, newIndexNode), - newIndexTarget.AttrEq(screl.TargetStatus, scpb.Status_PUBLIC), - newIndexNode.AttrEq(screl.CurrentStatus, scpb.Status_PUBLIC), + { + newIndex, newIndexTarget, newIndexNode := targetNodeVars("new-index") + oldIndex, oldIndexTarget, oldIndexNode := targetNodeVars("old-index") + var tableID rel.Var = "table-id" + + registerDepRule( + "primary index swap", + scgraph.SameStagePrecedence, + oldIndexNode, newIndexNode, + screl.MustQuery( + newIndex.Type((*scpb.PrimaryIndex)(nil)), + oldIndex.Type((*scpb.PrimaryIndex)(nil)), + tableID.Entities(screl.DescID, newIndex, oldIndex), + + rel.Filter( + "new-primary-index-depends-on-old", newIndex, oldIndex, + )(func(add, drop *scpb.PrimaryIndex) bool { + return add.SourceIndexID == drop.IndexID + }), + + screl.JoinTargetNode(newIndex, newIndexTarget, newIndexNode), + newIndexTarget.AttrEq(screl.TargetStatus, scpb.Status_PUBLIC), + newIndexNode.AttrEq(screl.CurrentStatus, scpb.Status_PUBLIC), + + screl.JoinTargetNode(oldIndex, oldIndexTarget, oldIndexNode), + oldIndexTarget.AttrEq(screl.TargetStatus, scpb.Status_ABSENT), + oldIndexNode.AttrEq(screl.CurrentStatus, scpb.Status_VALIDATED), + ), + ) + } - screl.JoinTargetNode(oldIndex, oldIndexTarget, oldIndexNode), - oldIndexTarget.AttrEq(screl.TargetStatus, scpb.Status_ABSENT), - oldIndexNode.AttrEq(screl.CurrentStatus, scpb.Status_VALIDATED), - ), - ) + { + newIndex, newIndexTarget, newIndexNode := targetNodeVars("new-index") + oldIndex, oldIndexTarget, oldIndexNode := targetNodeVars("old-index") + var tableID rel.Var = "table-id" + + registerDepRule( + "reverting a primary index swap", + scgraph.SameStagePrecedence, + newIndexNode, oldIndexNode, + screl.MustQuery( + newIndex.Type((*scpb.PrimaryIndex)(nil)), + oldIndex.Type((*scpb.PrimaryIndex)(nil)), + tableID.Entities(screl.DescID, newIndex, oldIndex), + + rel.Filter( + "new-primary-index-depends-on-old", newIndex, oldIndex, + )(func(add, drop *scpb.PrimaryIndex) bool { + return add.SourceIndexID == drop.IndexID + }), + + screl.JoinTargetNode(newIndex, newIndexTarget, newIndexNode), + newIndexTarget.AttrEq(screl.TargetStatus, scpb.Status_ABSENT), + newIndexNode.AttrEq(screl.CurrentStatus, scpb.Status_VALIDATED), + + screl.JoinTargetNode(oldIndex, oldIndexTarget, oldIndexNode), + oldIndexTarget.AttrEq(screl.TargetStatus, scpb.Status_PUBLIC), + oldIndexNode.AttrEq(screl.CurrentStatus, scpb.Status_PUBLIC), + ), + ) + } } // These rules ensure that index-dependent elements, like an index's name, its @@ -119,7 +152,6 @@ func init() { screl.DescID, screl.IndexID, ).register() - depRule( "index named right before index becomes public", scgraph.SameStagePrecedence, @@ -152,7 +184,6 @@ func init() { screl.DescID, screl.IndexID, ).register() - depRule( "dependents removed before index", scgraph.Precedence, @@ -225,7 +256,6 @@ func init() { screl.DescID, screl.ColumnID, ).register() - depRule( "column existence precedes column dependents", scgraph.Precedence, @@ -419,6 +449,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 + } column, columnTarget, columnNode := targetNodeVars("column") index, indexTarget, indexNode := targetNodeVars("index") @@ -445,7 +478,27 @@ func init() { screl.JoinTargetNode(index, indexTarget, indexNode), ), ) + registerDepRule( + "primary index should be cleaned up before newly added column when reverting", + scgraph.Precedence, + indexNode, columnNode, + screl.MustQuery( + status.Eq(scpb.Status_WRITE_ONLY), + targetStatus.Eq(scpb.Status_ABSENT), + + column.Type((*scpb.Column)(nil)), + index.Type((*scpb.PrimaryIndex)(nil)), + + tableID.Entities(screl.DescID, column, index), + rel.Filter("column-featured-in-index", column, index)(columnInPrimaryIndexSwap), + targetStatus.Entities(screl.TargetStatus, columnTarget, indexTarget), + status.Entities(screl.CurrentStatus, columnNode, indexNode), + + screl.JoinTargetNode(column, columnTarget, columnNode), + screl.JoinTargetNode(index, indexTarget, indexNode), + ), + ) depRule( "column existence precedes index existence", scgraph.Precedence, @@ -481,7 +534,83 @@ func init() { screl.JoinTargetNode(index, indexTarget, indexNode), ), ) - + 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 + } + depRule( + "primary index with new columns should exist before secondary/temp indexes", + scgraph.Precedence, + scpb.ToPublic, + element(scpb.Status_PUBLIC, + (*scpb.PrimaryIndex)(nil), + ), + element(scpb.Status_BACKFILL_ONLY, + (*scpb.SecondaryIndex)(nil), + (*scpb.TemporaryIndex)(nil), + ), + screl.DescID, + ).withFilter("new-column-featured-in-index", primaryIndexHasSecondaryColumns).register() + depRule( + "secondary indexes should be cleaned up before any primary index with columns when reverting", + scgraph.Precedence, + scpb.ToAbsent, + element(scpb.Status_ABSENT, + (*scpb.SecondaryIndex)(nil), + (*scpb.TemporaryIndex)(nil), + ), + element(scpb.Status_VALIDATED, + (*scpb.PrimaryIndex)(nil), + ), + screl.DescID, + ).withFilter("new-column-featured-in-index", secondaryIndexHasPrimarySwapColumns).register() } // 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 fa65e1113987..6471280be926 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules @@ -382,6 +382,28 @@ deprules - $old-index-node[Target] = $old-index-target - $old-index-target[TargetStatus] = ABSENT - $old-index-node[CurrentStatus] = VALIDATED +- name: reverting a 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-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 + - $new-index-target[TargetStatus] = ABSENT + - $new-index-node[CurrentStatus] = VALIDATED + - $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 + - $old-index-target[TargetStatus] = PUBLIC + - $old-index-node[CurrentStatus] = PUBLIC - name: index existence precedes index dependents from: from-node kind: Precedence @@ -856,6 +878,30 @@ deprules - $index-target[Element] = $index - $index-node[Type] = '*screl.Node' - $index-node[Target] = $index-target +- name: primary index should be cleaned up before newly added column when reverting + from: index-node + kind: Precedence + to: column-node + query: + - $status = WRITE_ONLY + - $target-status = ABSENT + - $column[Type] = '*scpb.Column' + - $index[Type] = '*scpb.PrimaryIndex' + - $column[DescID] = $table-id + - $index[DescID] = $table-id + - column-featured-in-index(*scpb.Column, *scpb.PrimaryIndex)($column, $index) + - $column-target[TargetStatus] = $target-status + - $index-target[TargetStatus] = $target-status + - $column-node[CurrentStatus] = $status + - $index-node[CurrentStatus] = $status + - $column-target[Type] = '*scpb.Target' + - $column-target[Element] = $column + - $column-node[Type] = '*screl.Node' + - $column-node[Target] = $column-target + - $index-target[Type] = '*scpb.Target' + - $index-target[Element] = $index + - $index-node[Type] = '*screl.Node' + - $index-node[Target] = $index-target - name: column existence precedes index existence from: from-node kind: Precedence @@ -902,6 +948,54 @@ deprules - $index-target[Element] = $index - $index-node[Type] = '*screl.Node' - $index-node[Target] = $index-target +- name: primary index with new columns should exist before secondary/temp indexes + from: from-node + kind: Precedence + to: to-node + query: + - $from[Type] = '*scpb.PrimaryIndex' + - $from-target[TargetStatus] = PUBLIC + - $to[Type] IN ['*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - $to-target[TargetStatus] = PUBLIC + - $from-node[CurrentStatus] = PUBLIC + - $to-node[CurrentStatus] = BACKFILL_ONLY + - $from-target[Type] = '*scpb.Target' + - $from-target[Element] = $from + - $from-node[Type] = '*screl.Node' + - $from-node[Target] = $from-target + - $to-target[Type] = '*scpb.Target' + - $to-target[Element] = $to + - $to-node[Type] = '*screl.Node' + - $to-node[Target] = $to-target + - $from[DescID] = $DescID-join-var + - $to[DescID] = $DescID-join-var + - $from[DescID] = $var-to-tell-rel-from-is-an-element + - $to[DescID] = $var-to-tell-rel-to-is-an-element + - new-column-featured-in-index(*scpb.PrimaryIndex, scpb.Element)($from, $to) +- name: secondary indexes should be cleaned up before any primary index with columns when reverting + from: from-node + kind: Precedence + to: to-node + query: + - $from[Type] IN ['*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - $from-target[TargetStatus] = ABSENT + - $to[Type] = '*scpb.PrimaryIndex' + - $to-target[TargetStatus] = ABSENT + - $from-node[CurrentStatus] = ABSENT + - $to-node[CurrentStatus] = VALIDATED + - $from-target[Type] = '*scpb.Target' + - $from-target[Element] = $from + - $from-node[Type] = '*screl.Node' + - $from-node[Target] = $from-target + - $to-target[Type] = '*scpb.Target' + - $to-target[Element] = $to + - $to-node[Type] = '*screl.Node' + - $to-node[Target] = $to-target + - $from[DescID] = $DescID-join-var + - $to[DescID] = $DescID-join-var + - $from[DescID] = $var-to-tell-rel-from-is-an-element + - $to[DescID] = $var-to-tell-rel-to-is-an-element + - new-column-featured-in-index(scpb.Element, *scpb.PrimaryIndex)($from, $to) - name: ensure columns are in increasing order from: laterColumn-node kind: SameStagePrecedence diff --git a/pkg/sql/schemachanger/testdata/alter_table_add_column b/pkg/sql/schemachanger/testdata/alter_table_add_column index f972f77791e8..2d3cc6da7981 100644 --- a/pkg/sql/schemachanger/testdata/alter_table_add_column +++ b/pkg/sql/schemachanger/testdata/alter_table_add_column @@ -11,7 +11,7 @@ CREATE SEQUENCE db.public.sq1; test -ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAULT nextval('db.public.sq1') +ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAULT nextval('db.public.sq1') UNIQUE ---- checking for feature: ALTER TABLE increment telemetry for sql.schema.alter_table @@ -22,7 +22,7 @@ begin transaction #1 # begin StatementPhase # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 11 MutationType ops +## PreCommitPhase stage 1 of 1 with 12 MutationType ops upsert descriptor #106 ... createAsOfTime: @@ -40,12 +40,16 @@ upsert descriptor #106 + - BACKFILL_ONLY + - ABSENT + - DELETE_ONLY + + - ABSENT + + - ABSENT + + - DELETE_ONLY + jobId: "1" + relevantStatements: + - statement: + redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹j› INT8 NOT - + NULL DEFAULT nextval(‹'db.public.sq1'›) - + statement: ALTER TABLE db.public.tbl ADD COLUMN j INT8 NOT NULL DEFAULT nextval('db.public.sq1') + + NULL UNIQUE DEFAULT nextval(‹'db.public.sq1'›) + + statement: ALTER TABLE db.public.tbl ADD COLUMN j INT8 NOT NULL UNIQUE DEFAULT + + nextval('db.public.sq1') + statementTag: ALTER TABLE + revertible: true + targetRanks: @@ -58,6 +62,9 @@ upsert descriptor #106 + - 6 + - 7 + - 8 + + - 9 + + - 10 + + - 11 + targets: + - elementProto: + primaryIndex: @@ -171,6 +178,51 @@ upsert descriptor #106 + metadata: + sourceElementId: 1 + subWorkId: 1 + + targetStatus: TRANSIENT_ABSENT + + - elementProto: + + secondaryIndex: + + embeddedIndex: + + indexId: 4 + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keySuffixColumnIds: + + - 1 + + sourceIndexId: 2 + + tableId: 106 + + temporaryIndexId: 5 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + indexName: + + indexId: 4 + + name: tbl_j_key + + tableId: 106 + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + + targetStatus: PUBLIC + + - elementProto: + + temporaryIndex: + + embeddedIndex: + + indexId: 5 + + isUnique: true + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keySuffixColumnIds: + + - 1 + + sourceIndexId: 2 + + tableId: 106 + + isUsingSecondaryEncoding: true + + metadata: + + sourceElementId: 1 + + subWorkId: 1 + targetStatus: TRANSIENT_ABSENT families: - columnIds: @@ -251,15 +303,40 @@ upsert descriptor #106 + useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + + state: DELETE_ONLY + + - direction: ADD + + index: + + constraintId: 4 + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 5 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keyColumnNames: + + - j + + keySuffixColumnIds: + + - 1 + + name: crdb_internal_index_5_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + state: DELETE_ONLY name: tbl - nextColumnId: 2 - nextConstraintId: 2 + nextColumnId: 3 - + nextConstraintId: 4 + + nextConstraintId: 5 nextFamilyId: 1 - nextIndexId: 2 - + nextIndexId: 4 + + nextIndexId: 6 nextMutationId: 1 parentId: 104 ... @@ -288,8 +365,8 @@ upsert descriptor #107 unexposedParentSchemaId: 105 - version: "1" + version: "2" -write *eventpb.AlterTable to event log for descriptor #106: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹j› INT8 NOT NULL DEFAULT nextval(‹'db.public.sq1'›) -create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN j INT8 NOT NULL DEFAULT nextval('db.public.sq1')" +write *eventpb.AlterTable to event log for descriptor #106: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹j› INT8 NOT NULL UNIQUE DEFAULT nextval(‹'db.public.sq1'›) +create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN j INT8 NOT NULL UNIQUE DEFAULT nextval('db.public.sq1')" descriptor IDs: [106 107] # end PreCommitPhase commit transaction #1 @@ -298,7 +375,7 @@ notified job registry to adopt jobs: [1] begin transaction #2 commit transaction #2 begin transaction #3 -## PostCommitPhase stage 1 of 7 with 5 MutationType ops +## PostCommitPhase stage 1 of 13 with 6 MutationType ops upsert descriptor #106 ... - PUBLIC @@ -311,6 +388,10 @@ upsert descriptor #106 - BACKFILL_ONLY - ABSENT - - DELETE_ONLY + + - WRITE_ONLY + - ABSENT + - ABSENT + - - DELETE_ONLY + - WRITE_ONLY jobId: "1" relevantStatements: @@ -325,6 +406,13 @@ upsert descriptor #106 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: 3 @@ -339,14 +427,14 @@ upsert descriptor #107 unexposedParentSchemaId: 105 - version: "2" + version: "3" -update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" +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 7 with 1 BackfillType op +## PostCommitPhase stage 2 of 13 with 1 BackfillType op backfill indexes [2] from index #1 in table #106 commit transaction #4 begin transaction #5 -## PostCommitPhase stage 3 of 7 with 4 MutationType ops +## PostCommitPhase stage 3 of 13 with 4 MutationType ops upsert descriptor #106 ... - PUBLIC @@ -373,10 +461,10 @@ upsert descriptor #107 unexposedParentSchemaId: 105 - version: "3" + version: "4" -update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" +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 7 with 4 MutationType ops +## PostCommitPhase stage 4 of 13 with 4 MutationType ops upsert descriptor #106 ... - PUBLIC @@ -403,18 +491,18 @@ upsert descriptor #107 unexposedParentSchemaId: 105 - version: "4" + version: "5" -update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" +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 7 with 1 BackfillType op +## PostCommitPhase stage 5 of 13 with 1 BackfillType op merge temporary indexes [3] into backfilled indexes [2] in table #106 commit transaction #7 begin transaction #8 -## PostCommitPhase stage 6 of 7 with 1 ValidationType op +## PostCommitPhase stage 6 of 13 with 1 ValidationType op validate forward indexes [2] in table #106 commit transaction #8 begin transaction #9 -## PostCommitPhase stage 7 of 7 with 9 MutationType ops +## PostCommitPhase stage 7 of 13 with 10 MutationType ops upsert descriptor #106 ... oid: 20 @@ -446,13 +534,11 @@ upsert descriptor #106 - - ABSENT + - PUBLIC - WRITE_ONLY + + - BACKFILL_ONLY + - ABSENT + - - ABSENT + - WRITE_ONLY jobId: "1" - ... - statement: ALTER TABLE db.public.tbl ADD COLUMN j INT8 NOT NULL DEFAULT nextval('db.public.sq1') - statementTag: ALTER TABLE - - revertible: true - targetRanks: - - 0 ... modificationTime: {} mutations: @@ -495,12 +581,35 @@ upsert descriptor #106 + useDeletePreservingEncoding: true version: 4 mutationId: 1 - state: DELETE_AND_WRITE_ONLY - - - direction: ADD - + - direction: DROP + ... + - direction: ADD index: - constraintId: 3 - - createdExplicitly: true + + constraintId: 4 + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 5 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keyColumnNames: + + - j + + keySuffixColumnIds: + + - 1 + + name: crdb_internal_index_5_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_AND_WRITE_ONLY + + - direction: DROP + + index: + constraintId: 1 + createdAtNanos: "1640995200000000000" encodingType: 1 @@ -525,6 +634,38 @@ upsert descriptor #106 - useDeletePreservingEncoding: true version: 4 mutationId: 1 + ... + - direction: ADD + index: + - constraintId: 4 + + constraintId: 5 + createdExplicitly: true + foreignKey: {} + geoConfig: {} + - id: 5 + + id: 4 + interleave: {} + keyColumnDirections: + ... + keySuffixColumnIds: + - 1 + - name: crdb_internal_index_5_name_placeholder + + name: crdb_internal_index_4_name_placeholder + partitioning: {} + sharded: {} + storeColumnNames: [] + unique: true + - useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + + state: BACKFILLING + name: tbl + nextColumnId: 3 + - nextConstraintId: 5 + + nextConstraintId: 6 + nextFamilyId: 1 + nextIndexId: 6 ... parentId: 104 primaryIndex: @@ -553,6 +694,165 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "5" + version: "6" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "5" + + version: "6" +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 [4] from index #2 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 + name: tbl + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "6" + + version: "7" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "6" + + version: "7" +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 + name: tbl + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "7" + + version: "8" +upsert descriptor #107 + ... + start: "1" + unexposedParentSchemaId: 105 + - version: "7" + + version: "8" +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 [5] into backfilled indexes [4] in table #106 +commit transaction #13 +begin transaction #14 +## PostCommitPhase stage 12 of 13 with 1 ValidationType op +validate forward indexes [4] 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: 5 + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 4 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keyColumnNames: + + - j + + keySuffixColumnIds: + + - 1 + + name: tbl_j_key + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + modificationTime: {} + mutations: + ... + mutationId: 1 + state: DELETE_AND_WRITE_ONLY + - - direction: ADD + - index: + - constraintId: 5 + - createdExplicitly: true + - foreignKey: {} + - geoConfig: {} + - id: 4 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 2 + - keyColumnNames: + - - j + - keySuffixColumnIds: + - - 1 + - name: crdb_internal_index_4_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnNames: [] + - unique: true + - version: 4 + - mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + name: tbl + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 105 + - version: "8" + + version: "9" upsert descriptor #107 ... userName: root @@ -563,14 +863,13 @@ upsert descriptor #107 ... start: "1" unexposedParentSchemaId: 105 - - version: "5" - + version: "6" -adding table for stats refresh: 106 -update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 1 of 2 with 2 MutationType ops pending" + - version: "8" + + version: "9" +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 #9 -begin transaction #10 -## PostCommitNonRevertiblePhase stage 1 of 2 with 5 MutationType ops +commit transaction #15 +begin transaction #16 +## PostCommitNonRevertiblePhase stage 1 of 2 with 6 MutationType ops upsert descriptor #106 ... userName: root @@ -583,6 +882,10 @@ upsert descriptor #106 - PUBLIC - PUBLIC - - WRITE_ONLY + + - TRANSIENT_DELETE_ONLY + - PUBLIC + - PUBLIC + - - WRITE_ONLY + - TRANSIENT_DELETE_ONLY jobId: "1" relevantStatements: @@ -590,6 +893,13 @@ upsert descriptor #106 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: @@ -603,18 +913,18 @@ upsert descriptor #106 ... time: {} unexposedParentSchemaId: 105 - - version: "6" - + version: "7" + - version: "9" + + version: "10" upsert descriptor #107 ... start: "1" unexposedParentSchemaId: 105 - - version: "6" - + version: "7" -update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 4 MutationType ops pending" -commit transaction #10 -begin transaction #11 -## PostCommitNonRevertiblePhase stage 2 of 2 with 7 MutationType ops + - version: "9" + + version: "10" +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: @@ -632,12 +942,16 @@ upsert descriptor #106 - - PUBLIC - - PUBLIC - - TRANSIENT_DELETE_ONLY + - - PUBLIC + - - PUBLIC + - - TRANSIENT_DELETE_ONLY - jobId: "1" - relevantStatements: - - statement: - redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹j› INT8 NOT - - NULL DEFAULT nextval(‹'db.public.sq1'›) - - statement: ALTER TABLE db.public.tbl ADD COLUMN j INT8 NOT NULL DEFAULT nextval('db.public.sq1') + - NULL UNIQUE DEFAULT nextval(‹'db.public.sq1'›) + - statement: ALTER TABLE db.public.tbl ADD COLUMN j INT8 NOT NULL UNIQUE DEFAULT + - nextval('db.public.sq1') - statementTag: ALTER TABLE - targetRanks: - - 0 @@ -649,6 +963,9 @@ upsert descriptor #106 - - 6 - - 7 - - 8 + - - 9 + - - 10 + - - 11 - targets: - - elementProto: - primaryIndex: @@ -762,11 +1079,56 @@ upsert descriptor #106 - metadata: - sourceElementId: 1 - subWorkId: 1 + - targetStatus: TRANSIENT_ABSENT + - - elementProto: + - secondaryIndex: + - embeddedIndex: + - indexId: 4 + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 2 + - keySuffixColumnIds: + - - 1 + - sourceIndexId: 2 + - tableId: 106 + - temporaryIndexId: 5 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - indexName: + - indexId: 4 + - name: tbl_j_key + - tableId: 106 + - metadata: + - sourceElementId: 1 + - subWorkId: 1 + - targetStatus: PUBLIC + - - elementProto: + - temporaryIndex: + - embeddedIndex: + - indexId: 5 + - isUnique: true + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 2 + - keySuffixColumnIds: + - - 1 + - sourceIndexId: 2 + - tableId: 106 + - isUsingSecondaryEncoding: true + - metadata: + - sourceElementId: 1 + - subWorkId: 1 - targetStatus: TRANSIENT_ABSENT families: - columnIds: ... - id: 106 + version: 4 modificationTime: {} - mutations: - - direction: ADD @@ -796,6 +1158,31 @@ upsert descriptor #106 - version: 4 - mutationId: 1 - state: DELETE_ONLY + - - direction: ADD + - index: + - constraintId: 4 + - createdExplicitly: true + - foreignKey: {} + - geoConfig: {} + - id: 5 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 2 + - keyColumnNames: + - - j + - keySuffixColumnIds: + - - 1 + - name: crdb_internal_index_5_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnNames: [] + - unique: true + - useDeletePreservingEncoding: true + - version: 4 + - mutationId: 1 + - state: DELETE_ONLY - - direction: DROP - index: - constraintId: 1 @@ -824,8 +1211,8 @@ upsert descriptor #106 ... time: {} unexposedParentSchemaId: 105 - - version: "7" - + version: "8" + - version: "10" + + version: "11" upsert descriptor #107 ... createAsOfTime: @@ -839,13 +1226,13 @@ upsert descriptor #107 ... start: "1" unexposedParentSchemaId: 105 - - version: "7" - + version: "8" + - version: "10" + + version: "11" write *eventpb.FinishSchemaChange to event log for descriptor 106 -create job #2 (non-cancelable: true): "GC for ALTER TABLE db.public.tbl ADD COLUMN j INT8 NOT NULL DEFAULT nextval('db.public.sq1')" +create job #2 (non-cancelable: true): "GC for ALTER TABLE db.public.tbl ADD COLUMN j INT8 NOT NULL UNIQUE DEFAULT nextval('db.public.sq1')" descriptor IDs: [106] update progress of schema change job #1: "all stages completed" -commit transaction #11 +commit transaction #17 notified job registry to adopt jobs: [2] # end PostCommitPhase @@ -970,7 +1357,7 @@ upsert descriptor #106 + primaryIndex: + embeddedIndex: + constraintId: 2 - + indexId: 4 + + indexId: 6 + isCreatedExplicitly: true + isUnique: true + keyColumnDirections: @@ -982,14 +1369,14 @@ upsert descriptor #106 + - 2 + - 3 + tableId: 106 - + temporaryIndexId: 5 + + temporaryIndexId: 7 + metadata: + sourceElementId: 1 + subWorkId: 1 + targetStatus: PUBLIC + - elementProto: + indexName: - + indexId: 4 + + indexId: 6 + name: tbl_pkey + tableId: 106 + metadata: @@ -1000,7 +1387,7 @@ upsert descriptor #106 + temporaryIndex: + embeddedIndex: + constraintId: 2 - + indexId: 5 + + indexId: 7 + isCreatedExplicitly: true + isUnique: true + keyColumnDirections: @@ -1028,7 +1415,7 @@ upsert descriptor #106 defaultColumnId: 2 name: primary ... - id: 106 + version: 4 modificationTime: {} + mutations: + - column: @@ -1045,12 +1432,12 @@ upsert descriptor #106 + state: DELETE_ONLY + - direction: ADD + index: - + constraintId: 4 + + constraintId: 6 + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} - + id: 4 + + id: 6 + interleave: {} + keyColumnDirections: + - ASC @@ -1058,7 +1445,7 @@ upsert descriptor #106 + - 1 + keyColumnNames: + - i - + name: crdb_internal_index_4_name_placeholder + + name: crdb_internal_index_6_name_placeholder + partitioning: {} + sharded: {} + storeColumnIds: @@ -1073,12 +1460,12 @@ upsert descriptor #106 + state: BACKFILLING + - direction: ADD + index: - + constraintId: 5 + + constraintId: 7 + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} - + id: 5 + + id: 7 + interleave: {} + keyColumnDirections: + - ASC @@ -1086,7 +1473,7 @@ upsert descriptor #106 + - 1 + keyColumnNames: + - i - + name: crdb_internal_index_5_name_placeholder + + name: crdb_internal_index_7_name_placeholder + partitioning: {} + sharded: {} + storeColumnIds: @@ -1102,19 +1489,19 @@ upsert descriptor #106 + state: DELETE_ONLY name: tbl - nextColumnId: 3 - - nextConstraintId: 4 + - nextConstraintId: 6 + nextColumnId: 4 - + nextConstraintId: 6 + + nextConstraintId: 8 nextFamilyId: 1 - - nextIndexId: 4 - + nextIndexId: 6 + - nextIndexId: 6 + + nextIndexId: 8 nextMutationId: 1 parentId: 104 ... time: {} unexposedParentSchemaId: 105 - - version: "8" - + version: "9" + - version: "11" + + version: "12" write *eventpb.AlterTable to event log for descriptor #106: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹k› INT8 NOT NULL DEFAULT ‹42› create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN k INT8 NOT NULL DEFAULT 42" descriptor IDs: [106] @@ -1158,13 +1545,13 @@ upsert descriptor #106 ... time: {} unexposedParentSchemaId: 105 - - version: "9" - + version: "10" + - version: "12" + + version: "13" update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 ## PostCommitPhase stage 2 of 7 with 1 BackfillType op -backfill indexes [4] from index #2 in table #106 +backfill indexes [6] from index #2 in table #106 commit transaction #4 begin transaction #5 ## PostCommitPhase stage 3 of 7 with 3 MutationType ops @@ -1186,8 +1573,8 @@ upsert descriptor #106 ... time: {} unexposedParentSchemaId: 105 - - version: "10" - + version: "11" + - version: "13" + + version: "14" update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -1210,17 +1597,17 @@ upsert descriptor #106 ... time: {} unexposedParentSchemaId: 105 - - version: "11" - + version: "12" + - version: "14" + + version: "15" update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 ## PostCommitPhase stage 5 of 7 with 1 BackfillType op -merge temporary indexes [5] into backfilled indexes [4] in table #106 +merge temporary indexes [7] into backfilled indexes [6] in table #106 commit transaction #7 begin transaction #8 ## PostCommitPhase stage 6 of 7 with 1 ValidationType op -validate forward indexes [4] in table #106 +validate forward indexes [6] in table #106 commit transaction #8 begin transaction #9 ## PostCommitPhase stage 7 of 7 with 8 MutationType ops @@ -1277,21 +1664,21 @@ upsert descriptor #106 - state: DELETE_AND_WRITE_ONLY - direction: ADD index: - - constraintId: 4 - + constraintId: 5 + - constraintId: 6 + + constraintId: 7 createdExplicitly: true encodingType: 1 foreignKey: {} geoConfig: {} - - id: 4 - + id: 5 + - id: 6 + + id: 7 interleave: {} keyColumnDirections: ... keyColumnNames: - i - - name: crdb_internal_index_4_name_placeholder - + name: crdb_internal_index_5_name_placeholder + - name: crdb_internal_index_6_name_placeholder + + name: crdb_internal_index_7_name_placeholder partitioning: {} sharded: {} ... @@ -1304,20 +1691,20 @@ upsert descriptor #106 - - direction: ADD + - direction: DROP index: - - constraintId: 5 + - constraintId: 7 + constraintId: 2 createdExplicitly: true encodingType: 1 foreignKey: {} geoConfig: {} - - id: 5 + - id: 7 + id: 2 interleave: {} keyColumnDirections: ... keyColumnNames: - i - - name: crdb_internal_index_5_name_placeholder + - name: crdb_internal_index_7_name_placeholder + name: tbl_pkey partitioning: {} sharded: {} @@ -1335,13 +1722,13 @@ upsert descriptor #106 parentId: 104 primaryIndex: - constraintId: 2 - + constraintId: 4 + + constraintId: 6 createdExplicitly: true encodingType: 1 foreignKey: {} geoConfig: {} - id: 2 - + id: 4 + + id: 6 interleave: {} keyColumnDirections: ... @@ -1356,8 +1743,8 @@ upsert descriptor #106 ... time: {} unexposedParentSchemaId: 105 - - version: "12" - + version: "13" + - version: "15" + + version: "16" adding table for stats refresh: 106 update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 1 of 2 with 2 MutationType ops pending" set schema change job #1 to non-cancellable @@ -1396,8 +1783,8 @@ upsert descriptor #106 ... time: {} unexposedParentSchemaId: 105 - - version: "13" - + version: "14" + - version: "16" + + version: "17" update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 4 MutationType ops pending" commit transaction #10 begin transaction #11 @@ -1509,7 +1896,7 @@ upsert descriptor #106 - primaryIndex: - embeddedIndex: - constraintId: 2 - - indexId: 4 + - indexId: 6 - isCreatedExplicitly: true - isUnique: true - keyColumnDirections: @@ -1521,14 +1908,14 @@ upsert descriptor #106 - - 2 - - 3 - tableId: 106 - - temporaryIndexId: 5 + - temporaryIndexId: 7 - metadata: - sourceElementId: 1 - subWorkId: 1 - targetStatus: PUBLIC - - elementProto: - indexName: - - indexId: 4 + - indexId: 6 - name: tbl_pkey - tableId: 106 - metadata: @@ -1539,7 +1926,7 @@ upsert descriptor #106 - temporaryIndex: - embeddedIndex: - constraintId: 2 - - indexId: 5 + - indexId: 7 - isCreatedExplicitly: true - isUnique: true - keyColumnDirections: @@ -1558,17 +1945,17 @@ upsert descriptor #106 families: - columnIds: ... - id: 106 + version: 4 modificationTime: {} - mutations: - - direction: ADD - index: - - constraintId: 5 + - constraintId: 7 - createdExplicitly: true - encodingType: 1 - foreignKey: {} - geoConfig: {} - - id: 5 + - id: 7 - interleave: {} - keyColumnDirections: - - ASC @@ -1576,7 +1963,7 @@ upsert descriptor #106 - - 1 - keyColumnNames: - - i - - name: crdb_internal_index_5_name_placeholder + - name: crdb_internal_index_7_name_placeholder - partitioning: {} - sharded: {} - storeColumnIds: @@ -1622,8 +2009,8 @@ upsert descriptor #106 ... time: {} unexposedParentSchemaId: 105 - - version: "14" - + version: "15" + - 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 k INT8 NOT NULL DEFAULT 42" descriptor IDs: [106]