diff --git a/pkg/ccl/schemachangerccl/testdata/decomp/multiregion b/pkg/ccl/schemachangerccl/testdata/decomp/multiregion index 03460d38b596..db268ec72ea1 100644 --- a/pkg/ccl/schemachangerccl/testdata/decomp/multiregion +++ b/pkg/ccl/schemachangerccl/testdata/decomp/multiregion @@ -190,37 +190,31 @@ ElementState: tableId: 110 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: b tableId: 110 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: rowid tableId: 110 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 110 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 110 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 110 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 110 @@ -591,37 +585,31 @@ ElementState: tableId: 109 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: a tableId: 109 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: rowid tableId: 109 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 109 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 109 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 109 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 109 @@ -1021,43 +1009,36 @@ ElementState: tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: k tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: v tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 3 name: crdb_region tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 108 diff --git a/pkg/ccl/schemachangerccl/testdata/decomp/partitioning b/pkg/ccl/schemachangerccl/testdata/decomp/partitioning index b0bf4475831e..f355ae5926be 100644 --- a/pkg/ccl/schemachangerccl/testdata/decomp/partitioning +++ b/pkg/ccl/schemachangerccl/testdata/decomp/partitioning @@ -111,43 +111,36 @@ ElementState: tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: pk tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: a tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 3 name: j tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 104 @@ -631,37 +624,31 @@ ElementState: tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: a tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: b tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 105 diff --git a/pkg/cmd/roachtest/registry/test_spec.go b/pkg/cmd/roachtest/registry/test_spec.go index 6342e17b6122..a61554cc0c02 100644 --- a/pkg/cmd/roachtest/registry/test_spec.go +++ b/pkg/cmd/roachtest/registry/test_spec.go @@ -244,10 +244,14 @@ const ( LeaderLeases // MetamorphicLeases randomly chooses epoch or expiration // leases (across the entire cluster). - // TODO(nvanbenschoten): add leader leases to this mix. MetamorphicLeases ) +// LeaseTypes contains all lease types. +// +// The list does not contain aliases like "default" and "metamorphic". +var LeaseTypes = []LeaseType{EpochLeases, ExpirationLeases, LeaderLeases} + // CloudSet represents a set of clouds. // // Instances of CloudSet are immutable. The uninitialized (zero) value is not diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 8b7f19bdecd8..56e228a1582d 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -900,7 +900,18 @@ func (r *testRunner) runWorker( c.clusterSettings = map[string]string{} c.virtualClusterSettings = map[string]string{} - switch testSpec.Leases { + leases := testSpec.Leases + if leases == registry.MetamorphicLeases { + // 50% change of using the default lease type, 50% change of choosing + // a random, specific lease type. + if prng.Intn(2) == 0 { + leases = registry.DefaultLeases + } else { + leases = registry.LeaseTypes[prng.Intn(len(registry.LeaseTypes))] + } + c.status(fmt.Sprintf("metamorphically using %s leases", leases)) + } + switch leases { case registry.DefaultLeases: case registry.EpochLeases: c.clusterSettings["kv.expiration_leases_only.enabled"] = "false" @@ -911,12 +922,9 @@ func (r *testRunner) runWorker( c.clusterSettings["kv.expiration_leases_only.enabled"] = "false" c.clusterSettings["kv.raft.leader_fortification.fraction_enabled"] = "1.0" case registry.MetamorphicLeases: - enabled := prng.Float64() < 0.5 - c.status(fmt.Sprintf("metamorphically setting kv.expiration_leases_only.enabled = %t", - enabled)) - c.clusterSettings["kv.expiration_leases_only.enabled"] = fmt.Sprintf("%t", enabled) + t.Fatalf("metamorphic leases handled above") default: - t.Fatalf("unknown lease type %s", testSpec.Leases) + t.Fatalf("unknown lease type %s", leases) } c.goCoverDir = t.GoCoverArtifactsDir() diff --git a/pkg/cmd/roachtest/tests/failover.go b/pkg/cmd/roachtest/tests/failover.go index 823c020a8cbf..11b0170e7d60 100644 --- a/pkg/cmd/roachtest/tests/failover.go +++ b/pkg/cmd/roachtest/tests/failover.go @@ -59,8 +59,7 @@ var rangeLeaseRenewalDuration = func() time.Duration { // requests are successful with nominal latencies. See also: // https://github.com/cockroachdb/cockroach/issues/103654 func registerFailover(r registry.Registry) { - leaseTypes := []registry.LeaseType{registry.EpochLeases, registry.ExpirationLeases, registry.LeaderLeases} - for _, leases := range leaseTypes { + for _, leases := range registry.LeaseTypes { var leasesStr string switch leases { case registry.EpochLeases: diff --git a/pkg/cmd/roachtest/tests/kv.go b/pkg/cmd/roachtest/tests/kv.go index 82bd0f887cb7..6297e877a9aa 100644 --- a/pkg/cmd/roachtest/tests/kv.go +++ b/pkg/cmd/roachtest/tests/kv.go @@ -719,15 +719,22 @@ func registerKVSplits(r registry.Registry) { // far past the number of replicas per node we support, at least if the // ranges start to unquiesce (which can set off a cascade due to resource // exhaustion). - {true, 300000, registry.EpochLeases, 2 * time.Hour}, + {true, 300_000, registry.EpochLeases, 2 * time.Hour}, // This version of the test prevents range quiescence to trigger the // badness described above more reliably for when we wish to improve // the performance. For now, just verify that 30k unquiesced ranges // is tenable. - {false, 30000, registry.EpochLeases, 2 * time.Hour}, + {false, 30_000, registry.EpochLeases, 2 * time.Hour}, // Expiration-based leases prevent quiescence, and are also more expensive // to keep alive. Again, just verify that 30k ranges is ok. - {false, 30000, registry.ExpirationLeases, 2 * time.Hour}, + {false, 30_000, registry.ExpirationLeases, 2 * time.Hour}, + // Leader leases don't need quiescence, as they use store liveness for + // failure detection and lease extension, so they don't issue raft + // heartbeats or periodic lease extensions. However, the cost of raft + // ticking is not entirely negligible (see #133885), so each range isn't + // completely free. Currently, they should be able to support 80k ranges in + // this cluster configuration. + {false, 80_000, registry.LeaderLeases, 2 * time.Hour}, } { item := item // for use in closure below r.Add(registry.TestSpec{ diff --git a/pkg/server/license/BUILD.bazel b/pkg/server/license/BUILD.bazel index 59b40379c335..521eff49a706 100644 --- a/pkg/server/license/BUILD.bazel +++ b/pkg/server/license/BUILD.bazel @@ -24,6 +24,7 @@ go_library( "//pkg/util/syncutil", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", + "@com_github_cockroachdb_redact//:redact", ], ) diff --git a/pkg/server/license/cclbridge.go b/pkg/server/license/cclbridge.go index bef19408b23d..2031dae86eca 100644 --- a/pkg/server/license/cclbridge.go +++ b/pkg/server/license/cclbridge.go @@ -9,6 +9,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/redact" ) // This file serves as a bridge to the license code in the CCL packages. @@ -24,6 +25,11 @@ var RegisterCallbackOnLicenseChange = func(context.Context, *cluster.Settings, * // enforcer. type LicType int +var _ redact.SafeValue = LicType(0) + +// SafeValue implements the redact.SafeValue interface. +func (i LicType) SafeValue() {} + //go:generate stringer -type=LicType -linecomment const ( LicTypeNone LicType = iota // none diff --git a/pkg/server/license/enforcer.go b/pkg/server/license/enforcer.go index 33ee54693a41..c5fe4b81529d 100644 --- a/pkg/server/license/enforcer.go +++ b/pkg/server/license/enforcer.go @@ -7,8 +7,6 @@ package license import ( "context" - "fmt" - "strings" "sync/atomic" "time" @@ -24,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" ) const ( @@ -231,7 +230,7 @@ func (e *Enforcer) initClusterMetadata(ctx context.Context, options options) err end := e.metadataAccessor.GetClusterInitGracePeriodTS() if end != 0 { e.clusterInitGracePeriodEndTS.Store(end) - log.Infof(ctx, "fetched cluster init grace period end time from system tenant: %s", e.GetClusterInitGracePeriodEndTS().String()) + log.Infof(ctx, "fetched cluster init grace period end time from system tenant: %s", e.GetClusterInitGracePeriodEndTS()) } else { // No timestamp was received, likely due to a mixed-version workload. // We'll use an estimate of 7 days from this node's startup time instead @@ -239,7 +238,7 @@ func (e *Enforcer) initClusterMetadata(ctx context.Context, options options) err // An update should be sent once the KV starts up on the new version. gracePeriodLength := e.getGracePeriodDuration(7 * 24 * time.Hour) e.clusterInitGracePeriodEndTS.Store(e.getStartTime().Add(gracePeriodLength).Unix()) - log.Infof(ctx, "estimated cluster init grace period end time as: %s", e.GetClusterInitGracePeriodEndTS().String()) + log.Infof(ctx, "estimated cluster init grace period end time as: %s", e.GetClusterInitGracePeriodEndTS()) e.continueToPollMetadataAccessor.Store(true) } return nil @@ -291,12 +290,12 @@ func (e *Enforcer) initClusterMetadata(ctx context.Context, options options) err } gracePeriodLength = e.getGracePeriodDuration(gracePeriodLength) // Allow the value to be shortened by env var end := e.getStartTime().Add(gracePeriodLength) - log.Infof(ctx, "generated new cluster init grace period end time: %s", end.UTC().String()) + log.Infof(ctx, "generated new cluster init grace period end time: %s", end.UTC()) e.clusterInitGracePeriodEndTS.Store(end.Unix()) return txn.KV().Put(ctx, keys.ClusterInitGracePeriodTimestamp, e.clusterInitGracePeriodEndTS.Load()) } e.clusterInitGracePeriodEndTS.Store(val.ValueInt()) - log.Infof(ctx, "fetched existing cluster init grace period end time: %s", e.GetClusterInitGracePeriodEndTS().String()) + log.Infof(ctx, "fetched existing cluster init grace period end time: %s", e.GetClusterInitGracePeriodEndTS()) return nil }) } @@ -485,18 +484,18 @@ func (e *Enforcer) RefreshForLicenseChange( e.licenseRequiresTelemetry.Store(false) } - var sb strings.Builder - sb.WriteString(fmt.Sprintf("enforcer license updated: type is %s, ", licType.String())) + var sb redact.StringBuilder + sb.Printf("enforcer license updated: type is %s, ", licType) gpEnd, _ := e.GetGracePeriodEndTS() if !gpEnd.IsZero() { - sb.WriteString(fmt.Sprintf("grace period ends at %q, ", gpEnd)) + sb.Printf("grace period ends at %q, ", gpEnd) } expiry := timeutil.Unix(e.licenseExpiryTS.Load(), 0) if !expiry.IsZero() { - sb.WriteString(fmt.Sprintf("expiration at %q, ", expiry)) + sb.Printf("expiration at %q, ", expiry) } - sb.WriteString(fmt.Sprintf("telemetry required: %t", e.licenseRequiresTelemetry.Load())) - log.Infof(ctx, "%s", sb.String()) + sb.Printf("telemetry required: %t", e.licenseRequiresTelemetry.Load()) + log.Infof(ctx, "%s", sb.RedactableString()) } // UpdateTrialLicenseExpiry returns the expiration timestamp of any trial license @@ -730,7 +729,7 @@ func (e *Enforcer) pollMetadataAccessor(ctx context.Context) { e.storeNewGracePeriodEndDate(e.GetClusterInitGracePeriodEndTS(), 0) e.continueToPollMetadataAccessor.Store(false) log.Infof(ctx, "late retrieval of cluster initialization grace period end time from system tenant: %s", - e.GetClusterInitGracePeriodEndTS().String()) + e.GetClusterInitGracePeriodEndTS()) } } } diff --git a/pkg/sql/schemachanger/dml_injection_test.go b/pkg/sql/schemachanger/dml_injection_test.go index bf9a037fd035..50f30650ea71 100644 --- a/pkg/sql/schemachanger/dml_injection_test.go +++ b/pkg/sql/schemachanger/dml_injection_test.go @@ -186,6 +186,7 @@ func TestAlterTableDMLInjection(t *testing.T) { "ALTER TABLE tbl ADD COLUMN new_col BIGINT NOT NULL DEFAULT 100", }, schemaChange: "ALTER TABLE tbl ALTER COLUMN new_col SET DATA TYPE TEXT", + query: "SELECT new_col FROM tbl LIMIT 1", }, { desc: "add column default udf", diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_column_type.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_column_type.go index d6678cc02670..a34de65a447c 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_column_type.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_column_type.go @@ -13,7 +13,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -302,8 +301,10 @@ func handleGeneralColumnConversion( oldColComment, newColComment := getColumnCommentForColumnReplacement(b, tbl.TableID, col.ColumnID, newColID) - // First set the target status of the old column to drop. We will replace this - // column with a new column. This column stays visible until the second backfill. + // First, set the target status of the old column to drop. This column will be + // replaced by a new one but remains visible until the new column is ready to be + // made public. A dependency rule ensures that the old and new columns are swapped + // within the same stage. b.Drop(col) b.Drop(colName) b.Drop(oldColType) @@ -337,7 +338,7 @@ func handleGeneralColumnConversion( // Add the spec for the new column. It will be identical to the column it is replacing, // except the type will differ, and it will have a transient computed expression. // This expression will reference the original column to facilitate the backfill. - // This column becomes visible after the first backfill. + // This column becomes visible in the same stage the old column becomes invisible. spec := addColumnSpec{ tbl: tbl, col: &scpb.Column{ @@ -370,24 +371,6 @@ func handleGeneralColumnConversion( fam: nil, } addColumn(b, spec, t) - - // The above operation will cause a backfill to occur twice. Once with both columns, - // then another time with the old column removed. Since both columns will exist at - // the same time for a short period of time, we need to rename the old column so that - // we can access either one. We add this name as a transient so that it is cleaned up - // prior to the old column being totally removed. - nameExists := func(name string) bool { - return getColumnIDFromColumnName(b, tbl.TableID, tree.Name(name), false /* required */) != 0 - } - oldColumnRename := tabledesc.GenerateUniqueName(fmt.Sprintf("%s_shadow", colName.Name), nameExists) - b.AddTransient(&scpb.ColumnName{ - TableID: tbl.TableID, - ColumnID: col.ColumnID, - Name: oldColumnRename, - // If we don't complete the operation, the column won't be dropped, so we - // need to remember the original name to preserve it. - AbsentName: colName.Name, - }) } func updateColumnType(b BuildCtx, oldColType, newColType *scpb.ColumnType) { diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_alter_column_type b/pkg/sql/schemachanger/scbuild/testdata/alter_table_alter_column_type index 6e02239b8104..23341f285b4e 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_alter_column_type +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_alter_column_type @@ -107,5 +107,3 @@ ALTER TABLE t ALTER COLUMN c2 SET DATA TYPE BIGINT USING c2::BIGINT {columnId: 4, indexId: 4, kind: STORED, ordinalInKind: 1, tableId: 104} - [[IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 5}, TRANSIENT_ABSENT], ABSENT] {columnId: 4, indexId: 5, kind: STORED, ordinalInKind: 1, tableId: 104} -- [[ColumnName:{DescID: 104, Name: c2_shadow, ColumnID: 2}, TRANSIENT_ABSENT], ABSENT] - {absentName: c2, columnId: 2, name: c2_shadow, tableId: 104} diff --git a/pkg/sql/schemachanger/scdecomp/testdata/other b/pkg/sql/schemachanger/scdecomp/testdata/other index 36aa46fa9121..a404173d9dab 100644 --- a/pkg/sql/schemachanger/scdecomp/testdata/other +++ b/pkg/sql/schemachanger/scdecomp/testdata/other @@ -285,31 +285,26 @@ ElementState: tableId: 112 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: id tableId: 112 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 112 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 112 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 112 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 112 @@ -570,37 +565,31 @@ ElementState: tableId: 113 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: k tableId: 113 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: id tableId: 113 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 113 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 113 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 113 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 113 diff --git a/pkg/sql/schemachanger/scdecomp/testdata/sequence b/pkg/sql/schemachanger/scdecomp/testdata/sequence index 1c83e8c70f7c..23c3e658b25d 100644 --- a/pkg/sql/schemachanger/scdecomp/testdata/sequence +++ b/pkg/sql/schemachanger/scdecomp/testdata/sequence @@ -186,37 +186,31 @@ ElementState: tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: id tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: cexpr tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 105 @@ -591,37 +585,31 @@ ElementState: tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: id tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: cexpr tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 105 diff --git a/pkg/sql/schemachanger/scdecomp/testdata/table b/pkg/sql/schemachanger/scdecomp/testdata/table index 2fe2002d23e1..07408e6896ff 100644 --- a/pkg/sql/schemachanger/scdecomp/testdata/table +++ b/pkg/sql/schemachanger/scdecomp/testdata/table @@ -85,31 +85,26 @@ ElementState: tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: id tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 104 @@ -466,43 +461,36 @@ ElementState: tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: id tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: name tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 3 name: price tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 105 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 105 @@ -1072,37 +1060,31 @@ ElementState: tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: id tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: j tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 104 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 104 @@ -1468,37 +1450,31 @@ ElementState: tableId: 109 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: v tableId: 109 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: rowid tableId: 109 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 109 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 109 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 109 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 109 diff --git a/pkg/sql/schemachanger/scdecomp/testdata/type b/pkg/sql/schemachanger/scdecomp/testdata/type index 28ae2c935ba8..a03d71bb5c8a 100644 --- a/pkg/sql/schemachanger/scdecomp/testdata/type +++ b/pkg/sql/schemachanger/scdecomp/testdata/type @@ -209,55 +209,46 @@ ElementState: tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: id tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: g tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 3 name: s tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4 name: other tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 108 Status: PUBLIC - ColumnName: - absentName: "" columnId: 5 name: name tableId: 108 @@ -1014,67 +1005,56 @@ ElementState: tableId: 111 Status: PUBLIC - ColumnName: - absentName: "" columnId: 1 name: id tableId: 111 Status: PUBLIC - ColumnName: - absentName: "" columnId: 2 name: c tableId: 111 Status: PUBLIC - ColumnName: - absentName: "" columnId: 3 name: cs tableId: 111 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4 name: cstored tableId: 111 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967292e+09 name: crdb_internal_origin_timestamp tableId: 111 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967293e+09 name: crdb_internal_origin_id tableId: 111 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967294e+09 name: tableoid tableId: 111 Status: PUBLIC - ColumnName: - absentName: "" columnId: 4.294967295e+09 name: crdb_internal_mvcc_timestamp tableId: 111 Status: PUBLIC - ColumnName: - absentName: "" columnId: 5 name: cvirtual tableId: 111 Status: PUBLIC - ColumnName: - absentName: "" columnId: 6 name: name tableId: 111 Status: PUBLIC - ColumnName: - absentName: "" columnId: 7 name: crdb_internal_idx_expr tableId: 111 diff --git a/pkg/sql/schemachanger/scpb/elements.proto b/pkg/sql/schemachanger/scpb/elements.proto index 410e7e421cc8..ed2c571f385c 100644 --- a/pkg/sql/schemachanger/scpb/elements.proto +++ b/pkg/sql/schemachanger/scpb/elements.proto @@ -645,9 +645,6 @@ message ColumnName { uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; uint32 column_id = 2 [(gogoproto.customname) = "ColumnID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.ColumnID"]; string name = 3; - // AbsentName is the name to use when ColumnName transitions to ABSENT. If this - // is omitted, a placeholder name is used. - string absent_name = 4; } message IndexName { diff --git a/pkg/sql/schemachanger/scpb/uml/table.puml b/pkg/sql/schemachanger/scpb/uml/table.puml index 8d905faf109f..8351452ec662 100644 --- a/pkg/sql/schemachanger/scpb/uml/table.puml +++ b/pkg/sql/schemachanger/scpb/uml/table.puml @@ -62,7 +62,6 @@ object ColumnName ColumnName : TableID ColumnName : ColumnID ColumnName : Name -ColumnName : AbsentName object ColumnNotNull diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_column_name.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_column_name.go index 1284606efd1a..ee0c23527906 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_column_name.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_column_name.go @@ -26,7 +26,6 @@ func init() { }), ), ), - toTransientAbsentLikePublic(), toAbsent( scpb.Status_PUBLIC, to(scpb.Status_ABSENT, @@ -36,10 +35,6 @@ func init() { ColumnID: this.ColumnID, Name: tabledesc.ColumnNamePlaceholder(this.ColumnID), } - // If a name was provided for the transition to absent, override the placeholder. - if this.AbsentName != "" { - op.Name = this.AbsentName - } return op }), ), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_column.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_column.go index 7811e0e18819..2a6127a97892 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_column.go @@ -27,6 +27,25 @@ func init() { to.TypeFilter(rulesVersionKey, isColumnDependent), JoinOnColumnID(from, to, "table-id", "col-id"), StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_PUBLIC), + IsNotAlterColumnTypeOp("table-id", "col-id"), + } + }, + ) + + // This rule is similar to the one above but omits the column name. This is specific + // to ALTER COLUMN ... TYPE, which replaces a column with the same name. The column + // name is assigned during the swap, as it replaces the old column. + registerDepRule( + "column existence precedes column dependents during an alter column type", + scgraph.Precedence, + "column", "dependent", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.Column)(nil)), + to.TypeFilter(rulesVersionKey, isColumnDependentExceptColumnName), + JoinOnColumnID(from, to, "table-id", "col-id"), + StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_PUBLIC), + rel.And(IsAlterColumnTypeOp("table-id", "col-id")...), } }, ) @@ -49,18 +68,57 @@ func init() { // Special cases of the above. func init() { registerDepRule( - "column name and type set right after column existence", + "column type set right after column existence", + scgraph.SameStagePrecedence, + "column", "column-type", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.Column)(nil)), + to.Type((*scpb.ColumnType)(nil)), + StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_PUBLIC), + JoinOnColumnID(from, to, "table-id", "col-id"), + } + }, + ) + + // There are two distinct rules for transitioning the column name to public, + // depending on whether an ALTER COLUMN TYPE operation is in progress. + // This rule applies when an ALTER COLUMN TYPE is *not* occurring and is + // identical to the prior rule for ColumnType, except that it applies to ColumnName. + registerDepRule( + "column name set right after column existence, except for alter column type", scgraph.SameStagePrecedence, "column", "column-name-or-type", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.Column)(nil)), - to.Type( - (*scpb.ColumnName)(nil), - (*scpb.ColumnType)(nil), - ), + to.Type((*scpb.ColumnName)(nil)), StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_PUBLIC), JoinOnColumnID(from, to, "table-id", "col-id"), + IsNotAlterColumnTypeOp("table-id", "col-id"), + } + }, + ) + + // This rule is similar to the previous one but applies during an ALTER COLUMN TYPE operation. + // In this scenario, we are replacing one column with another, both of which share the same name. + // To prevent any period where a column with that name is not public, we ensure that the column + // names are swapped in the same stage. + registerDepRule( + "during alter column type, column names for old and new columns are swapped in the same stage", + scgraph.SameStagePrecedence, + "old-column-name", "new-column-name", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.ColumnName)(nil)), + to.Type((*scpb.ColumnName)(nil)), + from.TargetStatus(scpb.ToAbsent), + from.CurrentStatus(scpb.Status_ABSENT), + to.TargetStatus(scpb.ToPublic), + to.CurrentStatus(scpb.Status_PUBLIC), + JoinOnDescID(from, to, "table-id"), + to.El.AttrEqVar(screl.ColumnID, "new-col-id"), + rel.And(IsAlterColumnTypeOp("table-id", "new-col-id")...), } }, ) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_column.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_column.go index b2c0560030eb..92efde464f70 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_column.go @@ -202,13 +202,39 @@ func init() { // storing all public columns within the table (as the column being dropped is still considered public // before it moves to WRITE_ONLY but the new primary index does not contain it since the schema changer // knows it is transitioning to a target status of ABSENT). + // + // This rule applies only when the operation is not ALTER COLUMN TYPE. A variant of this rule follows, + // allowing added and dropped columns to be swapped in the same stage during ALTER COLUMN TYPE. registerDepRule( "New primary index should go public only after columns being dropped move to WRITE_ONLY", scgraph.Precedence, "column", "new-primary-index", func(from, to NodeVars) rel.Clauses { ic := MkNodeVars("index-column") - relationID, columnID, indexID := rel.Var("table-id"), rel.Var("column-id"), rel.Var("index-id") + relationID, columnID, indexID := rel.Var("table-id"), rel.Var("old-column-id"), rel.Var("index-id") + return rel.Clauses{ + from.Type((*scpb.Column)(nil)), + to.Type((*scpb.PrimaryIndex)(nil)), + ColumnInSourcePrimaryIndex(ic, to, relationID, columnID, indexID), + JoinOnColumnID(ic, from, relationID, columnID), + from.TargetStatus(scpb.ToAbsent), + from.CurrentStatus(scpb.Status_WRITE_ONLY), + to.TargetStatus(scpb.ToPublic), + to.CurrentStatus(scpb.Status_PUBLIC), + IsNotDroppedColumnPartOfAlterColumnTypeOp("table-id", "old-column-id"), + } + }, + ) + + // This rule is similar to the previous one but applies specifically to ALTER COLUMN ... TYPE operations. + // It uses SameStagePrecedence to enable the swapping of dropped and added columns within the same stage. + registerDepRule( + "New primary index for alter column type should go public in the same stage as dropped column", + scgraph.SameStagePrecedence, + "column", "new-primary-index", + func(from, to NodeVars) rel.Clauses { + ic := MkNodeVars("index-column") + relationID, columnID, indexID := rel.Var("table-id"), rel.Var("old-column-id"), rel.Var("index-id") return rel.Clauses{ from.Type((*scpb.Column)(nil)), to.Type((*scpb.PrimaryIndex)(nil)), @@ -218,6 +244,7 @@ func init() { from.CurrentStatus(scpb.Status_WRITE_ONLY), to.TargetStatus(scpb.ToPublic), to.CurrentStatus(scpb.Status_PUBLIC), + rel.And(IsDroppedColumnPartOfAlterColumnTypeOp("table-id", "old-column-id")...), } }, ) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go index af8882d76f7a..0b6a5c468dc3 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go @@ -213,6 +213,14 @@ func isColumnDependent(e scpb.Element) bool { return isColumnTypeDependent(e) } +func isColumnDependentExceptColumnName(e scpb.Element) bool { + switch e.(type) { + case *scpb.ColumnName: + return false + } + return isColumnDependent(e) +} + func isColumnNotNull(e scpb.Element) bool { switch e.(type) { case *scpb.ColumnNotNull: diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules index 9d12e0f17f23..f81856c46fe4 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules @@ -890,6 +890,33 @@ deprules - descriptorIsDataNotBeingAdded-25.1($descID) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) +- name: New primary index for alter column type should go public in the same stage as dropped column + from: column-Node + kind: SameStagePrecedence + to: new-primary-index-Node + query: + - $column[Type] = '*scpb.Column' + - $new-primary-index[Type] = '*scpb.PrimaryIndex' + - ColumnInSourcePrimaryIndex($index-column, $new-primary-index, $table-id, $old-column-id, $index-id) + - joinOnColumnID($index-column, $column, $table-id, $old-column-id) + - $column-Target[TargetStatus] = ABSENT + - $column-Node[CurrentStatus] = WRITE_ONLY + - $new-primary-index-Target[TargetStatus] = PUBLIC + - $new-primary-index-Node[CurrentStatus] = PUBLIC + - $old-column-name[Type] = '*scpb.ColumnName' + - $new-column-name[Type] = '*scpb.ColumnName' + - joinOnColumnName($old-column-name, $new-column-name, $table-id, $column-name) + - $old-column-name[ColumnID] = $old-column-id + - $old-column-name-Target[TargetStatus] = ABSENT + - $new-column-name-Target[TargetStatus] = PUBLIC + - $compute-expression[Type] = '*scpb.ColumnComputeExpression' + - joinOnColumnID($new-column-name, $compute-expression, $table-id, $new-column-id) + - $compute-expression[Usage] = ALTER_TYPE_USING + - joinTargetNode($old-column-name, $old-column-name-Target, $old-column-name-Node) + - joinTargetNode($new-column-name, $new-column-name-Target, $new-column-name-Node) + - joinTargetNode($compute-expression, $compute-expression-Target, $compute-expression-Node) + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($new-primary-index, $new-primary-index-Target, $new-primary-index-Node) - name: New primary index should go public only after columns being dropped move to WRITE_ONLY from: column-Node kind: Precedence @@ -897,12 +924,13 @@ deprules query: - $column[Type] = '*scpb.Column' - $new-primary-index[Type] = '*scpb.PrimaryIndex' - - ColumnInSourcePrimaryIndex($index-column, $new-primary-index, $table-id, $column-id, $index-id) - - joinOnColumnID($index-column, $column, $table-id, $column-id) + - ColumnInSourcePrimaryIndex($index-column, $new-primary-index, $table-id, $old-column-id, $index-id) + - joinOnColumnID($index-column, $column, $table-id, $old-column-id) - $column-Target[TargetStatus] = ABSENT - $column-Node[CurrentStatus] = WRITE_ONLY - $new-primary-index-Target[TargetStatus] = PUBLIC - $new-primary-index-Node[CurrentStatus] = PUBLIC + - dropped column is not part of column type operation($table-id, $old-column-id) - joinTargetNode($column, $column-Target, $column-Node) - joinTargetNode($new-primary-index, $new-primary-index-Target, $new-primary-index-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -2504,6 +2532,26 @@ deprules - ToPublicOrTransient($column-Target, $dependent-Target) - $column-Node[CurrentStatus] = DELETE_ONLY - $dependent-Node[CurrentStatus] = PUBLIC + - no column type alteration in progress($table-id, $col-id) + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: column existence precedes column dependents during an alter column type + from: column-Node + kind: Precedence + to: dependent-Node + query: + - $column[Type] = '*scpb.Column' + - $dependent[Type] IN ['*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.IndexColumn', '*scpb.SequenceOwner'] + - joinOnColumnID($column, $dependent, $table-id, $col-id) + - ToPublicOrTransient($column-Target, $dependent-Target) + - $column-Node[CurrentStatus] = DELETE_ONLY + - $dependent-Node[CurrentStatus] = PUBLIC + - $column[Type] = '*scpb.Column' + - $compute-expression[Type] = '*scpb.ColumnComputeExpression' + - joinOnColumnID($column, $compute-expression, $table-id, $col-id) + - $compute-expression[Usage] = ALTER_TYPE_USING + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($compute-expression, $compute-expression-Target, $compute-expression-Node) - joinTargetNode($column, $column-Target, $column-Node) - joinTargetNode($dependent, $dependent-Target, $dependent-Node) - name: column existence precedes index existence @@ -2548,17 +2596,18 @@ deprules - $index-Node[CurrentStatus] = WRITE_ONLY - joinTargetNode($column, $column-Target, $column-Node) - joinTargetNode($index, $index-Target, $index-Node) -- name: column name and type set right after column existence +- name: column name set right after column existence, except for alter column type from: column-Node kind: SameStagePrecedence to: column-name-or-type-Node query: - $column[Type] = '*scpb.Column' - - $column-name-or-type[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType'] + - $column-name-or-type[Type] = '*scpb.ColumnName' - ToPublicOrTransient($column-Target, $column-name-or-type-Target) - $column-Node[CurrentStatus] = DELETE_ONLY - $column-name-or-type-Node[CurrentStatus] = PUBLIC - joinOnColumnID($column, $column-name-or-type, $table-id, $col-id) + - no column type alteration in progress($table-id, $col-id) - joinTargetNode($column, $column-Target, $column-Node) - joinTargetNode($column-name-or-type, $column-name-or-type-Target, $column-name-or-type-Node) - name: column no longer public before dependents @@ -2687,6 +2736,19 @@ deprules - $column-Node[CurrentStatus] = ABSENT - joinTargetNode($column-type, $column-type-Target, $column-type-Node) - joinTargetNode($column, $column-Target, $column-Node) +- name: column type set right after column existence + from: column-Node + kind: SameStagePrecedence + to: column-type-Node + query: + - $column[Type] = '*scpb.Column' + - $column-type[Type] = '*scpb.ColumnType' + - ToPublicOrTransient($column-Target, $column-type-Target) + - $column-Node[CurrentStatus] = DELETE_ONLY + - $column-type-Node[CurrentStatus] = PUBLIC + - joinOnColumnID($column, $column-type, $table-id, $col-id) + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($column-type, $column-type-Target, $column-type-Node) - name: column type update is decomposed as a drop then add from: old-column-type-Node kind: Precedence @@ -3298,6 +3360,27 @@ deprules - $column-type-Node[CurrentStatus] = ABSENT - joinTargetNode($dependent, $dependent-Target, $dependent-Node) - joinTargetNode($column-type, $column-type-Target, $column-type-Node) +- name: during alter column type, column names for old and new columns are swapped in the same stage + from: old-column-name-Node + kind: SameStagePrecedence + to: new-column-name-Node + query: + - $old-column-name[Type] = '*scpb.ColumnName' + - $new-column-name[Type] = '*scpb.ColumnName' + - $old-column-name-Target[TargetStatus] = ABSENT + - $old-column-name-Node[CurrentStatus] = ABSENT + - $new-column-name-Target[TargetStatus] = PUBLIC + - $new-column-name-Node[CurrentStatus] = PUBLIC + - joinOnDescID($old-column-name, $new-column-name, $table-id) + - $new-column-name[ColumnID] = $new-col-id + - $column[Type] = '*scpb.Column' + - $compute-expression[Type] = '*scpb.ColumnComputeExpression' + - joinOnColumnID($column, $compute-expression, $table-id, $new-col-id) + - $compute-expression[Usage] = ALTER_TYPE_USING + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($compute-expression, $compute-expression-Target, $compute-expression-Node) + - joinTargetNode($old-column-name, $old-column-name-Target, $old-column-name-Node) + - joinTargetNode($new-column-name, $new-column-name-Target, $new-column-name-Node) - name: ensure columns are in increasing order from: later-column-Node kind: Precedence @@ -5381,6 +5464,33 @@ deprules - descriptorIsDataNotBeingAdded-25.1($descID) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) +- name: New primary index for alter column type should go public in the same stage as dropped column + from: column-Node + kind: SameStagePrecedence + to: new-primary-index-Node + query: + - $column[Type] = '*scpb.Column' + - $new-primary-index[Type] = '*scpb.PrimaryIndex' + - ColumnInSourcePrimaryIndex($index-column, $new-primary-index, $table-id, $old-column-id, $index-id) + - joinOnColumnID($index-column, $column, $table-id, $old-column-id) + - $column-Target[TargetStatus] = ABSENT + - $column-Node[CurrentStatus] = WRITE_ONLY + - $new-primary-index-Target[TargetStatus] = PUBLIC + - $new-primary-index-Node[CurrentStatus] = PUBLIC + - $old-column-name[Type] = '*scpb.ColumnName' + - $new-column-name[Type] = '*scpb.ColumnName' + - joinOnColumnName($old-column-name, $new-column-name, $table-id, $column-name) + - $old-column-name[ColumnID] = $old-column-id + - $old-column-name-Target[TargetStatus] = ABSENT + - $new-column-name-Target[TargetStatus] = PUBLIC + - $compute-expression[Type] = '*scpb.ColumnComputeExpression' + - joinOnColumnID($new-column-name, $compute-expression, $table-id, $new-column-id) + - $compute-expression[Usage] = ALTER_TYPE_USING + - joinTargetNode($old-column-name, $old-column-name-Target, $old-column-name-Node) + - joinTargetNode($new-column-name, $new-column-name-Target, $new-column-name-Node) + - joinTargetNode($compute-expression, $compute-expression-Target, $compute-expression-Node) + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($new-primary-index, $new-primary-index-Target, $new-primary-index-Node) - name: New primary index should go public only after columns being dropped move to WRITE_ONLY from: column-Node kind: Precedence @@ -5388,12 +5498,13 @@ deprules query: - $column[Type] = '*scpb.Column' - $new-primary-index[Type] = '*scpb.PrimaryIndex' - - ColumnInSourcePrimaryIndex($index-column, $new-primary-index, $table-id, $column-id, $index-id) - - joinOnColumnID($index-column, $column, $table-id, $column-id) + - ColumnInSourcePrimaryIndex($index-column, $new-primary-index, $table-id, $old-column-id, $index-id) + - joinOnColumnID($index-column, $column, $table-id, $old-column-id) - $column-Target[TargetStatus] = ABSENT - $column-Node[CurrentStatus] = WRITE_ONLY - $new-primary-index-Target[TargetStatus] = PUBLIC - $new-primary-index-Node[CurrentStatus] = PUBLIC + - dropped column is not part of column type operation($table-id, $old-column-id) - joinTargetNode($column, $column-Target, $column-Node) - joinTargetNode($new-primary-index, $new-primary-index-Target, $new-primary-index-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -6995,6 +7106,26 @@ deprules - ToPublicOrTransient($column-Target, $dependent-Target) - $column-Node[CurrentStatus] = DELETE_ONLY - $dependent-Node[CurrentStatus] = PUBLIC + - no column type alteration in progress($table-id, $col-id) + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: column existence precedes column dependents during an alter column type + from: column-Node + kind: Precedence + to: dependent-Node + query: + - $column[Type] = '*scpb.Column' + - $dependent[Type] IN ['*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.IndexColumn', '*scpb.SequenceOwner'] + - joinOnColumnID($column, $dependent, $table-id, $col-id) + - ToPublicOrTransient($column-Target, $dependent-Target) + - $column-Node[CurrentStatus] = DELETE_ONLY + - $dependent-Node[CurrentStatus] = PUBLIC + - $column[Type] = '*scpb.Column' + - $compute-expression[Type] = '*scpb.ColumnComputeExpression' + - joinOnColumnID($column, $compute-expression, $table-id, $col-id) + - $compute-expression[Usage] = ALTER_TYPE_USING + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($compute-expression, $compute-expression-Target, $compute-expression-Node) - joinTargetNode($column, $column-Target, $column-Node) - joinTargetNode($dependent, $dependent-Target, $dependent-Node) - name: column existence precedes index existence @@ -7039,17 +7170,18 @@ deprules - $index-Node[CurrentStatus] = WRITE_ONLY - joinTargetNode($column, $column-Target, $column-Node) - joinTargetNode($index, $index-Target, $index-Node) -- name: column name and type set right after column existence +- name: column name set right after column existence, except for alter column type from: column-Node kind: SameStagePrecedence to: column-name-or-type-Node query: - $column[Type] = '*scpb.Column' - - $column-name-or-type[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType'] + - $column-name-or-type[Type] = '*scpb.ColumnName' - ToPublicOrTransient($column-Target, $column-name-or-type-Target) - $column-Node[CurrentStatus] = DELETE_ONLY - $column-name-or-type-Node[CurrentStatus] = PUBLIC - joinOnColumnID($column, $column-name-or-type, $table-id, $col-id) + - no column type alteration in progress($table-id, $col-id) - joinTargetNode($column, $column-Target, $column-Node) - joinTargetNode($column-name-or-type, $column-name-or-type-Target, $column-name-or-type-Node) - name: column no longer public before dependents @@ -7178,6 +7310,19 @@ deprules - $column-Node[CurrentStatus] = ABSENT - joinTargetNode($column-type, $column-type-Target, $column-type-Node) - joinTargetNode($column, $column-Target, $column-Node) +- name: column type set right after column existence + from: column-Node + kind: SameStagePrecedence + to: column-type-Node + query: + - $column[Type] = '*scpb.Column' + - $column-type[Type] = '*scpb.ColumnType' + - ToPublicOrTransient($column-Target, $column-type-Target) + - $column-Node[CurrentStatus] = DELETE_ONLY + - $column-type-Node[CurrentStatus] = PUBLIC + - joinOnColumnID($column, $column-type, $table-id, $col-id) + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($column-type, $column-type-Target, $column-type-Node) - name: column type update is decomposed as a drop then add from: old-column-type-Node kind: Precedence @@ -7789,6 +7934,27 @@ deprules - $column-type-Node[CurrentStatus] = ABSENT - joinTargetNode($dependent, $dependent-Target, $dependent-Node) - joinTargetNode($column-type, $column-type-Target, $column-type-Node) +- name: during alter column type, column names for old and new columns are swapped in the same stage + from: old-column-name-Node + kind: SameStagePrecedence + to: new-column-name-Node + query: + - $old-column-name[Type] = '*scpb.ColumnName' + - $new-column-name[Type] = '*scpb.ColumnName' + - $old-column-name-Target[TargetStatus] = ABSENT + - $old-column-name-Node[CurrentStatus] = ABSENT + - $new-column-name-Target[TargetStatus] = PUBLIC + - $new-column-name-Node[CurrentStatus] = PUBLIC + - joinOnDescID($old-column-name, $new-column-name, $table-id) + - $new-column-name[ColumnID] = $new-col-id + - $column[Type] = '*scpb.Column' + - $compute-expression[Type] = '*scpb.ColumnComputeExpression' + - joinOnColumnID($column, $compute-expression, $table-id, $new-col-id) + - $compute-expression[Usage] = ALTER_TYPE_USING + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($compute-expression, $compute-expression-Target, $compute-expression-Node) + - joinTargetNode($old-column-name, $old-column-name-Target, $old-column-name-Node) + - joinTargetNode($new-column-name, $new-column-name-Target, $new-column-name-Node) - name: ensure columns are in increasing order from: later-column-Node kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go index 7ca22358929f..187a3f35faab 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go @@ -105,6 +105,10 @@ func JoinOnColumnID(a, b NodeVars, relationIDVar, columnIDVar rel.Var) rel.Claus return joinOnColumnIDUntyped(a.El, b.El, relationIDVar, columnIDVar) } +func JoinOnColumnName(a, b NodeVars, relationIDVar, columnNameVar rel.Var) rel.Clause { + return joinOnColumnNameUntyped(a.El, b.El, relationIDVar, columnNameVar) +} + // JoinOnColumnFamilyID joins elements on column ID. func JoinOnColumnFamilyID(a, b NodeVars, relationIDVar, columnFamilyIDVar rel.Var) rel.Clause { return joinOnColumnFamilyIDUntyped(a.El, b.El, relationIDVar, columnFamilyIDVar) @@ -146,7 +150,10 @@ func ColumnInSourcePrimaryIndex( return columnInSourcePrimaryIndex(indexColumn.El, index.El, relationIDVar, columnIDVar, indexIDVar) } -// IsAlterColumnTypeOp checks if the specified column is undergoing a type alteration +// IsAlterColumnTypeOp checks if the specified column is undergoing a type alteration. +// columnIDVar represents the column ID of the new column in the operation. +// If only the dropped column ID is available, use the alternative function: +// IsDroppedColumnPartOfAlterColumnTypeOp. func IsAlterColumnTypeOp(tableIDVar, columnIDVar rel.Var) rel.Clauses { column := MkNodeVars("column") computeExpression := MkNodeVars("compute-expression") @@ -160,6 +167,30 @@ func IsAlterColumnTypeOp(tableIDVar, columnIDVar rel.Var) rel.Clauses { } } +// IsDroppedColumnPartOfAlterColumnTypeOp functions similarly to IsAlterColumnTypeOp +// but operates using the dropped column ID. It checks for a specific compute expression +// applied to the new column. This requires joining the old column with the new column +// by matching on the column name. +func IsDroppedColumnPartOfAlterColumnTypeOp(tableIDVar, oldColumnIDVar rel.Var) rel.Clauses { + oldColumnName := MkNodeVars("old-column-name") + newColumnName := MkNodeVars("new-column-name") + computeExpression := MkNodeVars("compute-expression") + return rel.Clauses{ + oldColumnName.Type((*scpb.ColumnName)(nil)), + newColumnName.Type((*scpb.ColumnName)(nil)), + JoinOnColumnName(oldColumnName, newColumnName, tableIDVar, "column-name"), + oldColumnName.El.AttrEqVar(screl.ColumnID, oldColumnIDVar), + oldColumnName.TargetStatus(scpb.ToAbsent), + newColumnName.TargetStatus(scpb.ToPublic), + computeExpression.Type((*scpb.ColumnComputeExpression)(nil)), + JoinOnColumnID(newColumnName, computeExpression, tableIDVar, "new-column-id"), + computeExpression.El.AttrEq(screl.Usage, scpb.ColumnComputeExpression_ALTER_TYPE_USING), + oldColumnName.JoinTargetNode(), + newColumnName.JoinTargetNode(), + computeExpression.JoinTargetNode(), + } +} + // IsPotentialSecondaryIndexSwap determines if a secondary index recreate is // occurring because of a primary key alter. func IsPotentialSecondaryIndexSwap(indexIdVar rel.Var, tableIDVar rel.Var) rel.Clauses { @@ -256,6 +287,18 @@ var ( } }, ) + + joinOnColumnNameUntyped = screl.Schema.Def4( + "joinOnColumnName", "a", "b", "desc-id", "column-name", func( + a, b, descID, columnName rel.Var, + ) rel.Clauses { + return rel.Clauses{ + JoinOnDescIDUntyped(a, b, descID), + columnName.Entities(screl.Name, a, b), + } + }, + ) + joinOnColumnFamilyIDUntyped = screl.Schema.Def4( "joinOnColumnFamilyID", "a", "b", "desc-id", "family-id", func( a, b, descID, familyID rel.Var, @@ -344,11 +387,15 @@ var ( return IsPotentialSecondaryIndexSwap(b, a) }) - // IsNotAlterColumnTypeOp determines if no column alteration in progress + // IsNotAlterColumnTypeOp determines if no column alteration in progress. The column + // passed in must for a new column. IsNotAlterColumnTypeOp = screl.Schema.DefNotJoin2("no column type alteration in progress", "table-id", "column-id", func(t, c rel.Var) rel.Clauses { return IsAlterColumnTypeOp(t, c) }) + + IsNotDroppedColumnPartOfAlterColumnTypeOp = screl.Schema.DefNotJoin2("dropped column is not part of column type operation", + "table-id", "old-column-id", func(t, c rel.Var) rel.Clauses { return IsDroppedColumnPartOfAlterColumnTypeOp(t, c) }) ) // ForEachElementInActiveVersion executes a function for each element supported within diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column b/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column index b4f56e3d21c5..78b99acfcfa2 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column @@ -2936,11 +2936,11 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 - from: [Column:{DescID: 109, ColumnID: 2}, DELETE_ONLY] to: [ColumnName:{DescID: 109, Name: g, ColumnID: 2}, PUBLIC] kind: SameStagePrecedence - rules: [column existence precedes column dependents; column name and type set right after column existence] + rules: [column existence precedes column dependents; column name set right after column existence, except for alter column type] - from: [Column:{DescID: 109, ColumnID: 2}, DELETE_ONLY] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2, TypeName: INT8}, PUBLIC] kind: SameStagePrecedence - rules: [column existence precedes column dependents; column name and type set right after column existence] + rules: [column existence precedes column dependents; column type set right after column existence] - from: [Column:{DescID: 109, ColumnID: 2}, DELETE_ONLY] to: [IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 2}, PUBLIC] kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table_alter_column_type b/pkg/sql/schemachanger/scplan/testdata/alter_table_alter_column_type index e0b26d9d44a3..676fbab2d3ea 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table_alter_column_type +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table_alter_column_type @@ -170,7 +170,7 @@ PostCommitNonRevertiblePhase stage 3 of 3 with 3 MutationType ops ops ALTER TABLE defaultdb.act ALTER COLUMN c1 SET DATA TYPE TEXT; ---- -StatementPhase stage 1 of 1 with 16 MutationType ops +StatementPhase stage 1 of 1 with 14 MutationType ops transitions: [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> BACKFILL_ONLY [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC @@ -183,23 +183,17 @@ StatementPhase stage 1 of 1 with 16 MutationType ops [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC [[IndexData:{DescID: 104, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC [[Column:{DescID: 104, ColumnID: 3}, PUBLIC], ABSENT] -> DELETE_ONLY - [[ColumnName:{DescID: 104, Name: c1, ColumnID: 3}, PUBLIC], ABSENT] -> PUBLIC [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 3, TypeName: STRING}, PUBLIC], ABSENT] -> PUBLIC [[ColumnComputeExpression:{DescID: 104, ColumnID: 3, Usage: ALTER_TYPE_USING}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC - [[ColumnName:{DescID: 104, Name: c1_shadow, ColumnID: 2}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC ops: *scop.MakeAbsentColumnDeleteOnly Column: ColumnID: 3 PgAttributeNum: 2 TableID: 104 - *scop.SetColumnName - ColumnID: 3 - Name: c1 - TableID: 104 *scop.UpsertColumnType ColumnType: ColumnFamilyOrderFollowsColumnID: 2 @@ -223,10 +217,6 @@ StatementPhase stage 1 of 1 with 16 MutationType ops - 2 TableID: 104 Usage: 1 - *scop.SetColumnName - ColumnID: 2 - Name: c1_shadow - TableID: 104 *scop.MakeAbsentIndexBackfilling Index: ConstraintID: 2 @@ -302,17 +292,15 @@ PreCommitPhase stage 1 of 2 with 1 MutationType op [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4}, PUBLIC], PUBLIC] -> ABSENT [[IndexData:{DescID: 104, IndexID: 4}, PUBLIC], PUBLIC] -> ABSENT [[Column:{DescID: 104, ColumnID: 3}, PUBLIC], DELETE_ONLY] -> ABSENT - [[ColumnName:{DescID: 104, Name: c1, ColumnID: 3}, PUBLIC], PUBLIC] -> ABSENT [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 3, TypeName: STRING}, PUBLIC], PUBLIC] -> ABSENT [[ColumnComputeExpression:{DescID: 104, ColumnID: 3, Usage: ALTER_TYPE_USING}, TRANSIENT_ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, TRANSIENT_ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, TRANSIENT_ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4}, PUBLIC], PUBLIC] -> ABSENT - [[ColumnName:{DescID: 104, Name: c1_shadow, ColumnID: 2}, TRANSIENT_ABSENT], PUBLIC] -> ABSENT ops: *scop.UndoAllInTxnImmediateMutationOpSideEffects {} -PreCommitPhase stage 2 of 2 with 21 MutationType ops +PreCommitPhase stage 2 of 2 with 19 MutationType ops transitions: [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> BACKFILL_ONLY [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC @@ -325,23 +313,17 @@ PreCommitPhase stage 2 of 2 with 21 MutationType ops [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC [[IndexData:{DescID: 104, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC [[Column:{DescID: 104, ColumnID: 3}, PUBLIC], ABSENT] -> DELETE_ONLY - [[ColumnName:{DescID: 104, Name: c1, ColumnID: 3}, PUBLIC], ABSENT] -> PUBLIC [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 3, TypeName: STRING}, PUBLIC], ABSENT] -> PUBLIC [[ColumnComputeExpression:{DescID: 104, ColumnID: 3, Usage: ALTER_TYPE_USING}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC - [[ColumnName:{DescID: 104, Name: c1_shadow, ColumnID: 2}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC ops: *scop.MakeAbsentColumnDeleteOnly Column: ColumnID: 3 PgAttributeNum: 2 TableID: 104 - *scop.SetColumnName - ColumnID: 3 - Name: c1 - TableID: 104 *scop.UpsertColumnType ColumnType: ColumnFamilyOrderFollowsColumnID: 2 @@ -365,10 +347,6 @@ PreCommitPhase stage 2 of 2 with 21 MutationType ops - 2 TableID: 104 Usage: 1 - *scop.SetColumnName - ColumnID: 2 - Name: c1_shadow - TableID: 104 *scop.MakeAbsentIndexBackfilling Index: ConstraintID: 2 @@ -650,22 +628,16 @@ PostCommitPhase stage 15 of 15 with 1 ValidationType op *scop.ValidateIndex IndexID: 4 TableID: 104 -PostCommitNonRevertiblePhase stage 1 of 4 with 16 MutationType ops +PostCommitNonRevertiblePhase stage 1 of 4 with 10 MutationType ops transitions: - [[Column:{DescID: 104, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY - [[ColumnName:{DescID: 104, Name: c1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> DELETE_ONLY - [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], TRANSIENT_DELETE_ONLY] -> TRANSIENT_ABSENT [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_ABSENT - [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_ABSENT [[TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, TRANSIENT_ABSENT], TRANSIENT_DELETE_ONLY] -> TRANSIENT_ABSENT [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_ABSENT [[ColumnComputeExpression:{DescID: 104, ColumnID: 3, Usage: ALTER_TYPE_USING}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_ABSENT [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_ABSENT [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_ABSENT - [[ColumnName:{DescID: 104, Name: c1_shadow, ColumnID: 2}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_ABSENT ops: *scop.RemoveColumnFromIndex ColumnID: 1 @@ -689,57 +661,47 @@ PostCommitNonRevertiblePhase stage 1 of 4 with 16 MutationType ops IndexID: 5 Kind: 2 TableID: 104 - *scop.MakePublicColumnWriteOnly - ColumnID: 2 - TableID: 104 - *scop.SetColumnName - ColumnID: 2 - Name: crdb_internal_column_2_name_placeholder - TableID: 104 *scop.MakeWriteOnlyIndexDeleteOnly IndexID: 1 TableID: 104 - *scop.RemoveColumnFromIndex - ColumnID: 2 - IndexID: 3 - Kind: 2 - TableID: 104 *scop.MakeIndexAbsent IndexID: 5 TableID: 104 - *scop.SetColumnName - ColumnID: 2 - Name: c1 - TableID: 104 *scop.RemoveColumnFromIndex ColumnID: 1 IndexID: 1 TableID: 104 - *scop.RemoveColumnFromIndex - ColumnID: 2 - IndexID: 1 - Kind: 2 - TableID: 104 - *scop.MakeIndexAbsent - IndexID: 3 - TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 *scop.UpdateSchemaChangerJob IsNonCancelable: true JobID: 1 -PostCommitNonRevertiblePhase stage 2 of 4 with 10 MutationType ops +PostCommitNonRevertiblePhase stage 2 of 4 with 15 MutationType ops transitions: - [[Column:{DescID: 104, ColumnID: 2}, ABSENT], WRITE_ONLY] -> DELETE_ONLY + [[Column:{DescID: 104, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 104, Name: c1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1}, ABSENT], DELETE_ONLY] -> ABSENT [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_VALIDATED [[IndexName:{DescID: 104, Name: act_pkey, IndexID: 2}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_ABSENT + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], TRANSIENT_DELETE_ONLY] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_ABSENT [[PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, PUBLIC], VALIDATED] -> PUBLIC [[IndexName:{DescID: 104, Name: act_pkey, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC [[Column:{DescID: 104, ColumnID: 3}, PUBLIC], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 104, Name: c1, ColumnID: 3}, PUBLIC], ABSENT] -> PUBLIC ops: - *scop.MakeWriteOnlyColumnDeleteOnly + *scop.MakePublicColumnWriteOnly + ColumnID: 2 + TableID: 104 + *scop.SetColumnName ColumnID: 2 + Name: crdb_internal_column_2_name_placeholder + TableID: 104 + *scop.RemoveColumnFromIndex + ColumnID: 2 + IndexID: 1 + Kind: 2 TableID: 104 *scop.MakeIndexAbsent IndexID: 1 @@ -751,10 +713,22 @@ PostCommitNonRevertiblePhase stage 2 of 4 with 10 MutationType ops IndexID: 2 Name: crdb_internal_index_2_name_placeholder TableID: 104 + *scop.RemoveColumnFromIndex + ColumnID: 2 + IndexID: 3 + Kind: 2 + TableID: 104 *scop.SetIndexName IndexID: 4 Name: act_pkey TableID: 104 + *scop.SetColumnName + ColumnID: 3 + Name: c1 + TableID: 104 + *scop.MakeIndexAbsent + IndexID: 3 + TableID: 104 *scop.MakeValidatedPrimaryIndexPublic IndexID: 4 TableID: 104 @@ -768,13 +742,17 @@ PostCommitNonRevertiblePhase stage 2 of 4 with 10 MutationType ops *scop.UpdateSchemaChangerJob IsNonCancelable: true JobID: 1 -PostCommitNonRevertiblePhase stage 3 of 4 with 6 MutationType ops +PostCommitNonRevertiblePhase stage 3 of 4 with 7 MutationType ops transitions: + [[Column:{DescID: 104, ColumnID: 2}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], TRANSIENT_VALIDATED] -> TRANSIENT_DELETE_ONLY [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_ABSENT [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_ABSENT [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, TRANSIENT_ABSENT], PUBLIC] -> TRANSIENT_ABSENT ops: + *scop.MakeWriteOnlyColumnDeleteOnly + ColumnID: 2 + TableID: 104 *scop.MakeWriteOnlyIndexDeleteOnly IndexID: 2 TableID: 104 diff --git a/pkg/sql/schemachanger/scplan/testdata/create_sequence b/pkg/sql/schemachanger/scplan/testdata/create_sequence index 62617956c364..8b4885ee02c6 100644 --- a/pkg/sql/schemachanger/scplan/testdata/create_sequence +++ b/pkg/sql/schemachanger/scplan/testdata/create_sequence @@ -259,7 +259,7 @@ CREATE SEQUENCE sq1 MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 3 - from: [Column:{DescID: 105, ColumnID: 1}, DELETE_ONLY] to: [ColumnName:{DescID: 105, Name: value, ColumnID: 1}, PUBLIC] kind: SameStagePrecedence - rules: [column existence precedes column dependents; column name and type set right after column existence] + rules: [column existence precedes column dependents; column name set right after column existence, except for alter column type] - from: [Column:{DescID: 105, ColumnID: 1}, DELETE_ONLY] to: [ColumnNotNull:{DescID: 105, ColumnID: 1, IndexID: 0}, PUBLIC] kind: Precedence @@ -267,7 +267,7 @@ CREATE SEQUENCE sq1 MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 3 - from: [Column:{DescID: 105, ColumnID: 1}, DELETE_ONLY] to: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1, TypeName: INT8}, PUBLIC] kind: SameStagePrecedence - rules: [column existence precedes column dependents; column name and type set right after column existence] + rules: [column existence precedes column dependents; column type set right after column existence] - from: [Column:{DescID: 105, ColumnID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 1}, PUBLIC] kind: Precedence diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general.explain index 79486c310cf8..969430220e53 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general.explain @@ -9,32 +9,28 @@ EXPLAIN (DDL) ALTER TABLE t ALTER COLUMN j SET DATA TYPE BIGINT USING j::BIGINT; Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER COLUMN ‹j› SET DATA TYPE INT8 USING ‹j›::INT8; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 7 elements transitioning toward PUBLIC + │ ├── 6 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey+), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey~)} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey+)} │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104 (t), IndexID: 4 (t_pkey+)} │ │ ├── ABSENT → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j+)} - │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j+)} │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 104 (t), ColumnFamilyID: 0 (primary), ColumnID: 3 (j+), TypeName: "INT8"} │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 4 (t_pkey+)} - │ ├── 11 elements transitioning toward TRANSIENT_ABSENT + │ ├── 10 elements transitioning toward TRANSIENT_ABSENT │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey~), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey-)} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey~)} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 2 (t_pkey~)} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 2 (t_pkey~)} │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104 (t), IndexID: 2 (t_pkey~)} │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey-)} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 3} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 3} │ │ ├── ABSENT → PUBLIC ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j+), Usage: ALTER_TYPE_USING} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 2 (t_pkey~)} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} - │ │ └── ABSENT → PUBLIC ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j-j_shadow~)} - │ └── 16 Mutation operations + │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} + │ └── 14 Mutation operations │ ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":3,"PgAttributeNum":2,"TableID":104}} - │ ├── SetColumnName {"ColumnID":3,"Name":"j","TableID":104} │ ├── UpsertColumnType {"ColumnType":{"ColumnFamilyOrderFollowsColumnID":2,"ColumnID":3,"IsNullable":true,"TableID":104}} │ ├── AddColumnComputeExpression {"ComputeExpression":{"ColumnID":3,"TableID":104,"Usage":1}} - │ ├── SetColumnName {"ColumnID":2,"Name":"j_shadow","TableID":104} │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":3}} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} @@ -48,55 +44,49 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER CO │ └── AddColumnToIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} ├── PreCommitPhase │ ├── Stage 1 of 2 in PreCommitPhase - │ │ ├── 7 elements transitioning toward PUBLIC + │ │ ├── 6 elements transitioning toward PUBLIC │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey+), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey~)} │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey+)} │ │ │ ├── PUBLIC → ABSENT IndexData:{DescID: 104 (t), IndexID: 4 (t_pkey+)} │ │ │ ├── DELETE_ONLY → ABSENT Column:{DescID: 104 (t), ColumnID: 3 (j+)} - │ │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j+)} │ │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 104 (t), ColumnFamilyID: 0 (primary), ColumnID: 3 (j+), TypeName: "INT8"} │ │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 4 (t_pkey+)} - │ │ ├── 11 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── 10 elements transitioning toward TRANSIENT_ABSENT │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey~), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey-)} │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey~)} - │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 2 (t_pkey~)} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 2 (t_pkey~)} │ │ │ ├── PUBLIC → ABSENT IndexData:{DescID: 104 (t), IndexID: 2 (t_pkey~)} │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey-)} │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 3} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 3} │ │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j+), Usage: ALTER_TYPE_USING} │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 2 (t_pkey~)} - │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} - │ │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j-j_shadow~)} + │ │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} │ │ └── 1 Mutation operation │ │ └── UndoAllInTxnImmediateMutationOpSideEffects │ └── Stage 2 of 2 in PreCommitPhase - │ ├── 7 elements transitioning toward PUBLIC + │ ├── 6 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey+), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey~)} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey+)} │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104 (t), IndexID: 4 (t_pkey+)} │ │ ├── ABSENT → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j+)} - │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j+)} │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 104 (t), ColumnFamilyID: 0 (primary), ColumnID: 3 (j+), TypeName: "INT8"} │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 4 (t_pkey+)} - │ ├── 11 elements transitioning toward TRANSIENT_ABSENT + │ ├── 10 elements transitioning toward TRANSIENT_ABSENT │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey~), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey-)} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey~)} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 2 (t_pkey~)} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 2 (t_pkey~)} │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104 (t), IndexID: 2 (t_pkey~)} │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey-)} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 3} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 3} │ │ ├── ABSENT → PUBLIC ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j+), Usage: ALTER_TYPE_USING} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 2 (t_pkey~)} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} - │ │ └── ABSENT → PUBLIC ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j-j_shadow~)} - │ └── 21 Mutation operations + │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} + │ └── 19 Mutation operations │ ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":3,"PgAttributeNum":2,"TableID":104}} - │ ├── SetColumnName {"ColumnID":3,"Name":"j","TableID":104} │ ├── UpsertColumnType {"ColumnType":{"ColumnFamilyOrderFollowsColumnID":2,"ColumnID":3,"IsNullable":true,"TableID":104}} │ ├── AddColumnComputeExpression {"ComputeExpression":{"ColumnID":3,"TableID":104,"Usage":1}} - │ ├── SetColumnName {"ColumnID":2,"Name":"j_shadow","TableID":104} │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":3}} │ ├── MaybeAddSplitForIndex {"IndexID":2,"TableID":104} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"TableID":104} @@ -233,56 +223,54 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER CO │ └── ValidateIndex {"IndexID":4,"TableID":104} └── PostCommitNonRevertiblePhase ├── Stage 1 of 4 in PostCommitNonRevertiblePhase - │ ├── 9 elements transitioning toward TRANSIENT_ABSENT - │ │ ├── TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey-)} + │ ├── 6 elements transitioning toward TRANSIENT_ABSENT │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 3} │ │ ├── TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey~)} │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── PUBLIC → TRANSIENT_ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j+), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} - │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 5} - │ │ └── PUBLIC → TRANSIENT_ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j-j_shadow~)} - │ ├── 5 elements transitioning toward ABSENT - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 2 (j-j_shadow~)} + │ │ └── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 5} + │ ├── 2 elements transitioning toward ABSENT │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 1 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 1 (t_pkey-)} │ │ └── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey-), ConstraintID: 1} - │ └── 16 Mutation operations + │ └── 10 Mutation operations │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":1,"TableID":104} - │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":5,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":1,"TableID":104} - │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":1,"Kind":2,"TableID":104} - │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104} │ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"PostCommitNonRev..."} ├── Stage 2 of 4 in PostCommitNonRevertiblePhase - │ ├── 3 elements transitioning toward PUBLIC - │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey+), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey~)} - │ │ ├── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 4 (t_pkey+)} - │ │ └── WRITE_ONLY → PUBLIC Column:{DescID: 104 (t), ColumnID: 3 (j+)} - │ ├── 2 elements transitioning toward TRANSIENT_ABSENT - │ │ ├── PUBLIC → TRANSIENT_VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey~), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey-)} - │ │ └── PUBLIC → TRANSIENT_ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey~)} - │ ├── 2 elements transitioning toward ABSENT - │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~)} - │ │ └── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey-), ConstraintID: 1} - │ └── 10 Mutation operations - │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":2,"TableID":104} + │ ├── 4 elements transitioning toward PUBLIC + │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey+), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey~)} + │ │ ├── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 4 (t_pkey+)} + │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104 (t), ColumnID: 3 (j+)} + │ │ └── ABSENT → PUBLIC ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j+)} + │ ├── 4 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── PUBLIC → TRANSIENT_VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey~), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey-)} + │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey~)} + │ │ ├── TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey-)} + │ │ └── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 3} + │ ├── 4 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104 (t), ColumnID: 2 (j-)} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 2 (j-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 1 (t_pkey-)} + │ │ └── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey-), ConstraintID: 1} + │ └── 15 Mutation operations + │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} + │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":1,"Kind":2,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":1,"TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── SetIndexName {"IndexID":4,"Name":"t_pkey","TableID":104} + │ ├── SetColumnName {"ColumnID":3,"Name":"j","TableID":104} + │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnPublic {"ColumnID":3,"TableID":104} │ ├── RefreshStats {"TableID":104} @@ -292,9 +280,12 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER CO │ ├── 4 elements transitioning toward TRANSIENT_ABSENT │ │ ├── TRANSIENT_VALIDATED → TRANSIENT_DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey~), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey-)} │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey~)} - │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 2 (t_pkey~)} + │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 2 (t_pkey~)} │ │ └── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 2 (t_pkey~)} - │ └── 6 Mutation operations + │ ├── 1 element transitioning toward ABSENT + │ │ └── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 2 (j-)} + │ └── 7 Mutation operations + │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":2,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} @@ -308,8 +299,8 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER CO │ ├── PUBLIC → TRANSIENT_ABSENT IndexData:{DescID: 104 (t), IndexID: 3} │ └── PUBLIC → TRANSIENT_ABSENT IndexData:{DescID: 104 (t), IndexID: 5} ├── 3 elements transitioning toward ABSENT - │ ├── DELETE_ONLY → ABSENT Column:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~)} - │ ├── PUBLIC → ABSENT ColumnType:{DescID: 104 (t), ColumnFamilyID: 0 (primary), ColumnID: 2 (j-j_shadow~), TypeName: "STRING"} + │ ├── DELETE_ONLY → ABSENT Column:{DescID: 104 (t), ColumnID: 2 (j-)} + │ ├── PUBLIC → ABSENT ColumnType:{DescID: 104 (t), ColumnFamilyID: 0 (primary), ColumnID: 2 (j-), TypeName: "STRING"} │ └── PUBLIC → ABSENT IndexData:{DescID: 104 (t), IndexID: 1 (t_pkey-)} └── 8 Mutation operations ├── CreateGCJobForIndex {"IndexID":1,"TableID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general.explain_shape b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general.explain_shape index d2d191ad8f14..4c46e962c28a 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general.explain_shape +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general.explain_shape @@ -9,7 +9,7 @@ EXPLAIN (DDL, SHAPE) ALTER TABLE t ALTER COLUMN j SET DATA TYPE BIGINT USING j:: Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER COLUMN ‹j› SET DATA TYPE INT8 USING ‹j›::INT8; ├── execute 2 system table mutations transactions ├── backfill using primary index t_pkey- in relation t - │ └── into t_pkey~ (i; j-j_shadow~, j+) + │ └── into t_pkey~ (i; j-, j+) ├── execute 2 system table mutations transactions ├── merge temporary indexes into backfilled indexes in relation t │ └── from t@[3] into t_pkey~ diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general.side_effects index b89a20d02dd5..4ed536ff8355 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general.side_effects +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general.side_effects @@ -14,32 +14,26 @@ begin transaction #1 checking for feature: ALTER TABLE increment telemetry for sql.schema.alter_table increment telemetry for sql.schema.alter_table.alter_column_type -## StatementPhase stage 1 of 1 with 16 MutationType ops +## StatementPhase stage 1 of 1 with 14 MutationType ops upsert descriptor #104 - ... - width: 64 - - id: 2 - - name: j - + name: j_shadow - nullable: true - type: ... - 1 - 2 + - 3 columnNames: - i - + - j_shadow - j + + - crdb_internal_column_3_name_placeholder defaultColumnId: 2 + name: primary ... id: 104 modificationTime: {} + mutations: + - column: - + computeExpr: j_shadow::INT8 + + computeExpr: j::INT8 + id: 3 - + name: j + + name: crdb_internal_column_3_name_placeholder + nullable: true + pgAttributeNum: 2 + type: @@ -71,8 +65,8 @@ upsert descriptor #104 + - 2 + - 3 + storeColumnNames: - + - j_shadow + - j + + - crdb_internal_column_3_name_placeholder + unique: true + version: 4 + mutationId: 1 @@ -99,8 +93,8 @@ upsert descriptor #104 + - 2 + - 3 + storeColumnNames: - + - j_shadow + - j + + - crdb_internal_column_3_name_placeholder + unique: true + useDeletePreservingEncoding: true + version: 4 @@ -127,7 +121,7 @@ upsert descriptor #104 + storeColumnIds: + - 3 + storeColumnNames: - + - j + + - crdb_internal_column_3_name_placeholder + unique: true + version: 4 + mutationId: 1 @@ -142,13 +136,6 @@ upsert descriptor #104 + nextIndexId: 5 nextMutationId: 1 parentId: 100 - ... - - 2 - storeColumnNames: - - - j - + - j_shadow - unique: true - version: 4 ... time: {} unexposedParentSchemaId: 101 @@ -159,15 +146,8 @@ upsert descriptor #104 ## PreCommitPhase stage 1 of 2 with 1 MutationType op undo all catalog changes within txn #1 persist all catalog changes to storage -## PreCommitPhase stage 2 of 2 with 21 MutationType ops +## PreCommitPhase stage 2 of 2 with 19 MutationType ops upsert descriptor #104 - ... - width: 64 - - id: 2 - - name: j - + name: j_shadow - nullable: true - type: ... createAsOfTime: wallTime: "1640995200000000000" @@ -205,17 +185,18 @@ upsert descriptor #104 + - 3 columnNames: - i - + - j_shadow - j + + - crdb_internal_column_3_name_placeholder defaultColumnId: 2 + name: primary ... id: 104 modificationTime: {} + mutations: + - column: - + computeExpr: j_shadow::INT8 + + computeExpr: j::INT8 + id: 3 - + name: j + + name: crdb_internal_column_3_name_placeholder + nullable: true + pgAttributeNum: 2 + type: @@ -247,8 +228,8 @@ upsert descriptor #104 + - 2 + - 3 + storeColumnNames: - + - j_shadow + - j + + - crdb_internal_column_3_name_placeholder + unique: true + version: 4 + mutationId: 1 @@ -275,8 +256,8 @@ upsert descriptor #104 + - 2 + - 3 + storeColumnNames: - + - j_shadow + - j + + - crdb_internal_column_3_name_placeholder + unique: true + useDeletePreservingEncoding: true + version: 4 @@ -303,7 +284,7 @@ upsert descriptor #104 + storeColumnIds: + - 3 + storeColumnNames: - + - j + + - crdb_internal_column_3_name_placeholder + unique: true + version: 4 + mutationId: 1 @@ -318,13 +299,6 @@ upsert descriptor #104 + nextIndexId: 5 nextMutationId: 1 parentId: 100 - ... - - 2 - storeColumnNames: - - - j - + - j_shadow - unique: true - version: 4 ... time: {} unexposedParentSchemaId: 101 @@ -466,7 +440,7 @@ upsert descriptor #104 partitioning: {} sharded: {} ... - - j + - crdb_internal_column_3_name_placeholder unique: true + useDeletePreservingEncoding: true version: 4 @@ -497,8 +471,8 @@ upsert descriptor #104 - - 2 - 3 storeColumnNames: - - - j_shadow - - j + - - j + - crdb_internal_column_3_name_placeholder unique: true - useDeletePreservingEncoding: true version: 4 @@ -526,7 +500,7 @@ upsert descriptor #104 + storeColumnIds: + - 2 + storeColumnNames: - + - j_shadow + + - j + unique: true + version: 4 + mutationId: 1 @@ -551,7 +525,7 @@ upsert descriptor #104 partitioning: {} sharded: {} ... - - j + - crdb_internal_column_3_name_placeholder unique: true + useDeletePreservingEncoding: true version: 4 @@ -584,8 +558,8 @@ upsert descriptor #104 - 2 + - 3 storeColumnNames: - - j_shadow - + - j + - j + + - crdb_internal_column_3_name_placeholder unique: true version: 4 ... @@ -695,80 +669,23 @@ begin transaction #17 validate forward indexes [4] in table #104 commit transaction #17 begin transaction #18 -## PostCommitNonRevertiblePhase stage 1 of 4 with 16 MutationType ops +## PostCommitNonRevertiblePhase stage 1 of 4 with 10 MutationType ops upsert descriptor #104 - ... - oid: 20 - width: 64 - - - id: 2 - - name: j_shadow - - nullable: true - - type: - - family: StringFamily - - oid: 25 - createAsOfTime: - wallTime: "1640995200000000000" ... statement: ALTER TABLE t ALTER COLUMN j SET DATA TYPE INT8 USING j::INT8 statementTag: ALTER TABLE - revertible: true targetRanks: targets: - ... - columnNames: - - i - - - j_shadow - - j - + - j - defaultColumnId: 2 - name: primary ... mutations: - column: - - computeExpr: j_shadow::INT8 + - computeExpr: j::INT8 id: 3 - name: j + name: crdb_internal_column_3_name_placeholder ... + version: 4 mutationId: 1 - state: WRITE_ONLY - - - direction: DROP - - index: - - constraintId: 3 - - createdExplicitly: true - - encodingType: 1 - - foreignKey: {} - - geoConfig: {} - - id: 3 - - interleave: {} - - keyColumnDirections: - - - ASC - - keyColumnIds: - - - 1 - - keyColumnNames: - - - i - - name: crdb_internal_index_3_name_placeholder - - partitioning: {} - - sharded: {} - - storeColumnIds: - - - 2 - - - 3 - - storeColumnNames: - - - j_shadow - - - j - - unique: true - - useDeletePreservingEncoding: true - - version: 4 - - mutationId: 1 - - state: DELETE_ONLY - - direction: ADD - index: - ... - - 2 - storeColumnNames: - - - j_shadow - - unique: true - - version: 4 - - mutationId: 1 - state: WRITE_ONLY - - direction: DROP - index: @@ -791,63 +708,56 @@ upsert descriptor #104 - storeColumnIds: - - 3 - storeColumnNames: - - j - unique: true + - - crdb_internal_column_3_name_placeholder + - unique: true - useDeletePreservingEncoding: true - version: 4 - mutationId: 1 + - version: 4 + - mutationId: 1 state: DELETE_ONLY - + - column: - + id: 2 - + name: j - + nullable: true - + type: - + family: StringFamily - + oid: 25 - + direction: DROP - + mutationId: 1 - + state: WRITE_ONLY name: t - nextColumnId: 4 - ... - - 3 - storeColumnNames: - - - j_shadow - - j - + - j - unique: true - version: 4 ... time: {} unexposedParentSchemaId: 101 - version: "11" + version: "12" persist all catalog changes to storage -update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 4 with 8 MutationType ops pending" +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 4 with 13 MutationType ops pending" set schema change job #1 to non-cancellable commit transaction #18 begin transaction #19 -## PostCommitNonRevertiblePhase stage 2 of 4 with 10 MutationType ops +## PostCommitNonRevertiblePhase stage 2 of 4 with 15 MutationType ops upsert descriptor #104 ... oid: 20 width: 64 + - - id: 2 + - id: 3 - + name: j - + nullable: true + name: j + nullable: true + pgAttributeNum: 2 - + type: + type: + - family: StringFamily + - oid: 25 + family: IntFamily + oid: 20 + width: 64 createAsOfTime: wallTime: "1640995200000000000" + ... + columnNames: + - i + + - crdb_internal_column_2_name_placeholder + - j + - - crdb_internal_column_3_name_placeholder + defaultColumnId: 2 + name: primary ... mutations: - column: - id: 3 + - name: crdb_internal_column_3_name_placeholder + id: 2 - name: j + + name: crdb_internal_column_2_name_placeholder nullable: true - pgAttributeNum: 2 type: @@ -859,37 +769,63 @@ upsert descriptor #104 + oid: 25 + direction: DROP mutationId: 1 - - state: WRITE_ONLY - - - direction: ADD - + state: DELETE_ONLY - + - direction: DROP + state: WRITE_ONLY + - direction: DROP index: - - constraintId: 4 + - constraintId: 3 + constraintId: 2 createdExplicitly: true encodingType: 1 foreignKey: {} geoConfig: {} - - id: 4 + - id: 3 + id: 2 interleave: {} keyColumnDirections: ... keyColumnNames: - i - - name: crdb_internal_index_4_name_placeholder + - name: crdb_internal_index_3_name_placeholder + name: crdb_internal_index_2_name_placeholder partitioning: {} sharded: {} - storeColumnIds: - + - 2 + ... - 3 storeColumnNames: + + - crdb_internal_column_2_name_placeholder - j + - - crdb_internal_column_3_name_placeholder + unique: true + - useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + - state: DELETE_ONLY + - - direction: ADD + - index: + - constraintId: 4 + - createdExplicitly: true + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 4 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - keyColumnNames: + - - i + - name: crdb_internal_index_4_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnIds: + - - 3 + - storeColumnNames: + - - crdb_internal_column_3_name_placeholder - unique: true - version: 4 - mutationId: 1 - - state: WRITE_ONLY + state: WRITE_ONLY - - direction: DROP - index: - constraintId: 1 @@ -911,22 +847,13 @@ upsert descriptor #104 - storeColumnIds: - - 2 - storeColumnNames: - - j - unique: true - version: 4 - mutationId: 1 - - state: DELETE_ONLY - - - column: - - id: 2 - - name: j - - nullable: true - - type: - - family: StringFamily - - oid: 25 - - direction: DROP + - - j + - unique: true + - version: 4 - mutationId: 1 - state: WRITE_ONLY + - state: DELETE_ONLY name: t + nextColumnId: 4 ... parentId: 100 primaryIndex: @@ -947,7 +874,7 @@ upsert descriptor #104 - 3 storeColumnNames: - j - - - j + - - crdb_internal_column_3_name_placeholder unique: true version: 4 ... @@ -957,11 +884,18 @@ upsert descriptor #104 + version: "13" persist all catalog changes to storage adding table for stats refresh: 104 -update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 4 with 4 MutationType ops pending" +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 4 with 5 MutationType ops pending" commit transaction #19 begin transaction #20 -## PostCommitNonRevertiblePhase stage 3 of 4 with 6 MutationType ops +## PostCommitNonRevertiblePhase stage 3 of 4 with 7 MutationType ops upsert descriptor #104 + ... + direction: DROP + mutationId: 1 + - state: WRITE_ONLY + + state: DELETE_ONLY + - direction: DROP + index: ... version: 4 mutationId: 1 @@ -1016,17 +950,16 @@ upsert descriptor #104 - 3 columnNames: - i + - - crdb_internal_column_2_name_placeholder - j - - - j defaultColumnId: 2 - name: primary ... id: 104 modificationTime: {} - mutations: - - column: - id: 2 - - name: j + - name: crdb_internal_column_2_name_placeholder - nullable: true - type: - family: StringFamily @@ -1056,7 +989,7 @@ upsert descriptor #104 - - 2 - - 3 - storeColumnNames: - - - j + - - crdb_internal_column_2_name_placeholder - - j - unique: true - version: 4 diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_10_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_10_of_15.explain index b2a1cccd5382..eab9412f5f1b 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_10_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_10_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -40,12 +38,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":5,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 5 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ └── 7 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_11_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_11_of_15.explain index 705be2efa8d3..6007463c3018 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_11_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_11_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -40,12 +38,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":5,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 5 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ └── 7 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_12_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_12_of_15.explain index 61ac0a2191e6..625b2c14a20f 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_12_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_12_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -40,12 +38,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":5,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 5 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ └── 7 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_13_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_13_of_15.explain index 78eee4f354eb..ea2648f2e73b 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_13_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_13_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── MERGE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -39,11 +37,9 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":5,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":4,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 6 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_14_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_14_of_15.explain index 69f7643d899e..922b053c9e86 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_14_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_14_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── MERGE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -39,11 +37,9 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":5,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":4,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 6 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_15_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_15_of_15.explain index e77f761f6116..efae6fa51a2c 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_15_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_15_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── WRITE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -40,12 +38,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":5,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 5 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ └── 7 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_1_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_1_of_15.explain index d82e231ed69e..5cf035122ca2 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_1_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_1_of_15.explain @@ -10,37 +10,33 @@ EXPLAIN (DDL) rollback at post-commit stage 1 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE INT8 USING ‹j›::INT8; └── PostCommitNonRevertiblePhase └── Stage 1 of 1 in PostCommitNonRevertiblePhase - ├── 18 elements transitioning toward ABSENT + ├── 16 elements transitioning toward ABSENT │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ ├── PUBLIC → ABSENT IndexData:{DescID: 104 (t), IndexID: 2 (t_pkey-)} │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ ├── PUBLIC → ABSENT IndexData:{DescID: 104 (t), IndexID: 4 (t_pkey-)} │ ├── DELETE_ONLY → ABSENT Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ ├── PUBLIC → ABSENT ColumnType:{DescID: 104 (t), ColumnFamilyID: 0 (primary), ColumnID: 3 (j-), TypeName: "INT8"} │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - └── 19 Mutation operations + │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + └── 17 Mutation operations ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} - ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":2,"Kind":2,"Ordinal":1,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} ├── MakeIndexAbsent {"IndexID":2,"TableID":104} ├── CreateGCJobForIndex {"IndexID":2,"TableID":104} ├── MakeIndexAbsent {"IndexID":3,"TableID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_2_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_2_of_15.explain index 87a70c161b3a..9139cad5fbcb 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_2_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_2_of_15.explain @@ -10,23 +10,21 @@ EXPLAIN (DDL) rollback at post-commit stage 2 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE INT8 USING ‹j›::INT8; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":3,"TableID":104} @@ -34,12 +32,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":2,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeIndexAbsent {"IndexID":2,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_3_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_3_of_15.explain index d8a693dbe077..da9a0c1ed4d9 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_3_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_3_of_15.explain @@ -10,23 +10,21 @@ EXPLAIN (DDL) rollback at post-commit stage 3 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE INT8 USING ‹j›::INT8; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":3,"TableID":104} @@ -34,12 +32,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":2,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeIndexAbsent {"IndexID":2,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_4_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_4_of_15.explain index b616cc33095b..342c02e61e93 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_4_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_4_of_15.explain @@ -10,23 +10,21 @@ EXPLAIN (DDL) rollback at post-commit stage 4 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE INT8 USING ‹j›::INT8; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":3,"TableID":104} @@ -34,12 +32,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":2,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeIndexAbsent {"IndexID":2,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_5_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_5_of_15.explain index 079a12ac6045..4121c718163f 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_5_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_5_of_15.explain @@ -10,33 +10,29 @@ EXPLAIN (DDL) rollback at post-commit stage 5 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE INT8 USING ‹j›::INT8; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── MERGE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_6_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_6_of_15.explain index a88c9ed35abb..258d655f10ae 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_6_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_6_of_15.explain @@ -10,33 +10,29 @@ EXPLAIN (DDL) rollback at post-commit stage 6 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE INT8 USING ‹j›::INT8; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── MERGE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_7_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_7_of_15.explain index bc53535d0e58..1fb4eedaf11b 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_7_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_7_of_15.explain @@ -10,23 +10,21 @@ EXPLAIN (DDL) rollback at post-commit stage 7 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE INT8 USING ‹j›::INT8; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── WRITE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} @@ -34,12 +32,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":2,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_8_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_8_of_15.explain index 20f00be37745..442c40b04db8 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_8_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_8_of_15.explain @@ -10,23 +10,21 @@ EXPLAIN (DDL) rollback at post-commit stage 8 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE INT8 USING ‹j›::INT8; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── WRITE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} @@ -34,12 +32,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":2,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_9_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_9_of_15.explain index 5ed5bad574f9..172c01485e99 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_9_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general/alter_table_alter_column_type_general__rollback_9_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -39,12 +37,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 4 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ └── 6 Mutation operations │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.definition b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.definition index 1022bfb6b2fd..a3f173d6a556 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.definition +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.definition @@ -10,7 +10,7 @@ INSERT INTO t VALUES (100+$stageKey, default); # One row is expected to be inserted after each stage. stage-query phase=PostCommitPhase stage=: -SELECT count(*)=$successfulStageCount FROM t WHERE i >= 100; +SELECT count(*)=$successfulStageCount FROM t WHERE i >= 100 AND j = '99'; ---- true diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.explain index db187216aa94..04214f5a1a16 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.explain @@ -9,32 +9,28 @@ EXPLAIN (DDL) ALTER TABLE t ALTER COLUMN j SET DATA TYPE TEXT; Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER COLUMN ‹j› SET DATA TYPE STRING; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 7 elements transitioning toward PUBLIC + │ ├── 6 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey+), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey~)} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey+)} │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104 (t), IndexID: 4 (t_pkey+)} │ │ ├── ABSENT → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j+)} - │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j+)} │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 104 (t), ColumnFamilyID: 0 (primary), ColumnID: 3 (j+), TypeName: "STRING"} │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 4 (t_pkey+)} - │ ├── 11 elements transitioning toward TRANSIENT_ABSENT + │ ├── 10 elements transitioning toward TRANSIENT_ABSENT │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey~), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey-)} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey~)} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 2 (t_pkey~)} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 2 (t_pkey~)} │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104 (t), IndexID: 2 (t_pkey~)} │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey-)} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 3} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 3} │ │ ├── ABSENT → PUBLIC ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j+), Usage: ALTER_TYPE_USING} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 2 (t_pkey~)} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} - │ │ └── ABSENT → PUBLIC ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j-j_shadow~)} - │ └── 16 Mutation operations + │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} + │ └── 14 Mutation operations │ ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":3,"PgAttributeNum":2,"TableID":104}} - │ ├── SetColumnName {"ColumnID":3,"Name":"j","TableID":104} │ ├── UpsertColumnType {"ColumnType":{"ColumnFamilyOrderFollowsColumnID":2,"ColumnID":3,"IsNullable":true,"TableID":104}} │ ├── AddColumnComputeExpression {"ComputeExpression":{"ColumnID":3,"TableID":104,"Usage":1}} - │ ├── SetColumnName {"ColumnID":2,"Name":"j_shadow","TableID":104} │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":3}} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} @@ -48,55 +44,49 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER CO │ └── AddColumnToIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} ├── PreCommitPhase │ ├── Stage 1 of 2 in PreCommitPhase - │ │ ├── 7 elements transitioning toward PUBLIC + │ │ ├── 6 elements transitioning toward PUBLIC │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey+), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey~)} │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey+)} │ │ │ ├── PUBLIC → ABSENT IndexData:{DescID: 104 (t), IndexID: 4 (t_pkey+)} │ │ │ ├── DELETE_ONLY → ABSENT Column:{DescID: 104 (t), ColumnID: 3 (j+)} - │ │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j+)} │ │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 104 (t), ColumnFamilyID: 0 (primary), ColumnID: 3 (j+), TypeName: "STRING"} │ │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 4 (t_pkey+)} - │ │ ├── 11 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── 10 elements transitioning toward TRANSIENT_ABSENT │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey~), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey-)} │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey~)} - │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 2 (t_pkey~)} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 2 (t_pkey~)} │ │ │ ├── PUBLIC → ABSENT IndexData:{DescID: 104 (t), IndexID: 2 (t_pkey~)} │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey-)} │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 3} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 3} │ │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j+), Usage: ALTER_TYPE_USING} │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 2 (t_pkey~)} - │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} - │ │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j-j_shadow~)} + │ │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} │ │ └── 1 Mutation operation │ │ └── UndoAllInTxnImmediateMutationOpSideEffects │ └── Stage 2 of 2 in PreCommitPhase - │ ├── 7 elements transitioning toward PUBLIC + │ ├── 6 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey+), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey~)} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey+)} │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104 (t), IndexID: 4 (t_pkey+)} │ │ ├── ABSENT → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j+)} - │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j+)} │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 104 (t), ColumnFamilyID: 0 (primary), ColumnID: 3 (j+), TypeName: "STRING"} │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 4 (t_pkey+)} - │ ├── 11 elements transitioning toward TRANSIENT_ABSENT + │ ├── 10 elements transitioning toward TRANSIENT_ABSENT │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey~), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey-)} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey~)} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 2 (t_pkey~)} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 2 (t_pkey~)} │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104 (t), IndexID: 2 (t_pkey~)} │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey-)} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 3} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 3} │ │ ├── ABSENT → PUBLIC ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j+), Usage: ALTER_TYPE_USING} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 2 (t_pkey~)} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} - │ │ └── ABSENT → PUBLIC ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j-j_shadow~)} - │ └── 21 Mutation operations + │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} + │ └── 19 Mutation operations │ ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":3,"PgAttributeNum":2,"TableID":104}} - │ ├── SetColumnName {"ColumnID":3,"Name":"j","TableID":104} │ ├── UpsertColumnType {"ColumnType":{"ColumnFamilyOrderFollowsColumnID":2,"ColumnID":3,"IsNullable":true,"TableID":104}} │ ├── AddColumnComputeExpression {"ComputeExpression":{"ColumnID":3,"TableID":104,"Usage":1}} - │ ├── SetColumnName {"ColumnID":2,"Name":"j_shadow","TableID":104} │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":3}} │ ├── MaybeAddSplitForIndex {"IndexID":2,"TableID":104} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"TableID":104} @@ -236,58 +226,56 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER CO │ ├── 2 elements transitioning toward PUBLIC │ │ ├── ABSENT → PUBLIC ColumnDefaultExpression:{DescID: 104 (t), ColumnID: 3 (j+), Expr: 99:::INT8} │ │ └── ABSENT → PUBLIC ColumnOnUpdateExpression:{DescID: 104 (t), ColumnID: 3 (j+)} - │ ├── 9 elements transitioning toward TRANSIENT_ABSENT - │ │ ├── TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey-)} + │ ├── 6 elements transitioning toward TRANSIENT_ABSENT │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 3} │ │ ├── TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey~)} │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── PUBLIC → TRANSIENT_ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j+), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 3} - │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 5} - │ │ └── PUBLIC → TRANSIENT_ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j-j_shadow~)} - │ ├── 5 elements transitioning toward ABSENT - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 2 (j-j_shadow~)} + │ │ └── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 5} + │ ├── 2 elements transitioning toward ABSENT │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 1 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 1 (t_pkey-)} │ │ └── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey-), ConstraintID: 1} - │ └── 18 Mutation operations + │ └── 12 Mutation operations │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":1,"TableID":104} - │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":5,"TableID":104} │ ├── AddColumnDefaultExpression {"Default":{"ColumnID":3,"TableID":104}} │ ├── AddColumnOnUpdateExpression {"OnUpdate":{"ColumnID":3,"TableID":104}} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":1,"TableID":104} - │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":1,"Kind":2,"TableID":104} - │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104} │ └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"PostCommitNonRev..."} ├── Stage 2 of 4 in PostCommitNonRevertiblePhase - │ ├── 3 elements transitioning toward PUBLIC - │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey+), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey~)} - │ │ ├── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 4 (t_pkey+)} - │ │ └── WRITE_ONLY → PUBLIC Column:{DescID: 104 (t), ColumnID: 3 (j+)} - │ ├── 2 elements transitioning toward TRANSIENT_ABSENT - │ │ ├── PUBLIC → TRANSIENT_VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey~), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey-)} - │ │ └── PUBLIC → TRANSIENT_ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey~)} - │ ├── 2 elements transitioning toward ABSENT - │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~)} - │ │ └── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey-), ConstraintID: 1} - │ └── 10 Mutation operations - │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":2,"TableID":104} + │ ├── 4 elements transitioning toward PUBLIC + │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey+), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey~)} + │ │ ├── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 4 (t_pkey+)} + │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104 (t), ColumnID: 3 (j+)} + │ │ └── ABSENT → PUBLIC ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j+)} + │ ├── 4 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── PUBLIC → TRANSIENT_VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey~), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey-)} + │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey~)} + │ │ ├── TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey-)} + │ │ └── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 3} + │ ├── 4 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104 (t), ColumnID: 2 (j-)} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 2 (j-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 1 (t_pkey-)} + │ │ └── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey-), ConstraintID: 1} + │ └── 15 Mutation operations + │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} + │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":1,"Kind":2,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":1,"TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} + │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── SetIndexName {"IndexID":4,"Name":"t_pkey","TableID":104} + │ ├── SetColumnName {"ColumnID":3,"Name":"j","TableID":104} + │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnPublic {"ColumnID":3,"TableID":104} │ ├── RefreshStats {"TableID":104} @@ -297,9 +285,12 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER CO │ ├── 4 elements transitioning toward TRANSIENT_ABSENT │ │ ├── TRANSIENT_VALIDATED → TRANSIENT_DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey~), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey-)} │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey~)} - │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), IndexID: 2 (t_pkey~)} + │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j-), IndexID: 2 (t_pkey~)} │ │ └── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j+), IndexID: 2 (t_pkey~)} - │ └── 6 Mutation operations + │ ├── 1 element transitioning toward ABSENT + │ │ └── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 2 (j-)} + │ └── 7 Mutation operations + │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":2,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} @@ -313,10 +304,10 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER CO │ ├── PUBLIC → TRANSIENT_ABSENT IndexData:{DescID: 104 (t), IndexID: 3} │ └── PUBLIC → TRANSIENT_ABSENT IndexData:{DescID: 104 (t), IndexID: 5} ├── 5 elements transitioning toward ABSENT - │ ├── DELETE_ONLY → ABSENT Column:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~)} - │ ├── PUBLIC → ABSENT ColumnType:{DescID: 104 (t), ColumnFamilyID: 0 (primary), ColumnID: 2 (j-j_shadow~), TypeName: "INT8"} - │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~), Expr: 99:::INT8} - │ ├── PUBLIC → ABSENT ColumnOnUpdateExpression:{DescID: 104 (t), ColumnID: 2 (j-j_shadow~)} + │ ├── DELETE_ONLY → ABSENT Column:{DescID: 104 (t), ColumnID: 2 (j-)} + │ ├── PUBLIC → ABSENT ColumnType:{DescID: 104 (t), ColumnFamilyID: 0 (primary), ColumnID: 2 (j-), TypeName: "INT8"} + │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 104 (t), ColumnID: 2 (j-), Expr: 99:::INT8} + │ ├── PUBLIC → ABSENT ColumnOnUpdateExpression:{DescID: 104 (t), ColumnID: 2 (j-)} │ └── PUBLIC → ABSENT IndexData:{DescID: 104 (t), IndexID: 1 (t_pkey-)} └── 10 Mutation operations ├── RemoveColumnDefaultExpression {"ColumnID":2,"TableID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.explain_shape b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.explain_shape index a690546cf4bf..eca98900b424 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.explain_shape +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.explain_shape @@ -9,7 +9,7 @@ EXPLAIN (DDL, SHAPE) ALTER TABLE t ALTER COLUMN j SET DATA TYPE TEXT; Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER COLUMN ‹j› SET DATA TYPE STRING; ├── execute 2 system table mutations transactions ├── backfill using primary index t_pkey- in relation t - │ └── into t_pkey~ (i; j-j_shadow~, j+) + │ └── into t_pkey~ (i; j-, j+) ├── execute 2 system table mutations transactions ├── merge temporary indexes into backfilled indexes in relation t │ └── from t@[3] into t_pkey~ diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.side_effects index bb09fb76a95b..9f809a48ae20 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.side_effects +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr.side_effects @@ -14,32 +14,26 @@ begin transaction #1 checking for feature: ALTER TABLE increment telemetry for sql.schema.alter_table increment telemetry for sql.schema.alter_table.alter_column_type -## StatementPhase stage 1 of 1 with 16 MutationType ops +## StatementPhase stage 1 of 1 with 14 MutationType ops upsert descriptor #104 - ... - - defaultExpr: 99:::INT8 - id: 2 - - name: j - + name: j_shadow - nullable: true - onUpdateExpr: (-1):::INT8 ... - 1 - 2 + - 3 columnNames: - i - + - j_shadow - j + + - crdb_internal_column_3_name_placeholder defaultColumnId: 2 + name: primary ... id: 104 modificationTime: {} + mutations: + - column: - + computeExpr: j_shadow::STRING + + computeExpr: j::STRING + id: 3 - + name: j + + name: crdb_internal_column_3_name_placeholder + nullable: true + pgAttributeNum: 2 + type: @@ -70,8 +64,8 @@ upsert descriptor #104 + - 2 + - 3 + storeColumnNames: - + - j_shadow + - j + + - crdb_internal_column_3_name_placeholder + unique: true + version: 4 + mutationId: 1 @@ -98,8 +92,8 @@ upsert descriptor #104 + - 2 + - 3 + storeColumnNames: - + - j_shadow + - j + + - crdb_internal_column_3_name_placeholder + unique: true + useDeletePreservingEncoding: true + version: 4 @@ -126,7 +120,7 @@ upsert descriptor #104 + storeColumnIds: + - 3 + storeColumnNames: - + - j + + - crdb_internal_column_3_name_placeholder + unique: true + version: 4 + mutationId: 1 @@ -141,13 +135,6 @@ upsert descriptor #104 + nextIndexId: 5 nextMutationId: 1 parentId: 100 - ... - - 2 - storeColumnNames: - - - j - + - j_shadow - unique: true - version: 4 ... time: {} unexposedParentSchemaId: 101 @@ -158,15 +145,8 @@ upsert descriptor #104 ## PreCommitPhase stage 1 of 2 with 1 MutationType op undo all catalog changes within txn #1 persist all catalog changes to storage -## PreCommitPhase stage 2 of 2 with 21 MutationType ops +## PreCommitPhase stage 2 of 2 with 19 MutationType ops upsert descriptor #104 - ... - - defaultExpr: 99:::INT8 - id: 2 - - name: j - + name: j_shadow - nullable: true - onUpdateExpr: (-1):::INT8 ... createAsOfTime: wallTime: "1640995200000000000" @@ -204,17 +184,18 @@ upsert descriptor #104 + - 3 columnNames: - i - + - j_shadow - j + + - crdb_internal_column_3_name_placeholder defaultColumnId: 2 + name: primary ... id: 104 modificationTime: {} + mutations: + - column: - + computeExpr: j_shadow::STRING + + computeExpr: j::STRING + id: 3 - + name: j + + name: crdb_internal_column_3_name_placeholder + nullable: true + pgAttributeNum: 2 + type: @@ -245,8 +226,8 @@ upsert descriptor #104 + - 2 + - 3 + storeColumnNames: - + - j_shadow + - j + + - crdb_internal_column_3_name_placeholder + unique: true + version: 4 + mutationId: 1 @@ -273,8 +254,8 @@ upsert descriptor #104 + - 2 + - 3 + storeColumnNames: - + - j_shadow + - j + + - crdb_internal_column_3_name_placeholder + unique: true + useDeletePreservingEncoding: true + version: 4 @@ -301,7 +282,7 @@ upsert descriptor #104 + storeColumnIds: + - 3 + storeColumnNames: - + - j + + - crdb_internal_column_3_name_placeholder + unique: true + version: 4 + mutationId: 1 @@ -316,13 +297,6 @@ upsert descriptor #104 + nextIndexId: 5 nextMutationId: 1 parentId: 100 - ... - - 2 - storeColumnNames: - - - j - + - j_shadow - unique: true - version: 4 ... time: {} unexposedParentSchemaId: 101 @@ -464,7 +438,7 @@ upsert descriptor #104 partitioning: {} sharded: {} ... - - j + - crdb_internal_column_3_name_placeholder unique: true + useDeletePreservingEncoding: true version: 4 @@ -495,8 +469,8 @@ upsert descriptor #104 - - 2 - 3 storeColumnNames: - - - j_shadow - - j + - - j + - crdb_internal_column_3_name_placeholder unique: true - useDeletePreservingEncoding: true version: 4 @@ -524,7 +498,7 @@ upsert descriptor #104 + storeColumnIds: + - 2 + storeColumnNames: - + - j_shadow + + - j + unique: true + version: 4 + mutationId: 1 @@ -549,7 +523,7 @@ upsert descriptor #104 partitioning: {} sharded: {} ... - - j + - crdb_internal_column_3_name_placeholder unique: true + useDeletePreservingEncoding: true version: 4 @@ -582,8 +556,8 @@ upsert descriptor #104 - 2 + - 3 storeColumnNames: - - j_shadow - + - j + - j + + - crdb_internal_column_3_name_placeholder unique: true version: 4 ... @@ -693,88 +667,28 @@ begin transaction #17 validate forward indexes [4] in table #104 commit transaction #17 begin transaction #18 -## PostCommitNonRevertiblePhase stage 1 of 4 with 18 MutationType ops +## PostCommitNonRevertiblePhase stage 1 of 4 with 12 MutationType ops upsert descriptor #104 - ... - oid: 20 - width: 64 - - - defaultExpr: 99:::INT8 - - id: 2 - - name: j_shadow - - nullable: true - - onUpdateExpr: (-1):::INT8 - - type: - - family: IntFamily - - oid: 20 - - width: 64 - createAsOfTime: - wallTime: "1640995200000000000" ... statement: ALTER TABLE t ALTER COLUMN j SET DATA TYPE STRING statementTag: ALTER TABLE - revertible: true targetRanks: targets: - ... - columnNames: - - i - - - j_shadow - - j - + - j - defaultColumnId: 2 - name: primary ... mutations: - column: - - computeExpr: j_shadow::STRING + - computeExpr: j::STRING + defaultExpr: 99:::INT8 id: 3 - name: j + name: crdb_internal_column_3_name_placeholder nullable: true + onUpdateExpr: (-1):::INT8 pgAttributeNum: 2 type: ... + version: 4 mutationId: 1 - state: WRITE_ONLY - - - direction: DROP - - index: - - constraintId: 3 - - createdExplicitly: true - - encodingType: 1 - - foreignKey: {} - - geoConfig: {} - - id: 3 - - interleave: {} - - keyColumnDirections: - - - ASC - - keyColumnIds: - - - 1 - - keyColumnNames: - - - i - - name: crdb_internal_index_3_name_placeholder - - partitioning: {} - - sharded: {} - - storeColumnIds: - - - 2 - - - 3 - - storeColumnNames: - - - j_shadow - - - j - - unique: true - - useDeletePreservingEncoding: true - - version: 4 - - mutationId: 1 - - state: DELETE_ONLY - - direction: ADD - index: - ... - - 2 - storeColumnNames: - - - j_shadow - - unique: true - - version: 4 - - mutationId: 1 - state: WRITE_ONLY - - direction: DROP - index: @@ -797,67 +711,57 @@ upsert descriptor #104 - storeColumnIds: - - 3 - storeColumnNames: - - j - unique: true + - - crdb_internal_column_3_name_placeholder + - unique: true - useDeletePreservingEncoding: true - version: 4 - mutationId: 1 + - version: 4 + - mutationId: 1 state: DELETE_ONLY - + - column: - + defaultExpr: 99:::INT8 - + id: 2 - + name: j - + nullable: true - + onUpdateExpr: (-1):::INT8 - + type: - + family: IntFamily - + oid: 20 - + width: 64 - + direction: DROP - + mutationId: 1 - + state: WRITE_ONLY name: t - nextColumnId: 4 - ... - - 3 - storeColumnNames: - - - j_shadow - - j - + - j - unique: true - version: 4 ... time: {} unexposedParentSchemaId: 101 - version: "11" + version: "12" persist all catalog changes to storage -update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 4 with 8 MutationType ops pending" +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 4 with 13 MutationType ops pending" set schema change job #1 to non-cancellable commit transaction #18 begin transaction #19 -## PostCommitNonRevertiblePhase stage 2 of 4 with 10 MutationType ops +## PostCommitNonRevertiblePhase stage 2 of 4 with 15 MutationType ops upsert descriptor #104 ... - oid: 20 width: 64 - + - defaultExpr: 99:::INT8 + - defaultExpr: 99:::INT8 + - id: 2 + id: 3 - + name: j - + nullable: true - + onUpdateExpr: (-1):::INT8 + name: j + nullable: true + onUpdateExpr: (-1):::INT8 + pgAttributeNum: 2 - + type: + type: + - family: IntFamily + - oid: 20 + - width: 64 + family: StringFamily + oid: 25 createAsOfTime: wallTime: "1640995200000000000" + ... + columnNames: + - i + + - crdb_internal_column_2_name_placeholder + - j + - - crdb_internal_column_3_name_placeholder + defaultColumnId: 2 + name: primary ... - column: defaultExpr: 99:::INT8 - id: 3 + - name: crdb_internal_column_3_name_placeholder + id: 2 - name: j + + name: crdb_internal_column_2_name_placeholder nullable: true onUpdateExpr: (-1):::INT8 - pgAttributeNum: 2 @@ -870,37 +774,63 @@ upsert descriptor #104 + width: 64 + direction: DROP mutationId: 1 - - state: WRITE_ONLY - - - direction: ADD - + state: DELETE_ONLY - + - direction: DROP + state: WRITE_ONLY + - direction: DROP index: - - constraintId: 4 + - constraintId: 3 + constraintId: 2 createdExplicitly: true encodingType: 1 foreignKey: {} geoConfig: {} - - id: 4 + - id: 3 + id: 2 interleave: {} keyColumnDirections: ... keyColumnNames: - i - - name: crdb_internal_index_4_name_placeholder + - name: crdb_internal_index_3_name_placeholder + name: crdb_internal_index_2_name_placeholder partitioning: {} sharded: {} - storeColumnIds: - + - 2 + ... - 3 storeColumnNames: + + - crdb_internal_column_2_name_placeholder - j + - - crdb_internal_column_3_name_placeholder + unique: true + - useDeletePreservingEncoding: true + version: 4 + mutationId: 1 + - state: DELETE_ONLY + - - direction: ADD + - index: + - constraintId: 4 + - createdExplicitly: true + - encodingType: 1 + - foreignKey: {} + - geoConfig: {} + - id: 4 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - keyColumnNames: + - - i + - name: crdb_internal_index_4_name_placeholder + - partitioning: {} + - sharded: {} + - storeColumnIds: + - - 3 + - storeColumnNames: + - - crdb_internal_column_3_name_placeholder - unique: true - version: 4 - mutationId: 1 - - state: WRITE_ONLY + state: WRITE_ONLY - - direction: DROP - index: - constraintId: 1 @@ -922,25 +852,13 @@ upsert descriptor #104 - storeColumnIds: - - 2 - storeColumnNames: - - j - unique: true - version: 4 - mutationId: 1 - - state: DELETE_ONLY - - - column: - - defaultExpr: 99:::INT8 - - id: 2 - - name: j - - nullable: true - - onUpdateExpr: (-1):::INT8 - - type: - - family: IntFamily - - oid: 20 - - width: 64 - - direction: DROP + - - j + - unique: true + - version: 4 - mutationId: 1 - state: WRITE_ONLY + - state: DELETE_ONLY name: t + nextColumnId: 4 ... parentId: 100 primaryIndex: @@ -961,7 +879,7 @@ upsert descriptor #104 - 3 storeColumnNames: - j - - - j + - - crdb_internal_column_3_name_placeholder unique: true version: 4 ... @@ -971,11 +889,18 @@ upsert descriptor #104 + version: "13" persist all catalog changes to storage adding table for stats refresh: 104 -update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 4 with 4 MutationType ops pending" +update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 4 with 5 MutationType ops pending" commit transaction #19 begin transaction #20 -## PostCommitNonRevertiblePhase stage 3 of 4 with 6 MutationType ops +## PostCommitNonRevertiblePhase stage 3 of 4 with 7 MutationType ops upsert descriptor #104 + ... + direction: DROP + mutationId: 1 + - state: WRITE_ONLY + + state: DELETE_ONLY + - direction: DROP + index: ... version: 4 mutationId: 1 @@ -1030,10 +955,9 @@ upsert descriptor #104 - 3 columnNames: - i + - - crdb_internal_column_2_name_placeholder - j - - - j defaultColumnId: 2 - name: primary ... id: 104 modificationTime: {} @@ -1041,7 +965,7 @@ upsert descriptor #104 - - column: - defaultExpr: 99:::INT8 - id: 2 - - name: j + - name: crdb_internal_column_2_name_placeholder - nullable: true - onUpdateExpr: (-1):::INT8 - type: @@ -1073,7 +997,7 @@ upsert descriptor #104 - - 2 - - 3 - storeColumnNames: - - - j + - - crdb_internal_column_2_name_placeholder - - j - unique: true - version: 4 diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_10_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_10_of_15.explain index d1dd2b2fb497..8d58731d70f4 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_10_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_10_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -40,12 +38,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":5,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 5 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ └── 7 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_11_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_11_of_15.explain index e87cfe6c0c56..2db5e0f44e94 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_11_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_11_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -40,12 +38,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":5,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 5 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ └── 7 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_12_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_12_of_15.explain index c54c048d78a9..18856ce3c4cf 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_12_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_12_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -40,12 +38,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":5,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 5 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ └── 7 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_13_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_13_of_15.explain index 114fe940d461..56a2f65ad390 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_13_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_13_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── MERGE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -39,11 +37,9 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":5,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":4,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 6 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_14_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_14_of_15.explain index 23c93aee0633..1564d0326f08 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_14_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_14_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── MERGE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -39,11 +37,9 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":5,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":4,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 6 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_15_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_15_of_15.explain index 9e7f65167b05..b07fa8161aa5 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_15_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_15_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── WRITE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -40,12 +38,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":5,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 5 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ └── 7 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_1_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_1_of_15.explain index 6dc1f3ce303f..14bd6c3fc938 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_1_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_1_of_15.explain @@ -10,37 +10,33 @@ EXPLAIN (DDL) rollback at post-commit stage 1 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE STRING; └── PostCommitNonRevertiblePhase └── Stage 1 of 1 in PostCommitNonRevertiblePhase - ├── 18 elements transitioning toward ABSENT + ├── 16 elements transitioning toward ABSENT │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ ├── PUBLIC → ABSENT IndexData:{DescID: 104 (t), IndexID: 2 (t_pkey-)} │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ ├── PUBLIC → ABSENT IndexData:{DescID: 104 (t), IndexID: 4 (t_pkey-)} │ ├── DELETE_ONLY → ABSENT Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ ├── PUBLIC → ABSENT ColumnType:{DescID: 104 (t), ColumnFamilyID: 0 (primary), ColumnID: 3 (j-), TypeName: "STRING"} │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - └── 19 Mutation operations + │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + └── 17 Mutation operations ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} - ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":2,"Kind":2,"Ordinal":1,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} ├── MakeIndexAbsent {"IndexID":2,"TableID":104} ├── CreateGCJobForIndex {"IndexID":2,"TableID":104} ├── MakeIndexAbsent {"IndexID":3,"TableID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_2_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_2_of_15.explain index 57de2b432d2c..a836dd87c477 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_2_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_2_of_15.explain @@ -10,23 +10,21 @@ EXPLAIN (DDL) rollback at post-commit stage 2 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE STRING; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":3,"TableID":104} @@ -34,12 +32,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":2,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeIndexAbsent {"IndexID":2,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_3_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_3_of_15.explain index 7b5ceed34e52..e5fb36531e0d 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_3_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_3_of_15.explain @@ -10,23 +10,21 @@ EXPLAIN (DDL) rollback at post-commit stage 3 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE STRING; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":3,"TableID":104} @@ -34,12 +32,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":2,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeIndexAbsent {"IndexID":2,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_4_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_4_of_15.explain index c0186f945992..6df9b6afb3d4 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_4_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_4_of_15.explain @@ -10,23 +10,21 @@ EXPLAIN (DDL) rollback at post-commit stage 4 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE STRING; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":3,"TableID":104} @@ -34,12 +32,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":2,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeIndexAbsent {"IndexID":2,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_5_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_5_of_15.explain index bca78ce52859..89aafe44e06a 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_5_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_5_of_15.explain @@ -10,33 +10,29 @@ EXPLAIN (DDL) rollback at post-commit stage 5 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE STRING; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── MERGE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_6_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_6_of_15.explain index cfb8ed06f9ea..2062e9eca50b 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_6_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_6_of_15.explain @@ -10,33 +10,29 @@ EXPLAIN (DDL) rollback at post-commit stage 6 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE STRING; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── MERGE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_7_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_7_of_15.explain index b8d9d9d780dc..d81d85d11f95 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_7_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_7_of_15.explain @@ -10,23 +10,21 @@ EXPLAIN (DDL) rollback at post-commit stage 7 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE STRING; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── WRITE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} @@ -34,12 +32,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":2,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_8_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_8_of_15.explain index d93cc9ab4d57..7a29e2d9e9ed 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_8_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_8_of_15.explain @@ -10,23 +10,21 @@ EXPLAIN (DDL) rollback at post-commit stage 8 of 15; Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ALTER COLUMN ‹j› SET DATA TYPE STRING; └── PostCommitNonRevertiblePhase ├── Stage 1 of 2 in PostCommitNonRevertiblePhase - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── WRITE_ONLY → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 17 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} + │ └── 15 Mutation operations │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":104} @@ -34,12 +32,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":2,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_9_of_15.explain b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_9_of_15.explain index dcfe82823cb1..01c387e50d9f 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_9_of_15.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_column_type_general_expr/alter_table_alter_column_type_general_expr__rollback_9_of_15.explain @@ -13,24 +13,22 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 2 elements transitioning toward PUBLIC │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 104 (t), IndexID: 1 (t_pkey+), ConstraintID: 1} │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 1 (t_pkey+)} - │ ├── 16 elements transitioning toward ABSENT + │ ├── 14 elements transitioning toward ABSENT │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (t), Name: "t_pkey", IndexID: 2 (t_pkey-)} │ │ ├── TRANSIENT_DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 3, ConstraintID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 3} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 3} │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104 (t), IndexID: 4 (t_pkey-), ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 4 (t_pkey-)} │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104 (t), IndexID: 5, ConstraintID: 5, SourceIndexID: 2 (t_pkey-)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 5} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104 (t), ColumnID: 3 (j-)} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j", ColumnID: 3 (j-)} │ │ ├── PUBLIC → ABSENT ColumnComputeExpression:{DescID: 104 (t), ColumnID: 3 (j-), Usage: ALTER_TYPE_USING} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 3} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 4 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} - │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104 (t), Name: "j_shadow", ColumnID: 2 (j_shadow-j+)} - │ └── 20 Mutation operations + │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 5} + │ └── 18 Mutation operations │ ├── SetIndexName {"IndexID":1,"Name":"t_pkey","TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetIndexName {"IndexID":2,"Name":"crdb_internal_in...","TableID":104} @@ -39,12 +37,10 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} - │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComputeExpression {"ColumnID":3,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"Ordinal":1,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":5,"Kind":2,"TableID":104} - │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":104} │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":3,"TableID":104} │ ├── MakeIndexAbsent {"IndexID":4,"TableID":104} @@ -55,7 +51,7 @@ Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› A │ ├── 4 elements transitioning toward ABSENT │ │ ├── VALIDATED → DELETE_ONLY PrimaryIndex:{DescID: 104 (t), IndexID: 2 (t_pkey-), ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1 (t_pkey+)} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 1 (i), IndexID: 2 (t_pkey-)} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j_shadow-j+), IndexID: 2 (t_pkey-)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 2 (j+), IndexID: 2 (t_pkey-)} │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104 (t), ColumnID: 3 (j-), IndexID: 2 (t_pkey-)} │ └── 6 Mutation operations │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} diff --git a/pkg/testutils/lint/passes/redactcheck/redactcheck.go b/pkg/testutils/lint/passes/redactcheck/redactcheck.go index 26f97ce8dec3..06799cc14d80 100644 --- a/pkg/testutils/lint/passes/redactcheck/redactcheck.go +++ b/pkg/testutils/lint/passes/redactcheck/redactcheck.go @@ -168,6 +168,9 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { "github.com/cockroachdb/cockroach/pkg/rpc/rpcpb": { "ConnectionClass": {}, }, + "github.com/cockroachdb/cockroach/pkg/server/license": { + "LicType": {}, + }, "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb": { "JobID": {}, "ScheduleID": {},