From 9d89ec3ad0d1d42104b871dd8dcb596d8078aa09 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Tue, 26 Jul 2022 18:16:19 -0400 Subject: [PATCH 01/11] sql/catalog/tabledesc: fixed an issue for estimating number of keys for a row in an index We encounter an issue for this case ``` CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL, FAMILY (i), FAMILY (j)); INSERT INTO t VALUES (23, 24); ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); ``` where the last `ALTER PRIMARY KEY` will result in duplicate entries in the new primary index, because the number of estimated keys for a row in this new primary index is incorrectly estimated to be 1, rather than 2 (the number of column families). --- pkg/sql/catalog/tabledesc/structured.go | 2 +- .../logictest/testdata/logic_test/alter_primary_key | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 3157a01fe4da..4a460fbc2473 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -111,7 +111,7 @@ func (desc *wrapper) GetParentSchemaID() descpb.ID { // IndexKeysPerRow implements the TableDescriptor interface. func (desc *wrapper) IndexKeysPerRow(idx catalog.Index) int { - if desc.PrimaryIndex.ID == idx.GetID() { + if desc.PrimaryIndex.ID == idx.GetID() || idx.GetEncodingType() == descpb.PrimaryIndexEncoding { return len(desc.Families) } if idx.NumSecondaryStoredColumns() == 0 || len(desc.Families) == 1 { diff --git a/pkg/sql/logictest/testdata/logic_test/alter_primary_key b/pkg/sql/logictest/testdata/logic_test/alter_primary_key index 63ece68e1625..b6efd9e6ac24 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_primary_key +++ b/pkg/sql/logictest/testdata/logic_test/alter_primary_key @@ -1706,3 +1706,14 @@ t_pkey i ASC t_pkey j N/A t_j_key j ASC t_j_key i ASC + + +subtest alter_primary_key_with_more_than_one_column_families +statement ok +CREATE TABLE t_multiple_cf (i INT PRIMARY KEY, j INT NOT NULL, FAMILY (i), FAMILY (j)) + +statement ok +INSERT INTO t_multiple_cf VALUES (23, 24) + +statement ok +ALTER TABLE t_multiple_cf ALTER PRIMARY KEY USING COLUMNS (j) From aaa609c42b19bdbd55bbc4fe0596ed43f9c17b41 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Mon, 1 Aug 2022 17:41:41 -0400 Subject: [PATCH 02/11] sql: removed outdated notice message for ALTER PRIMARY KEY Previously, we issued a notice to customer for `ALTER PRIMARY KEY`, saying the altering primary key is done asynchronously and subsequent schema change might be rejected. This was no longer true (see the slace conversation https://cockroachlabs.slack.com/archives/C04U1BTF8/p1640124375172500 for some history. This PR removes this notice and modified the test that tests this notice. --- pkg/sql/alter_primary_key.go | 16 ++++++---------- .../testdata/logic_test/alter_primary_key | 4 +--- 2 files changed, 7 insertions(+), 13 deletions(-) diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index 59db617e3450..07c0ec298793 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -568,18 +568,14 @@ func (p *planner) AlterPrimaryKey( // Send a notice to users about how this job is asynchronous. // TODO(knz): Mention the job ID in the client notice. - noticeStr := "primary key changes are finalized asynchronously" if alterPrimaryKeyLocalitySwap != nil { - noticeStr = "LOCALITY changes will be finalized asynchronously" + p.BufferClientNotice( + ctx, + pgnotice.Newf( + "LOCALITY changes will be finalized asynchronously; "+ + "further schema changes on this table may be restricted until the job completes"), + ) } - p.BufferClientNotice( - ctx, - pgnotice.Newf( - "%s; further schema changes on this table may be restricted "+ - "until the job completes", - noticeStr, - ), - ) return nil } diff --git a/pkg/sql/logictest/testdata/logic_test/alter_primary_key b/pkg/sql/logictest/testdata/logic_test/alter_primary_key index b6efd9e6ac24..0c42f5fc555e 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_primary_key +++ b/pkg/sql/logictest/testdata/logic_test/alter_primary_key @@ -4,10 +4,8 @@ CREATE TABLE t (x INT PRIMARY KEY, y INT NOT NULL, z INT NOT NULL, w INT, INDEX statement ok INSERT INTO t VALUES (1, 2, 3, 4), (5, 6, 7, 8) -query T noticetrace +statement ok ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (y, z) ----- -NOTICE: primary key changes are finalized asynchronously; further schema changes on this table may be restricted until the job completes query IIII rowsort SELECT * FROM t@t_pkey From a79d11b98f05f9d0eade8911f662284eb68bd466 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Fri, 15 Jul 2022 17:53:22 -0400 Subject: [PATCH 03/11] sql/schemachanger: implemented `ALTER PRIMARY KEY` for vanilla case The PR implements `ALTER PRIMARY KEY` under the declarative schema changer framework that handles the simplest, "vanilla" case like ``` CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j) ``` This is the first of a series PRs where followup PRs will expand its capabilities to be able to handle more complex cases, including 1. Allow the requested new primary key to be hash-sharded; 2. Consider the case where the old primary index is on the implicitly created `rowid` column, in which case we also need to drop that column; 3. Consider the case where altering primary key requires us to modify existing secondary indexes(see the legacy schema change about in what cases we should rewrite) 4. Consider partitioning and locality (I'm not sure what they are, and why they play a role when `ALTER PRIMARY KEY` but I've seen them in the old schema changer, so I assume we ought to do something about them too here). Release note: None --- .../scbuild/internal/scbuildstmt/BUILD.bazel | 3 + .../internal/scbuildstmt/alter_table.go | 5 +- .../alter_table_alter_primary_key.go | 766 ++++++++++++++++++ .../scbuild/internal/scbuildstmt/helpers.go | 14 + 4 files changed, 786 insertions(+), 2 deletions(-) create mode 100644 pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_primary_key.go diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel index 049985121e4c..69a1d17d94e6 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel @@ -6,6 +6,7 @@ go_library( srcs = [ "alter_table.go", "alter_table_add_column.go", + "alter_table_alter_primary_key.go", "alter_table_drop_column.go", "comment_on.go", "create_index.go", @@ -24,6 +25,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild/internal/scbuildstmt", visibility = ["//pkg/sql/schemachanger/scbuild:__subpackages__"], deps = [ + "//pkg/clusterversion", "//pkg/security/username", "//pkg/server/telemetry", "//pkg/settings/cluster", @@ -35,6 +37,7 @@ go_library( "//pkg/sql/catalog/tabledesc", "//pkg/sql/catalog/typedesc", "//pkg/sql/decodeusername", + "//pkg/sql/paramparse", "//pkg/sql/parser", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go index 51413905506d..288c3c5ae5e6 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go @@ -28,8 +28,9 @@ import ( // declarative schema changer. Operations marked as non-fully supported can // only be with the use_declarative_schema_changer session variable. var supportedAlterTableStatements = map[reflect.Type]supportedStatement{ - reflect.TypeOf((*tree.AlterTableAddColumn)(nil)): {alterTableAddColumn, true}, - reflect.TypeOf((*tree.AlterTableDropColumn)(nil)): {alterTableDropColumn, true}, + reflect.TypeOf((*tree.AlterTableAddColumn)(nil)): {alterTableAddColumn, true}, + reflect.TypeOf((*tree.AlterTableDropColumn)(nil)): {alterTableDropColumn, true}, + reflect.TypeOf((*tree.AlterTableAlterPrimaryKey)(nil)): {alterTableAlterPrimaryKey, false}, } func init() { diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_primary_key.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_primary_key.go new file mode 100644 index 000000000000..8cd1fe131964 --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_primary_key.go @@ -0,0 +1,766 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scbuildstmt + +import ( + "fmt" + "sort" + "strings" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/paramparse" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "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/types" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +func alterTableAlterPrimaryKey( + b BuildCtx, tn *tree.TableName, tbl *scpb.Table, t *tree.AlterTableAlterPrimaryKey, +) { + // Panic on certain forbidden `ALTER PRIMARY KEY` cases (e.g. one of + // the new primary key column is a virtual column). See the comments + // for a full list of preconditions we check. + checkForEarlyExit(b, tbl, t) + + // Nothing to do if the requested new pk is the same as the old one. + if isNewPrimaryKeySameAsOldPrimaryKey(b, tbl, t) { + return + } + + // TODO (xiang): This section contains all fall-back cases and need to + // be removed to fully support `ALTER PRIMARY KEY`. + fallBackIfConcurrentSchemaChange(b, tbl.TableID) + fallBackIfRequestToBeSharded(t) + fallBackIfSecondaryIndexExists(b, tbl.TableID) + fallBackIfRegionalByRowTable(b, tbl.TableID) + fallBackIfDescColInRowLevelTTLTables(b, tbl.TableID, t) + + // Retrieve old primary index and its name elements. + oldPrimaryIndexElem, newPrimaryIndexElem := getPrimaryIndexes(b, tbl.TableID) + if newPrimaryIndexElem != nil { + // TODO (xiang): some other DDL stmt preceded this `ALTER PRIMARY KEY` and + // thus a new primary index has already been created. We'd like + // to support this use case one day + // (e.g. `ALTER TABLE t ADD COLUMN ..., ALTER PRIMARY KEY ...;`). + // Note that such scenarios should be caught above in + // `fallBackIfConcurrentSchemaChange` and an unimplemented error + // should be returned, so, here we panic with an programming error. + panic(errors.AssertionFailedf("programming error: new primary index has already existed.")) + } + newPrimaryIndexElem = createNewPrimaryIndex(b, tbl, oldPrimaryIndexElem, func( + b BuildCtx, newPrimaryIndex *scpb.PrimaryIndex, _ []*scpb.IndexColumn, + ) (newColumns []*scpb.IndexColumn) { + allColumns := getSortedAllColumnIDsInTable(b, tbl.TableID) + + // Get all KEY columns from t.Columns + allColumnsNameToIDMapping := getAllColumnsNameToIDMapping(b, tbl.TableID) + allKeyColumnIDs := make(map[catid.ColumnID]bool) + for i, col := range t.Columns { + if colID, exist := allColumnsNameToIDMapping[string(col.Column)]; !exist { + panic(fmt.Sprintf("table %v does not have a column named %v", tn.String(), col.Column)) + } else { + ic := &scpb.IndexColumn{ + TableID: tbl.TableID, + IndexID: newPrimaryIndex.IndexID, + ColumnID: colID, + OrdinalInKind: uint32(i), + Kind: scpb.IndexColumn_KEY, + Direction: indexColumnDirection(col.Direction), + } + b.Add(ic) + newColumns = append(newColumns, ic) + allKeyColumnIDs[colID] = true + } + } + + // What's left are STORED columns, excluding virtual columns and system columns + i := 0 + for _, colID := range allColumns { + if _, isKeyCol := allKeyColumnIDs[colID]; isKeyCol || + mustRetrieveColumnTypeElem(b, tbl.TableID, colID).IsVirtual || + colinfo.IsColIDSystemColumn(colID) { + continue + } + ic := &scpb.IndexColumn{ + TableID: tbl.TableID, + IndexID: newPrimaryIndex.IndexID, + ColumnID: colID, + OrdinalInKind: uint32(i), + Kind: scpb.IndexColumn_STORED, + } + b.Add(ic) + newColumns = append(newColumns, ic) + i++ + } + + return newColumns + }) + newPrimaryIndexElem.Sharding = makeShardedDescriptor(b, t) + + // Construct and add elements for a unique secondary index created on + // the old primary key columns. + // This is a CRDB unique feature that exists in the legacy schema changer. + maybeAddUniqueIndexForOldPrimaryKey(b, tn, tbl, t, oldPrimaryIndexElem, newPrimaryIndexElem) +} + +// checkForEarlyExit asserts several precondition for a +// `ALTER PRIMARY KEY`, including +// 1. no expression columns allowed; +// 2. no columns that are in `DROPPED` state; +// 3. no inaccessible columns; +// 4. no nullable columns; +// 5. no virtual columns (starting from v22.1); +// 6. add more here +// Panic if any precondition is found unmet. +func checkForEarlyExit(b BuildCtx, tbl *scpb.Table, t *tree.AlterTableAlterPrimaryKey) { + if err := paramparse.ValidateUniqueConstraintParams( + t.StorageParams, + paramparse.UniqueConstraintParamContext{ + IsPrimaryKey: true, + IsSharded: t.Sharded != nil, + }, + ); err != nil { + panic(err) + } + + for _, col := range t.Columns { + if col.Column == "" && col.Expr != nil { + panic(errors.WithHint( + pgerror.Newf( + pgcode.InvalidColumnDefinition, + "expressions such as %q are not allowed in primary index definition", + col.Expr.String(), + ), + "use columns instead", + )) + } + + colElems := b.ResolveColumn(tbl.TableID, col.Column, ResolveParams{ + IsExistenceOptional: false, + RequiredPrivilege: privilege.CREATE, + }) + + colCurrentStatus, _, colElem := scpb.FindColumn(colElems) + if colElem == nil { + panic(errors.AssertionFailedf("programming error: resolving column %v does not give a "+ + "Column element.", col.Column)) + } + if colCurrentStatus == scpb.Status_DROPPED || colCurrentStatus == scpb.Status_ABSENT { + panic(pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, + "column %q is being dropped", col.Column)) + } + if colElem.IsInaccessible { + panic(pgerror.Newf(pgcode.InvalidSchemaDefinition, "cannot use inaccessible "+ + "column %q in primary key", col.Column)) + } + _, _, colTypeElem := scpb.FindColumnType(colElems) + if colTypeElem == nil { + panic(errors.AssertionFailedf("programming error: resolving column %v does not give a "+ + "ColumnType element.", col.Column)) + } + if colTypeElem.IsNullable { + panic(pgerror.Newf(pgcode.InvalidSchemaDefinition, "cannot use nullable column "+ + "%q in primary key", col.Column)) + } + + if !b.EvalCtx().Settings.Version.IsActive(b, clusterversion.Start22_1) { + if colTypeElem.IsVirtual { + panic(pgerror.Newf(pgcode.FeatureNotSupported, "cannot use virtual column %q "+ + "in primary key", col.Column)) + } + } + } +} + +// isNewPrimaryKeySameAsOldPrimaryKey returns whether the requested new +// primary key is the same as the old primary key. +func isNewPrimaryKeySameAsOldPrimaryKey( + b BuildCtx, tbl *scpb.Table, t *tree.AlterTableAlterPrimaryKey, +) bool { + oldPrimaryIndexElem := mustRetrievePrimaryIndexElement(b, tbl.TableID) + oldPrimaryIndexKeyColumns := mustRetrieveKeyIndexColumns(b, tbl.TableID, oldPrimaryIndexElem.IndexID) + + // Check whether they have the same number of key columns. + if len(oldPrimaryIndexKeyColumns) != len(t.Columns) { + return false + } + + // Check whether they are both sharded or both not sharded. + if (oldPrimaryIndexElem.Sharding == nil) != (t.Sharded == nil) { + return false + } + + // Check whether all key columns (ID and directions) are the same. + for i, col := range t.Columns { + colElems := b.ResolveColumn(tbl.TableID, col.Column, ResolveParams{ + IsExistenceOptional: false, + RequiredPrivilege: privilege.CREATE, + }) + _, _, colElem := scpb.FindColumn(colElems) + if (oldPrimaryIndexKeyColumns[i].ColumnID != colElem.ColumnID) || + oldPrimaryIndexKeyColumns[i].Direction != indexColumnDirection(col.Direction) { + return false + } + } + + // If both are sharded, check whether they have the same bucket count. + if oldPrimaryIndexElem.Sharding != nil { + shardBucketsInNewPrimaryIndex, err := tabledesc.EvalShardBucketCount(b, b.SemaCtx(), b.EvalCtx(), + t.Sharded.ShardBuckets, t.StorageParams) + if err != nil { + panic(err) + } + if oldPrimaryIndexElem.Sharding.ShardBuckets != shardBucketsInNewPrimaryIndex { + return false + } + } + + return true +} + +// fallBackIfConcurrentSchemaChange panics with an unimplemented error if +// there are any other concurrent schema change on this table. This is determined +// by searching for any element that is currently not in its terminal status. +func fallBackIfConcurrentSchemaChange(b BuildCtx, tableID catid.DescID) { + b.QueryByID(tableID).ForEachElementStatus(func(current scpb.Status, target scpb.TargetStatus, e scpb.Element) { + if current != target.Status() { + _, _, ns := scpb.FindNamespace(b.QueryByID(tableID)) + if ns == nil { + panic(errors.AssertionFailedf("programming error: resolving table %v does not "+ + "give a Namespace element", tableID)) + } + panic(scerrors.NotImplementedErrorf( + nil, + "cannot perform a primary key change on %v with other schema changes on %v in the same transaction", + ns.Name, ns.Name)) + } + }) +} + +// fallBackIfRequestToBeSharded panics with an unimplemented error +// if it is requested to be hash-sharded. +func fallBackIfRequestToBeSharded(t *tree.AlterTableAlterPrimaryKey) { + if t.Sharded != nil { + panic(scerrors.NotImplementedErrorf(nil, "ALTER PRIMARY KEY USING HASH is not yet supported.")) + } +} + +// fallBackIfSecondaryIndexExists panics with an unimplemented +// error if there exists secondary indexes on the table, which might +// need to be rewritten. +func fallBackIfSecondaryIndexExists(b BuildCtx, tableID catid.DescID) { + _, _, sie := scpb.FindSecondaryIndex(b.QueryByID(tableID)) + if sie != nil { + panic(scerrors.NotImplementedErrorf(nil, "ALTER PRIMARY KEY on a table with secondary index"+ + "is not yet supported because they might need to be rewritten.")) + } +} + +// fallBackIfRegionalByRowTable panics with an unimplemented +// error if it's a REGIONAL BY ROW table because we need to +// include the implicit REGION column when constructing the +// new primary key. +func fallBackIfRegionalByRowTable(b BuildCtx, tableID catid.DescID) { + _, _, rbrElem := scpb.FindTableLocalityRegionalByRow(b.QueryByID(tableID)) + if rbrElem != nil { + panic(scerrors.NotImplementedErrorf(nil, "ALTER PRIMARY KEY on a REGIONAL BY ROW table "+ + "is not yet supported.")) + } +} + +// fallBackIfDescColInRowLevelTTLTables panics with an unimplemented +// error if the table is a (row-level-ttl table && (it has a descending +// key column || it has any inbound/outbound FK constraint)). +func fallBackIfDescColInRowLevelTTLTables( + b BuildCtx, tableID catid.DescID, t *tree.AlterTableAlterPrimaryKey, +) { + _, _, rowLevelTTLElem := scpb.FindRowLevelTTL(b.QueryByID(tableID)) + if rowLevelTTLElem == nil { + return + } + + // It's a row-level-ttl table. Ensure it has no non-descending + // key columns, and there is no inbound/outbound foreign keys. + for _, col := range t.Columns { + if indexColumnDirection(col.Direction) != catpb.IndexColumn_ASC { + panic(scerrors.NotImplementedErrorf(nil, "non-ascending ordering on PRIMARY KEYs are not supported")) + } + } + + _, _, ns := scpb.FindNamespace(b.QueryByID(tableID)) + hasFKConstraintError := scerrors.NotImplementedErrorf(nil, fmt.Sprintf(`foreign keys to/from +table with TTL "%s" are not permitted`, ns.Name)) + // Panic if there is any inbound/outbound FK constraints. + _, _, inboundFKElem := scpb.FindForeignKeyConstraint(b.BackReferences(tableID)) + if inboundFKElem != nil { + panic(hasFKConstraintError) + } + scpb.ForEachForeignKeyConstraint(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.ForeignKeyConstraint, + ) { + panic(hasFKConstraintError) + }) +} + +func mustRetrievePrimaryIndexElement(b BuildCtx, tableID catid.DescID) (res *scpb.PrimaryIndex) { + scpb.ForEachPrimaryIndex(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.PrimaryIndex, + ) { + // TODO (xiang): for now, we assume there is one primary index, which + // will not be true when there are DDL stmts (e.g. ADD/DROP COLUMN) + // before this `ALTER PRIMARY KEY`. + if current == scpb.Status_PUBLIC { + res = e + } + }) + if res == nil { + panic(errors.AssertionFailedf("programming error: resolving table %v does not give "+ + "a PrimaryIndex element", tableID)) + } + return res +} + +func mustRetrieveColumnElem( + b BuildCtx, tableID catid.DescID, columnID catid.ColumnID, +) (column *scpb.Column) { + scpb.ForEachColumn(b.QueryByID(tableID), func(current scpb.Status, target scpb.TargetStatus, e *scpb.Column) { + if e.ColumnID == columnID { + column = e + } + }) + if column == nil { + panic(errors.AssertionFailedf("programming error: cannot find a Column element for column ID %v", columnID)) + } + return column +} + +func mustRetrieveColumnNameElem( + b BuildCtx, tableID catid.DescID, columnID catid.ColumnID, +) (columnName *scpb.ColumnName) { + scpb.ForEachColumnName(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.ColumnName, + ) { + if e.ColumnID == columnID { + columnName = e + } + }) + if columnName == nil { + panic(errors.AssertionFailedf("programming error: cannot find a ColumnName element for column ID %v", columnID)) + } + return columnName +} + +func mustRetrieveColumnTypeElem( + b BuildCtx, tableID catid.DescID, columnID catid.ColumnID, +) (columnType *scpb.ColumnType) { + scpb.ForEachColumnType(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.ColumnType, + ) { + if e.ColumnID == columnID { + columnType = e + } + }) + if columnType == nil { + panic(errors.AssertionFailedf("programming error: cannot find a ColumnType element for column ID %v", columnID)) + } + return columnType +} + +func mustRetrieveIndexElement( + b BuildCtx, tableID catid.DescID, indexID catid.IndexID, +) (indexElem *scpb.Index) { + scpb.ForEachSecondaryIndex(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.SecondaryIndex, + ) { + if e.IndexID == indexID { + indexElem = &e.Index + } + }) + scpb.ForEachPrimaryIndex(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.PrimaryIndex, + ) { + if e.IndexID == indexID { + indexElem = &e.Index + } + }) + if indexElem == nil { + panic(errors.AssertionFailedf("programming error: cannot find an index with ID %v from table %v", + indexID, tableID)) + } + return indexElem +} + +func mustRetrieveKeyIndexColumns( + b BuildCtx, tableID catid.DescID, indexID catid.IndexID, +) (indexColumns []*scpb.IndexColumn) { + scpb.ForEachIndexColumn(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.IndexColumn, + ) { + if e.IndexID == indexID && e.Kind == scpb.IndexColumn_KEY { + indexColumns = append(indexColumns, e) + } + }) + if indexColumns == nil { + panic(errors.AssertionFailedf("programming error: cannot find any KEY index columns in "+ + "index %v from table %v", indexID, tableID)) + } + return indexColumns +} + +// makeShardedDescriptor construct a sharded descriptor for the new primary key. +// Return nil if the new primary key is not hash-sharded. +func makeShardedDescriptor(b BuildCtx, t *tree.AlterTableAlterPrimaryKey) *catpb.ShardedDescriptor { + if t.Sharded == nil { + return nil + } + + shardBuckets, err := tabledesc.EvalShardBucketCount(b, b.SemaCtx(), b.EvalCtx(), + t.Sharded.ShardBuckets, t.StorageParams) + if err != nil { + panic(err) + } + columnNames := make([]string, len(t.Columns)) + for i, col := range t.Columns { + columnNames[i] = col.Column.String() + } + + return &catpb.ShardedDescriptor{ + IsSharded: true, + Name: tabledesc.GetShardColumnName(columnNames, shardBuckets), + ShardBuckets: shardBuckets, + ColumnNames: columnNames, + } +} + +// maybeAddUniqueIndexForOldPrimaryKey constructs and adds all necessary elements +// for a unique index on the old primary key columns, if certain conditions are +// met (see comments of shouldCreateUniqueIndexOnOldPrimaryKeyColumns for details). +// Namely, it includes +// 1. a SecondaryIndex element; +// 2. a set of IndexColumn elements for the secondary index; +// 3. a TemporaryIndex elements; +// 4. a set of IndexColumn elements for the temporary index; +// 5. a IndexName element; +// This is a CRDB unique feature that helps optimize the performance of +// queries that still filter on old primary key columns. +func maybeAddUniqueIndexForOldPrimaryKey( + b BuildCtx, + tn *tree.TableName, + tbl *scpb.Table, + t *tree.AlterTableAlterPrimaryKey, + oldPrimaryIndex, newPrimaryIndex *scpb.PrimaryIndex, +) { + if shouldCreateUniqueIndexOnOldPrimaryKeyColumns(b, tn, tbl, t, + oldPrimaryIndex.IndexID, newPrimaryIndex.IndexID) { + newUniqueSecondaryIndex, tempIndex := addNewUniqueSecondaryIndexAndTempIndex(b, tn, tbl, oldPrimaryIndex) + addIndexColumnsForNewUniqueSecondaryIndexAndTempIndex(b, tn, tbl, t, + oldPrimaryIndex.IndexID, newUniqueSecondaryIndex.IndexID, tempIndex.IndexID) + addIndexNameForNewUniqueSecondaryIndex(b, tbl, newUniqueSecondaryIndex.IndexID) + } +} + +// addNewUniqueSecondaryIndexAndTempIndex constructs and adds elements for +// a new secondary index and its associated temporary index. +func addNewUniqueSecondaryIndexAndTempIndex( + b BuildCtx, tn *tree.TableName, tbl *scpb.Table, oldPrimaryIndexElem *scpb.PrimaryIndex, +) (*scpb.SecondaryIndex, *scpb.TemporaryIndex) { + newSecondaryIndexElem := &scpb.SecondaryIndex{Index: scpb.Index{ + TableID: tbl.TableID, + IndexID: nextRelationIndexID(b, tbl), + IsUnique: true, + IsInverted: oldPrimaryIndexElem.IsInverted, + Sharding: oldPrimaryIndexElem.Sharding, + IsCreatedExplicitly: false, + SourceIndexID: oldPrimaryIndexElem.IndexID, + TemporaryIndexID: 0, + }} + b.Add(newSecondaryIndexElem) + + temporaryIndexElemForNewSecondaryIndex := &scpb.TemporaryIndex{ + Index: protoutil.Clone(newSecondaryIndexElem).(*scpb.SecondaryIndex).Index, + IsUsingSecondaryEncoding: true, + } + b.AddTransient(temporaryIndexElemForNewSecondaryIndex) + + temporaryIndexElemForNewSecondaryIndex.IndexID = nextRelationIndexID(b, tbl) + newSecondaryIndexElem.TemporaryIndexID = temporaryIndexElemForNewSecondaryIndex.IndexID + + return newSecondaryIndexElem, temporaryIndexElemForNewSecondaryIndex +} + +// addIndexColumnsForNewUniqueSecondaryIndexAndTempIndex constructs and adds IndexColumn +// elements for the new primary index and its associated temporary index. +func addIndexColumnsForNewUniqueSecondaryIndexAndTempIndex( + b BuildCtx, + tn *tree.TableName, + tbl *scpb.Table, + t *tree.AlterTableAlterPrimaryKey, + oldPrimaryIndexID catid.IndexID, + newUniqueSecondaryIndexID catid.IndexID, + temporaryIndexIDForNewUniqueSecondaryIndex catid.IndexID, +) { + // KEY columns = old primary key columns + oldPrimaryIndexKeyColumns := mustRetrieveKeyIndexColumns(b, tbl.TableID, oldPrimaryIndexID) + oldPrimaryIndexKeyColumnIDs := make([]catid.ColumnID, len(oldPrimaryIndexKeyColumns)) + for i, keyIndexCol := range oldPrimaryIndexKeyColumns { + oldPrimaryIndexKeyColumnIDs[i] = keyIndexCol.ColumnID + } + + for _, keyIndexColumn := range oldPrimaryIndexKeyColumns { + b.Add(&scpb.IndexColumn{ + TableID: tbl.TableID, + IndexID: newUniqueSecondaryIndexID, + ColumnID: keyIndexColumn.ColumnID, + OrdinalInKind: keyIndexColumn.OrdinalInKind, + Kind: scpb.IndexColumn_KEY, + Direction: keyIndexColumn.Direction, + }) + b.Add(&scpb.IndexColumn{ + TableID: tbl.TableID, + IndexID: temporaryIndexIDForNewUniqueSecondaryIndex, + ColumnID: keyIndexColumn.ColumnID, + OrdinalInKind: keyIndexColumn.OrdinalInKind, + Kind: scpb.IndexColumn_KEY, + Direction: keyIndexColumn.Direction, + }) + } + + // SUFFIX_KEY columns = new primary index columns - old primary key columns + // First find column IDs and dirs by their names, as specified in t.Columns. + newPrimaryIndexKeyColumnIDs := make([]catid.ColumnID, len(t.Columns)) + newPrimaryIndexKeyColumnDirs := make([]catpb.IndexColumn_Direction, len(t.Columns)) + allColumnsNameToIDMapping := getAllColumnsNameToIDMapping(b, tbl.TableID) + for i, col := range t.Columns { + if colID, exist := allColumnsNameToIDMapping[string(col.Column)]; !exist { + panic(fmt.Sprintf("table %v does not have a column named %v", tn.String(), col.Column)) + } else { + newPrimaryIndexKeyColumnIDs[i] = colID + newPrimaryIndexKeyColumnDirs[i] = indexColumnDirection(col.Direction) + } + } + + // Add each column that is not in the old primary key as a SUFFIX_KEY column. + for i, keyColIDInNewPrimaryIndex := range newPrimaryIndexKeyColumnIDs { + if !descpb.ColumnIDs(oldPrimaryIndexKeyColumnIDs).Contains(keyColIDInNewPrimaryIndex) { + b.Add(&scpb.IndexColumn{ + TableID: tbl.TableID, + IndexID: newUniqueSecondaryIndexID, + ColumnID: keyColIDInNewPrimaryIndex, + OrdinalInKind: uint32(i), + Kind: scpb.IndexColumn_KEY_SUFFIX, + Direction: newPrimaryIndexKeyColumnDirs[i], + }) + b.Add(&scpb.IndexColumn{ + TableID: tbl.TableID, + IndexID: temporaryIndexIDForNewUniqueSecondaryIndex, + ColumnID: keyColIDInNewPrimaryIndex, + OrdinalInKind: uint32(i), + Kind: scpb.IndexColumn_KEY_SUFFIX, + Direction: newPrimaryIndexKeyColumnDirs[i], + }) + } + } +} + +// addIndexNameForNewUniqueSecondaryIndex constructs and adds an IndexName +// element for the new, unique secondary index on the old primary key. +func addIndexNameForNewUniqueSecondaryIndex(b BuildCtx, tbl *scpb.Table, indexID catid.IndexID) { + indexName := getImplicitSecondaryIndexName(b, tbl, indexID, 0 /* numImplicitColumns */) + b.Add(&scpb.IndexName{ + TableID: tbl.TableID, + IndexID: indexID, + Name: indexName, + }) +} + +// We only recreate the old primary key of the table as a unique secondary +// index if: +// * The table has a primary key (no DROP PRIMARY KEY statements have +// been executed). +// * The primary key is not the default rowid primary key. +// * The new primary key isn't the same set of columns and directions +// other than hash sharding. +// * There is no partitioning change. +// * There is no existing secondary index on the old primary key columns. +func shouldCreateUniqueIndexOnOldPrimaryKeyColumns( + b BuildCtx, + tn *tree.TableName, + tbl *scpb.Table, + t *tree.AlterTableAlterPrimaryKey, + oldPrimaryIndexID catid.IndexID, + newPrimaryIndexID catid.IndexID, +) bool { + // A function that retrieves all KEY columns of this index. + // If excludeShardedCol, sharded column is excluded, if any. + keyColumnIDsAndDirsOfIndex := func( + b BuildCtx, tableID catid.DescID, indexID catid.IndexID, excludeShardedCol bool, + ) ( + columnIDs descpb.ColumnIDs, + columnDirs []catpb.IndexColumn_Direction, + ) { + sharding := mustRetrieveIndexElement(b, tableID, indexID).Sharding + allKeyIndexColumns := mustRetrieveKeyIndexColumns(b, tableID, indexID) + for _, keyIndexCol := range allKeyIndexColumns { + if !excludeShardedCol || sharding == nil || + mustRetrieveColumnNameElem(b, tableID, keyIndexCol.ColumnID).Name != sharding.Name { + columnIDs = append(columnIDs, keyIndexCol.ColumnID) + columnDirs = append(columnDirs, keyIndexCol.Direction) + } + } + return columnIDs, columnDirs + } + + // A function that checks whether two indexes have matching columns and directions, + // excluding shard column if specified. + keyColumnIDsAndDirsMatch := func( + b BuildCtx, tableID catid.DescID, oldIndexID, newIndexID catid.IndexID, excludeShardedCol bool, + ) bool { + oldIDs, oldDirs := keyColumnIDsAndDirsOfIndex(b, tableID, oldIndexID, excludeShardedCol) + newIDs, newDirs := keyColumnIDsAndDirsOfIndex(b, tableID, newIndexID, excludeShardedCol) + if !oldIDs.Equals(newIDs) { + return false + } + for i := range oldDirs { + if oldDirs[i] != newDirs[i] { + return false + } + } + return true + } + + // A function that checks whether there exists a secondary index + // that is "identical" to the old primary index. + // It is used to avoid creating duplicate secondary index during + // `ALTER PRIMARY KEY`. + alreadyHasSecondaryIndexOnPKColumns := func( + b BuildCtx, tableID catid.DescID, oldPrimaryIndexID catid.IndexID, + ) (found bool) { + scpb.ForEachSecondaryIndex(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, candidate *scpb.SecondaryIndex, + ) { + if !mustRetrieveIndexElement(b, tableID, candidate.IndexID).IsUnique { + return + } + if !keyColumnIDsAndDirsMatch(b, tableID, oldPrimaryIndexID, + candidate.IndexID, false /* excludeShardedCol */) { + return + } + // This secondary index is non-partial, unique, and has exactly the same + // key columns (and same directions) as the old primary index! + found = true + }) + return found + } + + return !isPrimaryIndexDefaultRowID(b, tbl.TableID, oldPrimaryIndexID) && + !keyColumnIDsAndDirsMatch(b, tbl.TableID, oldPrimaryIndexID, newPrimaryIndexID, true /* excludeShardedCol */) && + !alreadyHasSecondaryIndexOnPKColumns(b, tbl.TableID, oldPrimaryIndexID) +} + +// isPrimaryIndexDefaultRowID checks whether the index is on the +// implicitly created, hidden column 'rowid'. +func isPrimaryIndexDefaultRowID( + b BuildCtx, tableID catid.DescID, indexID catid.IndexID, +) (res bool) { + // Sanity check: input `indexID` should really be the index of + // a primary index. + var primaryIndex *scpb.PrimaryIndex + scpb.ForEachPrimaryIndex(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.PrimaryIndex, + ) { + if current == scpb.Status_PUBLIC && e.IndexID == indexID { + primaryIndex = e + } + }) + if primaryIndex == nil { + panic(fmt.Sprintf("internal error: input indexID %v is not the primary index of table %v", indexID, tableID)) + } + + // This primary index should have only one column. + indexColumns := mustRetrieveKeyIndexColumns(b, tableID, indexID) + if len(indexColumns) != 1 { + return false + } + + columnID := indexColumns[0].ColumnID + + // That one column should be hidden. + column := mustRetrieveColumnElem(b, tableID, columnID) + if !column.IsHidden { + return false + } + + // That one column's name should be 'rowid' or prefixed by 'rowid'. + columnName := mustRetrieveColumnNameElem(b, tableID, columnID) + if !strings.HasPrefix(columnName.Name, "rowid") { + return false + } + + // That column should be of type INT. + columnType := mustRetrieveColumnTypeElem(b, tableID, columnID) + if !columnType.Type.Equal(types.Int) { + return false + } + + // That column should have default expression that is equal to "unique_rowid()". + var columnDefaultExpression *scpb.ColumnDefaultExpression + scpb.ForEachColumnDefaultExpression(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.ColumnDefaultExpression, + ) { + if e.ColumnID == column.ColumnID { + columnDefaultExpression = e + } + }) + if columnDefaultExpression == nil || columnDefaultExpression.Expr != "unique_rowid()" { + return false + } + + // All checks are satisfied, return true! + return true +} + +// getAllColumnsNameToIDMapping constructs a name to ID mapping +// for all non-system columns. +func getAllColumnsNameToIDMapping( + b BuildCtx, tableID catid.DescID, +) (res map[string]catid.ColumnID) { + res = make(map[string]catid.ColumnID) + scpb.ForEachColumnName(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.ColumnName, + ) { + res[e.Name] = e.ColumnID + }) + return res +} + +// getSortedAllColumnIDsInTable returns sorted IDs of all columns in table. +func getSortedAllColumnIDsInTable(b BuildCtx, tableID catid.DescID) (res []catid.ColumnID) { + scpb.ForEachColumn(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.Column, + ) { + res = append(res, e.ColumnID) + }) + sort.Slice(res, func(i, j int) bool { + return res[i] < res[j] + }) + return res +} diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go index db4437add65c..7f898069537b 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go @@ -15,6 +15,7 @@ import ( "sort" "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" @@ -22,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/errors" ) @@ -356,3 +358,15 @@ func getPrimaryIndexes( } return existing, freshlyAdded } + +// indexColumnDirection converts tree.Direction to catpb.IndexColumn_Direction. +func indexColumnDirection(d tree.Direction) catpb.IndexColumn_Direction { + switch d { + case tree.DefaultDirection, tree.Ascending: + return catpb.IndexColumn_ASC + case tree.Descending: + return catpb.IndexColumn_DESC + default: + panic(errors.AssertionFailedf("unknown direction %s", d)) + } +} From dfc168c000a1e89c161a218f196a2b216211d22b Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Fri, 15 Jul 2022 17:59:32 -0400 Subject: [PATCH 04/11] sql/schemachanger: added tests for the `ALTER PRIMARY KEY` vanilla case Release note: None --- .../scbuild/testdata/alter_table_add_column | 44 ++ .../testdata/unimplemented_alter_table | 4 - .../scplan/testdata/alter_table_add_column | 318 +++++++++ .../alter_table_alter_primary_key_vanilla | 636 ++++++++++++++++++ .../alter_table_alter_primary_key_vanilla | 149 ++++ ..._alter_primary_key_vanilla.rollback_1_of_7 | 43 ++ ..._alter_primary_key_vanilla.rollback_2_of_7 | 52 ++ ..._alter_primary_key_vanilla.rollback_3_of_7 | 52 ++ ..._alter_primary_key_vanilla.rollback_4_of_7 | 52 ++ ..._alter_primary_key_vanilla.rollback_5_of_7 | 56 ++ ..._alter_primary_key_vanilla.rollback_6_of_7 | 56 ++ ..._alter_primary_key_vanilla.rollback_7_of_7 | 56 ++ .../alter_table_alter_primary_key_vanilla | 612 +++++++++++++++++ ..._alter_primary_key_vanilla.rollback_1_of_7 | 207 ++++++ ..._alter_primary_key_vanilla.rollback_2_of_7 | 235 +++++++ ..._alter_primary_key_vanilla.rollback_3_of_7 | 235 +++++++ ..._alter_primary_key_vanilla.rollback_4_of_7 | 235 +++++++ ..._alter_primary_key_vanilla.rollback_5_of_7 | 249 +++++++ ..._alter_primary_key_vanilla.rollback_6_of_7 | 249 +++++++ ..._alter_primary_key_vanilla.rollback_7_of_7 | 249 +++++++ 20 files changed, 3785 insertions(+), 4 deletions(-) create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_vanilla create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_1_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_2_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_3_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_4_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_5_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_6_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_7_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_1_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_2_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_3_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_4_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_5_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_6_of_7 create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_7_of_7 diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_column b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_column index 614a2fe864b0..2701e677912b 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_column +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_column @@ -145,3 +145,47 @@ ALTER TABLE defaultdb.bar ADD COLUMN b INT; {columnId: 3, isNullable: true, tableId: 105, type: {family: IntFamily, oid: 20, width: 64}} - [[IndexColumn:{DescID: 105, ColumnID: 3, IndexID: 1}, PUBLIC], ABSENT] {columnId: 3, indexId: 1, kind: STORED, ordinalInKind: 1, tableId: 105} + +setup +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL) +---- + +build +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j) +---- +- [[IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] + {columnId: 1, indexId: 1, tableId: 106} +- [[IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] + {columnId: 2, indexId: 1, kind: STORED, tableId: 106} +- [[PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] + {constraintId: 1, indexId: 1, isUnique: true, tableId: 106} +- [[IndexName:{DescID: 106, Name: t_pkey, IndexID: 1}, ABSENT], PUBLIC] + {indexId: 1, name: t_pkey, tableId: 106} +- [[IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] + {columnId: 2, indexId: 2, tableId: 106} +- [[IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT] + {columnId: 1, indexId: 2, kind: STORED, tableId: 106} +- [[PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], ABSENT] + {constraintId: 1, indexId: 2, isUnique: true, sourceIndexId: 1, tableId: 106, temporaryIndexId: 3} +- [[IndexName:{DescID: 106, Name: t_pkey, IndexID: 2}, PUBLIC], ABSENT] + {indexId: 2, name: t_pkey, tableId: 106} +- [[TemporaryIndex:{DescID: 106, IndexID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] + {constraintId: 1, indexId: 3, isUnique: true, sourceIndexId: 1, tableId: 106} +- [[IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3}, PUBLIC], ABSENT] + {columnId: 2, indexId: 3, tableId: 106} +- [[IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3}, PUBLIC], ABSENT] + {columnId: 1, indexId: 3, kind: STORED, tableId: 106} +- [[SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], ABSENT] + {indexId: 4, isUnique: true, sourceIndexId: 1, tableId: 106, temporaryIndexId: 5} +- [[TemporaryIndex:{DescID: 106, IndexID: 5, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] + {indexId: 5, isUnique: true, isUsingSecondaryEncoding: true, sourceIndexId: 1, tableId: 106} +- [[IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 4}, PUBLIC], ABSENT] + {columnId: 1, indexId: 4, tableId: 106} +- [[IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5}, PUBLIC], ABSENT] + {columnId: 1, indexId: 5, tableId: 106} +- [[IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 4}, PUBLIC], ABSENT] + {columnId: 2, indexId: 4, kind: KEY_SUFFIX, tableId: 106} +- [[IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 5}, PUBLIC], ABSENT] + {columnId: 2, indexId: 5, kind: KEY_SUFFIX, tableId: 106} +- [[IndexName:{DescID: 106, Name: t_i_key, IndexID: 4}, PUBLIC], ABSENT] + {indexId: 4, name: t_i_key, tableId: 106} diff --git a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table index 2cd33ad8f7c5..ce6dc16a0c29 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table +++ b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table @@ -32,10 +32,6 @@ unimplemented ALTER TABLE defaultdb.foo ALTER COLUMN i SET DATA TYPE STRING ---- -unimplemented -ALTER TABLE defaultdb.foo ALTER PRIMARY KEY USING COLUMNS (i) ----- - unimplemented ALTER TABLE defaultdb.foo DROP COLUMN k ---- diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column b/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column index 9de7f368e600..f4eaeb0b24d1 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column @@ -1241,3 +1241,321 @@ PostCommitPhase stage 2 of 2 with 7 MutationType ops *scop.UpdateSchemaChangerJob IsNonCancelable: true JobID: 1 + +setup +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL) +---- + +ops +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j) +---- +StatementPhase stage 1 of 1 with 12 MutationType ops + transitions: + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 108, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[TemporaryIndex:{DescID: 108, IndexID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[SecondaryIndex:{DescID: 108, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[TemporaryIndex:{DescID: 108, IndexID: 5, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 5}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 5}, PUBLIC], ABSENT] -> PUBLIC + ops: + *scop.MakeAddedIndexBackfilling + Index: + ConstraintID: 1 + IndexID: 2 + IsUnique: true + SourceIndexID: 1 + TableID: 108 + TemporaryIndexID: 3 + *scop.MakeAddedTempIndexDeleteOnly + Index: + ConstraintID: 1 + IndexID: 3 + IsUnique: true + SourceIndexID: 1 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 3 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 3 + Kind: 2 + TableID: 108 + *scop.MakeAddedIndexBackfilling + Index: + IndexID: 4 + IsUnique: true + SourceIndexID: 1 + TableID: 108 + TemporaryIndexID: 5 + IsSecondaryIndex: true + *scop.MakeAddedTempIndexDeleteOnly + Index: + IndexID: 5 + IsUnique: true + SourceIndexID: 1 + TableID: 108 + IsSecondaryIndex: true + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 4 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 5 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 4 + Kind: 1 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 5 + Kind: 1 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 2 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 2 + Kind: 2 + TableID: 108 +PreCommitPhase stage 1 of 1 with 2 MutationType ops + transitions: + ops: + *scop.SetJobStateOnDescriptor + DescriptorID: 108 + Initialize: true + *scop.CreateSchemaChangerJob + Authorization: + UserName: root + DescriptorIDs: + - 108 + JobID: 1 + RunningStatus: PostCommitPhase stage 1 of 7 with 2 MutationType ops pending + Statements: + - statement: ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j) + redactedstatement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS + (‹j›) + statementtag: ALTER TABLE +PostCommitPhase stage 1 of 7 with 4 MutationType ops + transitions: + [[TemporaryIndex:{DescID: 108, IndexID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> WRITE_ONLY + [[TemporaryIndex:{DescID: 108, IndexID: 5, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> WRITE_ONLY + ops: + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 3 + TableID: 108 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 5 + TableID: 108 + *scop.SetJobStateOnDescriptor + DescriptorID: 108 + *scop.UpdateSchemaChangerJob + JobID: 1 +PostCommitPhase stage 2 of 7 with 2 BackfillType ops + transitions: + [[PrimaryIndex:{DescID: 108, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], BACKFILL_ONLY] -> BACKFILLED + [[SecondaryIndex:{DescID: 108, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], BACKFILL_ONLY] -> BACKFILLED + ops: + *scop.BackfillIndex + IndexID: 2 + SourceIndexID: 1 + TableID: 108 + *scop.BackfillIndex + IndexID: 4 + SourceIndexID: 1 + TableID: 108 +PostCommitPhase stage 3 of 7 with 4 MutationType ops + transitions: + [[PrimaryIndex:{DescID: 108, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], BACKFILLED] -> DELETE_ONLY + [[SecondaryIndex:{DescID: 108, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], BACKFILLED] -> DELETE_ONLY + ops: + *scop.MakeBackfillingIndexDeleteOnly + IndexID: 2 + TableID: 108 + *scop.MakeBackfillingIndexDeleteOnly + IndexID: 4 + TableID: 108 + *scop.SetJobStateOnDescriptor + DescriptorID: 108 + *scop.UpdateSchemaChangerJob + JobID: 1 +PostCommitPhase stage 4 of 7 with 4 MutationType ops + transitions: + [[PrimaryIndex:{DescID: 108, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], DELETE_ONLY] -> MERGE_ONLY + [[SecondaryIndex:{DescID: 108, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], DELETE_ONLY] -> MERGE_ONLY + ops: + *scop.MakeBackfilledIndexMerging + IndexID: 2 + TableID: 108 + *scop.MakeBackfilledIndexMerging + IndexID: 4 + TableID: 108 + *scop.SetJobStateOnDescriptor + DescriptorID: 108 + *scop.UpdateSchemaChangerJob + JobID: 1 +PostCommitPhase stage 5 of 7 with 2 BackfillType ops + transitions: + [[PrimaryIndex:{DescID: 108, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], MERGE_ONLY] -> MERGED + [[SecondaryIndex:{DescID: 108, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], MERGE_ONLY] -> MERGED + ops: + *scop.MergeIndex + BackfilledIndexID: 2 + TableID: 108 + TemporaryIndexID: 3 + *scop.MergeIndex + BackfilledIndexID: 4 + TableID: 108 + TemporaryIndexID: 5 +PostCommitPhase stage 6 of 7 with 4 MutationType ops + transitions: + [[PrimaryIndex:{DescID: 108, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], MERGED] -> WRITE_ONLY + [[SecondaryIndex:{DescID: 108, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], MERGED] -> WRITE_ONLY + ops: + *scop.MakeMergedIndexWriteOnly + IndexID: 2 + TableID: 108 + *scop.MakeMergedIndexWriteOnly + IndexID: 4 + TableID: 108 + *scop.SetJobStateOnDescriptor + DescriptorID: 108 + *scop.UpdateSchemaChangerJob + JobID: 1 +PostCommitPhase stage 7 of 7 with 2 ValidationType ops + transitions: + [[PrimaryIndex:{DescID: 108, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], WRITE_ONLY] -> VALIDATED + [[SecondaryIndex:{DescID: 108, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], WRITE_ONLY] -> VALIDATED + ops: + *scop.ValidateUniqueIndex + IndexID: 2 + TableID: 108 + *scop.ValidateUniqueIndex + IndexID: 4 + TableID: 108 +PostCommitNonRevertiblePhase stage 1 of 3 with 12 MutationType ops + transitions: + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[IndexName:{DescID: 108, Name: t_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[PrimaryIndex:{DescID: 108, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], VALIDATED] -> PUBLIC + [[IndexName:{DescID: 108, Name: t_pkey, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[TemporaryIndex:{DescID: 108, IndexID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], WRITE_ONLY] -> TRANSIENT_DELETE_ONLY + [[SecondaryIndex:{DescID: 108, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], VALIDATED] -> PUBLIC + [[TemporaryIndex:{DescID: 108, IndexID: 5, SourceIndexID: 1}, TRANSIENT_ABSENT], WRITE_ONLY] -> TRANSIENT_DELETE_ONLY + [[IndexName:{DescID: 108, Name: t_i_key, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC + ops: + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + IndexID: 1 + TableID: 108 + *scop.SetIndexName + IndexID: 1 + Name: crdb_internal_index_1_name_placeholder + TableID: 108 + *scop.SetIndexName + IndexID: 2 + Name: t_pkey + TableID: 108 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 3 + TableID: 108 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 5 + TableID: 108 + *scop.SetIndexName + IndexID: 4 + Name: t_i_key + TableID: 108 + *scop.RemoveColumnFromIndex + ColumnID: 1 + IndexID: 1 + TableID: 108 + *scop.RemoveColumnFromIndex + ColumnID: 2 + IndexID: 1 + Kind: 2 + TableID: 108 + *scop.MakeAddedPrimaryIndexPublic + EventBase: + Authorization: + UserName: root + Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + StatementTag: ALTER TABLE + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + IndexID: 2 + TableID: 108 + *scop.MakeAddedSecondaryIndexPublic + IndexID: 4 + TableID: 108 + *scop.SetJobStateOnDescriptor + DescriptorID: 108 + *scop.UpdateSchemaChangerJob + IsNonCancelable: true + JobID: 1 +PostCommitNonRevertiblePhase stage 2 of 3 with 3 MutationType ops + transitions: + [[PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}, ABSENT], WRITE_ONLY] -> DELETE_ONLY + ops: + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 108 + *scop.SetJobStateOnDescriptor + DescriptorID: 108 + *scop.UpdateSchemaChangerJob + IsNonCancelable: true + JobID: 1 +PostCommitNonRevertiblePhase stage 3 of 3 with 8 MutationType ops + transitions: + [[PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}, ABSENT], DELETE_ONLY] -> ABSENT + [[TemporaryIndex:{DescID: 108, IndexID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], TRANSIENT_DELETE_ONLY] -> TRANSIENT_ABSENT + [[TemporaryIndex:{DescID: 108, IndexID: 5, SourceIndexID: 1}, TRANSIENT_ABSENT], TRANSIENT_DELETE_ONLY] -> TRANSIENT_ABSENT + ops: + *scop.CreateGcJobForIndex + IndexID: 3 + TableID: 108 + *scop.MakeIndexAbsent + IndexID: 3 + TableID: 108 + *scop.CreateGcJobForIndex + IndexID: 5 + TableID: 108 + *scop.MakeIndexAbsent + IndexID: 5 + TableID: 108 + *scop.CreateGcJobForIndex + IndexID: 1 + StatementForDropJob: + Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + TableID: 108 + *scop.MakeIndexAbsent + EventBase: + Authorization: + UserName: root + Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + StatementTag: ALTER TABLE + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + IndexID: 1 + TableID: 108 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 108 + JobID: 1 + *scop.UpdateSchemaChangerJob + IsNonCancelable: true + JobID: 1 diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_vanilla b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_vanilla new file mode 100644 index 000000000000..27b65cea2b35 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_vanilla @@ -0,0 +1,636 @@ +setup +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL) +---- +... ++object {100 101 t} -> 104 + + +test +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j) +---- +begin transaction #1 +# begin StatementPhase +checking for feature: ALTER TABLE +increment telemetry for sql.schema.alter_table +## StatementPhase stage 1 of 1 with 12 MutationType ops +upsert descriptor #104 + ... + formatVersion: 3 + id: 104 + - modificationTime: + - wallTime: "1640995200000000000" + + modificationTime: {} + + mutations: + + - direction: ADD + + index: + + constraintId: 2 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keyColumnNames: + + - j + + name: crdb_internal_index_2_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 1 + + storeColumnNames: + + - i + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keyColumnNames: + + - j + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 1 + + storeColumnNames: + + - i + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + + - direction: ADD + + index: + + constraintId: 4 + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 4 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + keySuffixColumnIds: + + - 2 + + name: crdb_internal_index_4_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 5 + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 5 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + keySuffixColumnIds: + + - 2 + + name: crdb_internal_index_5_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: t + nextColumnId: 3 + - nextConstraintId: 2 + + nextConstraintId: 6 + nextFamilyId: 1 + - nextIndexId: 2 + + nextIndexId: 6 + nextMutationId: 1 + parentId: 100 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "1" + + version: "2" +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 1 with 2 MutationType ops +upsert descriptor #104 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PRIMARY KEY + + USING COLUMNS (‹j›) + + statement: ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j) + + statementTag: ALTER TABLE + + revertible: true + + targetRanks: + + targets: + families: + - columnIds: + ... + formatVersion: 3 + id: 104 + - modificationTime: {} + + modificationTime: + + wallTime: "1640995200000000001" + mutations: + - direction: ADD + ... +create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j)" + descriptor IDs: [104] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitPhase stage 1 of 7 with 4 MutationType ops +upsert descriptor #104 + ... + formatVersion: 3 + id: 104 + - modificationTime: + - wallTime: "1640995200000000001" + + modificationTime: {} + mutations: + - direction: ADD + ... + 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: t + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "2" + + version: "3" +update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 2 BackfillType ops pending" +commit transaction #3 +begin transaction #4 +## PostCommitPhase stage 2 of 7 with 2 BackfillType ops +backfill indexes [2 4] from index #1 in table #104 +commit transaction #4 +begin transaction #5 +## PostCommitPhase stage 3 of 7 with 4 MutationType ops +upsert descriptor #104 + ... + formatVersion: 3 + id: 104 + - modificationTime: + - wallTime: "1640995200000000003" + + modificationTime: {} + mutations: + - direction: ADD + ... + version: 4 + mutationId: 1 + - state: BACKFILLING + + state: DELETE_ONLY + - direction: ADD + index: + ... + version: 4 + mutationId: 1 + - state: BACKFILLING + + state: DELETE_ONLY + - direction: ADD + index: + ... + time: {} + unexposedParentSchemaId: 101 + - version: "3" + + version: "4" +update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 2 MutationType ops pending" +commit transaction #5 +begin transaction #6 +## PostCommitPhase stage 4 of 7 with 4 MutationType ops +upsert descriptor #104 + ... + formatVersion: 3 + id: 104 + - modificationTime: + - wallTime: "1640995200000000005" + + modificationTime: {} + mutations: + - direction: ADD + ... + version: 4 + mutationId: 1 + - state: DELETE_ONLY + + state: MERGING + - direction: ADD + index: + ... + version: 4 + mutationId: 1 + - state: DELETE_ONLY + + state: MERGING + - direction: ADD + index: + ... + time: {} + unexposedParentSchemaId: 101 + - version: "4" + + version: "5" +update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 2 BackfillType ops pending" +commit transaction #6 +begin transaction #7 +## PostCommitPhase stage 5 of 7 with 2 BackfillType ops +merge temporary indexes [3 5] into backfilled indexes [2 4] in table #104 +commit transaction #7 +begin transaction #8 +## PostCommitPhase stage 6 of 7 with 4 MutationType ops +upsert descriptor #104 + ... + formatVersion: 3 + id: 104 + - modificationTime: + - wallTime: "1640995200000000006" + + modificationTime: {} + mutations: + - direction: ADD + ... + version: 4 + mutationId: 1 + - state: MERGING + + state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + ... + version: 4 + mutationId: 1 + - state: MERGING + + state: DELETE_AND_WRITE_ONLY + - direction: ADD + index: + ... + time: {} + unexposedParentSchemaId: 101 + - version: "5" + + version: "6" +update progress of schema change job #1: "PostCommitPhase stage 7 of 7 with 2 ValidationType ops pending" +commit transaction #8 +begin transaction #9 +## PostCommitPhase stage 7 of 7 with 2 ValidationType ops +validate forward indexes [2] in table #104 +commit transaction #9 +begin transaction #10 +## PostCommitNonRevertiblePhase stage 1 of 3 with 12 MutationType ops +upsert descriptor #104 + ... + statement: ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j) + statementTag: ALTER TABLE + - revertible: true + targetRanks: + targets: + ... + formatVersion: 3 + id: 104 + - modificationTime: + - wallTime: "1640995200000000008" + + indexes: + + - constraintId: 4 + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 4 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + keySuffixColumnIds: + + - 2 + + name: t_i_key + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + + modificationTime: {} + mutations: + - - direction: ADD + + - direction: DROP + index: + - constraintId: 2 + - createdExplicitly: true + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 2 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 2 + - keyColumnNames: + - - j + - name: crdb_internal_index_2_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnIds: + - - 1 + - storeColumnNames: + - - i + - unique: true + - version: 4 + - mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + - - direction: ADD + - index: + constraintId: 3 + createdExplicitly: true + ... + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + - - direction: ADD + + state: DELETE_ONLY + + - direction: DROP + index: + - constraintId: 4 + + constraintId: 5 + createdExplicitly: true + foreignKey: {} + geoConfig: {} + - id: 4 + + id: 5 + interleave: {} + keyColumnDirections: + ... + keySuffixColumnIds: + - 2 + - name: crdb_internal_index_4_name_placeholder + + name: crdb_internal_index_5_name_placeholder + partitioning: {} + sharded: {} + storeColumnNames: [] + unique: true + + useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + - - direction: ADD + + state: DELETE_ONLY + + - direction: DROP + index: + - constraintId: 5 + - createdExplicitly: true + + constraintId: 1 + + createdAtNanos: "1640995200000000000" + + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 5 + + id: 1 + interleave: {} + keyColumnDirections: + ... + keyColumnNames: + - i + - keySuffixColumnIds: + - - 2 + - name: crdb_internal_index_5_name_placeholder + + name: crdb_internal_index_1_name_placeholder + partitioning: {} + sharded: {} + - storeColumnNames: [] + + storeColumnIds: + + - 2 + + storeColumnNames: + + - j + unique: true + - useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + ... + parentId: 100 + primaryIndex: + - constraintId: 1 + - createdAtNanos: "1640995200000000000" + + constraintId: 2 + + createdExplicitly: true + encodingType: 1 + foreignKey: {} + geoConfig: {} + - id: 1 + + id: 2 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - - 1 + + - 2 + keyColumnNames: + - - i + + - j + name: t_pkey + partitioning: {} + sharded: {} + storeColumnIds: + - - 2 + + - 1 + storeColumnNames: + - - j + + - i + unique: true + version: 4 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "6" + + version: "7" +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 3 with 1 MutationType op pending" +set schema change job #1 to non-cancellable +commit transaction #10 +begin transaction #11 +## PostCommitNonRevertiblePhase stage 2 of 3 with 3 MutationType ops +upsert descriptor #104 + ... + unique: true + version: 4 + - modificationTime: + - wallTime: "1640995200000000010" + + modificationTime: {} + mutations: + - direction: DROP + ... + version: 4 + mutationId: 1 + - state: DELETE_AND_WRITE_ONLY + + state: DELETE_ONLY + name: t + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "7" + + version: "8" +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 3 with 6 MutationType ops pending" +commit transaction #11 +begin transaction #12 +## PostCommitNonRevertiblePhase stage 3 of 3 with 8 MutationType ops +upsert descriptor #104 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PRIMARY KEY + - USING COLUMNS (‹j›) + - statement: ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j) + - statementTag: ALTER TABLE + - targetRanks: + - targets: + families: + - columnIds: + ... + unique: true + version: 4 + - modificationTime: + - wallTime: "1640995200000000011" + - mutations: + - - direction: DROP + - index: + - constraintId: 3 + - createdExplicitly: true + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 3 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 2 + - keyColumnNames: + - - j + - name: crdb_internal_index_3_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnIds: + - - 1 + - storeColumnNames: + - - i + - unique: true + - useDeletePreservingEncoding: true + - version: 4 + - mutationId: 1 + - state: DELETE_ONLY + - - direction: DROP + - index: + - constraintId: 5 + - createdExplicitly: true + - foreignKey: {} + - geoConfig: {} + - id: 5 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - keyColumnNames: + - - i + - keySuffixColumnIds: + - - 2 + - 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 + - createdAtNanos: "1640995200000000000" + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 1 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - keyColumnNames: + - - i + - name: crdb_internal_index_1_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnIds: + - - 2 + - storeColumnNames: + - - j + - unique: true + - version: 4 + - mutationId: 1 + - state: DELETE_ONLY + + modificationTime: {} + + mutations: [] + name: t + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "8" + + version: "9" +write *eventpb.FinishSchemaChange to event log for descriptor 104 +create job #2 (non-cancelable: true): "GC for ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j)" + descriptor IDs: [104] +update progress of schema change job #1: "all stages completed" +commit transaction #12 +notified job registry to adopt jobs: [2] +# end PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla new file mode 100644 index 000000000000..02c3e10e7436 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla @@ -0,0 +1,149 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +EXPLAIN (ddl) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +---- +Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); + ├── StatementPhase + │ └── Stage 1 of 1 in StatementPhase + │ ├── 10 elements transitioning toward PUBLIC + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── ABSENT → BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ ├── 2 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ └── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ └── 12 Mutation operations + │ ├── MakeAddedIndexBackfilling {"Index":{"ConstraintID":1,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":3}} + │ ├── MakeAddedTempIndexDeleteOnly {"Index":{"ConstraintID":1,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104}} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"Kind":2,"TableID":104} + │ ├── MakeAddedIndexBackfilling {"IsSecondaryIndex":true} + │ ├── MakeAddedTempIndexDeleteOnly {"IsSecondaryIndex":true} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":5,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":4,"Kind":1,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":5,"Kind":1,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"TableID":104} + │ └── AddColumnToIndex {"ColumnID":1,"IndexID":2,"Kind":2,"TableID":104} + ├── PreCommitPhase + │ └── Stage 1 of 1 in PreCommitPhase + │ └── 2 Mutation operations + │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} + │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} + ├── PostCommitPhase + │ ├── Stage 1 of 7 in PostCommitPhase + │ │ ├── 2 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ └── DELETE_ONLY → WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ └── 4 Mutation operations + │ │ ├── MakeAddedIndexDeleteAndWriteOnly {"IndexID":3,"TableID":104} + │ │ ├── MakeAddedIndexDeleteAndWriteOnly {"IndexID":5,"TableID":104} + │ │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ │ └── UpdateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} + │ ├── Stage 2 of 7 in PostCommitPhase + │ │ ├── 2 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILL_ONLY → BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── BACKFILL_ONLY → BACKFILLED SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ └── 2 Backfill operations + │ │ ├── BackfillIndex {"IndexID":2,"SourceIndexID":1,"TableID":104} + │ │ └── BackfillIndex {"IndexID":4,"SourceIndexID":1,"TableID":104} + │ ├── Stage 3 of 7 in PostCommitPhase + │ │ ├── 2 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILLED → DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── BACKFILLED → DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ └── 4 Mutation operations + │ │ ├── MakeBackfillingIndexDeleteOnly {"IndexID":2,"TableID":104} + │ │ ├── MakeBackfillingIndexDeleteOnly {"IndexID":4,"TableID":104} + │ │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ │ └── UpdateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} + │ ├── Stage 4 of 7 in PostCommitPhase + │ │ ├── 2 elements transitioning toward PUBLIC + │ │ │ ├── DELETE_ONLY → MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── DELETE_ONLY → MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ └── 4 Mutation operations + │ │ ├── MakeBackfilledIndexMerging {"IndexID":2,"TableID":104} + │ │ ├── MakeBackfilledIndexMerging {"IndexID":4,"TableID":104} + │ │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ │ └── UpdateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} + │ ├── Stage 5 of 7 in PostCommitPhase + │ │ ├── 2 elements transitioning toward PUBLIC + │ │ │ ├── MERGE_ONLY → MERGED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── MERGE_ONLY → MERGED SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ └── 2 Backfill operations + │ │ ├── MergeIndex {"BackfilledIndexID":2,"TableID":104,"TemporaryIndexID":3} + │ │ └── MergeIndex {"BackfilledIndexID":4,"TableID":104,"TemporaryIndexID":5} + │ ├── Stage 6 of 7 in PostCommitPhase + │ │ ├── 2 elements transitioning toward PUBLIC + │ │ │ ├── MERGED → WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── MERGED → WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ └── 4 Mutation operations + │ │ ├── MakeMergedIndexWriteOnly {"IndexID":2,"TableID":104} + │ │ ├── MakeMergedIndexWriteOnly {"IndexID":4,"TableID":104} + │ │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ │ └── UpdateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} + │ └── Stage 7 of 7 in PostCommitPhase + │ ├── 2 elements transitioning toward PUBLIC + │ │ ├── WRITE_ONLY → VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ └── WRITE_ONLY → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ └── 2 Validation operations + │ ├── ValidateUniqueIndex {"IndexID":2,"TableID":104} + │ └── ValidateUniqueIndex {"IndexID":4,"TableID":104} + └── PostCommitNonRevertiblePhase + ├── Stage 1 of 3 in PostCommitNonRevertiblePhase + │ ├── 4 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1} + │ │ ├── PUBLIC → WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ └── PUBLIC → ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} + │ ├── 4 elements transitioning toward PUBLIC + │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} + │ │ ├── VALIDATED → PUBLIC SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} + │ ├── 2 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── WRITE_ONLY → TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ └── WRITE_ONLY → TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ └── 12 Mutation operations + │ ├── MakeDroppedPrimaryIndexDeleteAndWriteOnly {"IndexID":1,"TableID":104} + │ ├── SetIndexName {"IndexID":1,"Name":"crdb_internal_in...","TableID":104} + │ ├── SetIndexName {"IndexID":2,"Name":"t_pkey","TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":3,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":5,"TableID":104} + │ ├── SetIndexName {"IndexID":4,"Name":"t_i_key","TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":1,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":1,"Kind":2,"TableID":104} + │ ├── MakeAddedPrimaryIndexPublic {"IndexID":2,"TableID":104} + │ ├── MakeAddedSecondaryIndexPublic {"IndexID":4,"TableID":104} + │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"PostCommitNonRev..."} + ├── Stage 2 of 3 in PostCommitNonRevertiblePhase + │ ├── 1 element transitioning toward ABSENT + │ │ └── WRITE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ └── 3 Mutation operations + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":1,"TableID":104} + │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"PostCommitNonRev..."} + └── Stage 3 of 3 in PostCommitNonRevertiblePhase + ├── 1 element transitioning toward ABSENT + │ └── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + ├── 2 elements transitioning toward TRANSIENT_ABSENT + │ ├── TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ └── TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + └── 8 Mutation operations + ├── CreateGcJobForIndex {"IndexID":3,"TableID":104} + ├── MakeIndexAbsent {"IndexID":3,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":5,"TableID":104} + ├── MakeIndexAbsent {"IndexID":5,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":1,"TableID":104} + ├── MakeIndexAbsent {"IndexID":1,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_1_of_7 new file mode 100644 index 000000000000..780bd08c9a70 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_1_of_7 @@ -0,0 +1,43 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl) rollback at post-commit stage 1 of 7; +---- +Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); + └── PostCommitNonRevertiblePhase + └── Stage 1 of 1 in PostCommitNonRevertiblePhase + ├── 12 elements transitioning toward ABSENT + │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ ├── BACKFILL_ONLY → ABSENT SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + └── 19 Mutation operations + ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"TableID":104} + ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"Kind":2,"TableID":104} + ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"TableID":104} + ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"Kind":2,"TableID":104} + ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} + ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} + ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":4,"Kind":1,"TableID":104} + ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":5,"Kind":1,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":2,"TableID":104} + ├── MakeIndexAbsent {"IndexID":2,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":3,"TableID":104} + ├── MakeIndexAbsent {"IndexID":3,"TableID":104} + ├── LogEvent {"TargetStatus":1} + ├── CreateGcJobForIndex {"IndexID":4,"TableID":104} + ├── MakeIndexAbsent {"IndexID":4,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":5,"TableID":104} + ├── MakeIndexAbsent {"IndexID":5,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_2_of_7 new file mode 100644 index 000000000000..2ef6171625bd --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_2_of_7 @@ -0,0 +1,52 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl) rollback at post-commit stage 2 of 7; +---- +Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); + └── PostCommitNonRevertiblePhase + ├── Stage 1 of 2 in PostCommitNonRevertiblePhase + │ ├── 12 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── BACKFILL_ONLY → ABSENT SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ └── 17 Mutation operations + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"Kind":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":3,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"Kind":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":5,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":4,"Kind":1,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":5,"Kind":1,"TableID":104} + │ ├── CreateGcJobForIndex {"IndexID":2,"TableID":104} + │ ├── MakeIndexAbsent {"IndexID":2,"TableID":104} + │ ├── LogEvent {"TargetStatus":1} + │ ├── CreateGcJobForIndex {"IndexID":4,"TableID":104} + │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} + │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"PostCommitNonRev..."} + └── Stage 2 of 2 in PostCommitNonRevertiblePhase + ├── 2 elements transitioning toward ABSENT + │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + └── 6 Mutation operations + ├── CreateGcJobForIndex {"IndexID":3,"TableID":104} + ├── MakeIndexAbsent {"IndexID":3,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":5,"TableID":104} + ├── MakeIndexAbsent {"IndexID":5,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_3_of_7 new file mode 100644 index 000000000000..64e47659c0d8 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_3_of_7 @@ -0,0 +1,52 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl) rollback at post-commit stage 3 of 7; +---- +Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); + └── PostCommitNonRevertiblePhase + ├── Stage 1 of 2 in PostCommitNonRevertiblePhase + │ ├── 12 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── BACKFILL_ONLY → ABSENT SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ └── 17 Mutation operations + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"Kind":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":3,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"Kind":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":5,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":4,"Kind":1,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":5,"Kind":1,"TableID":104} + │ ├── CreateGcJobForIndex {"IndexID":2,"TableID":104} + │ ├── MakeIndexAbsent {"IndexID":2,"TableID":104} + │ ├── LogEvent {"TargetStatus":1} + │ ├── CreateGcJobForIndex {"IndexID":4,"TableID":104} + │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} + │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"PostCommitNonRev..."} + └── Stage 2 of 2 in PostCommitNonRevertiblePhase + ├── 2 elements transitioning toward ABSENT + │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + └── 6 Mutation operations + ├── CreateGcJobForIndex {"IndexID":3,"TableID":104} + ├── MakeIndexAbsent {"IndexID":3,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":5,"TableID":104} + ├── MakeIndexAbsent {"IndexID":5,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_4_of_7 new file mode 100644 index 000000000000..06a77d6f5349 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_4_of_7 @@ -0,0 +1,52 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl) rollback at post-commit stage 4 of 7; +---- +Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); + └── PostCommitNonRevertiblePhase + ├── Stage 1 of 2 in PostCommitNonRevertiblePhase + │ ├── 12 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── DELETE_ONLY → ABSENT SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ └── 17 Mutation operations + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"Kind":2,"TableID":104} + │ ├── CreateGcJobForIndex {"IndexID":2,"TableID":104} + │ ├── MakeIndexAbsent {"IndexID":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":3,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"Kind":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":5,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":4,"Kind":1,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":5,"Kind":1,"TableID":104} + │ ├── LogEvent {"TargetStatus":1} + │ ├── CreateGcJobForIndex {"IndexID":4,"TableID":104} + │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} + │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"PostCommitNonRev..."} + └── Stage 2 of 2 in PostCommitNonRevertiblePhase + ├── 2 elements transitioning toward ABSENT + │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + └── 6 Mutation operations + ├── CreateGcJobForIndex {"IndexID":3,"TableID":104} + ├── MakeIndexAbsent {"IndexID":3,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":5,"TableID":104} + ├── MakeIndexAbsent {"IndexID":5,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_5_of_7 new file mode 100644 index 000000000000..08b856d789bc --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_5_of_7 @@ -0,0 +1,56 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl) rollback at post-commit stage 5 of 7; +---- +Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); + └── PostCommitNonRevertiblePhase + ├── Stage 1 of 2 in PostCommitNonRevertiblePhase + │ ├── 12 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ ├── MERGE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── MERGE_ONLY → DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ └── 14 Mutation operations + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"Kind":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":3,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"Kind":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":5,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":5,"Kind":1,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":4,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":4,"Kind":1,"TableID":104} + │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"PostCommitNonRev..."} + └── Stage 2 of 2 in PostCommitNonRevertiblePhase + ├── 4 elements transitioning toward ABSENT + │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ ├── DELETE_ONLY → ABSENT SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + └── 11 Mutation operations + ├── CreateGcJobForIndex {"IndexID":2,"TableID":104} + ├── MakeIndexAbsent {"IndexID":2,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":3,"TableID":104} + ├── MakeIndexAbsent {"IndexID":3,"TableID":104} + ├── LogEvent {"TargetStatus":1} + ├── CreateGcJobForIndex {"IndexID":4,"TableID":104} + ├── MakeIndexAbsent {"IndexID":4,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":5,"TableID":104} + ├── MakeIndexAbsent {"IndexID":5,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_6_of_7 new file mode 100644 index 000000000000..2c77516c8b18 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_6_of_7 @@ -0,0 +1,56 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl) rollback at post-commit stage 6 of 7; +---- +Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); + └── PostCommitNonRevertiblePhase + ├── Stage 1 of 2 in PostCommitNonRevertiblePhase + │ ├── 12 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ ├── MERGE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── MERGE_ONLY → DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ └── 14 Mutation operations + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"Kind":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":3,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"Kind":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":5,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":5,"Kind":1,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":4,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":4,"Kind":1,"TableID":104} + │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"PostCommitNonRev..."} + └── Stage 2 of 2 in PostCommitNonRevertiblePhase + ├── 4 elements transitioning toward ABSENT + │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ ├── DELETE_ONLY → ABSENT SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + └── 11 Mutation operations + ├── CreateGcJobForIndex {"IndexID":2,"TableID":104} + ├── MakeIndexAbsent {"IndexID":2,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":3,"TableID":104} + ├── MakeIndexAbsent {"IndexID":3,"TableID":104} + ├── LogEvent {"TargetStatus":1} + ├── CreateGcJobForIndex {"IndexID":4,"TableID":104} + ├── MakeIndexAbsent {"IndexID":4,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":5,"TableID":104} + ├── MakeIndexAbsent {"IndexID":5,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_7_of_7 new file mode 100644 index 000000000000..3fa15fcc4692 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla.rollback_7_of_7 @@ -0,0 +1,56 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl) rollback at post-commit stage 7 of 7; +---- +Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); + └── PostCommitNonRevertiblePhase + ├── Stage 1 of 2 in PostCommitNonRevertiblePhase + │ ├── 12 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ ├── WRITE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── WRITE_ONLY → DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ └── 14 Mutation operations + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"Kind":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":3,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"Kind":2,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":4,"TableID":104} + │ ├── MakeDroppedIndexDeleteOnly {"IndexID":5,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":4,"Kind":1,"TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":5,"Kind":1,"TableID":104} + │ ├── SetJobStateOnDescriptor {"DescriptorID":104} + │ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"PostCommitNonRev..."} + └── Stage 2 of 2 in PostCommitNonRevertiblePhase + ├── 4 elements transitioning toward ABSENT + │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ ├── DELETE_ONLY → ABSENT SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + └── 11 Mutation operations + ├── CreateGcJobForIndex {"IndexID":2,"TableID":104} + ├── MakeIndexAbsent {"IndexID":2,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":3,"TableID":104} + ├── MakeIndexAbsent {"IndexID":3,"TableID":104} + ├── LogEvent {"TargetStatus":1} + ├── CreateGcJobForIndex {"IndexID":4,"TableID":104} + ├── MakeIndexAbsent {"IndexID":4,"TableID":104} + ├── CreateGcJobForIndex {"IndexID":5,"TableID":104} + ├── MakeIndexAbsent {"IndexID":5,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla new file mode 100644 index 000000000000..ad560924225e --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla @@ -0,0 +1,612 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +---- +• Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); +│ +├── • StatementPhase +│ │ +│ └── • Stage 1 of 1 in StatementPhase +│ │ +│ ├── • 10 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index-column added to index after index exists" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index-column added to index after index exists" +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ ABSENT → BACKFILL_ONLY +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index exists before columns, partitioning, and partial" +│ │ │ rule: "index-column added to index after temp index exists" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index exists before columns, partitioning, and partial" +│ │ │ rule: "index-column added to index after temp index exists" +│ │ │ +│ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ ABSENT → BACKFILL_ONLY +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ rule: "index-column added to index after index exists" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} +│ │ │ rule: "temp index exists before columns, partitioning, and partial" +│ │ │ rule: "index-column added to index after temp index exists" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ rule: "index-column added to index after index exists" +│ │ │ +│ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} +│ │ rule: "temp index exists before columns, partitioning, and partial" +│ │ rule: "index-column added to index after temp index exists" +│ │ +│ ├── • 2 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} +│ │ │ ABSENT → DELETE_ONLY +│ │ │ +│ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} +│ │ ABSENT → DELETE_ONLY +│ │ +│ └── • 12 Mutation operations +│ │ +│ ├── • MakeAddedIndexBackfilling +│ │ Index: +│ │ ConstraintID: 1 +│ │ IndexID: 2 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ TemporaryIndexID: 3 +│ │ +│ ├── • MakeAddedTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 1 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 3 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ Kind: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAddedIndexBackfilling +│ │ Index: +│ │ IndexID: 4 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ TemporaryIndexID: 5 +│ │ IsSecondaryIndex: true +│ │ +│ ├── • MakeAddedTempIndexDeleteOnly +│ │ Index: +│ │ IndexID: 5 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ IsSecondaryIndex: true +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 4 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 5 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 4 +│ │ Kind: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 5 +│ │ Kind: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ └── • AddColumnToIndex +│ ColumnID: 1 +│ IndexID: 2 +│ Kind: 2 +│ TableID: 104 +│ +├── • PreCommitPhase +│ │ +│ └── • Stage 1 of 1 in PreCommitPhase +│ │ +│ └── • 2 Mutation operations +│ │ +│ ├── • SetJobStateOnDescriptor +│ │ DescriptorID: 104 +│ │ Initialize: true +│ │ +│ └── • CreateSchemaChangerJob +│ Authorization: +│ UserName: root +│ DescriptorIDs: +│ - 104 +│ JobID: 1 +│ RunningStatus: PostCommitPhase stage 1 of 7 with 2 MutationType ops pending +│ Statements: +│ - statement: ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j) +│ redactedstatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PRIMARY KEY USING +│ COLUMNS (‹j›) +│ statementtag: ALTER TABLE +│ +├── • PostCommitPhase +│ │ +│ ├── • Stage 1 of 7 in PostCommitPhase +│ │ │ +│ │ ├── • 2 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} +│ │ │ │ │ DELETE_ONLY → WRITE_ONLY +│ │ │ │ │ +│ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} +│ │ │ │ │ rule: "index-column added to index before temp index receives writes" +│ │ │ │ │ +│ │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ rule: "index-column added to index before temp index receives writes" +│ │ │ │ +│ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → WRITE_ONLY +│ │ │ │ +│ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} +│ │ │ │ rule: "index-column added to index before temp index receives writes" +│ │ │ │ +│ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} +│ │ │ rule: "index-column added to index before temp index receives writes" +│ │ │ +│ │ └── • 4 Mutation operations +│ │ │ +│ │ ├── • MakeAddedIndexDeleteAndWriteOnly +│ │ │ IndexID: 3 +│ │ │ TableID: 104 +│ │ │ +│ │ ├── • MakeAddedIndexDeleteAndWriteOnly +│ │ │ IndexID: 5 +│ │ │ TableID: 104 +│ │ │ +│ │ ├── • SetJobStateOnDescriptor +│ │ │ DescriptorID: 104 +│ │ │ +│ │ └── • UpdateSchemaChangerJob +│ │ JobID: 1 +│ │ RunningStatus: PostCommitPhase stage 2 of 7 with 2 BackfillType ops pending +│ │ +│ ├── • Stage 2 of 7 in PostCommitPhase +│ │ │ +│ │ ├── • 2 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ │ BACKFILL_ONLY → BACKFILLED +│ │ │ │ │ +│ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ │ rule: "index-column added to index before index is backfilled" +│ │ │ │ │ +│ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ │ rule: "index-column added to index before index is backfilled" +│ │ │ │ │ +│ │ │ │ └── • Precedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} +│ │ │ │ rule: "temp index is WRITE_ONLY before backfill" +│ │ │ │ +│ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → BACKFILLED +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} +│ │ │ │ rule: "temp index is WRITE_ONLY before backfill" +│ │ │ │ +│ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ rule: "index-column added to index before index is backfilled" +│ │ │ │ +│ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} +│ │ │ rule: "index-column added to index before index is backfilled" +│ │ │ +│ │ └── • 2 Backfill operations +│ │ │ +│ │ ├── • BackfillIndex +│ │ │ IndexID: 2 +│ │ │ SourceIndexID: 1 +│ │ │ TableID: 104 +│ │ │ +│ │ └── • BackfillIndex +│ │ IndexID: 4 +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ +│ ├── • Stage 3 of 7 in PostCommitPhase +│ │ │ +│ │ ├── • 2 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ BACKFILLED → DELETE_ONLY +│ │ │ │ +│ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ BACKFILLED → DELETE_ONLY +│ │ │ +│ │ └── • 4 Mutation operations +│ │ │ +│ │ ├── • MakeBackfillingIndexDeleteOnly +│ │ │ IndexID: 2 +│ │ │ TableID: 104 +│ │ │ +│ │ ├── • MakeBackfillingIndexDeleteOnly +│ │ │ IndexID: 4 +│ │ │ TableID: 104 +│ │ │ +│ │ ├── • SetJobStateOnDescriptor +│ │ │ DescriptorID: 104 +│ │ │ +│ │ └── • UpdateSchemaChangerJob +│ │ JobID: 1 +│ │ RunningStatus: PostCommitPhase stage 4 of 7 with 2 MutationType ops pending +│ │ +│ ├── • Stage 4 of 7 in PostCommitPhase +│ │ │ +│ │ ├── • 2 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → MERGE_ONLY +│ │ │ │ +│ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ DELETE_ONLY → MERGE_ONLY +│ │ │ +│ │ └── • 4 Mutation operations +│ │ │ +│ │ ├── • MakeBackfilledIndexMerging +│ │ │ IndexID: 2 +│ │ │ TableID: 104 +│ │ │ +│ │ ├── • MakeBackfilledIndexMerging +│ │ │ IndexID: 4 +│ │ │ TableID: 104 +│ │ │ +│ │ ├── • SetJobStateOnDescriptor +│ │ │ DescriptorID: 104 +│ │ │ +│ │ └── • UpdateSchemaChangerJob +│ │ JobID: 1 +│ │ RunningStatus: PostCommitPhase stage 5 of 7 with 2 BackfillType ops pending +│ │ +│ ├── • Stage 5 of 7 in PostCommitPhase +│ │ │ +│ │ ├── • 2 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ MERGE_ONLY → MERGED +│ │ │ │ +│ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ MERGE_ONLY → MERGED +│ │ │ +│ │ └── • 2 Backfill operations +│ │ │ +│ │ ├── • MergeIndex +│ │ │ BackfilledIndexID: 2 +│ │ │ TableID: 104 +│ │ │ TemporaryIndexID: 3 +│ │ │ +│ │ └── • MergeIndex +│ │ BackfilledIndexID: 4 +│ │ TableID: 104 +│ │ TemporaryIndexID: 5 +│ │ +│ ├── • Stage 6 of 7 in PostCommitPhase +│ │ │ +│ │ ├── • 2 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ MERGED → WRITE_ONLY +│ │ │ │ +│ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ MERGED → WRITE_ONLY +│ │ │ +│ │ └── • 4 Mutation operations +│ │ │ +│ │ ├── • MakeMergedIndexWriteOnly +│ │ │ IndexID: 2 +│ │ │ TableID: 104 +│ │ │ +│ │ ├── • MakeMergedIndexWriteOnly +│ │ │ IndexID: 4 +│ │ │ TableID: 104 +│ │ │ +│ │ ├── • SetJobStateOnDescriptor +│ │ │ DescriptorID: 104 +│ │ │ +│ │ └── • UpdateSchemaChangerJob +│ │ JobID: 1 +│ │ RunningStatus: PostCommitPhase stage 7 of 7 with 2 ValidationType ops pending +│ │ +│ └── • Stage 7 of 7 in PostCommitPhase +│ │ +│ ├── • 2 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ WRITE_ONLY → VALIDATED +│ │ │ +│ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ WRITE_ONLY → VALIDATED +│ │ +│ └── • 2 Validation operations +│ │ +│ ├── • ValidateUniqueIndex +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ └── • ValidateUniqueIndex +│ IndexID: 4 +│ TableID: 104 +│ +└── • PostCommitNonRevertiblePhase + │ + ├── • Stage 1 of 3 in PostCommitNonRevertiblePhase + │ │ + │ ├── • 4 elements transitioning toward ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ rule: "index no longer public before dependents removed" + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ rule: "index no longer public before dependents removed" + │ │ │ + │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ PUBLIC → WRITE_ONLY + │ │ │ + │ │ └── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ └── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ rule: "index no longer public before dependents removed" + │ │ + │ ├── • 4 elements transitioning toward PUBLIC + │ │ │ + │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ │ VALIDATED → PUBLIC + │ │ │ │ + │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ │ rule: "primary index swap" + │ │ │ │ + │ │ │ └── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} + │ │ │ rule: "index named right before index becomes public" + │ │ │ + │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} + │ │ │ │ ABSENT → PUBLIC + │ │ │ │ + │ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ rule: "index existence precedes index name and comment" + │ │ │ + │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ │ VALIDATED → PUBLIC + │ │ │ │ + │ │ │ └── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} + │ │ │ rule: "index named right before index becomes public" + │ │ │ + │ │ └── • IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} + │ │ │ ABSENT → PUBLIC + │ │ │ + │ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ rule: "index existence precedes index name and comment" + │ │ + │ ├── • 2 elements transitioning toward TRANSIENT_ABSENT + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY + │ │ │ + │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY + │ │ + │ └── • 12 Mutation operations + │ │ + │ ├── • MakeDroppedPrimaryIndexDeleteAndWriteOnly + │ │ IndexID: 1 + │ │ TableID: 104 + │ │ + │ ├── • SetIndexName + │ │ IndexID: 1 + │ │ Name: crdb_internal_index_1_name_placeholder + │ │ TableID: 104 + │ │ + │ ├── • SetIndexName + │ │ IndexID: 2 + │ │ Name: t_pkey + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • SetIndexName + │ │ IndexID: 4 + │ │ Name: t_i_key + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 1 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 1 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeAddedPrimaryIndexPublic + │ │ EventBase: + │ │ Authorization: + │ │ UserName: root + │ │ Statement: ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PRIMARY KEY USING COLUMNS + │ │ (‹j›) + │ │ StatementTag: ALTER TABLE + │ │ TargetMetadata: + │ │ SourceElementID: 1 + │ │ SubWorkID: 1 + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeAddedSecondaryIndexPublic + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • SetJobStateOnDescriptor + │ │ DescriptorID: 104 + │ │ + │ └── • UpdateSchemaChangerJob + │ IsNonCancelable: true + │ JobID: 1 + │ RunningStatus: PostCommitNonRevertiblePhase stage 2 of 3 with 1 MutationType op pending + │ + ├── • Stage 2 of 3 in PostCommitNonRevertiblePhase + │ │ + │ ├── • 1 element transitioning toward ABSENT + │ │ │ + │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ WRITE_ONLY → DELETE_ONLY + │ │ + │ └── • 3 Mutation operations + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 1 + │ │ TableID: 104 + │ │ + │ ├── • SetJobStateOnDescriptor + │ │ DescriptorID: 104 + │ │ + │ └── • UpdateSchemaChangerJob + │ IsNonCancelable: true + │ JobID: 1 + │ RunningStatus: PostCommitNonRevertiblePhase stage 3 of 3 with 6 MutationType ops pending + │ + └── • Stage 3 of 3 in PostCommitNonRevertiblePhase + │ + ├── • 1 element transitioning toward ABSENT + │ │ + │ └── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ DELETE_ONLY → ABSENT + │ │ + │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • SameStagePrecedence dependency from TRANSIENT_ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ rule: "temp indexes reach absent at the same time as other indexes" + │ │ + │ └── • SameStagePrecedence dependency from TRANSIENT_ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ rule: "temp indexes reach absent at the same time as other indexes" + │ + ├── • 2 elements transitioning toward TRANSIENT_ABSENT + │ │ + │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT + │ │ + │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT + │ + └── • 8 Mutation operations + │ + ├── • CreateGcJobForIndex + │ IndexID: 3 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 3 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 5 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 5 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 1 + │ StatementForDropJob: + │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ EventBase: + │ Authorization: + │ UserName: root + │ Statement: ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PRIMARY KEY USING COLUMNS + │ (‹j›) + │ StatementTag: ALTER TABLE + │ TargetMetadata: + │ SourceElementID: 1 + │ SubWorkID: 1 + │ IndexID: 1 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_1_of_7 new file mode 100644 index 000000000000..1deaaa599d41 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_1_of_7 @@ -0,0 +1,207 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; +---- +• Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); +│ +└── • PostCommitNonRevertiblePhase + │ + └── • Stage 1 of 1 in PostCommitNonRevertiblePhase + │ + ├── • 12 elements transitioning toward ABSENT + │ │ + │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ PUBLIC → ABSENT + │ │ + │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ PUBLIC → ABSENT + │ │ + │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ BACKFILL_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ PUBLIC → ABSENT + │ │ + │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ PUBLIC → ABSENT + │ │ + │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ BACKFILL_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ + │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ PUBLIC → ABSENT + │ │ + │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ + │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ PUBLIC → ABSENT + │ + └── • 19 Mutation operations + │ + ├── • RemoveColumnFromIndex + │ ColumnID: 2 + │ IndexID: 2 + │ TableID: 104 + │ + ├── • RemoveColumnFromIndex + │ ColumnID: 1 + │ IndexID: 2 + │ Kind: 2 + │ TableID: 104 + │ + ├── • RemoveColumnFromIndex + │ ColumnID: 2 + │ IndexID: 3 + │ TableID: 104 + │ + ├── • RemoveColumnFromIndex + │ ColumnID: 1 + │ IndexID: 3 + │ Kind: 2 + │ TableID: 104 + │ + ├── • RemoveColumnFromIndex + │ ColumnID: 1 + │ IndexID: 4 + │ TableID: 104 + │ + ├── • RemoveColumnFromIndex + │ ColumnID: 1 + │ IndexID: 5 + │ TableID: 104 + │ + ├── • RemoveColumnFromIndex + │ ColumnID: 2 + │ IndexID: 4 + │ Kind: 1 + │ TableID: 104 + │ + ├── • RemoveColumnFromIndex + │ ColumnID: 2 + │ IndexID: 5 + │ Kind: 1 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 2 + │ StatementForDropJob: + │ Rollback: true + │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ EventBase: + │ Authorization: + │ UserName: root + │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ StatementTag: ALTER TABLE + │ TargetMetadata: + │ SourceElementID: 1 + │ SubWorkID: 1 + │ IndexID: 2 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 3 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 3 + │ TableID: 104 + │ + ├── • LogEvent + │ Element: + │ SecondaryIndex: + │ indexId: 4 + │ isUnique: true + │ sourceIndexId: 1 + │ tableId: 104 + │ temporaryIndexId: 5 + │ EventBase: + │ Authorization: + │ UserName: root + │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ StatementTag: ALTER TABLE + │ TargetMetadata: + │ SourceElementID: 1 + │ SubWorkID: 1 + │ TargetStatus: 1 + │ + ├── • CreateGcJobForIndex + │ IndexID: 4 + │ StatementForDropJob: + │ Rollback: true + │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 4 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 5 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 5 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_2_of_7 new file mode 100644 index 000000000000..faeb36dd5096 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_2_of_7 @@ -0,0 +1,235 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; +---- +• Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); +│ +└── • PostCommitNonRevertiblePhase + │ + ├── • Stage 1 of 2 in PostCommitNonRevertiblePhase + │ │ + │ ├── • 12 elements transitioning toward ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ │ BACKFILL_ONLY → ABSENT + │ │ │ │ + │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ │ rule: "dependents removed before index" + │ │ │ │ + │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ │ rule: "dependents removed before index" + │ │ │ │ + │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ │ BACKFILL_ONLY → ABSENT + │ │ │ │ + │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ │ rule: "dependents removed before index" + │ │ │ │ + │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ │ rule: "dependents removed before index" + │ │ │ │ + │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ │ + │ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ │ PUBLIC → ABSENT + │ │ + │ └── • 17 Mutation operations + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 2 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 3 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 4 + │ │ Kind: 1 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 5 + │ │ Kind: 1 + │ │ TableID: 104 + │ │ + │ ├── • CreateGcJobForIndex + │ │ IndexID: 2 + │ │ StatementForDropJob: + │ │ Rollback: true + │ │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ │ TableID: 104 + │ │ + │ ├── • MakeIndexAbsent + │ │ EventBase: + │ │ Authorization: + │ │ UserName: root + │ │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ │ StatementTag: ALTER TABLE + │ │ TargetMetadata: + │ │ SourceElementID: 1 + │ │ SubWorkID: 1 + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • LogEvent + │ │ Element: + │ │ SecondaryIndex: + │ │ indexId: 4 + │ │ isUnique: true + │ │ sourceIndexId: 1 + │ │ tableId: 104 + │ │ temporaryIndexId: 5 + │ │ EventBase: + │ │ Authorization: + │ │ UserName: root + │ │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ │ StatementTag: ALTER TABLE + │ │ TargetMetadata: + │ │ SourceElementID: 1 + │ │ SubWorkID: 1 + │ │ TargetStatus: 1 + │ │ + │ ├── • CreateGcJobForIndex + │ │ IndexID: 4 + │ │ StatementForDropJob: + │ │ Rollback: true + │ │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ │ TableID: 104 + │ │ + │ ├── • MakeIndexAbsent + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • SetJobStateOnDescriptor + │ │ DescriptorID: 104 + │ │ + │ └── • UpdateSchemaChangerJob + │ IsNonCancelable: true + │ JobID: 1 + │ RunningStatus: PostCommitNonRevertiblePhase stage 2 of 2 with 4 MutationType ops pending + │ + └── • Stage 2 of 2 in PostCommitNonRevertiblePhase + │ + ├── • 2 elements transitioning toward ABSENT + │ │ + │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ rule: "dependents removed before index" + │ │ + │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ DELETE_ONLY → ABSENT + │ │ + │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ rule: "dependents removed before index" + │ │ + │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ rule: "dependents removed before index" + │ + └── • 6 Mutation operations + │ + ├── • CreateGcJobForIndex + │ IndexID: 3 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 3 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 5 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 5 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_3_of_7 new file mode 100644 index 000000000000..c6c8fb5a1fb5 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_3_of_7 @@ -0,0 +1,235 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; +---- +• Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); +│ +└── • PostCommitNonRevertiblePhase + │ + ├── • Stage 1 of 2 in PostCommitNonRevertiblePhase + │ │ + │ ├── • 12 elements transitioning toward ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ │ BACKFILL_ONLY → ABSENT + │ │ │ │ + │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ │ rule: "dependents removed before index" + │ │ │ │ + │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ │ rule: "dependents removed before index" + │ │ │ │ + │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ │ BACKFILL_ONLY → ABSENT + │ │ │ │ + │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ │ rule: "dependents removed before index" + │ │ │ │ + │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ │ rule: "dependents removed before index" + │ │ │ │ + │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ │ + │ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ │ PUBLIC → ABSENT + │ │ + │ └── • 17 Mutation operations + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 2 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 3 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 4 + │ │ Kind: 1 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 5 + │ │ Kind: 1 + │ │ TableID: 104 + │ │ + │ ├── • CreateGcJobForIndex + │ │ IndexID: 2 + │ │ StatementForDropJob: + │ │ Rollback: true + │ │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ │ TableID: 104 + │ │ + │ ├── • MakeIndexAbsent + │ │ EventBase: + │ │ Authorization: + │ │ UserName: root + │ │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ │ StatementTag: ALTER TABLE + │ │ TargetMetadata: + │ │ SourceElementID: 1 + │ │ SubWorkID: 1 + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • LogEvent + │ │ Element: + │ │ SecondaryIndex: + │ │ indexId: 4 + │ │ isUnique: true + │ │ sourceIndexId: 1 + │ │ tableId: 104 + │ │ temporaryIndexId: 5 + │ │ EventBase: + │ │ Authorization: + │ │ UserName: root + │ │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ │ StatementTag: ALTER TABLE + │ │ TargetMetadata: + │ │ SourceElementID: 1 + │ │ SubWorkID: 1 + │ │ TargetStatus: 1 + │ │ + │ ├── • CreateGcJobForIndex + │ │ IndexID: 4 + │ │ StatementForDropJob: + │ │ Rollback: true + │ │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ │ TableID: 104 + │ │ + │ ├── • MakeIndexAbsent + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • SetJobStateOnDescriptor + │ │ DescriptorID: 104 + │ │ + │ └── • UpdateSchemaChangerJob + │ IsNonCancelable: true + │ JobID: 1 + │ RunningStatus: PostCommitNonRevertiblePhase stage 2 of 2 with 4 MutationType ops pending + │ + └── • Stage 2 of 2 in PostCommitNonRevertiblePhase + │ + ├── • 2 elements transitioning toward ABSENT + │ │ + │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ rule: "dependents removed before index" + │ │ + │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ DELETE_ONLY → ABSENT + │ │ + │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ rule: "dependents removed before index" + │ │ + │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ rule: "dependents removed before index" + │ + └── • 6 Mutation operations + │ + ├── • CreateGcJobForIndex + │ IndexID: 3 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 3 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 5 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 5 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_4_of_7 new file mode 100644 index 000000000000..e660120de7a7 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_4_of_7 @@ -0,0 +1,235 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; +---- +• Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); +│ +└── • PostCommitNonRevertiblePhase + │ + ├── • Stage 1 of 2 in PostCommitNonRevertiblePhase + │ │ + │ ├── • 12 elements transitioning toward ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ │ DELETE_ONLY → ABSENT + │ │ │ │ + │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ │ rule: "dependents removed before index" + │ │ │ │ + │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ │ rule: "dependents removed before index" + │ │ │ │ + │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ │ DELETE_ONLY → ABSENT + │ │ │ │ + │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ │ rule: "dependents removed before index" + │ │ │ │ + │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ │ rule: "dependents removed before index" + │ │ │ │ + │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ │ + │ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ │ PUBLIC → ABSENT + │ │ + │ └── • 17 Mutation operations + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 2 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • CreateGcJobForIndex + │ │ IndexID: 2 + │ │ StatementForDropJob: + │ │ Rollback: true + │ │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ │ TableID: 104 + │ │ + │ ├── • MakeIndexAbsent + │ │ EventBase: + │ │ Authorization: + │ │ UserName: root + │ │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ │ StatementTag: ALTER TABLE + │ │ TargetMetadata: + │ │ SourceElementID: 1 + │ │ SubWorkID: 1 + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 3 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 4 + │ │ Kind: 1 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 5 + │ │ Kind: 1 + │ │ TableID: 104 + │ │ + │ ├── • LogEvent + │ │ Element: + │ │ SecondaryIndex: + │ │ indexId: 4 + │ │ isUnique: true + │ │ sourceIndexId: 1 + │ │ tableId: 104 + │ │ temporaryIndexId: 5 + │ │ EventBase: + │ │ Authorization: + │ │ UserName: root + │ │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ │ StatementTag: ALTER TABLE + │ │ TargetMetadata: + │ │ SourceElementID: 1 + │ │ SubWorkID: 1 + │ │ TargetStatus: 1 + │ │ + │ ├── • CreateGcJobForIndex + │ │ IndexID: 4 + │ │ StatementForDropJob: + │ │ Rollback: true + │ │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ │ TableID: 104 + │ │ + │ ├── • MakeIndexAbsent + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • SetJobStateOnDescriptor + │ │ DescriptorID: 104 + │ │ + │ └── • UpdateSchemaChangerJob + │ IsNonCancelable: true + │ JobID: 1 + │ RunningStatus: PostCommitNonRevertiblePhase stage 2 of 2 with 4 MutationType ops pending + │ + └── • Stage 2 of 2 in PostCommitNonRevertiblePhase + │ + ├── • 2 elements transitioning toward ABSENT + │ │ + │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ rule: "dependents removed before index" + │ │ + │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ DELETE_ONLY → ABSENT + │ │ + │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ rule: "dependents removed before index" + │ │ + │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ rule: "dependents removed before index" + │ + └── • 6 Mutation operations + │ + ├── • CreateGcJobForIndex + │ IndexID: 3 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 3 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 5 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 5 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_5_of_7 new file mode 100644 index 000000000000..f63ebfca19f3 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_5_of_7 @@ -0,0 +1,249 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; +---- +• Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); +│ +└── • PostCommitNonRevertiblePhase + │ + ├── • Stage 1 of 2 in PostCommitNonRevertiblePhase + │ │ + │ ├── • 12 elements transitioning toward ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ MERGE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ MERGE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ │ + │ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ │ PUBLIC → ABSENT + │ │ + │ └── • 14 Mutation operations + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 2 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 3 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 5 + │ │ Kind: 1 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 4 + │ │ Kind: 1 + │ │ TableID: 104 + │ │ + │ ├── • SetJobStateOnDescriptor + │ │ DescriptorID: 104 + │ │ + │ └── • UpdateSchemaChangerJob + │ IsNonCancelable: true + │ JobID: 1 + │ RunningStatus: PostCommitNonRevertiblePhase stage 2 of 2 with 9 MutationType ops pending + │ + └── • Stage 2 of 2 in PostCommitNonRevertiblePhase + │ + ├── • 4 elements transitioning toward ABSENT + │ │ + │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} + │ │ rule: "dependents removed before index" + │ │ + │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ DELETE_ONLY → ABSENT + │ │ + │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ rule: "dependents removed before index" + │ │ + │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ rule: "dependents removed before index" + │ + └── • 11 Mutation operations + │ + ├── • CreateGcJobForIndex + │ IndexID: 2 + │ StatementForDropJob: + │ Rollback: true + │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ EventBase: + │ Authorization: + │ UserName: root + │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ StatementTag: ALTER TABLE + │ TargetMetadata: + │ SourceElementID: 1 + │ SubWorkID: 1 + │ IndexID: 2 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 3 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 3 + │ TableID: 104 + │ + ├── • LogEvent + │ Element: + │ SecondaryIndex: + │ indexId: 4 + │ isUnique: true + │ sourceIndexId: 1 + │ tableId: 104 + │ temporaryIndexId: 5 + │ EventBase: + │ Authorization: + │ UserName: root + │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ StatementTag: ALTER TABLE + │ TargetMetadata: + │ SourceElementID: 1 + │ SubWorkID: 1 + │ TargetStatus: 1 + │ + ├── • CreateGcJobForIndex + │ IndexID: 4 + │ StatementForDropJob: + │ Rollback: true + │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 4 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 5 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 5 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_6_of_7 new file mode 100644 index 000000000000..9d9a49eaa8a8 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_6_of_7 @@ -0,0 +1,249 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; +---- +• Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); +│ +└── • PostCommitNonRevertiblePhase + │ + ├── • Stage 1 of 2 in PostCommitNonRevertiblePhase + │ │ + │ ├── • 12 elements transitioning toward ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ MERGE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ MERGE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ │ + │ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ │ PUBLIC → ABSENT + │ │ + │ └── • 14 Mutation operations + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 2 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 3 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 5 + │ │ Kind: 1 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 4 + │ │ Kind: 1 + │ │ TableID: 104 + │ │ + │ ├── • SetJobStateOnDescriptor + │ │ DescriptorID: 104 + │ │ + │ └── • UpdateSchemaChangerJob + │ IsNonCancelable: true + │ JobID: 1 + │ RunningStatus: PostCommitNonRevertiblePhase stage 2 of 2 with 9 MutationType ops pending + │ + └── • Stage 2 of 2 in PostCommitNonRevertiblePhase + │ + ├── • 4 elements transitioning toward ABSENT + │ │ + │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} + │ │ rule: "dependents removed before index" + │ │ + │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ DELETE_ONLY → ABSENT + │ │ + │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ rule: "dependents removed before index" + │ │ + │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ rule: "dependents removed before index" + │ + └── • 11 Mutation operations + │ + ├── • CreateGcJobForIndex + │ IndexID: 2 + │ StatementForDropJob: + │ Rollback: true + │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ EventBase: + │ Authorization: + │ UserName: root + │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ StatementTag: ALTER TABLE + │ TargetMetadata: + │ SourceElementID: 1 + │ SubWorkID: 1 + │ IndexID: 2 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 3 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 3 + │ TableID: 104 + │ + ├── • LogEvent + │ Element: + │ SecondaryIndex: + │ indexId: 4 + │ isUnique: true + │ sourceIndexId: 1 + │ tableId: 104 + │ temporaryIndexId: 5 + │ EventBase: + │ Authorization: + │ UserName: root + │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ StatementTag: ALTER TABLE + │ TargetMetadata: + │ SourceElementID: 1 + │ SubWorkID: 1 + │ TargetStatus: 1 + │ + ├── • CreateGcJobForIndex + │ IndexID: 4 + │ StatementForDropJob: + │ Rollback: true + │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 4 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 5 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 5 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_7_of_7 new file mode 100644 index 000000000000..f4aa0439a937 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_7_of_7 @@ -0,0 +1,249 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL); + +/* test */ +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); +EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; +---- +• Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›); +│ +└── • PostCommitNonRevertiblePhase + │ + ├── • Stage 1 of 2 in PostCommitNonRevertiblePhase + │ │ + │ ├── • 12 elements transitioning toward ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ │ WRITE_ONLY → DELETE_ONLY + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ │ PUBLIC → ABSENT + │ │ │ + │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ │ PUBLIC → ABSENT + │ │ │ │ + │ │ │ └── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ rule: "secondary index in DELETE_ONLY before removing columns" + │ │ │ + │ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ │ PUBLIC → ABSENT + │ │ + │ └── • 14 Mutation operations + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 2 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 3 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 3 + │ │ Kind: 2 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • MakeDroppedIndexDeleteOnly + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 4 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 1 + │ │ IndexID: 5 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 4 + │ │ Kind: 1 + │ │ TableID: 104 + │ │ + │ ├── • RemoveColumnFromIndex + │ │ ColumnID: 2 + │ │ IndexID: 5 + │ │ Kind: 1 + │ │ TableID: 104 + │ │ + │ ├── • SetJobStateOnDescriptor + │ │ DescriptorID: 104 + │ │ + │ └── • UpdateSchemaChangerJob + │ IsNonCancelable: true + │ JobID: 1 + │ RunningStatus: PostCommitNonRevertiblePhase stage 2 of 2 with 9 MutationType ops pending + │ + └── • Stage 2 of 2 in PostCommitNonRevertiblePhase + │ + ├── • 4 elements transitioning toward ABSENT + │ │ + │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 1, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ rule: "dependents removed before index" + │ │ + │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ DELETE_ONLY → ABSENT + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ rule: "secondary index columns removed before removing the index" + │ │ │ rule: "dependents removed before index" + │ │ │ + │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} + │ │ rule: "dependents removed before index" + │ │ + │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, SourceIndexID: 1} + │ │ DELETE_ONLY → ABSENT + │ │ + │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ rule: "dependents removed before index" + │ │ + │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ rule: "dependents removed before index" + │ + └── • 11 Mutation operations + │ + ├── • CreateGcJobForIndex + │ IndexID: 2 + │ StatementForDropJob: + │ Rollback: true + │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ EventBase: + │ Authorization: + │ UserName: root + │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ StatementTag: ALTER TABLE + │ TargetMetadata: + │ SourceElementID: 1 + │ SubWorkID: 1 + │ IndexID: 2 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 3 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 3 + │ TableID: 104 + │ + ├── • LogEvent + │ Element: + │ SecondaryIndex: + │ indexId: 4 + │ isUnique: true + │ sourceIndexId: 1 + │ tableId: 104 + │ temporaryIndexId: 5 + │ EventBase: + │ Authorization: + │ UserName: root + │ Statement: ALTER TABLE ‹defaultdb›.public.‹t› ALTER PRIMARY KEY USING COLUMNS (‹j›) + │ StatementTag: ALTER TABLE + │ TargetMetadata: + │ SourceElementID: 1 + │ SubWorkID: 1 + │ TargetStatus: 1 + │ + ├── • CreateGcJobForIndex + │ IndexID: 4 + │ StatementForDropJob: + │ Rollback: true + │ Statement: ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j) + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 4 + │ TableID: 104 + │ + ├── • CreateGcJobForIndex + │ IndexID: 5 + │ TableID: 104 + │ + ├── • MakeIndexAbsent + │ IndexID: 5 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed From 8a253326c737addf26bb3d13dce0a1d895c5a606 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Fri, 29 Jul 2022 11:15:29 -0400 Subject: [PATCH 05/11] turn ALTER PRIMARY KEY on A few tests failed because they test/reply on specific behavior in the old schema changer, so we disabled them in this pr. --- pkg/sql/schema_changer_test.go | 47 ++++++++----------- .../internal/scbuildstmt/alter_table.go | 2 +- 2 files changed, 20 insertions(+), 29 deletions(-) diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 558d4908da4a..9606e5c7ca17 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -2645,7 +2645,10 @@ CREATE TABLE t.test (k INT NOT NULL, v INT, v2 INT NOT NULL)`); err != nil { wg.Add(1) // Alter the primary key of the table. go func() { - if _, err := sqlDB.Exec(`ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (v2)`); err != nil { + if _, err := sqlDB.Exec(` +SET use_declarative_schema_changer = off; +ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (v2); +SET use_declarative_schema_changer = on;`); err != nil { t.Error(err) } wg.Done() @@ -2654,7 +2657,10 @@ CREATE TABLE t.test (k INT NOT NULL, v INT, v2 INT NOT NULL)`); err != nil { <-backfillNotif // This must be rejected, because there is a primary key change already in progress. - _, err := sqlDB.Exec(`ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (k)`) + _, err := sqlDB.Exec(` +SET use_declarative_schema_changer = off; +ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (k); +SET use_declarative_schema_changer = on;`) if !testutils.IsError(err, "pq: unimplemented: table test is currently undergoing a schema change") { t.Errorf("expected to concurrent primary key change to error, but got %+v", err) } @@ -2718,7 +2724,8 @@ CREATE TABLE t.test (k INT NOT NULL, v INT); go func() { if _, err := sqlDB.Exec(` SET use_declarative_schema_changer = off; -ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (k)`); err != nil { +ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (k); +SET use_declarative_schema_changer = on;`); err != nil { t.Error(err) } wg.Done() @@ -2729,7 +2736,8 @@ ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (k)`); err != nil { // Test that trying different schema changes results an error. _, err := sqlDB.Exec(` SET use_declarative_schema_changer = off; -ALTER TABLE t.test ADD COLUMN z INT`) +ALTER TABLE t.test ADD COLUMN z INT; +SET use_declarative_schema_changer = on;`) expected := fmt.Sprintf(`pq: relation "test" \(%d\): unimplemented: cannot perform a schema change operation while a primary key change is in progress`, tableID) if !testutils.IsError(err, expected) { t.Fatalf("expected to find error %s but found %+v", expected, err) @@ -3277,7 +3285,7 @@ func TestPrimaryKeyChangeWithCancel(t *testing.T) { if _, err := db.Exec(`CANCEL JOB ( SELECT job_id FROM [SHOW JOBS] WHERE - job_type = 'SCHEMA CHANGE' AND + job_type = 'NEW SCHEMA CHANGE' AND status = $1 AND description NOT LIKE 'ROLL BACK%' )`, jobs.StatusRunning); err != nil { @@ -3372,7 +3380,8 @@ CREATE TABLE t.test (k INT NOT NULL, v INT); `) require.NoError(t, err) - _, err = sqlDB.Exec(`ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (k)`) + _, err = sqlDB.Exec(`SET use_declarative_schema_changer = off; +ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (k)`) require.NoError(t, err) // Wait until the testing knob has notified that canceling the job has been @@ -6202,16 +6211,6 @@ ALTER TABLE t.public.test DROP COLUMN v;`) require.Equal(t, [][]string{ {"1", "2"}, }, rows) - - // Validate the job cancellation metrics. - rows = runner.QueryStr(t, "SELECT * FROM crdb_internal.feature_usage WHERE feature_name LIKE 'job.%.canceled'") - if len(rows) != 1 || - len(rows[0]) != 2 || - rows[0][0] != "job.schema_change.canceled" { - require.Failf(t, "Unexpected result set", "Rows: %s", rows) - } else if val, err := strconv.ParseInt(rows[0][1], 10, 32); err != nil || val < 0 { - require.Failf(t, "Invalid integer or value", "Error: %s Val: %d", err, val) - } } // TestRetriableErrorDuringRollback tests that a retriable error while rolling @@ -6651,16 +6650,6 @@ SELECT job_id FROM crdb_internal.jobs withJobsToFail(func(m map[jobspb.JobID]struct{}) { require.Len(t, m, 0) }) - - // Validate the job cancellation metrics. - rows := tdb.QueryStr(t, "SELECT * FROM crdb_internal.feature_usage WHERE feature_name LIKE 'job.%.canceled'") - if len(rows) != 1 || - len(rows[0]) != 2 || - rows[0][0] != "job.schema_change.canceled" { - require.Failf(t, "Unexpected result set", "Rows: %s", rows) - } else if val, err := strconv.ParseInt(rows[0][1], 10, 32); err != nil || val < 2 { - require.Failf(t, "Invalid integer or value", "Error: %s Val: %d", err, val) - } } // TestCancelMultipleQueued tests that canceling schema changes when there are @@ -8113,11 +8102,13 @@ func TestOperationAtRandomStateTransition(t *testing.T) { for _, tc := range []testCase{ { name: "update during alter table with multiple column families", - setupSQL: `CREATE DATABASE t; + setupSQL: `SET use_declarative_schema_changer = off; +CREATE DATABASE t; CREATE TABLE t.test (pk INT PRIMARY KEY, a INT NOT NULL, b INT, FAMILY (pk, a), FAMILY (b)); INSERT INTO t.test (pk, a, b) VALUES (1, 1, 1), (2, 2, 2), (3, 3, 3); `, - schemaChangeSQL: `ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (a)`, + schemaChangeSQL: `SET use_declarative_schema_changer = off; +ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (a)`, operation: func(sqlDB *gosql.DB, kvDB *kv.DB) error { _, err := sqlDB.Exec("UPDATE t.test SET b = 22 WHERE pk = 1") return err diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go index 288c3c5ae5e6..311549bb623d 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go @@ -30,7 +30,7 @@ import ( var supportedAlterTableStatements = map[reflect.Type]supportedStatement{ reflect.TypeOf((*tree.AlterTableAddColumn)(nil)): {alterTableAddColumn, true}, reflect.TypeOf((*tree.AlterTableDropColumn)(nil)): {alterTableDropColumn, true}, - reflect.TypeOf((*tree.AlterTableAlterPrimaryKey)(nil)): {alterTableAlterPrimaryKey, false}, + reflect.TypeOf((*tree.AlterTableAlterPrimaryKey)(nil)): {alterTableAlterPrimaryKey, true}, } func init() { From 71f0298c19fb5fd5153290f0fe052b33b89005b9 Mon Sep 17 00:00:00 2001 From: Santamaura Date: Thu, 28 Jul 2022 16:53:06 -0400 Subject: [PATCH 06/11] sql, server: add new system privileges for observability This patch introduces 2 new system privileges VIEWDEBUG and VIEWCLUSTERMETADATA. VIEWDEBUG will now be used to gate taking traces and viewing debug endpoints. VIEWCLUSTERMETADATA will now be used to gate the node and range reports. Resolves #83844, #83856, #83857, #83858, #83861 Release note (sql change): add VIEWDEBUG and VIEWCLUSTERMETADATA system privileges. --- docs/generated/sql/bnf/stmt_block.bnf | 2 + pkg/server/admin.go | 63 +++++++++++++++++++++++---- pkg/server/admin_test.go | 22 ++++++++++ pkg/server/server_http.go | 10 ++--- pkg/server/status.go | 24 +++++----- pkg/sql/parser/sql.y | 6 ++- pkg/sql/privilege/kind_string.go | 6 ++- pkg/sql/privilege/privilege.go | 9 ++-- 8 files changed, 108 insertions(+), 34 deletions(-) diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index c87b9ae53b63..5cbf4ffc58e2 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -1359,7 +1359,9 @@ unreserved_keyword ::= | 'VIEW' | 'VIEWACTIVITY' | 'VIEWACTIVITYREDACTED' + | 'VIEWCLUSTERMETADATA' | 'VIEWCLUSTERSETTING' + | 'VIEWDEBUG' | 'VISIBLE' | 'VOLATILE' | 'VOTERS' diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 361a4c69ecf6..9f36b374fc0f 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -1515,14 +1515,17 @@ func (s *adminServer) RangeLog( ctx = s.server.AnnotateCtx(ctx) // Range keys, even when pretty-printed, contain PII. - userName, err := s.requireAdminUser(ctx) + user, _, err := s.getUserAndRole(ctx) + if err != nil { + return nil, err + } + + err = s.requireViewClusterMetadataPermission(ctx) if err != nil { - // NB: not using serverError() here since the priv checker - // already returns a proper gRPC error status. return nil, err } - r, err := s.rangeLogHelper(ctx, req, userName) + r, err := s.rangeLogHelper(ctx, req, user) if err != nil { return nil, serverError(ctx, err) } @@ -3543,6 +3546,50 @@ func (c *adminPrivilegeChecker) requireViewActivityAndNoViewActivityRedactedPerm return nil } +// requireViewClusterMetadataPermission requires the user have admin or the VIEWCLUSTERMETADATA +// system privilege and returns an error if the user does not have it. +func (c *adminPrivilegeChecker) requireViewClusterMetadataPermission( + ctx context.Context, +) (err error) { + userName, isAdmin, err := c.getUserAndRole(ctx) + if err != nil { + return serverError(ctx, err) + } + if !isAdmin { + if c.st.Version.IsActive(ctx, clusterversion.SystemPrivilegesTable) { + if hasViewClusterMetadata := c.checkHasSystemPrivilege(ctx, userName, privilege.VIEWCLUSTERMETADATA); !hasViewClusterMetadata { + return status.Errorf( + codes.PermissionDenied, "this operation requires the %s system privilege", + privilege.VIEWCLUSTERMETADATA) + } + } else { + return status.Error(codes.PermissionDenied, "this operation requires admin privilege") + } + } + return nil +} + +// requireViewDebugPermission requires the user have admin or the VIEWDEBUG system privilege +// and returns an error if the user does not have it. +func (c *adminPrivilegeChecker) requireViewDebugPermission(ctx context.Context) (err error) { + userName, isAdmin, err := c.getUserAndRole(ctx) + if err != nil { + return serverError(ctx, err) + } + if !isAdmin { + if c.st.Version.IsActive(ctx, clusterversion.SystemPrivilegesTable) { + if hasViewDebug := c.checkHasSystemPrivilege(ctx, userName, privilege.VIEWDEBUG); !hasViewDebug { + return status.Errorf( + codes.PermissionDenied, "this operation requires the %s system privilege", + privilege.VIEWDEBUG) + } + } else { + return status.Error(codes.PermissionDenied, "this operation requires admin privilege") + } + } + return nil +} + // Note that the function returns plain errors, and it is the caller's // responsibility to convert them to serverErrors. func (c *adminPrivilegeChecker) getUserAndRole( @@ -3638,7 +3685,7 @@ func (s *adminServer) ListTracingSnapshots( ctx context.Context, req *serverpb.ListTracingSnapshotsRequest, ) (*serverpb.ListTracingSnapshotsResponse, error) { ctx = s.server.AnnotateCtx(ctx) - _, err := s.requireAdminUser(ctx) + err := s.requireViewDebugPermission(ctx) if err != nil { return nil, err } @@ -3665,7 +3712,7 @@ func (s *adminServer) TakeTracingSnapshot( ctx context.Context, req *serverpb.TakeTracingSnapshotRequest, ) (*serverpb.TakeTracingSnapshotResponse, error) { ctx = s.server.AnnotateCtx(ctx) - _, err := s.requireAdminUser(ctx) + err := s.requireViewDebugPermission(ctx) if err != nil { return nil, err } @@ -3709,7 +3756,7 @@ func (s *adminServer) GetTracingSnapshot( ctx context.Context, req *serverpb.GetTracingSnapshotRequest, ) (*serverpb.GetTracingSnapshotResponse, error) { ctx = s.server.AnnotateCtx(ctx) - _, err := s.requireAdminUser(ctx) + err := s.requireViewDebugPermission(ctx) if err != nil { return nil, err } @@ -3768,7 +3815,7 @@ func (s *adminServer) GetTrace( ctx context.Context, req *serverpb.GetTraceRequest, ) (*serverpb.GetTraceResponse, error) { ctx = s.server.AnnotateCtx(ctx) - _, err := s.requireAdminUser(ctx) + err := s.requireViewDebugPermission(ctx) if err != nil { return nil, err } diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index fbec9541db01..2a0dd0518380 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -2834,6 +2834,20 @@ func TestAdminPrivilegeChecker(t *testing.T) { withAdmin: false, withVa: false, withVaRedacted: true, withVaAndRedacted: true, withoutPrivs: true, }, }, + { + "requireViewClusterMetadataPermission", + underTest.requireViewClusterMetadataPermission, + map[username.SQLUsername]bool{ + withAdmin: false, withoutPrivs: true, + }, + }, + { + "requireViewDebugPermission", + underTest.requireViewDebugPermission, + map[username.SQLUsername]bool{ + withAdmin: false, withoutPrivs: true, + }, + }, } // test system privileges if valid version if s.ClusterSettings().Version.IsActive(ctx, clusterversion.SystemPrivilegesTable) { @@ -2844,10 +2858,16 @@ func TestAdminPrivilegeChecker(t *testing.T) { sqlDB.Exec(t, "CREATE USER withvaandredactedsystemprivilege") sqlDB.Exec(t, "GRANT SYSTEM VIEWACTIVITY TO withvaandredactedsystemprivilege") sqlDB.Exec(t, "GRANT SYSTEM VIEWACTIVITYREDACTED TO withvaandredactedsystemprivilege") + sqlDB.Exec(t, "CREATE USER withviewclustermetadata") + sqlDB.Exec(t, "GRANT SYSTEM VIEWCLUSTERMETADATA TO withviewclustermetadata") + sqlDB.Exec(t, "CREATE USER withviewdebug") + sqlDB.Exec(t, "GRANT SYSTEM VIEWDEBUG TO withviewdebug") withVaSystemPrivilege := username.MakeSQLUsernameFromPreNormalizedString("withvasystemprivilege") withVaRedactedSystemPrivilege := username.MakeSQLUsernameFromPreNormalizedString("withvaredactedsystemprivilege") withVaAndRedactedSystemPrivilege := username.MakeSQLUsernameFromPreNormalizedString("withvaandredactedsystemprivilege") + withviewclustermetadata := username.MakeSQLUsernameFromPreNormalizedString("withviewclustermetadata") + withViewDebug := username.MakeSQLUsernameFromPreNormalizedString("withviewdebug") tests[0].usernameWantErr[withVaSystemPrivilege] = false tests[1].usernameWantErr[withVaSystemPrivilege] = false @@ -2858,6 +2878,8 @@ func TestAdminPrivilegeChecker(t *testing.T) { tests[0].usernameWantErr[withVaAndRedactedSystemPrivilege] = false tests[1].usernameWantErr[withVaAndRedactedSystemPrivilege] = false tests[2].usernameWantErr[withVaAndRedactedSystemPrivilege] = true + tests[3].usernameWantErr[withviewclustermetadata] = false + tests[4].usernameWantErr[withViewDebug] = false } for _, tt := range tests { diff --git a/pkg/server/server_http.go b/pkg/server/server_http.go index f417eec4daa6..43a87e677ef2 100644 --- a/pkg/server/server_http.go +++ b/pkg/server/server_http.go @@ -185,13 +185,9 @@ func makeAdminAuthzCheckHandler( md := forwardAuthenticationMetadata(req.Context(), req) authCtx := metadata.NewIncomingContext(req.Context(), md) // Check the privileges of the requester. - _, err := adminAuthzCheck.requireAdminUser(authCtx) - if errors.Is(err, errRequiresAdmin) { - http.Error(w, "admin privilege required", http.StatusUnauthorized) - return - } else if err != nil { - log.Ops.Infof(authCtx, "web session error: %s", err) - http.Error(w, "error checking authentication", http.StatusInternalServerError) + err := adminAuthzCheck.requireViewDebugPermission(authCtx) + if err != nil { + http.Error(w, "admin privilege or VIEWDEBUG system privilege required", http.StatusUnauthorized) return } // Forward the request to the inner handler. diff --git a/pkg/server/status.go b/pkg/server/status.go index e7d385dd4f71..026bea2bdafc 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -800,9 +800,8 @@ func (s *statusServer) AllocatorRange( ctx = propagateGatewayMetadata(ctx) ctx = s.AnnotateCtx(ctx) - if _, err := s.privilegeChecker.requireAdminUser(ctx); err != nil { - // NB: not using serverError() here since the priv checker - // already returns a proper gRPC error status. + err := s.privilegeChecker.requireViewClusterMetadataPermission(ctx) + if err != nil { return nil, err } @@ -1453,7 +1452,7 @@ func (s *statusServer) Nodes( ctx = propagateGatewayMetadata(ctx) ctx = s.AnnotateCtx(ctx) - err := s.privilegeChecker.requireViewActivityPermission(ctx) + err := s.privilegeChecker.requireViewClusterMetadataPermission(ctx) if err != nil { return nil, err } @@ -1471,14 +1470,14 @@ func (s *statusServer) NodesUI( ctx = propagateGatewayMetadata(ctx) ctx = s.AnnotateCtx(ctx) - hasViewActivity := false - err := s.privilegeChecker.requireViewActivityPermission(ctx) + hasViewClusterMetadata := false + err := s.privilegeChecker.requireViewClusterMetadataPermission(ctx) if err != nil { if !grpcutil.IsAuthError(err) { return nil, err } } else { - hasViewActivity = true + hasViewClusterMetadata = true } internalResp, _, err := s.nodesHelper(ctx, 0 /* limit */, 0 /* offset */) @@ -1490,13 +1489,13 @@ func (s *statusServer) NodesUI( LivenessByNodeID: internalResp.LivenessByNodeID, } for i, nodeStatus := range internalResp.Nodes { - resp.Nodes[i] = nodeStatusToResp(&nodeStatus, hasViewActivity) + resp.Nodes[i] = nodeStatusToResp(&nodeStatus, hasViewClusterMetadata) } return resp, nil } -func nodeStatusToResp(n *statuspb.NodeStatus, hasViewActivity bool) serverpb.NodeResponse { +func nodeStatusToResp(n *statuspb.NodeStatus, hasViewClusterMetadata bool) serverpb.NodeResponse { tiers := make([]serverpb.Tier, len(n.Desc.Locality.Tiers)) for j, t := range n.Desc.Locality.Tiers { tiers[j] = serverpb.Tier{ @@ -1552,7 +1551,7 @@ func nodeStatusToResp(n *statuspb.NodeStatus, hasViewActivity bool) serverpb.Nod sfsprops := &roachpb.FileStoreProperties{ FsType: fsprops.FsType, } - if hasViewActivity { + if hasViewClusterMetadata { sfsprops.Path = fsprops.Path sfsprops.BlockDevice = fsprops.BlockDevice sfsprops.MountPoint = fsprops.MountPoint @@ -1577,7 +1576,7 @@ func nodeStatusToResp(n *statuspb.NodeStatus, hasViewActivity bool) serverpb.Nod NumCpus: n.NumCpus, } - if hasViewActivity { + if hasViewClusterMetadata { resp.Args = n.Args resp.Env = n.Env resp.Desc.Attrs = n.Desc.Attrs @@ -1916,7 +1915,8 @@ func (s *statusServer) rangesHelper( ctx = propagateGatewayMetadata(ctx) ctx = s.AnnotateCtx(ctx) - if _, err := s.privilegeChecker.requireAdminUser(ctx); err != nil { + err := s.privilegeChecker.requireViewClusterMetadataPermission(ctx) + if err != nil { return nil, 0, err } diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index f4facdca74c2..8b7a48efa6de 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -932,8 +932,8 @@ func (u *sqlSymUnion) functionObjs() tree.FuncObjs { %token UNBOUNDED UNCOMMITTED UNION UNIQUE UNKNOWN UNLOGGED UNSPLIT %token UPDATE UPSERT UNSET UNTIL USE USER USERS USING UUID -%token VALID VALIDATE VALUE VALUES VARBIT VARCHAR VARIADIC VIEW VARYING VIEWACTIVITY VIEWACTIVITYREDACTED -%token VIEWCLUSTERSETTING VIRTUAL VISIBLE VOLATILE VOTERS +%token VALID VALIDATE VALUE VALUES VARBIT VARCHAR VARIADIC VIEW VARYING VIEWACTIVITY VIEWACTIVITYREDACTED VIEWDEBUG +%token VIEWCLUSTERMETADATA VIEWCLUSTERSETTING VIRTUAL VISIBLE VOLATILE VOTERS %token WHEN WHERE WINDOW WITH WITHIN WITHOUT WORK WRITE @@ -15198,7 +15198,9 @@ unreserved_keyword: | VIEW | VIEWACTIVITY | VIEWACTIVITYREDACTED +| VIEWCLUSTERMETADATA | VIEWCLUSTERSETTING +| VIEWDEBUG | VISIBLE | VOLATILE | VOTERS diff --git a/pkg/sql/privilege/kind_string.go b/pkg/sql/privilege/kind_string.go index 396cfe3c3e68..67157a8514f8 100644 --- a/pkg/sql/privilege/kind_string.go +++ b/pkg/sql/privilege/kind_string.go @@ -28,11 +28,13 @@ func _() { _ = x[CANCELQUERY-18] _ = x[NOSQLLOGIN-19] _ = x[EXECUTE-20] + _ = x[VIEWCLUSTERMETADATA-21] + _ = x[VIEWDEBUG-22] } -const _Kind_name = "ALLCREATEDROPGRANTSELECTINSERTDELETEUPDATEUSAGEZONECONFIGCONNECTRULEMODIFYCLUSTERSETTINGEXTERNALCONNECTIONVIEWACTIVITYVIEWACTIVITYREDACTEDVIEWCLUSTERSETTINGCANCELQUERYNOSQLLOGINEXECUTE" +const _Kind_name = "ALLCREATEDROPGRANTSELECTINSERTDELETEUPDATEUSAGEZONECONFIGCONNECTRULEMODIFYCLUSTERSETTINGEXTERNALCONNECTIONVIEWACTIVITYVIEWACTIVITYREDACTEDVIEWCLUSTERSETTINGCANCELQUERYNOSQLLOGINEXECUTEVIEWCLUSTERMETADATAVIEWDEBUG" -var _Kind_index = [...]uint8{0, 3, 9, 13, 18, 24, 30, 36, 42, 47, 57, 64, 68, 88, 106, 118, 138, 156, 167, 177, 184} +var _Kind_index = [...]uint8{0, 3, 9, 13, 18, 24, 30, 36, 42, 47, 57, 64, 68, 88, 106, 118, 138, 156, 167, 177, 184, 203, 212} func (i Kind) String() string { i -= 1 diff --git a/pkg/sql/privilege/privilege.go b/pkg/sql/privilege/privilege.go index ad8ab256fce2..65adf7faf529 100644 --- a/pkg/sql/privilege/privilege.go +++ b/pkg/sql/privilege/privilege.go @@ -53,6 +53,8 @@ const ( CANCELQUERY Kind = 18 NOSQLLOGIN Kind = 19 EXECUTE Kind = 20 + VIEWCLUSTERMETADATA Kind = 21 + VIEWDEBUG Kind = 22 ) // Privilege represents a privilege parsed from an Access Privilege Inquiry @@ -120,7 +122,7 @@ var ( // certain privileges unavailable after upgrade migration. // Note that "CREATE, INSERT, DELETE, ZONECONFIG" are no-op privileges on sequences. SequencePrivileges = List{ALL, USAGE, SELECT, UPDATE, CREATE, DROP, INSERT, DELETE, ZONECONFIG} - SystemPrivileges = List{ALL, MODIFYCLUSTERSETTING, EXTERNALCONNECTION, VIEWACTIVITY, VIEWACTIVITYREDACTED, VIEWCLUSTERSETTING, CANCELQUERY, NOSQLLOGIN} + SystemPrivileges = List{ALL, MODIFYCLUSTERSETTING, EXTERNALCONNECTION, VIEWACTIVITY, VIEWACTIVITYREDACTED, VIEWCLUSTERSETTING, CANCELQUERY, NOSQLLOGIN, VIEWCLUSTERMETADATA, VIEWDEBUG} VirtualTablePrivileges = List{ALL, SELECT} ExternalConnectionPrivileges = List{ALL, USAGE} ) @@ -137,8 +139,7 @@ func (k Kind) IsSetIn(bits uint32) bool { // ByValue is just an array of privilege kinds sorted by value. var ByValue = [...]Kind{ - ALL, CREATE, DROP, SELECT, INSERT, DELETE, UPDATE, USAGE, ZONECONFIG, CONNECT, RULE, MODIFYCLUSTERSETTING, - EXTERNALCONNECTION, VIEWACTIVITY, VIEWACTIVITYREDACTED, VIEWCLUSTERSETTING, CANCELQUERY, NOSQLLOGIN, EXECUTE, + ALL, CREATE, DROP, SELECT, INSERT, DELETE, UPDATE, USAGE, ZONECONFIG, CONNECT, RULE, MODIFYCLUSTERSETTING, EXTERNALCONNECTION, VIEWACTIVITY, VIEWACTIVITYREDACTED, VIEWCLUSTERSETTING, CANCELQUERY, NOSQLLOGIN, EXECUTE, VIEWCLUSTERMETADATA, VIEWDEBUG, } // ByName is a map of string -> kind value. @@ -162,6 +163,8 @@ var ByName = map[string]Kind{ "CANCELQUERY": CANCELQUERY, "NOSQLLOGIN": NOSQLLOGIN, "EXECUTE": EXECUTE, + "VIEWCLUSTERMETADATA": VIEWCLUSTERMETADATA, + "VIEWDEBUG": VIEWDEBUG, } // List is a list of privileges. From 51c5a382cf033cc37b84c1d0f1ee4cb5addd185d Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Tue, 9 Aug 2022 11:54:08 -0400 Subject: [PATCH 07/11] kv: use max timestamp during below-Raft scan to gossip liveness This commit switches `MaybeGossipNodeLivenessRaftMuLocked` to evaluate its scan at the maximum timestamp instead of at the local node's HLC time. This ensures that we gossip the most recent liveness record, regardless of what timestamp it is written at. --- pkg/kv/kvserver/replica_gossip.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pkg/kv/kvserver/replica_gossip.go b/pkg/kv/kvserver/replica_gossip.go index e9e2d4e798f5..96c40a425b30 100644 --- a/pkg/kv/kvserver/replica_gossip.go +++ b/pkg/kv/kvserver/replica_gossip.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -83,7 +84,9 @@ func (r *Replica) MaybeGossipNodeLivenessRaftMuLocked( } ba := roachpb.BatchRequest{} - ba.Timestamp = r.store.Clock().Now() + // Read at the maximum timestamp to ensure that we see the most recent + // liveness record, regardless of what timestamp it is written at. + ba.Timestamp = hlc.MaxTimestamp ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeaderFromSpan(span)}) // Call evaluateBatch instead of Send to avoid reacquiring latches. rec := NewReplicaEvalContext( From fd1b51f88cd69fda12e74770bad22cc23c9ffc02 Mon Sep 17 00:00:00 2001 From: Thomas Hardy Date: Wed, 20 Jul 2022 11:34:52 -0400 Subject: [PATCH 08/11] sql: populate query-level stats earlier & add contention to telemetry log Addresses: #71328 This change adds contention time (measured in nanoseconds) to the `SampledQuery` telemetry log. To accomodate this change, we needed to collect query-level statistics earlier. Previously, query-level statistics were fetched when we called `Finish` under the `instrumentationHelper`, however this occurred after we had already emitted our query execution logs. Now, we collect query-level stats in `dispatchToExecutionEngine` after we've executed the query. As a tradeoff to collecting query-level stats earlier, we need to fetch the trace twice: - once when populating query-level stats (trace is required to compute query-level stats) at `populateQueryLevelStats` in `dispatchToExecutionEngine` after query execution - once during the instrumentation helper's `Finish` (as we do currently) This allows us to collect query-level stats earlier without omitting any tracing events we record currently. This approach is safer, with the additional overhead of fetching the trace twice only occuring at the tracing sampling rate of 1-2%, which is fairly conservative. The concern with only fetching the trace at query-level stats population was the ommission of a number of events that occur in `commitSQLTransactionInternal` (or any execution paths that don't lead to `dispatchToExecutionEngine`). Release note (sql change): Add `ContentionTime` field to the `SampledQuery` telemetry log. Query-level statistics are collected earlier to facilitate the adding of contention time to query execution logs. The earlier collection of query-level statistics requires the additional overhead of fetching the query's trace twice (instead of previously once). --- docs/generated/eventlog.md | 1 + pkg/sql/conn_executor_exec.go | 31 ++++++++++ pkg/sql/exec_log.go | 2 + pkg/sql/execstats/traceanalyzer.go | 16 +++++ pkg/sql/executor_statement_metrics.go | 15 ++++- pkg/sql/instrumentation.go | 59 +++++++++---------- pkg/sql/telemetry_logging.go | 11 ++++ pkg/sql/telemetry_logging_test.go | 38 +++++++++++- pkg/util/log/eventpb/json_encode_generated.go | 9 +++ pkg/util/log/eventpb/telemetry.proto | 3 + 10 files changed, 151 insertions(+), 34 deletions(-) diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index 768064079aa7..f516fb2935a3 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -2553,6 +2553,7 @@ contains common SQL event/execution details. | `InvertedJoinCount` | The number of inverted joins in the query plan. | no | | `ApplyJoinCount` | The number of apply joins in the query plan. | no | | `ZigZagJoinCount` | The number of zig zag joins in the query plan. | no | +| `ContentionNanos` | The duration of time in nanoseconds that the query experienced contention. | no | #### Common fields diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index bec283f1be10..df7c99442910 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -48,6 +48,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -1194,6 +1195,8 @@ func (ex *connExecutor) dispatchToExecutionEngine( ex.extraTxnState.bytesRead += stats.bytesRead ex.extraTxnState.rowsWritten += stats.rowsWritten + populateQueryLevelStats(ctx, planner) + // Record the statement summary. This also closes the plan if the // plan has not been closed earlier. stmtFingerprintID = ex.recordStatementSummary( @@ -1211,6 +1214,34 @@ func (ex *connExecutor) dispatchToExecutionEngine( return err } +// populateQueryLevelStats collects query-level execution statistics and +// populates it in the instrumentationHelper's queryLevelStatsWithErr field. +// Query-level execution statistics are collected using the statement's trace +// and the plan's flow metadata. +func populateQueryLevelStats(ctx context.Context, p *planner) { + ih := &p.instrumentation + if _, ok := ih.Tracing(); !ok { + return + } + // Get the query-level stats. + var flowsMetadata []*execstats.FlowsMetadata + for _, flowInfo := range p.curPlan.distSQLFlowInfos { + flowsMetadata = append(flowsMetadata, flowInfo.flowsMetadata) + } + trace := ih.sp.GetRecording(tracingpb.RecordingStructured) + var err error + queryLevelStats, err := execstats.GetQueryLevelStats( + trace, p.execCfg.TestingKnobs.DeterministicExplain, flowsMetadata) + ih.queryLevelStatsWithErr = execstats.MakeQueryLevelStatsWithErr(queryLevelStats, err) + if err != nil { + const msg = "error getting query level stats for statement: %s: %+v" + if buildutil.CrdbTestBuild { + panic(fmt.Sprintf(msg, ih.fingerprint, err)) + } + log.VInfof(ctx, 1, msg, ih.fingerprint, err) + } +} + type txnRowsWrittenLimitErr struct { eventpb.CommonTxnRowsLimitDetails } diff --git a/pkg/sql/exec_log.go b/pkg/sql/exec_log.go index 895394804361..b14ed17e3f51 100644 --- a/pkg/sql/exec_log.go +++ b/pkg/sql/exec_log.go @@ -391,6 +391,7 @@ func (p *planner) maybeLogStatementInternal( requiredTimeElapsed = 0 } if telemetryMetrics.maybeUpdateLastEmittedTime(telemetryMetrics.timeNow(), requiredTimeElapsed) { + contentionNanos := telemetryMetrics.getContentionTime(p.instrumentation.queryLevelStatsWithErr.Stats.ContentionTime.Nanoseconds()) skippedQueries := telemetryMetrics.resetSkippedQueryCount() sampledQuery := eventpb.SampledQuery{ CommonSQLExecDetails: execDetails, @@ -426,6 +427,7 @@ func (p *planner) maybeLogStatementInternal( InvertedJoinCount: int64(p.curPlan.instrumentation.joinAlgorithmCounts[exec.InvertedJoin]), ApplyJoinCount: int64(p.curPlan.instrumentation.joinAlgorithmCounts[exec.ApplyJoin]), ZigZagJoinCount: int64(p.curPlan.instrumentation.joinAlgorithmCounts[exec.ZigZagJoin]), + ContentionNanos: contentionNanos, } p.logOperationalEventsOnlyExternally(ctx, eventLogEntry{event: &sampledQuery}) } else { diff --git a/pkg/sql/execstats/traceanalyzer.go b/pkg/sql/execstats/traceanalyzer.go index f3aa593af932..27f7273cd7cf 100644 --- a/pkg/sql/execstats/traceanalyzer.go +++ b/pkg/sql/execstats/traceanalyzer.go @@ -126,6 +126,22 @@ type QueryLevelStats struct { Regions []string } +// QueryLevelStatsWithErr is the same as QueryLevelStats, but also tracks +// if an error occurred while getting query-level stats. +type QueryLevelStatsWithErr struct { + Stats QueryLevelStats + Err error +} + +// MakeQueryLevelStatsWithErr creates a QueryLevelStatsWithErr from a +// QueryLevelStats and error. +func MakeQueryLevelStatsWithErr(stats QueryLevelStats, err error) QueryLevelStatsWithErr { + return QueryLevelStatsWithErr{ + stats, + err, + } +} + // Accumulate accumulates other's stats into the receiver. func (s *QueryLevelStats) Accumulate(other QueryLevelStats) { s.NetworkBytesSent += other.NetworkBytesSent diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index 1c32fe8dbb5c..dcde246cf541 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -102,7 +102,7 @@ var _ metric.Struct = GuardrailMetrics{} // MetricStruct is part of the metric.Struct interface. func (GuardrailMetrics) MetricStruct() {} -// recordStatementSummery gathers various details pertaining to the +// recordStatementSummary gathers various details pertaining to the // last executed statement/query and performs the associated // accounting in the passed-in EngineMetrics. // - distSQLUsed reports whether the query was distributed. @@ -205,6 +205,17 @@ func (ex *connExecutor) recordStatementSummary( ex.server.ServerMetrics.StatsMetrics.DiscardedStatsCount.Inc(1) } + // Record statement execution statistics if span is recorded and no error was + // encountered while collecting query-level statistics. + if _, ok := planner.instrumentation.Tracing(); ok && planner.instrumentation.queryLevelStatsWithErr.Err == nil { + err = ex.statsCollector.RecordStatementExecStats(recordedStmtStatsKey, planner.instrumentation.queryLevelStatsWithErr.Stats) + if err != nil { + if log.V(2 /* level */) { + log.Warningf(ctx, "unable to record statement exec stats: %s", err) + } + } + } + // Do some transaction level accounting for the transaction this statement is // a part of. @@ -264,7 +275,7 @@ func shouldIncludeStmtInLatencyMetrics(stmt *Statement) bool { func getNodesFromPlanner(planner *planner) []int64 { // Retrieve the list of all nodes which the statement was executed on. var nodes []int64 - if planner.instrumentation.sp != nil { + if _, ok := planner.instrumentation.Tracing(); !ok { trace := planner.instrumentation.sp.GetRecording(tracingpb.RecordingStructured) // ForEach returns nodes in order. execinfrapb.ExtractNodesFromSpans(planner.EvalContext().Context, trace).ForEach(func(i int) { diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go index d698bbf8c738..c6de23e21413 100644 --- a/pkg/sql/instrumentation.go +++ b/pkg/sql/instrumentation.go @@ -101,13 +101,24 @@ type instrumentationHelper struct { stmtDiagnosticsRecorder *stmtdiagnostics.Registry withStatementTrace func(trace tracingpb.Recording, stmt string) + // sp is always populated by the instrumentationHelper Setup method, except in + // the scenario where we do not need tracing information. This scenario occurs + // with the confluence of: + // - not collecting a bundle (collectBundle is false) + // - withStatementTrace is nil (only populated by testing knobs) + // - outputMode is unmodifiedOutput (i.e. outputMode not specified) + // - not collecting execution statistics (collectExecStats is false) + // TODO(yuzefovich): refactor statement span creation #85820 sp *tracing.Span + // shouldFinishSpan determines whether sp needs to be finished in // instrumentationHelper.Finish. shouldFinishSpan bool origCtx context.Context evalCtx *eval.Context + queryLevelStatsWithErr execstats.QueryLevelStatsWithErr + // If savePlanForStats is true, the explainPlan will be collected and returned // via PlanForStats(). savePlanForStats bool @@ -172,6 +183,15 @@ const ( explainAnalyzeDistSQLOutput ) +// Tracing returns the current value of the instrumentation helper's span, +// along with a boolean that determines whether the span is populated. +func (ih *instrumentationHelper) Tracing() (sp *tracing.Span, ok bool) { + if ih.sp != nil { + return ih.sp, true + } + return nil, false +} + // SetOutputMode can be called before Setup, if we are running an EXPLAIN // ANALYZE variant. func (ih *instrumentationHelper) SetOutputMode(outputMode outputMode, explainFlags explain.Flags) { @@ -284,13 +304,15 @@ func (ih *instrumentationHelper) Finish( retErr error, ) error { ctx := ih.origCtx - if ih.sp == nil { + if _, ok := ih.Tracing(); !ok { return retErr } // Record the statement information that we've collected. // Note that in case of implicit transactions, the trace contains the auto-commit too. var trace tracingpb.Recording + queryLevelStatsWithErr := ih.queryLevelStatsWithErr + if ih.shouldFinishSpan { trace = ih.sp.FinishAndGetConfiguredRecording() } else { @@ -310,34 +332,11 @@ func (ih *instrumentationHelper) Finish( ) } - // Get the query-level stats. - var flowsMetadata []*execstats.FlowsMetadata - for _, flowInfo := range p.curPlan.distSQLFlowInfos { - flowsMetadata = append(flowsMetadata, flowInfo.flowsMetadata) - } - queryLevelStats, err := execstats.GetQueryLevelStats(trace, cfg.TestingKnobs.DeterministicExplain, flowsMetadata) - if err != nil { - const msg = "error getting query level stats for statement: %s: %+v" - if buildutil.CrdbTestBuild { - panic(fmt.Sprintf(msg, ih.fingerprint, err)) - } - log.VInfof(ctx, 1, msg, ih.fingerprint, err) - } else { - stmtStatsKey := roachpb.StatementStatisticsKey{ - Query: ih.fingerprint, - ImplicitTxn: ih.implicitTxn, - Database: p.SessionData().Database, - Failed: retErr != nil, - PlanHash: ih.planGist.Hash(), - } - err = statsCollector.RecordStatementExecStats(stmtStatsKey, queryLevelStats) - if err != nil { - if log.V(2 /* level */) { - log.Warningf(ctx, "unable to record statement exec stats: %s", err) - } - } + // Accumulate txn stats if no error was encountered while collecting + // query-level statistics. + if queryLevelStatsWithErr.Err == nil { if collectExecStats || ih.implicitTxn { - txnStats.Accumulate(queryLevelStats) + txnStats.Accumulate(queryLevelStatsWithErr.Stats) } } @@ -355,7 +354,7 @@ func (ih *instrumentationHelper) Finish( ob := ih.emitExplainAnalyzePlanToOutputBuilder( explain.Flags{Verbose: true, ShowTypes: true}, phaseTimes, - &queryLevelStats, + &queryLevelStatsWithErr.Stats, ) bundle = buildStatementBundle( ih.origCtx, cfg.DB, ie.(*InternalExecutor), &p.curPlan, ob.BuildString(), trace, placeholders, @@ -381,7 +380,7 @@ func (ih *instrumentationHelper) Finish( if ih.outputMode == explainAnalyzeDistSQLOutput { flows = p.curPlan.distSQLFlowInfos } - return ih.setExplainAnalyzeResult(ctx, res, statsCollector.PhaseTimes(), &queryLevelStats, flows, trace) + return ih.setExplainAnalyzeResult(ctx, res, statsCollector.PhaseTimes(), &queryLevelStatsWithErr.Stats, flows, trace) default: return nil diff --git a/pkg/sql/telemetry_logging.go b/pkg/sql/telemetry_logging.go index 7ae14f8142c0..0f343aedb82b 100644 --- a/pkg/sql/telemetry_logging.go +++ b/pkg/sql/telemetry_logging.go @@ -53,6 +53,10 @@ type TelemetryLoggingTestingKnobs struct { // getTimeNow allows tests to override the timeutil.Now() function used // when updating rolling query counts. getTimeNow func() time.Time + // getContentionNanos allows tests to override the recorded contention time + // for the query. Used to stub non-zero values to populate the log's contention + // time field. + getContentionNanos func() int64 } // ModuleTestingKnobs implements base.ModuleTestingKnobs interface. @@ -83,6 +87,13 @@ func (t *TelemetryLoggingMetrics) maybeUpdateLastEmittedTime( return false } +func (t *TelemetryLoggingMetrics) getContentionTime(contentionTimeInNanoseconds int64) int64 { + if t.Knobs != nil && t.Knobs.getContentionNanos != nil { + return t.Knobs.getContentionNanos() + } + return contentionTimeInNanoseconds +} + func (t *TelemetryLoggingMetrics) resetSkippedQueryCount() (res uint64) { return atomic.SwapUint64(&t.skippedQueryCount, 0) } diff --git a/pkg/sql/telemetry_logging_test.go b/pkg/sql/telemetry_logging_test.go index 41d90427f2b7..02e69da4b86b 100644 --- a/pkg/sql/telemetry_logging_test.go +++ b/pkg/sql/telemetry_logging_test.go @@ -51,6 +51,23 @@ func (s *stubTime) TimeNow() time.Time { return s.t } +type stubQueryMetrics struct { + syncutil.RWMutex + contentionNanos int64 +} + +func (s *stubQueryMetrics) setContentionNanos(t int64) { + s.RWMutex.Lock() + defer s.RWMutex.Unlock() + s.contentionNanos = t +} + +func (s *stubQueryMetrics) ContentionNanos() int64 { + s.RWMutex.RLock() + defer s.RWMutex.RUnlock() + return s.contentionNanos +} + func installTelemetryLogFileSink(sc *log.TestLogScope, t *testing.T) func() { // Enable logging channels. log.TestingResetActive() @@ -83,11 +100,13 @@ func TestTelemetryLogging(t *testing.T) { defer cleanup() st := stubTime{} + sqm := stubQueryMetrics{} s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ TelemetryLoggingKnobs: &TelemetryLoggingTestingKnobs{ - getTimeNow: st.TimeNow, + getTimeNow: st.TimeNow, + getContentionNanos: sqm.ContentionNanos, }, }, }) @@ -138,6 +157,7 @@ func TestTelemetryLogging(t *testing.T) { expectedRead bool expectedWrite bool expectedErr string // Empty string means no error is expected. + contentionNanos int64 }{ { // Test case with statement that is not of type DML. @@ -157,6 +177,7 @@ func TestTelemetryLogging(t *testing.T) { expectedStatsAvailable: false, expectedRead: false, expectedWrite: false, + contentionNanos: 0, }, { // Test case with statement that is of type DML. @@ -174,6 +195,7 @@ func TestTelemetryLogging(t *testing.T) { expectedStatsAvailable: false, expectedRead: false, expectedWrite: false, + contentionNanos: 1, }, { // Test case with statement that is of type DML. @@ -192,6 +214,7 @@ func TestTelemetryLogging(t *testing.T) { expectedStatsAvailable: true, expectedRead: true, expectedWrite: false, + contentionNanos: 2, }, { // Test case with statement that is of type DML. @@ -209,6 +232,7 @@ func TestTelemetryLogging(t *testing.T) { expectedStatsAvailable: true, expectedRead: true, expectedWrite: false, + contentionNanos: 3, }, { // Test case with a full scan. @@ -226,6 +250,7 @@ func TestTelemetryLogging(t *testing.T) { expectedStatsAvailable: true, expectedRead: true, expectedWrite: false, + contentionNanos: 0, }, { // Test case with a write. @@ -243,6 +268,7 @@ func TestTelemetryLogging(t *testing.T) { expectedStatsAvailable: true, expectedRead: true, expectedWrite: true, + contentionNanos: 0, }, // Not of type DML so not sampled { @@ -268,6 +294,7 @@ func TestTelemetryLogging(t *testing.T) { for _, execTimestamp := range tc.execTimestampsSeconds { stubTime := timeutil.FromUnixMicros(int64(execTimestamp * 1e6)) st.setTime(stubTime) + sqm.setContentionNanos(tc.contentionNanos) _, err := db.DB.ExecContext(context.Background(), tc.query) if err != nil && tc.expectedErr == "" { t.Errorf("unexpected error executing query `%s`: %v", tc.query, err) @@ -420,7 +447,6 @@ func TestTelemetryLogging(t *testing.T) { if RowsReadRe.MatchString(e.Message) { t.Errorf("expected not to find RowsRead but it was found in: %s", e.Message) } - } RowsWrittenRe := regexp.MustCompile("\"RowsWritten\":[0-9]*") if tc.expectedWrite { @@ -432,6 +458,14 @@ func TestTelemetryLogging(t *testing.T) { t.Errorf("expected not to find RowsWritten but it was found in: %s", e.Message) } } + contentionNanos := regexp.MustCompile("\"ContentionNanos\":[0-9]*") + if tc.contentionNanos > 0 && !contentionNanos.MatchString(e.Message) { + // If we have contention, we expect the ContentionNanos field to be populated. + t.Errorf("expected to find ContentionNanos but none was found") + } else if tc.contentionNanos == 0 && contentionNanos.MatchString(e.Message) { + // If we do not have contention, expect no ContentionNanos field. + t.Errorf("expected no ContentionNanos field, but was found") + } if tc.expectedErr != "" { if !strings.Contains(e.Message, tc.expectedErr) { t.Errorf("%s: missing error %s in message %s", tc.name, tc.expectedErr, e.Message) diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index 27b8f5799d70..98b3868381a6 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -3789,6 +3789,15 @@ func (m *SampledQuery) AppendJSONFields(printComma bool, b redact.RedactableByte b = strconv.AppendInt(b, int64(m.ZigZagJoinCount), 10) } + if m.ContentionNanos != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"ContentionNanos\":"...) + b = strconv.AppendInt(b, int64(m.ContentionNanos), 10) + } + return printComma, b } diff --git a/pkg/util/log/eventpb/telemetry.proto b/pkg/util/log/eventpb/telemetry.proto index 5dd654ba5aa4..966b3595bda2 100644 --- a/pkg/util/log/eventpb/telemetry.proto +++ b/pkg/util/log/eventpb/telemetry.proto @@ -136,6 +136,9 @@ message SampledQuery { // The number of zig zag joins in the query plan. int64 zig_zag_join_count = 36 [(gogoproto.jsontag) = ",omitempty"]; + // The duration of time in nanoseconds that the query experienced contention. + int64 contention_nanos = 37 [(gogoproto.jsontag) = ',omitempty']; + reserved 12; } From d4490c94ef04c4c2f72edf40332517a70eda85fd Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 8 Aug 2022 19:25:13 -0700 Subject: [PATCH 09/11] colbuilder: add a microbenchmark for running many render expressions This commit adds a microbenchmark of queries with many render expressions. It'll be used in the following commit to tune when we fall back to wrapping a row-by-row processor to handle those renders. Release note: None --- pkg/sql/colexec/colbuilder/execplan_test.go | 41 +++++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/pkg/sql/colexec/colbuilder/execplan_test.go b/pkg/sql/colexec/colbuilder/execplan_test.go index 28582acba521..abc7b4e49225 100644 --- a/pkg/sql/colexec/colbuilder/execplan_test.go +++ b/pkg/sql/colexec/colbuilder/execplan_test.go @@ -12,6 +12,8 @@ package colbuilder import ( "context" + "fmt" + "strings" "testing" "github.com/cockroachdb/cockroach/pkg/base" @@ -159,3 +161,42 @@ func TestNewColOperatorExpectedTypeSchema(t *testing.T) { } require.Equal(t, numRows, rowIdx) } + +// BenchmarkRenderPlanning benchmarks how long it takes to run a query with many +// render expressions inside. With small number of rows to read, the overhead of +// allocating the initial vectors for the projection operators dominates. +func BenchmarkRenderPlanning(b *testing.B) { + defer leaktest.AfterTest(b)() + defer log.Scope(b).Close(b) + + ctx := context.Background() + s, db, _ := serverutils.StartServer(b, base.TestServerArgs{SQLMemoryPoolSize: 10 << 30}) + defer s.Stopper().Stop(ctx) + + jsonValue := `'{"string": "string", "int": 123, "null": null, "nested": {"string": "string", "int": 123, "null": null, "nested": {"string": "string", "int": 123, "null": null}}}'` + + sqlDB := sqlutils.MakeSQLRunner(db) + for _, numRows := range []int{1, 1 << 3, 1 << 6, 1 << 9} { + sqlDB.Exec(b, "DROP TABLE IF EXISTS bench") + sqlDB.Exec(b, "CREATE TABLE bench (id INT PRIMARY KEY, state JSONB)") + sqlDB.Exec(b, fmt.Sprintf(`INSERT INTO bench SELECT i, %s FROM generate_series(1, %d) AS g(i)`, jsonValue, numRows)) + sqlDB.Exec(b, "ANALYZE bench") + for _, numRenders := range []int{1, 1 << 4, 1 << 8, 1 << 12} { + var sb strings.Builder + sb.WriteString("SELECT ") + for i := 0; i < numRenders; i++ { + if i > 0 { + sb.WriteString(", ") + } + sb.WriteString(fmt.Sprintf("state->'nested'->>'nested' AS test%d", i+1)) + } + sb.WriteString(" FROM bench") + query := sb.String() + b.Run(fmt.Sprintf("rows=%d/renders=%d", numRows, numRenders), func(b *testing.B) { + for i := 0; i < b.N; i++ { + sqlDB.Exec(b, query) + } + }) + } + } +} From f5a28fb3689fd08465eb376db356551e8df230ba Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 8 Aug 2022 20:44:46 -0700 Subject: [PATCH 10/11] colbuilder: fall back to row-by-row processor wrapping for many renders MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit introduces a mechanism to handle render expressions by wrapping a row-by-row processor into the vectorized flow when 1. the estimated number of rows to go through the renders is relatively small 2. the number of renders is relatively high. The idea is that the vectorized projection operators have higher overhead when many of them are planned AND there is not enough data to amortize the overhead, so to improve the performance in those cases we'll use the row-by-row noop processor. Both of the thresholds are controlled by cluster settings and the defaults were chosen based on a representative microbenchmark. It's worth pointing out that we only have the estimated row count for the scan operators, so the change has limited applicability. ``` RenderPlanning/rows=1/renders=1-24 407µs ± 2% 408µs ± 2% ~ (p=0.684 n=10+10) RenderPlanning/rows=1/renders=8-24 516µs ± 1% 537µs ± 1% +4.05% (p=0.000 n=10+10) RenderPlanning/rows=1/renders=32-24 832µs ± 1% 811µs ± 1% -2.59% (p=0.000 n=10+10) RenderPlanning/rows=1/renders=64-24 1.22ms ± 0% 1.14ms ± 1% -6.62% (p=0.000 n=9+10) RenderPlanning/rows=1/renders=128-24 2.02ms ± 0% 1.80ms ± 1% -11.18% (p=0.000 n=8+9) RenderPlanning/rows=1/renders=512-24 7.75ms ± 1% 5.75ms ± 1% -25.77% (p=0.000 n=10+9) RenderPlanning/rows=1/renders=4096-24 160ms ± 1% 62ms ± 1% -61.51% (p=0.000 n=10+9) RenderPlanning/rows=4/renders=1-24 438µs ± 2% 438µs ± 1% ~ (p=0.853 n=10+10) RenderPlanning/rows=4/renders=8-24 603µs ± 1% 633µs ± 2% +5.06% (p=0.000 n=10+10) RenderPlanning/rows=4/renders=32-24 1.08ms ± 1% 1.08ms ± 1% ~ (p=0.105 n=10+10) RenderPlanning/rows=4/renders=64-24 1.72ms ± 0% 1.62ms ± 0% -5.83% (p=0.000 n=9+9) RenderPlanning/rows=4/renders=128-24 3.01ms ± 1% 2.75ms ± 1% -8.78% (p=0.000 n=10+10) RenderPlanning/rows=4/renders=512-24 11.6ms ± 1% 9.6ms ± 2% -17.58% (p=0.000 n=10+10) RenderPlanning/rows=4/renders=4096-24 192ms ± 2% 91ms ± 2% -52.58% (p=0.000 n=10+10) RenderPlanning/rows=16/renders=1-24 494µs ± 1% 499µs ± 1% +1.03% (p=0.006 n=10+8) RenderPlanning/rows=16/renders=8-24 855µs ± 1% 901µs ± 1% +5.37% (p=0.000 n=10+10) RenderPlanning/rows=16/renders=32-24 2.03ms ± 1% 2.04ms ± 1% ~ (p=0.190 n=10+10) RenderPlanning/rows=16/renders=64-24 3.58ms ± 1% 3.42ms ± 1% -4.56% (p=0.000 n=10+9) RenderPlanning/rows=16/renders=128-24 6.74ms ± 1% 6.31ms ± 1% -6.37% (p=0.000 n=10+10) RenderPlanning/rows=16/renders=512-24 26.9ms ± 1% 24.7ms ± 1% -8.24% (p=0.000 n=9+10) RenderPlanning/rows=16/renders=4096-24 329ms ± 2% 218ms ± 2% -33.66% (p=0.000 n=10+10) RenderPlanning/rows=64/renders=1-24 666µs ± 1% 659µs ± 2% -1.07% (p=0.007 n=10+10) RenderPlanning/rows=64/renders=8-24 1.79ms ± 1% 1.84ms ± 1% +3.01% (p=0.000 n=10+10) RenderPlanning/rows=64/renders=32-24 5.53ms ± 1% 5.79ms ± 2% +4.74% (p=0.000 n=10+10) RenderPlanning/rows=64/renders=64-24 10.8ms ± 1% 11.0ms ± 1% +1.87% (p=0.000 n=10+9) RenderPlanning/rows=64/renders=128-24 21.2ms ± 1% 21.7ms ± 1% +2.71% (p=0.000 n=10+10) RenderPlanning/rows=64/renders=512-24 83.6ms ± 0% 84.9ms ± 0% +1.47% (p=0.000 n=10+7) RenderPlanning/rows=64/renders=4096-24 824ms ± 1% 751ms ± 2% -8.88% (p=0.000 n=10+10) RenderPlanning/rows=128/renders=1-24 853µs ± 1% 851µs ± 1% ~ (p=0.481 n=10+10) RenderPlanning/rows=128/renders=8-24 2.98ms ± 1% 3.11ms ± 1% +4.32% (p=0.000 n=10+10) RenderPlanning/rows=128/renders=32-24 10.4ms ± 1% 10.9ms ± 1% +5.44% (p=0.000 n=10+10) RenderPlanning/rows=128/renders=64-24 20.1ms ± 1% 21.3ms ± 1% +5.99% (p=0.000 n=10+10) RenderPlanning/rows=128/renders=128-24 39.7ms ± 1% 42.1ms ± 2% +5.98% (p=0.000 n=10+10) RenderPlanning/rows=128/renders=512-24 160ms ± 1% 168ms ± 2% +5.13% (p=0.000 n=9+10) RenderPlanning/rows=128/renders=4096-24 1.44s ± 1% 1.48s ± 2% +3.15% (p=0.000 n=9+10) RenderPlanning/rows=256/renders=1-24 1.22ms ± 1% 1.21ms ± 1% -1.01% (p=0.000 n=10+10) RenderPlanning/rows=256/renders=8-24 5.22ms ± 0% 5.19ms ± 1% -0.54% (p=0.011 n=8+9) RenderPlanning/rows=256/renders=32-24 19.9ms ± 1% 20.0ms ± 1% ~ (p=0.182 n=9+10) RenderPlanning/rows=256/renders=64-24 39.0ms ± 0% 38.9ms ± 0% -0.33% (p=0.023 n=10+10) RenderPlanning/rows=256/renders=128-24 76.8ms ± 1% 76.7ms ± 1% ~ (p=0.739 n=10+10) RenderPlanning/rows=256/renders=512-24 316ms ± 1% 319ms ± 1% +1.15% (p=0.001 n=9+10) RenderPlanning/rows=256/renders=4096-24 2.75s ± 1% 2.73s ± 1% -0.64% (p=0.002 n=8+9) ``` Release note: None --- pkg/sql/colexec/colbuilder/BUILD.bazel | 1 + pkg/sql/colexec/colbuilder/execplan.go | 75 +++++++++++++++++- .../execbuilder/testdata/vectorize_wrapping | 78 +++++++++++++++++++ .../execbuilder/tests/local/generated_test.go | 7 ++ 4 files changed, 157 insertions(+), 4 deletions(-) create mode 100644 pkg/sql/opt/exec/execbuilder/testdata/vectorize_wrapping diff --git a/pkg/sql/colexec/colbuilder/BUILD.bazel b/pkg/sql/colexec/colbuilder/BUILD.bazel index 3206277b95a4..3e915e25dbf5 100644 --- a/pkg/sql/colexec/colbuilder/BUILD.bazel +++ b/pkg/sql/colexec/colbuilder/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "//pkg/col/coldata", "//pkg/col/coldataext", "//pkg/col/typeconv", + "//pkg/settings", "//pkg/sql/catalog/descpb", "//pkg/sql/colconv", "//pkg/sql/colexec", diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 23233989c752..5f44473dba6a 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec" @@ -1495,7 +1496,7 @@ func NewColOperator( Op: result.Root, ColumnTypes: result.ColumnTypes, } - err = ppr.planPostProcessSpec(ctx, flowCtx, args, post, factory, &r.Releasables) + err = ppr.planPostProcessSpec(ctx, flowCtx, args, post, factory, &r.Releasables, args.Spec.EstimatedRowCount) if err != nil { err = result.wrapPostProcessSpec(ctx, flowCtx, args, post, spec.ProcessorID, factory, err) } else { @@ -1647,6 +1648,46 @@ func (r opResult) wrapPostProcessSpec( ) } +// renderExprCountVisitor counts how many projection operators need to be +// planned across render expressions. +type renderExprCountVisitor struct { + renderCount int64 +} + +var _ tree.Visitor = &renderExprCountVisitor{} + +func (r *renderExprCountVisitor) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) { + if _, ok := expr.(*tree.IndexedVar); ok { + // IndexedVars don't get a projection operator (we just refer to the + // vector by index), so they don't contribute to the render count. + return false, expr + } + r.renderCount++ + return true, expr +} + +func (r *renderExprCountVisitor) VisitPost(expr tree.Expr) tree.Expr { + return expr +} + +var renderWrappingRowCountThreshold = settings.RegisterIntSetting( + settings.TenantWritable, + "sql.distsql.vectorize_render_wrapping.max_row_count", + "determines the maximum number of estimated rows that flow through the render "+ + "expressions up to which we handle those renders by wrapping a row-by-row processor", + 128, + settings.NonNegativeInt, +) + +var renderWrappingRenderCountThreshold = settings.RegisterIntSetting( + settings.TenantWritable, + "sql.distsql.vectorize_render_wrapping.min_render_count", + "determines the minimum number of render expressions for which we fall "+ + "back to handling renders by wrapping a row-by-row processor", + 16, + settings.NonNegativeInt, +) + // planPostProcessSpec plans the post processing stage specified in post on top // of r.Op. func (r *postProcessResult) planPostProcessSpec( @@ -1656,16 +1697,42 @@ func (r *postProcessResult) planPostProcessSpec( post *execinfrapb.PostProcessSpec, factory coldata.ColumnFactory, releasables *[]execreleasable.Releasable, + estimatedRowCount uint64, ) error { if post.Projection { r.Op, r.ColumnTypes = addProjection(r.Op, r.ColumnTypes, post.OutputColumns) } else if post.RenderExprs != nil { - var renderedCols []uint32 - for _, renderExpr := range post.RenderExprs { - expr, err := args.ExprHelper.ProcessExpr(renderExpr, flowCtx.EvalCtx, r.ColumnTypes) + // Deserialize expressions upfront. + exprs := make([]tree.TypedExpr, len(post.RenderExprs)) + var err error + for i := range exprs { + exprs[i], err = args.ExprHelper.ProcessExpr(post.RenderExprs[i], flowCtx.EvalCtx, r.ColumnTypes) if err != nil { return err } + } + // If we have an estimated row count and it doesn't exceed the wrapping + // row count threshold, we might need to fall back to wrapping a + // row-by-row processor to handle the render expressions (for better + // performance). + if estimatedRowCount != 0 && + estimatedRowCount <= uint64(renderWrappingRowCountThreshold.Get(&flowCtx.Cfg.Settings.SV)) { + renderCountThreshold := renderWrappingRenderCountThreshold.Get(&flowCtx.Cfg.Settings.SV) + // Walk over all expressions and estimate how many projection + // operators will need to be created. + var v renderExprCountVisitor + for _, expr := range exprs { + tree.WalkExpr(&v, expr) + if v.renderCount >= renderCountThreshold { + return errors.Newf( + "falling back to wrapping a row-by-row processor for at least "+ + "%d renders, estimated row count = %d", v.renderCount, estimatedRowCount, + ) + } + } + } + var renderedCols []uint32 + for _, expr := range exprs { var outputIdx int r.Op, outputIdx, r.ColumnTypes, err = planProjectionOperators( ctx, flowCtx.EvalCtx, expr, r.ColumnTypes, r.Op, args.StreamingMemAccount, factory, releasables, diff --git a/pkg/sql/opt/exec/execbuilder/testdata/vectorize_wrapping b/pkg/sql/opt/exec/execbuilder/testdata/vectorize_wrapping new file mode 100644 index 000000000000..29db1e965821 --- /dev/null +++ b/pkg/sql/opt/exec/execbuilder/testdata/vectorize_wrapping @@ -0,0 +1,78 @@ +# LogicTest: local + +statement ok +CREATE TABLE t85632 (k INT PRIMARY KEY); + +statement ok +ALTER TABLE t85632 INJECT STATISTICS '[ + { + "avg_size": 1, + "columns": [ + "k" + ], + "created_at": "2022-08-09 09:00:00.00000", + "distinct_count": 1000, + "name": "__auto__", + "null_count": 0, + "row_count": 1000 + } + ]' + +# Use experimental_always vectorize mode so that we error out when trying to +# wrap a row-by-row processor. +statement ok +SET vectorize=experimental_always; + +# Both the estimated row count exceeds the max wrapping row count and number of +# render is smaller that the min render count, so we use native projection +# operators. +query T +EXPLAIN (VEC) SELECT k + k + k + k FROM t85632 +---- +│ +└ Node 1 + └ *colexecproj.projPlusInt64Int64Op + └ *colexecproj.projPlusInt64Int64Op + └ *colexecproj.projPlusInt64Int64Op + └ *colfetcher.ColBatchScan + +statement ok +SET CLUSTER SETTING sql.distsql.vectorize_render_wrapping.min_render_count = 3; + +# The estimated row count still exceeds the max wrapping row count. +query T +EXPLAIN (VEC) SELECT k + k + k + k FROM t85632 +---- +│ +└ Node 1 + └ *colexecproj.projPlusInt64Int64Op + └ *colexecproj.projPlusInt64Int64Op + └ *colexecproj.projPlusInt64Int64Op + └ *colfetcher.ColBatchScan + +statement ok +SET CLUSTER SETTING sql.distsql.vectorize_render_wrapping.max_row_count = 1000; + +# Now both wrapping conditions are satisfied. +query error falling back to wrapping a row-by-row processor +EXPLAIN (VEC) SELECT k + k + k + k FROM t85632 + +statement ok +RESET CLUSTER SETTING sql.distsql.vectorize_render_wrapping.min_render_count; + +# The render count isn't sufficient for wrapping to kick in. +query T +EXPLAIN (VEC) SELECT k + k + k + k FROM t85632 +---- +│ +└ Node 1 + └ *colexecproj.projPlusInt64Int64Op + └ *colexecproj.projPlusInt64Int64Op + └ *colexecproj.projPlusInt64Int64Op + └ *colfetcher.ColBatchScan + +statement ok +RESET CLUSTER SETTING sql.distsql.vectorize_render_wrapping.max_row_count; + +statement ok +RESET vectorize diff --git a/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go b/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go index 159af9953155..26aeca7622d4 100644 --- a/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go +++ b/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go @@ -578,6 +578,13 @@ func TestExecBuild_vectorize_overloads( runExecBuildLogicTest(t, "vectorize_overloads") } +func TestExecBuild_vectorize_wrapping( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runExecBuildLogicTest(t, "vectorize_wrapping") +} + func TestExecBuild_virtual( t *testing.T, ) { From 3b404780c9de750dab1fc52abdfbdfc6f0f0a55d Mon Sep 17 00:00:00 2001 From: Jayant Shrivastava Date: Mon, 1 Aug 2022 13:18:12 -0400 Subject: [PATCH 11/11] changefeedccl: allow retries for core changefeeds Previously, core changefeeds would stop entirely due to transient errors or certain schema changes. This change adds a retry loop to the core changefeed distributed SQL workflow. This change updates related tests which omitted sinkless feeds since they could not handle schema changes. Fixes https://github.com/cockroachdb/cockroach/issues/85008 Release note (general change): Changefeeds without a specified sink will not longer terminate when schema changes occur. --- .../changefeedccl/changefeed_processors.go | 2 +- pkg/ccl/changefeedccl/changefeed_stmt.go | 49 +++++++--- pkg/ccl/changefeedccl/changefeed_test.go | 89 +++++++------------ .../changefeedccl/changefeedbase/BUILD.bazel | 1 + .../changefeedccl/changefeedbase/errors.go | 27 +++++- 5 files changed, 101 insertions(+), 67 deletions(-) diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index eac7ee4217c9..d52a9bf9ec5f 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -1038,7 +1038,7 @@ func (cf *changeFrontier) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetad // Detect whether this boundary should be used to kill or restart the // changefeed. if cf.frontier.boundaryType == jobspb.ResolvedSpan_RESTART { - err = changefeedbase.MarkRetryableError(err) + err = changefeedbase.MarkRetryableErrorWithTimestamp(err, cf.frontier.boundaryTime) } } diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index f23409079b1e..960449fe80d0 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -57,6 +57,12 @@ import ( "github.com/cockroachdb/errors" ) +var changefeedRetryOptions = retry.Options{ + InitialBackoff: 5 * time.Millisecond, + Multiplier: 2, + MaxBackoff: 10 * time.Second, +} + // featureChangefeedEnabled is used to enable and disable the CHANGEFEED feature. var featureChangefeedEnabled = settings.RegisterBoolSetting( settings.TenantWritable, @@ -196,10 +202,38 @@ func changefeedPlanHook( telemetry.Count(`changefeed.create.core`) logChangefeedCreateTelemetry(ctx, jr) - err := distChangefeedFlow(ctx, p, 0 /* jobID */, details, progress, resultsCh) - if err != nil { - telemetry.Count(`changefeed.core.error`) + + var err error + for r := retry.StartWithCtx(ctx, changefeedRetryOptions); r.Next(); { + if err = distChangefeedFlow(ctx, p, 0 /* jobID */, details, progress, resultsCh); err == nil { + return nil + } + + if knobs, ok := p.ExecCfg().DistSQLSrv.TestingKnobs.Changefeed.(*TestingKnobs); ok { + if knobs != nil && knobs.HandleDistChangefeedError != nil { + err = knobs.HandleDistChangefeedError(err) + } + } + + if !changefeedbase.IsRetryableError(err) { + log.Warningf(ctx, `CHANGEFEED returning with error: %+v`, err) + return err + } + + // Check for a schemachange boundary timestamp returned via a + // retryable error. Retrying without updating the changefeed progress + // will result in the changefeed performing the schema change again, + // causing an infinite loop. + if ts, ok := changefeedbase.MaybeGetRetryableErrorTimestamp(err); ok { + progress = jobspb.Progress{ + Progress: &jobspb.Progress_HighWater{HighWater: &ts}, + Details: &jobspb.Progress_Changefeed{ + Changefeed: &jobspb.ChangefeedProgress{}, + }, + } + } } + telemetry.Count(`changefeed.core.error`) return changefeedbase.MaybeStripRetryableErrorMarker(err) } @@ -934,15 +968,10 @@ func (b *changefeedResumer) resumeWithRetries( // bubbles up to this level, we'd like to "retry" the flow if possible. This // could be because the sink is down or because a cockroach node has crashed // or for many other reasons. - opts := retry.Options{ - InitialBackoff: 5 * time.Millisecond, - Multiplier: 2, - MaxBackoff: 10 * time.Second, - } var err error var lastRunStatusUpdate time.Time - for r := retry.StartWithCtx(ctx, opts); r.Next(); { + for r := retry.StartWithCtx(ctx, changefeedRetryOptions); r.Next(); { // startedCh is normally used to signal back to the creator of the job that // the job has started; however, in this case nothing will ever receive // on the channel, causing the changefeed flow to block. Replace it with @@ -959,7 +988,7 @@ func (b *changefeedResumer) resumeWithRetries( } } - // Retry changefeed is error is retryable. In addition, we want to handle + // Retry changefeed if error is retryable. In addition, we want to handle // context cancellation as retryable, but only if the resumer context has not been cancelled. // (resumer context is canceled by the jobs framework -- so we should respect it). isRetryableErr := changefeedbase.IsRetryableError(err) || diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 92f236fb43e6..5948dedaaf92 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -1027,7 +1027,7 @@ func TestNoStopAfterNonTargetColumnDrop(t *testing.T) { } } - cdcTest(t, testFn, feedTestOmitSinks("sinkless")) + cdcTest(t, testFn) } // If we drop columns which are not targeted by the changefeed, it should not backfill. @@ -1060,7 +1060,7 @@ func TestNoBackfillAfterNonTargetColumnDrop(t *testing.T) { }) } - cdcTest(t, testFn, feedTestOmitSinks("sinkless")) + cdcTest(t, testFn) } func TestChangefeedColumnDropsWithFamilyAndNonFamilyTargets(t *testing.T) { @@ -1105,7 +1105,7 @@ func TestChangefeedColumnDropsWithFamilyAndNonFamilyTargets(t *testing.T) { }) } - cdcTest(t, testFn, feedTestOmitSinks("sinkless")) + cdcTest(t, testFn) } func TestChangefeedColumnDropsOnMultipleFamiliesWithTheSameName(t *testing.T) { @@ -1150,7 +1150,7 @@ func TestChangefeedColumnDropsOnMultipleFamiliesWithTheSameName(t *testing.T) { }) } - cdcTest(t, testFn, feedTestOmitSinks("sinkless")) + cdcTest(t, testFn) } func TestChangefeedColumnDropsOnTheSameTableWithMultipleFamilies(t *testing.T) { @@ -1182,7 +1182,7 @@ func TestChangefeedColumnDropsOnTheSameTableWithMultipleFamilies(t *testing.T) { }) } - cdcTest(t, testFn, feedTestOmitSinks("sinkless")) + cdcTest(t, testFn) } func TestChangefeedExternalIODisabled(t *testing.T) { @@ -5294,36 +5294,6 @@ func TestChangefeedPrimaryKeyChangeWorks(t *testing.T) { foo := feed(t, f, baseStmt) defer closeFeed(t, foo) - // maybeHandleRestart deals with the fact that sinkless changefeeds don't - // gracefully handle primary index changes but rather force the client to - // deal with restarting the changefeed as of the last resolved timestamp. - // - // This ends up being pretty sane; sinkless changefeeds already require this - // behavior in the face of other transient failures so clients already need - // to implement this logic. - maybeHandleRestart := func(t *testing.T) (cleanup func()) { - return func() {} - } - if strings.HasSuffix(t.Name(), "sinkless") { - maybeHandleRestart = func(t *testing.T) func() { - var resolved hlc.Timestamp - for { - m, err := foo.Next() - if err != nil { - assert.Contains(t, err.Error(), - fmt.Sprintf("schema change occurred at %s", resolved.Next().AsOfSystemTime())) - break - } - resolved = extractResolvedTimestamp(t, m) - } - const restartStmt = baseStmt + ", cursor = $1" - foo = feed(t, f, restartStmt, resolved.AsOfSystemTime()) - return func() { - closeFeed(t, foo) - } - } - } - // 'initial' is skipped because only the latest value ('updated') is // emitted by the initial scan. assertPayloads(t, foo, []string{ @@ -5337,7 +5307,6 @@ func TestChangefeedPrimaryKeyChangeWorks(t *testing.T) { }) sqlDB.Exec(t, `ALTER TABLE foo ALTER PRIMARY KEY USING COLUMNS (b)`) - defer maybeHandleRestart(t)() sqlDB.Exec(t, `INSERT INTO foo VALUES (3, 'c'), (4, 'd')`) assertPayloads(t, foo, []string{ `foo: ["c"]->{"after": {"a": 3, "b": "c"}}`, @@ -5360,7 +5329,6 @@ INSERT INTO foo VALUES (1, 'f'); `foo: ["a"]->{"after": {"a": 6, "b": "a"}}`, `foo: ["e"]->{"after": {"a": 5, "b": "e"}}`, }) - defer maybeHandleRestart(t)() assertPayloads(t, foo, []string{ `foo: [1]->{"after": {"a": 1, "b": "f"}}`, }) @@ -5406,24 +5374,6 @@ func TestChangefeedPrimaryKeyChangeWorksWithMultipleTables(t *testing.T) { maybeHandleRestart := func(t *testing.T) (cleanup func()) { return func() {} } - if strings.HasSuffix(t.Name(), "sinkless") { - maybeHandleRestart = func(t *testing.T) func() { - var resolvedTS hlc.Timestamp - for { - m, err := cf.Next() - if err != nil { - assert.Contains(t, err.Error(), fmt.Sprintf("schema change occurred at %s", resolvedTS.Next().AsOfSystemTime())) - break - } - resolvedTS = extractResolvedTimestamp(t, m) - } - const restartStmt = baseStmt + ", cursor = $1" - cf = feed(t, f, restartStmt, resolvedTS.AsOfSystemTime()) - return func() { - closeFeed(t, cf) - } - } - } // 'initial' is skipped because only the latest value ('updated') is // emitted by the initial scan. @@ -7110,3 +7060,32 @@ func TestChangefeedTestTimesOut(t *testing.T) { cdcTest(t, testFn) } + +// Regression for #85008. +func TestSchemachangeDoesNotBreakSinklessFeed(t *testing.T) { + defer leaktest.AfterTest(t)() + testFn := func(t *testing.T, s TestServer, f cdctest.TestFeedFactory) { + sqlDB := sqlutils.MakeSQLRunner(s.DB) + + sqlDB.Exec(t, `CREATE TABLE mytable (id INT PRIMARY KEY)`) + sqlDB.Exec(t, `INSERT INTO mytable VALUES (0)`) + + // Open up the changefeed. + cf := feed(t, f, `CREATE CHANGEFEED FOR TABLE mytable`) + defer closeFeed(t, cf) + assertPayloads(t, cf, []string{ + `mytable: [0]->{"after": {"id": 0}}`, + }) + + sqlDB.Exec(t, `ALTER TABLE mytable ADD COLUMN val INT DEFAULT 0`) + assertPayloads(t, cf, []string{ + `mytable: [0]->{"after": {"id": 0, "val": 0}}`, + }) + sqlDB.Exec(t, `INSERT INTO mytable VALUES (1,1)`) + assertPayloads(t, cf, []string{ + `mytable: [1]->{"after": {"id": 1, "val": 1}}`, + }) + } + + cdcTest(t, testFn, feedTestForceSink("sinkless")) +} diff --git a/pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel b/pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel index 64d92e46876e..f34ecdaffd42 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel +++ b/pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "//pkg/sql", "//pkg/sql/catalog/descpb", "//pkg/sql/flowinfra", + "//pkg/util/hlc", "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/ccl/changefeedccl/changefeedbase/errors.go b/pkg/ccl/changefeedccl/changefeedbase/errors.go index a10415b4ad4c..5dd16ca3debe 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/errors.go +++ b/pkg/ccl/changefeedccl/changefeedbase/errors.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" ) @@ -77,7 +78,14 @@ func (e *taggedError) Unwrap() error { return e.wrapped } const retryableErrorString = "retryable changefeed error" type retryableError struct { - wrapped error + // A schema change may result in a changefeed returning retryableError, + // which can signal the changefeed to restart. + // boundaryTimestamp can be returned inside this error so + // the changefeed knows where to restart from. Note this is + // only useful for sinkless/core changefeeds because they do not have + // the ability to read/write their state to jobs tables during restarts. + boundaryTimestamp hlc.Timestamp + wrapped error } // MarkRetryableError wraps the given error, marking it as retryable to @@ -86,6 +94,12 @@ func MarkRetryableError(e error) error { return &retryableError{wrapped: e} } +// MarkRetryableErrorWithTimestamp wraps the given error, marks it as +// retryable, and attaches a timestamp to the error. +func MarkRetryableErrorWithTimestamp(e error, ts hlc.Timestamp) error { + return &retryableError{boundaryTimestamp: ts, wrapped: e} +} + // Error implements the error interface. func (e *retryableError) Error() string { return fmt.Sprintf("%s: %s", retryableErrorString, e.wrapped.Error()) @@ -125,6 +139,17 @@ func IsRetryableError(err error) bool { errors.Is(err, sql.ErrPlanChanged)) } +// MaybeGetRetryableErrorTimestamp will get the timestamp of an error if +// the error is a retryableError and the timestamp field is populated. +func MaybeGetRetryableErrorTimestamp(err error) (timestamp hlc.Timestamp, ok bool) { + if retryableErr := (*retryableError)(nil); errors.As(err, &retryableErr) { + if !retryableErr.boundaryTimestamp.IsEmpty() { + return retryableErr.boundaryTimestamp, true + } + } + return hlc.Timestamp{}, false +} + // MaybeStripRetryableErrorMarker performs some minimal attempt to clean the // RetryableError marker out. This won't do anything if the RetryableError // itself has been wrapped, but that's okay, we'll just have an uglier string.